From 4cc28cde6a815017035ac04786e3e29622c6a329 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 04:02:33 +0100 Subject: [PATCH 01/65] Feat: Add Kafka integration for Parseable server #936 . --- Cargo.lock | 517 ++++++++- Cargo.toml | 11 +- scripts/kafka_log_stream_generator.py | 123 ++ src/cli.rs | 1024 ++++++++++++----- src/connectors/common/config.rs | 164 +++ src/connectors/common/mod.rs | 31 + src/connectors/common/processor.rs | 11 + src/connectors/common/shutdown.rs | 118 ++ src/connectors/common/types/mod.rs | 45 + src/connectors/kafka/config.rs | 627 ++++++++++ src/connectors/kafka/consumer.rs | 169 +++ src/connectors/kafka/metrics.rs | 37 + src/connectors/kafka/mod.rs | 231 ++++ .../kafka/partition_stream_queue.rs | 90 ++ src/connectors/kafka/processor.rs | 154 +++ src/connectors/kafka/rebalance_listener.rs | 65 ++ src/connectors/kafka/sink.rs | 68 ++ src/connectors/kafka/state.rs | 50 + src/connectors/mod.rs | 85 ++ src/lib.rs | 1 + src/main.rs | 34 +- src/metadata.rs | 14 + 22 files changed, 3313 insertions(+), 356 deletions(-) create mode 100644 scripts/kafka_log_stream_generator.py create mode 100644 src/connectors/common/config.rs create mode 100644 src/connectors/common/mod.rs create mode 100644 src/connectors/common/processor.rs create mode 100644 src/connectors/common/shutdown.rs create mode 100644 src/connectors/common/types/mod.rs create mode 100644 src/connectors/kafka/config.rs create mode 100644 src/connectors/kafka/consumer.rs create mode 100644 src/connectors/kafka/metrics.rs create mode 100644 src/connectors/kafka/mod.rs create mode 100644 src/connectors/kafka/partition_stream_queue.rs create mode 100644 src/connectors/kafka/processor.rs create mode 100644 src/connectors/kafka/rebalance_listener.rs create mode 100644 src/connectors/kafka/sink.rs create mode 100644 src/connectors/kafka/state.rs create mode 100644 src/connectors/mod.rs diff --git a/Cargo.lock b/Cargo.lock index 1b98329a3..15ca371a8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -892,6 +892,56 @@ dependencies = [ "generic-array", ] +[[package]] +name = "bollard" +version = "0.17.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d41711ad46fda47cd701f6908e59d1bd6b9a2b7464c0d0aeab95c6d37096ff8a" +dependencies = [ + "base64 0.22.0", + "bollard-stubs", + "bytes", + "futures-core", + "futures-util", + "hex", + "home", + "http 1.1.0", + "http-body-util", + "hyper 1.4.1", + "hyper-named-pipe", + "hyper-rustls 0.27.3", + "hyper-util", + "hyperlocal", + "log", + "pin-project-lite", + "rustls 0.23.13", + "rustls-native-certs 0.7.3", + "rustls-pemfile 2.1.2", + "rustls-pki-types", + "serde", + "serde_derive", + "serde_json", + "serde_repr", + "serde_urlencoded", + "thiserror", + "tokio", + "tokio-util", + "tower-service", + "url", + "winapi", +] + +[[package]] +name = "bollard-stubs" +version = "1.45.0-rc.26.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d7c5415e3a6bc6d3e99eff6268e488fd4ee25e7b28c10f08fa6760bd9de16e4" +dependencies = [ + "serde", + "serde_repr", + "serde_with", +] + [[package]] name = "brotli" version = "6.0.0" @@ -1124,6 +1174,15 @@ dependencies = [ "chrono", ] +[[package]] +name = "cmake" +version = "0.1.52" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c682c223677e0e5b6b7f63a64b9351844c3f1b1678a68b7ee617e30fb082620e" +dependencies = [ + "cc", +] + [[package]] name = "colorchoice" version = "1.0.0" @@ -1766,6 +1825,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b42b6fa04a440b495c8b04d0e71b707c585f83cb9cb28cf8cd0d976c315e31b4" dependencies = [ "powerfmt", + "serde", ] [[package]] @@ -1814,11 +1874,34 @@ dependencies = [ "syn 2.0.79", ] +[[package]] +name = "docker_credential" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "31951f49556e34d90ed28342e1df7e1cb7a229c4cab0aecc627b5d91edd41d07" +dependencies = [ + "base64 0.21.7", + "serde", + "serde_json", +] + +[[package]] +name = "duct" +version = "0.13.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4ab5718d1224b63252cd0c6f74f6480f9ffeb117438a2e0f5cf6d9a4798929c" +dependencies = [ + "libc", + "once_cell", + "os_pipe", + "shared_child", +] + [[package]] name = "either" -version = "1.11.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47c1c47d2f5964e29c61246e81db715514cd532db6b5116a25ea3c03d6780a2" +checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" [[package]] name = "encoding_rs" @@ -1845,6 +1928,17 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "etcetera" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "136d1b5283a1ab77bd9257427ffd09d8667ced0570b6f938942bc7568ed5b943" +dependencies = [ + "cfg-if", + "home", + "windows-sys 0.48.0", +] + [[package]] name = "fastrand" version = "1.9.0" @@ -1860,6 +1954,18 @@ version = "2.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "658bd65b1cf4c852a3cc96f18a8ce7b5640f6b703f905c7d74532294c2a63984" +[[package]] +name = "filetime" +version = "0.2.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35c0522e981e68cbfa8c3f978441a5f34b30b96e146b33cd3359176b50fe8586" +dependencies = [ + "cfg-if", + "libc", + "libredox", + "windows-sys 0.59.0", +] + [[package]] name = "fixedbitset" version = "0.4.2" @@ -1924,9 +2030,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" dependencies = [ "futures-core", "futures-sink", @@ -1934,9 +2040,9 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" +checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" [[package]] name = "futures-executor" @@ -1951,9 +2057,9 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" +checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" [[package]] name = "futures-lite" @@ -1972,9 +2078,9 @@ dependencies = [ [[package]] name = "futures-macro" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", @@ -1983,15 +2089,15 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" +checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" [[package]] name = "futures-task" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" +checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" [[package]] name = "futures-timer" @@ -2001,9 +2107,9 @@ checksum = "f288b0a4f20f9a56b5d1da57e2227c661b7b16168e2f72365f57b63326e29b24" [[package]] name = "futures-util" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" dependencies = [ "futures-channel", "futures-core", @@ -2160,6 +2266,15 @@ version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" +[[package]] +name = "home" +version = "0.5.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +dependencies = [ + "windows-sys 0.52.0", +] + [[package]] name = "hostname" version = "0.4.0" @@ -2315,6 +2430,21 @@ dependencies = [ "want", ] +[[package]] +name = "hyper-named-pipe" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73b7d8abf35697b81a825e386fc151e0d503e8cb5fcb93cc8669c376dfd6f278" +dependencies = [ + "hex", + "hyper 1.4.1", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", + "winapi", +] + [[package]] name = "hyper-rustls" version = "0.24.2" @@ -2340,7 +2470,7 @@ dependencies = [ "hyper 1.4.1", "hyper-util", "rustls 0.23.13", - "rustls-native-certs", + "rustls-native-certs 0.8.0", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", @@ -2381,6 +2511,21 @@ dependencies = [ "tracing", ] +[[package]] +name = "hyperlocal" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "986c5ce3b994526b3cd75578e62554abd09f0899d6206de48b3e96ab34ccc8c7" +dependencies = [ + "hex", + "http-body-util", + "hyper 1.4.1", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", +] + [[package]] name = "iana-time-zone" version = "0.1.60" @@ -2434,6 +2579,7 @@ checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" dependencies = [ "autocfg", "hashbrown 0.12.3", + "serde", ] [[package]] @@ -2444,6 +2590,7 @@ checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" dependencies = [ "equivalent", "hashbrown 0.14.5", + "serde", ] [[package]] @@ -2611,6 +2758,29 @@ version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" +[[package]] +name = "libredox" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" +dependencies = [ + "bitflags 2.5.0", + "libc", + "redox_syscall 0.5.7", +] + +[[package]] +name = "libz-sys" +version = "1.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2d16453e800a8cf6dd2fc3eb4bc99b786a9b90c663b8559a5b1a041bf89e472" +dependencies = [ + "cc", + "libc", + "pkg-config", + "vcpkg", +] + [[package]] name = "linux-raw-sys" version = "0.1.4" @@ -2920,6 +3090,27 @@ dependencies = [ "libc", ] +[[package]] +name = "num_enum" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e613fc340b2220f734a8595782c551f1250e969d87d3be1ae0579e8d4065179" +dependencies = [ + "num_enum_derive", +] + +[[package]] +name = "num_enum_derive" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af1844ef2428cc3e1cb900be36181049ef3d3193c63e43026cfe202983b27a56" +dependencies = [ + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "num_threads" version = "0.1.7" @@ -2999,6 +3190,18 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" +[[package]] +name = "openssl-sys" +version = "0.9.104" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "45abf306cbf99debc8195b66b7346498d7b10c210de50418b5ccd7ceba08c741" +dependencies = [ + "cc", + "libc", + "pkg-config", + "vcpkg", +] + [[package]] name = "ordered-float" version = "2.10.1" @@ -3008,6 +3211,16 @@ dependencies = [ "num-traits", ] +[[package]] +name = "os_pipe" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ffd2b0a5634335b135d5728d84c5e0fd726954b87111f7506a61c502280d982" +dependencies = [ + "libc", + "windows-sys 0.59.0", +] + [[package]] name = "overload" version = "0.1.1" @@ -3038,7 +3251,7 @@ checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" dependencies = [ "cfg-if", "libc", - "redox_syscall", + "redox_syscall 0.4.1", "smallvec", "windows-targets 0.48.5", ] @@ -3079,6 +3292,31 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "parse-display" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "914a1c2265c98e2446911282c6ac86d8524f495792c38c5bd884f80499c7538a" +dependencies = [ + "parse-display-derive", + "regex", + "regex-syntax 0.8.5", +] + +[[package]] +name = "parse-display-derive" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ae7800a4c974efd12df917266338e79a7a74415173caf7e70aa0a0707345281" +dependencies = [ + "proc-macro2", + "quote", + "regex", + "regex-syntax 0.8.5", + "structmeta", + "syn 2.0.79", +] + [[package]] name = "parse-zoneinfo" version = "0.3.0" @@ -3147,6 +3385,7 @@ dependencies = [ "prost", "prost-build", "rand", + "rdkafka", "regex", "relative-path", "reqwest 0.11.27", @@ -3161,10 +3400,13 @@ dependencies = [ "sha2", "static-files", "sysinfo", + "testcontainers", + "testcontainers-modules", "thiserror", "thread-priority", "tokio", "tokio-stream", + "tokio-util", "tonic", "tonic-web", "tower-http 0.6.1", @@ -3617,6 +3859,49 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "rdkafka" +version = "0.37.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "14b52c81ac3cac39c9639b95c20452076e74b8d9a71bc6fc4d83407af2ea6fff" +dependencies = [ + "futures-channel", + "futures-util", + "libc", + "log", + "rdkafka-sys", + "serde", + "serde_derive", + "serde_json", + "slab", + "tokio", + "tracing", +] + +[[package]] +name = "rdkafka-sys" +version = "4.8.0+2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ced38182dc436b3d9df0c77976f37a67134df26b050df1f0006688e46fc4c8be" +dependencies = [ + "cmake", + "libc", + "libz-sys", + "num_enum", + "openssl-sys", + "pkg-config", + "sasl2-sys", +] + +[[package]] +name = "redox_syscall" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" +dependencies = [ + "bitflags 1.3.2", +] + [[package]] name = "redox_syscall" version = "0.4.1" @@ -3626,6 +3911,15 @@ dependencies = [ "bitflags 1.3.2", ] +[[package]] +name = "redox_syscall" +version = "0.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" +dependencies = [ + "bitflags 2.5.0", +] + [[package]] name = "regex" version = "1.11.0" @@ -3752,7 +4046,7 @@ dependencies = [ "pin-project-lite", "quinn", "rustls 0.23.13", - "rustls-native-certs", + "rustls-native-certs 0.8.0", "rustls-pemfile 2.1.2", "rustls-pki-types", "serde", @@ -3906,6 +4200,19 @@ dependencies = [ "zeroize", ] +[[package]] +name = "rustls-native-certs" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" +dependencies = [ + "openssl-probe", + "rustls-pemfile 2.1.2", + "rustls-pki-types", + "schannel", + "security-framework", +] + [[package]] name = "rustls-native-certs" version = "0.8.0" @@ -3986,6 +4293,18 @@ dependencies = [ "winapi-util", ] +[[package]] +name = "sasl2-sys" +version = "0.1.22+2.1.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f2a7f7efd9fc98b3a9033272df10709f5ee3fa0eabbd61a527a3a1ed6bd3c6" +dependencies = [ + "cc", + "duct", + "libc", + "pkg-config", +] + [[package]] name = "schannel" version = "0.1.23" @@ -4113,6 +4432,36 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_with" +version = "3.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ad483d2ab0149d5a5ebcd9972a3852711e0153d863bf5a5d0391d28883c4a20" +dependencies = [ + "base64 0.22.0", + "chrono", + "hex", + "indexmap 1.9.3", + "indexmap 2.5.0", + "serde", + "serde_derive", + "serde_json", + "serde_with_macros", + "time", +] + +[[package]] +name = "serde_with_macros" +version = "3.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "65569b702f41443e8bc8bbb1c5779bd0450bbe723b56198980e80ec45780bce2" +dependencies = [ + "darling", + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "sha1" version = "0.10.6" @@ -4150,6 +4499,16 @@ dependencies = [ "lazy_static", ] +[[package]] +name = "shared_child" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09fa9338aed9a1df411814a5b2252f7cd206c55ae9bf2fa763f8de84603aa60c" +dependencies = [ + "libc", + "windows-sys 0.59.0", +] + [[package]] name = "shlex" version = "1.3.0" @@ -4300,6 +4659,29 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" +[[package]] +name = "structmeta" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e1575d8d40908d70f6fd05537266b90ae71b15dbbe7a8b7dffa2b759306d329" +dependencies = [ + "proc-macro2", + "quote", + "structmeta-derive", + "syn 2.0.79", +] + +[[package]] +name = "structmeta-derive" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "strum" version = "0.26.2" @@ -4411,6 +4793,44 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "testcontainers" +version = "0.23.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f40cc2bd72e17f328faf8ca7687fe337e61bccd8acf9674fa78dd3792b045e1" +dependencies = [ + "async-trait", + "bollard", + "bollard-stubs", + "bytes", + "docker_credential", + "either", + "etcetera", + "futures", + "log", + "memchr", + "parse-display", + "pin-project-lite", + "serde", + "serde_json", + "serde_with", + "thiserror", + "tokio", + "tokio-stream", + "tokio-tar", + "tokio-util", + "url", +] + +[[package]] +name = "testcontainers-modules" +version = "0.11.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "064a2677e164cad39ef3c1abddb044d5a25c49d27005804563d8c4227aac8bd0" +dependencies = [ + "testcontainers", +] + [[package]] name = "thiserror" version = "1.0.64" @@ -4525,28 +4945,27 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.37.0" +version = "1.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1adbebffeca75fcfd058afa480fb6c0b81e165a0323f9c9d39c9697e37c46787" +checksum = "5cec9b21b0450273377fc97bd4c33a8acffc8c996c987a7c5b319a0083707551" dependencies = [ "backtrace", "bytes", "libc", - "mio 0.8.11", - "num_cpus", + "mio 1.0.2", "parking_lot", "pin-project-lite", "signal-hook-registry", "socket2", "tokio-macros", - "windows-sys 0.48.0", + "windows-sys 0.52.0", ] [[package]] name = "tokio-macros" -version = "2.2.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" +checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", @@ -4596,6 +5015,21 @@ dependencies = [ "tokio", ] +[[package]] +name = "tokio-tar" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d5714c010ca3e5c27114c1cdeb9d14641ace49874aa5626d7149e47aedace75" +dependencies = [ + "filetime", + "futures-core", + "libc", + "redox_syscall 0.3.5", + "tokio", + "tokio-stream", + "xattr", +] + [[package]] name = "tokio-util" version = "0.7.10" @@ -4817,6 +5251,7 @@ dependencies = [ "sharded-slab", "smallvec", "thread_local", + "time", "tracing", "tracing-core", "tracing-log", @@ -4952,9 +5387,9 @@ checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "uuid" -version = "1.8.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a183cf7feeba97b4dd1c0d46788634f6221d87fa961b305bed08c851829efcc0" +checksum = "f8c5f0a0af699448548ad1a2fbf920fb4bee257eae39953ba95cb84891a0446a" dependencies = [ "getrandom", ] @@ -4995,6 +5430,12 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" +[[package]] +name = "vcpkg" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" + [[package]] name = "vergen" version = "8.3.1" @@ -5326,6 +5767,15 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", +] + [[package]] name = "windows-targets" version = "0.42.2" @@ -5523,6 +5973,17 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "xattr" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8da84f1a25939b27f6820d92aed108f83ff920fdf11a7b19366c27c4cda81d4f" +dependencies = [ + "libc", + "linux-raw-sys 0.4.13", + "rustix 0.38.34", +] + [[package]] name = "xxhash-rust" version = "0.8.10" diff --git a/Cargo.toml b/Cargo.toml index 455321afe..92d6fa2de 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -30,6 +30,11 @@ actix-web-prometheus = { version = "0.1" } actix-web-static-files = "4.0" mime = "0.3.17" +### connectors dependencies +rdkafka = { version = "0.37", features = ["cmake-build", "tracing", "sasl", "ssl", "libz-static"] } +testcontainers = "0.23" +testcontainers-modules = { version = "0.11", features = ["kafka"] } + ### other dependencies anyhow = { version = "1.0", features = ["backtrace"] } argon2 = "0.5.0" @@ -81,13 +86,15 @@ static-files = "0.2" sysinfo = "0.31.4" thiserror = "1.0.64" thread-priority = "1.0.0" -tokio = { version = "1.28", default-features = false, features = [ +tokio = { version = "1.42", default-features = false, features = [ "sync", "macros", "fs", + "rt-multi-thread" ] } tokio-stream = { version = "0.1", features = ["fs"] } -tracing-subscriber = { version = "0.3.18", features = ["env-filter"] } +tokio-util = "0.7" +tracing-subscriber = { version = "0.3.18", features = ["env-filter", "time"] } ulid = { version = "1.0", features = ["serde"] } uptime_lib = "0.3.0" xxhash-rust = { version = "0.8", features = ["xxh3"] } diff --git a/scripts/kafka_log_stream_generator.py b/scripts/kafka_log_stream_generator.py new file mode 100644 index 000000000..93eed25b3 --- /dev/null +++ b/scripts/kafka_log_stream_generator.py @@ -0,0 +1,123 @@ +import json +import time +from datetime import datetime, timezone +from random import choice, randint +from uuid import uuid4 +from confluent_kafka import Producer + +# Configuration +config = { + "kafka_broker": "localhost:9092", # Replace with your Kafka broker address + "kafka_topic": "log-stream", # Replace with your Kafka topic name + "log_rate": 500, # Logs per second + "log_template": { + "timestamp": "", # Timestamp will be added dynamically + "correlation_id": "", # Unique identifier for tracing requests + "level": "INFO", # Log level (e.g., INFO, ERROR, DEBUG) + "message": "", # Main log message to be dynamically set + "pod": { + "name": "example-pod", # Kubernetes pod name + "namespace": "default", # Kubernetes namespace + "node": "node-01" # Kubernetes node name + }, + "request": { + "method": "", # HTTP method + "path": "", # HTTP request path + "remote_address": "" # IP address of the client + }, + "response": { + "status_code": 200, # HTTP response status code + "latency_ms": 0 # Latency in milliseconds + }, + "metadata": { + "container_id": "", # Container ID + "image": "example/image:1.0", # Docker image + "environment": "prod" # Environment (e.g., dev, staging, prod) + } + } +} + +producer = Producer({"bootstrap.servers": config["kafka_broker"]}) + + +def delivery_report(err, msg): + if err is not None: + print(f"Delivery failed for message {msg.key()}: {err}") + else: + print(f"Message delivered to {msg.topic()} [{msg.partition()}]") + + +def generate_log(): + log = config["log_template"].copy() + log["timestamp"] = datetime.now(timezone.utc).isoformat() + log["correlation_id"] = str(uuid4()) + + levels = ["INFO", "WARNING", "ERROR", "DEBUG"] + messages = [ + "Received incoming HTTP request", + "Processed request successfully", + "Failed to process request", + "Request timeout encountered", + "Service unavailable" + ] + log["level"] = choice(levels) + log["message"] = choice(messages) + + # Populate request fields + methods = ["GET", "POST", "PUT", "DELETE"] + paths = ["/api/resource", "/api/login", "/api/logout", "/api/data"] + log["request"] = { + "method": choice(methods), + "path": choice(paths), + "remote_address": f"192.168.1.{randint(1, 255)}" + } + + # Populate response fields + log["response"] = { + "status_code": choice([200, 201, 400, 401, 403, 404, 500]), + "latency_ms": randint(10, 1000) + } + + # Populate pod and metadata fields + log["pod"] = { + "name": f"pod-{randint(1, 100)}", + "namespace": choice(["default", "kube-system", "production", "staging"]), + "node": f"node-{randint(1, 10)}" + } + + log["metadata"] = { + "container_id": f"container-{randint(1000, 9999)}", + "image": f"example/image:{randint(1, 5)}.0", + "environment": choice(["dev", "staging", "prod"]) + } + + return log + + +def main(): + try: + while True: + # Generate log message + log_message = generate_log() + log_json = json.dumps(log_message) + + # Send to Kafka + producer.produce( + config["kafka_topic"], + value=log_json, + callback=delivery_report + ) + + # Flush the producer to ensure delivery + producer.flush() + + # Wait based on the log rate + time.sleep(1 / config["log_rate"]) + except KeyboardInterrupt: + print("Stopped log generation.") + finally: + producer.flush() + + +if __name__ == "__main__": + main() diff --git a/src/cli.rs b/src/cli.rs index 982a2a765..9cb1ae840 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -18,9 +18,14 @@ use clap::{value_parser, Arg, ArgGroup, Command, FromArgMatches}; use std::path::PathBuf; - +use std::time::Duration; +use tracing::warn; use url::Url; +use crate::connectors::common::config::ConnectorConfig; +use crate::connectors::common::types::ConnectorType; +use crate::connectors::common::BadData; +use crate::connectors::kafka::config::{ConsumerConfig, KafkaConfig, SourceOffset}; use crate::{ oidc::{self, OpenidConfig}, option::{validation, Compression, Mode}, @@ -119,6 +124,8 @@ pub struct Cli { pub trino_auth: Option, pub trino_schema: Option, pub trino_catalog: Option, + //Connectors config + pub connector_config: Option, } impl Cli { @@ -164,6 +171,46 @@ impl Cli { pub const TRINO_AUTHORIZATION: &'static str = "p-trino-authorization"; pub const TRINO_SCHEMA: &'static str = "p-trino-schema"; + // ConnectorConfig arguments + pub const CONNECTOR_NAME: &'static str = "connector-name"; + pub const CONNECTOR_BUFFER_SIZE: &'static str = "connector-buffer-size"; + pub const CONNECTOR_BUFFER_TIMEOUT: &'static str = "connector-buffer-timeout"; + pub const CONNECTOR_OFFSET_MODE: &'static str = "connector-offset-mode"; // earliest, latest, group + pub const CONNECTOR_BAD_DATA_POLICY: &'static str = "connector-bad-data-policy"; // e.g. "drop", "fail" , "dlt" + pub const CONNECTOR_MAX_RETRIES: &'static str = "connector-max-retries"; + pub const CONNECTOR_RETRY_INTERVAL_MS: &'static str = "connector-retry-interval-ms"; + pub const CONNECTOR_METRICS_ENABLED: &'static str = "connector-metrics-enabled"; + pub const CONNECTOR_INSTANCE_ID: &'static str = "connector-instance-id"; + + // ConsumerConfig arguments + pub const CONSUMER_GROUP_INSTANCE_ID: &'static str = "consumer-group-instance-id"; + pub const CONSUMER_PARTITION_ASSIGNMENT_STRATEGY: &'static str = + "consumer-partition-assignment-strategy"; + pub const CONSUMER_SESSION_TIMEOUT_MS: &'static str = "consumer-session-timeout-ms"; + pub const CONSUMER_HEARTBEAT_INTERVAL_MS: &'static str = "consumer-heartbeat-interval-ms"; + pub const CONSUMER_MAX_POLL_INTERVAL_MS: &'static str = "consumer-max-poll-interval-ms"; + pub const CONSUMER_ENABLE_AUTO_COMMIT: &'static str = "consumer-enable-auto-commit"; + pub const CONSUMER_AUTO_COMMIT_INTERVAL_MS: &'static str = "consumer-auto-commit-interval-ms"; + pub const CONSUMER_ENABLE_AUTO_OFFSET_STORE: &'static str = "consumer-enable-auto-offset-store"; + pub const CONSUMER_AUTO_OFFSET_RESET: &'static str = "consumer-auto-offset-reset"; + pub const CONSUMER_FETCH_MIN_BYTES: &'static str = "consumer-fetch-min-bytes"; + pub const CONSUMER_FETCH_MAX_BYTES: &'static str = "consumer-fetch-max-bytes"; + pub const CONSUMER_FETCH_MAX_WAIT_MS: &'static str = "consumer-fetch-max-wait-ms"; + pub const CONSUMER_MAX_PARTITION_FETCH_BYTES: &'static str = + "consumer-max-partition-fetch-bytes"; + pub const CONSUMER_QUEUED_MIN_MESSAGES: &'static str = "consumer-queued-min-messages"; + pub const CONSUMER_QUEUED_MAX_MESSAGES_KBYTES: &'static str = + "consumer-queued-max-messages-kbytes"; + pub const CONSUMER_ENABLE_PARTITION_EOF: &'static str = "consumer-enable-partition-eof"; + pub const CONSUMER_CHECK_CRCS: &'static str = "consumer-check-crcs"; + pub const CONSUMER_ISOLATION_LEVEL: &'static str = "consumer-isolation-level"; + pub const CONSUMER_FETCH_MESSAGE_MAX_BYTES: &'static str = "consumer-fetch-message-max-bytes"; + pub const CONSUMER_STATS_INTERVAL_MS: &'static str = "consumer-stats-interval-ms"; + + pub const KAFKA_TOPICS: &'static str = "kafka-topics"; + pub const KAFKA_BOOTSTRAP_SERVERS: &'static str = "kafka-bootstrap-servers"; + pub const KAFKA_GROUP_ID: &'static str = "kafka-group-id"; + pub fn local_stream_data_path(&self, stream_name: &str) -> PathBuf { self.local_staging_path.join(stream_name) } @@ -177,57 +224,57 @@ impl Cli { pub fn create_cli_command_with_clap(name: &'static str) -> Command { Command::new(name).next_line_help(false) - .arg( - Arg::new(Self::TRINO_ENDPOINT) - .long(Self::TRINO_ENDPOINT) - .env("P_TRINO_ENDPOINT") - .value_name("STRING") - .help("Address and port for Trino HTTP(s) server"), - ) - .arg( - Arg::new(Self::TRINO_CATALOG_NAME) - .long(Self::TRINO_CATALOG_NAME) - .env("P_TRINO_CATALOG_NAME") - .value_name("STRING") - .help("Name of the catalog to be queried (Translates to X-Trino-Catalog)"), - ) - .arg( - Arg::new(Self::TRINO_SCHEMA) - .long(Self::TRINO_SCHEMA) - .env("P_TRINO_SCHEMA") - .value_name("STRING") - .help("Name of schema to be queried (Translates to X-Trino-Schema)"), - ) - .arg( - Arg::new(Self::TRINO_USER_NAME) - .long(Self::TRINO_USER_NAME) - .env("P_TRINO_USER_NAME") - .value_name("STRING") - .help("Name of Trino user (Translates to X-Trino-User)"), - ) - .arg( - Arg::new(Self::TRINO_AUTHORIZATION) - .long(Self::TRINO_AUTHORIZATION) - .env("P_TRINO_AUTHORIZATION") - .value_name("STRING") - .help("Base 64 encoded in the format username:password"), - ) - .arg( - Arg::new(Self::TLS_CERT) - .long(Self::TLS_CERT) - .env("P_TLS_CERT_PATH") - .value_name("PATH") - .value_parser(validation::file_path) - .help("Local path on this device where certificate file is located. Required to enable TLS"), - ) - .arg( - Arg::new(Self::TLS_KEY) - .long(Self::TLS_KEY) - .env("P_TLS_KEY_PATH") - .value_name("PATH") - .value_parser(validation::file_path) - .help("Local path on this device where private key file is located. Required to enable TLS"), - ) + .arg( + Arg::new(Self::TRINO_ENDPOINT) + .long(Self::TRINO_ENDPOINT) + .env("P_TRINO_ENDPOINT") + .value_name("STRING") + .help("Address and port for Trino HTTP(s) server"), + ) + .arg( + Arg::new(Self::TRINO_CATALOG_NAME) + .long(Self::TRINO_CATALOG_NAME) + .env("P_TRINO_CATALOG_NAME") + .value_name("STRING") + .help("Name of the catalog to be queried (Translates to X-Trino-Catalog)"), + ) + .arg( + Arg::new(Self::TRINO_SCHEMA) + .long(Self::TRINO_SCHEMA) + .env("P_TRINO_SCHEMA") + .value_name("STRING") + .help("Name of schema to be queried (Translates to X-Trino-Schema)"), + ) + .arg( + Arg::new(Self::TRINO_USER_NAME) + .long(Self::TRINO_USER_NAME) + .env("P_TRINO_USER_NAME") + .value_name("STRING") + .help("Name of Trino user (Translates to X-Trino-User)"), + ) + .arg( + Arg::new(Self::TRINO_AUTHORIZATION) + .long(Self::TRINO_AUTHORIZATION) + .env("P_TRINO_AUTHORIZATION") + .value_name("STRING") + .help("Base 64 encoded in the format username:password"), + ) + .arg( + Arg::new(Self::TLS_CERT) + .long(Self::TLS_CERT) + .env("P_TLS_CERT_PATH") + .value_name("PATH") + .value_parser(validation::file_path) + .help("Local path on this device where certificate file is located. Required to enable TLS"), + ) + .arg( + Arg::new(Self::TLS_KEY) + .long(Self::TLS_KEY) + .env("P_TLS_KEY_PATH") + .value_name("PATH") + .value_parser(validation::file_path) + .help("Local path on this device where private key file is located. Required to enable TLS"), + ) .arg( Arg::new(Self::TRUSTED_CA_CERTS_PATH) .long(Self::TRUSTED_CA_CERTS_PATH) @@ -236,273 +283,492 @@ impl Cli { .value_parser(validation::canonicalize_path) .help("Local path on this device where all trusted certificates are located.") ) - .arg( - Arg::new(Self::ADDRESS) - .long(Self::ADDRESS) - .env("P_ADDR") - .value_name("ADDR:PORT") - .default_value("0.0.0.0:8000") - .value_parser(validation::socket_addr) - .help("Address and port for Parseable HTTP(s) server"), - ) - .arg( - Arg::new(Self::STAGING) - .long(Self::STAGING) - .env("P_STAGING_DIR") - .value_name("DIR") - .default_value("./staging") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used as landing point for incoming events") - .next_line_help(true), - ) - .arg( - Arg::new(Self::CACHE) - .long(Self::CACHE) - .env("P_CACHE_DIR") - .value_name("DIR") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used for caching data") - .next_line_help(true), - ) - .arg( - Arg::new(Self::CACHE_SIZE) - .long(Self::CACHE_SIZE) - .env("P_CACHE_SIZE") - .value_name("size") - .default_value("1GiB") - .value_parser(validation::cache_size) - .help("Maximum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") - .next_line_help(true), - ) - .arg( - Arg::new(Self::QUERY_CACHE) - .long(Self::QUERY_CACHE) - .env("P_QUERY_CACHE_DIR") - .value_name("DIR") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used for caching data") - .next_line_help(true), - ) - .arg( - Arg::new(Self::QUERY_CACHE_SIZE) - .long(Self::QUERY_CACHE_SIZE) - .env("P_QUERY_CACHE_SIZE") - .value_name("size") - .default_value("1GiB") - .value_parser(validation::cache_size) - .help("Maximum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") - .next_line_help(true), - ) - .arg( - Arg::new(Self::USERNAME) - .long(Self::USERNAME) - .env("P_USERNAME") - .value_name("STRING") - .required(true) - .help("Admin username to be set for this Parseable server"), - ) - .arg( - Arg::new(Self::PASSWORD) - .long(Self::PASSWORD) - .env("P_PASSWORD") - .value_name("STRING") - .required(true) - .help("Admin password to be set for this Parseable server"), - ) - .arg( - Arg::new(Self::CHECK_UPDATE) - .long(Self::CHECK_UPDATE) - .env("P_CHECK_UPDATE") - .value_name("BOOL") - .required(false) - .default_value("true") - .value_parser(value_parser!(bool)) - .help("Enable/Disable checking for new Parseable release"), - ) - .arg( - Arg::new(Self::SEND_ANALYTICS) - .long(Self::SEND_ANALYTICS) - .env("P_SEND_ANONYMOUS_USAGE_DATA") - .value_name("BOOL") - .required(false) - .default_value("true") - .value_parser(value_parser!(bool)) - .help("Enable/Disable anonymous telemetry data collection"), - ) - .arg( - Arg::new(Self::OPEN_AI_KEY) - .long(Self::OPEN_AI_KEY) - .env("P_OPENAI_API_KEY") - .value_name("STRING") - .required(false) - .help("OpenAI key to enable llm features"), - ) - .arg( - Arg::new(Self::OPENID_CLIENT_ID) - .long(Self::OPENID_CLIENT_ID) - .env("P_OIDC_CLIENT_ID") - .value_name("STRING") - .required(false) - .help("Client id for OIDC provider"), - ) - .arg( - Arg::new(Self::OPENID_CLIENT_SECRET) - .long(Self::OPENID_CLIENT_SECRET) - .env("P_OIDC_CLIENT_SECRET") - .value_name("STRING") - .required(false) - .help("Client secret for OIDC provider"), - ) - .arg( - Arg::new(Self::OPENID_ISSUER) - .long(Self::OPENID_ISSUER) - .env("P_OIDC_ISSUER") - .value_name("URL") - .required(false) - .value_parser(validation::url) - .help("OIDC provider's host address"), - ) - .arg( - Arg::new(Self::DOMAIN_URI) - .long(Self::DOMAIN_URI) - .env("P_ORIGIN_URI") - .value_name("URL") - .required(false) - .value_parser(validation::url) - .help("Parseable server global domain address"), - ) - .arg( - Arg::new(Self::GRPC_PORT) - .long(Self::GRPC_PORT) - .env("P_GRPC_PORT") - .value_name("PORT") - .default_value("8001") - .required(false) - .value_parser(value_parser!(u16)) - .help("Port for gRPC server"), - ) - .arg( - Arg::new(Self::FLIGHT_PORT) - .long(Self::FLIGHT_PORT) - .env("P_FLIGHT_PORT") - .value_name("PORT") - .default_value("8002") - .required(false) - .value_parser(value_parser!(u16)) - .help("Port for Arrow Flight Querying Engine"), - ) - .arg( - Arg::new(Self::CORS) - .long(Self::CORS) - .env("P_CORS") - .value_name("BOOL") - .required(false) - .default_value("true") - .value_parser(value_parser!(bool)) - .help("Enable/Disable CORS, default disabled"), - ) - .arg( - Arg::new(Self::LIVETAIL_CAPACITY) - .long(Self::LIVETAIL_CAPACITY) - .env("P_LIVETAIL_CAPACITY") - .value_name("NUMBER") - .default_value("1000") - .required(false) - .value_parser(value_parser!(usize)) - .help("Number of rows in livetail channel"), - ) - .arg( - Arg::new(Self::QUERY_MEM_POOL_SIZE) - .long(Self::QUERY_MEM_POOL_SIZE) - .env("P_QUERY_MEMORY_LIMIT") - .value_name("Gib") - .required(false) - .value_parser(value_parser!(u8)) - .help("Set a fixed memory limit for query"), - ) - .arg( - // RowGroupSize controls the number of rows present in one row group - // More rows = better compression but HIGHER Memory consumption during read/write - // 1048576 is the default value for DataFusion - Arg::new(Self::ROW_GROUP_SIZE) - .long(Self::ROW_GROUP_SIZE) - .env("P_PARQUET_ROW_GROUP_SIZE") - .value_name("NUMBER") - .required(false) - .default_value("1048576") - .value_parser(value_parser!(usize)) - .help("Number of rows in a row group"), - ).arg( - Arg::new(Self::MODE) - .long(Self::MODE) - .env("P_MODE") - .value_name("STRING") - .required(false) - .default_value("all") - .value_parser([ - "query", - "ingest", - "all"]) - .help("Mode of operation"), - ) - .arg( - Arg::new(Self::INGESTOR_ENDPOINT) - .long(Self::INGESTOR_ENDPOINT) - .env("P_INGESTOR_ENDPOINT") - .value_name("URL") - .required(false) - .help("URL to connect to this specific ingestor. Default is the address of the server.") - ) - .arg( - Arg::new(Self::PARQUET_COMPRESSION_ALGO) - .long(Self::PARQUET_COMPRESSION_ALGO) - .env("P_PARQUET_COMPRESSION_ALGO") - .value_name("[UNCOMPRESSED, SNAPPY, GZIP, LZO, BROTLI, LZ4, ZSTD]") - .required(false) - .default_value("lz4") - .value_parser([ - "uncompressed", - "snappy", - "gzip", - "lzo", - "brotli", - "lz4", - "zstd"]) - .help("Parquet compression algorithm"), - ) - .arg( - Arg::new(Self::HOT_TIER_PATH) - .long(Self::HOT_TIER_PATH) - .env("P_HOT_TIER_DIR") - .value_name("DIR") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used for hot tier data") - .next_line_help(true), - ) - .arg( - Arg::new(Self::MAX_DISK_USAGE) - .long(Self::MAX_DISK_USAGE) - .env("P_MAX_DISK_USAGE_PERCENT") - .value_name("percentage") - .default_value("80.0") - .value_parser(validation::validate_disk_usage) - .help("Maximum allowed disk usage in percentage e.g 90.0 for 90%") - .next_line_help(true), - ) - .arg( - Arg::new(Self::MS_CLARITY_TAG) - .long(Self::MS_CLARITY_TAG) - .env("P_MS_CLARITY_TAG") - .value_name("STRING") - .required(false) - .help("Tag for MS Clarity"), - ) - .group( - ArgGroup::new("oidc") - .args([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) - .requires_all([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) - .multiple(true) - ) + .arg( + Arg::new(Self::ADDRESS) + .long(Self::ADDRESS) + .env("P_ADDR") + .value_name("ADDR:PORT") + .default_value("0.0.0.0:8000") + .value_parser(validation::socket_addr) + .help("Address and port for Parseable HTTP(s) server"), + ) + .arg( + Arg::new(Self::STAGING) + .long(Self::STAGING) + .env("P_STAGING_DIR") + .value_name("DIR") + .default_value("./staging") + .value_parser(validation::canonicalize_path) + .help("Local path on this device to be used as landing point for incoming events") + .next_line_help(true), + ) + .arg( + Arg::new(Self::CACHE) + .long(Self::CACHE) + .env("P_CACHE_DIR") + .value_name("DIR") + .value_parser(validation::canonicalize_path) + .help("Local path on this device to be used for caching data") + .next_line_help(true), + ) + .arg( + Arg::new(Self::CACHE_SIZE) + .long(Self::CACHE_SIZE) + .env("P_CACHE_SIZE") + .value_name("size") + .default_value("1GiB") + .value_parser(validation::cache_size) + .help("Maximum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") + .next_line_help(true), + ) + .arg( + Arg::new(Self::QUERY_CACHE) + .long(Self::QUERY_CACHE) + .env("P_QUERY_CACHE_DIR") + .value_name("DIR") + .value_parser(validation::canonicalize_path) + .help("Local path on this device to be used for caching data") + .next_line_help(true), + ) + .arg( + Arg::new(Self::QUERY_CACHE_SIZE) + .long(Self::QUERY_CACHE_SIZE) + .env("P_QUERY_CACHE_SIZE") + .value_name("size") + .default_value("1GiB") + .value_parser(validation::cache_size) + .help("Maximum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") + .next_line_help(true), + ) + .arg( + Arg::new(Self::USERNAME) + .long(Self::USERNAME) + .env("P_USERNAME") + .value_name("STRING") + .required(true) + .help("Admin username to be set for this Parseable server"), + ) + .arg( + Arg::new(Self::PASSWORD) + .long(Self::PASSWORD) + .env("P_PASSWORD") + .value_name("STRING") + .required(true) + .help("Admin password to be set for this Parseable server"), + ) + .arg( + Arg::new(Self::CHECK_UPDATE) + .long(Self::CHECK_UPDATE) + .env("P_CHECK_UPDATE") + .value_name("BOOL") + .required(false) + .default_value("true") + .value_parser(value_parser!(bool)) + .help("Enable/Disable checking for new Parseable release"), + ) + .arg( + Arg::new(Self::SEND_ANALYTICS) + .long(Self::SEND_ANALYTICS) + .env("P_SEND_ANONYMOUS_USAGE_DATA") + .value_name("BOOL") + .required(false) + .default_value("true") + .value_parser(value_parser!(bool)) + .help("Enable/Disable anonymous telemetry data collection"), + ) + .arg( + Arg::new(Self::OPEN_AI_KEY) + .long(Self::OPEN_AI_KEY) + .env("P_OPENAI_API_KEY") + .value_name("STRING") + .required(false) + .help("OpenAI key to enable llm features"), + ) + .arg( + Arg::new(Self::OPENID_CLIENT_ID) + .long(Self::OPENID_CLIENT_ID) + .env("P_OIDC_CLIENT_ID") + .value_name("STRING") + .required(false) + .help("Client id for OIDC provider"), + ) + .arg( + Arg::new(Self::OPENID_CLIENT_SECRET) + .long(Self::OPENID_CLIENT_SECRET) + .env("P_OIDC_CLIENT_SECRET") + .value_name("STRING") + .required(false) + .help("Client secret for OIDC provider"), + ) + .arg( + Arg::new(Self::OPENID_ISSUER) + .long(Self::OPENID_ISSUER) + .env("P_OIDC_ISSUER") + .value_name("URL") + .required(false) + .value_parser(validation::url) + .help("OIDC provider's host address"), + ) + .arg( + Arg::new(Self::DOMAIN_URI) + .long(Self::DOMAIN_URI) + .env("P_ORIGIN_URI") + .value_name("URL") + .required(false) + .value_parser(validation::url) + .help("Parseable server global domain address"), + ) + .arg( + Arg::new(Self::GRPC_PORT) + .long(Self::GRPC_PORT) + .env("P_GRPC_PORT") + .value_name("PORT") + .default_value("8001") + .required(false) + .value_parser(value_parser!(u16)) + .help("Port for gRPC server"), + ) + .arg( + Arg::new(Self::FLIGHT_PORT) + .long(Self::FLIGHT_PORT) + .env("P_FLIGHT_PORT") + .value_name("PORT") + .default_value("8002") + .required(false) + .value_parser(value_parser!(u16)) + .help("Port for Arrow Flight Querying Engine"), + ) + .arg( + Arg::new(Self::CORS) + .long(Self::CORS) + .env("P_CORS") + .value_name("BOOL") + .required(false) + .default_value("true") + .value_parser(value_parser!(bool)) + .help("Enable/Disable CORS, default disabled"), + ) + .arg( + Arg::new(Self::LIVETAIL_CAPACITY) + .long(Self::LIVETAIL_CAPACITY) + .env("P_LIVETAIL_CAPACITY") + .value_name("NUMBER") + .default_value("1000") + .required(false) + .value_parser(value_parser!(usize)) + .help("Number of rows in livetail channel"), + ) + .arg( + Arg::new(Self::QUERY_MEM_POOL_SIZE) + .long(Self::QUERY_MEM_POOL_SIZE) + .env("P_QUERY_MEMORY_LIMIT") + .value_name("Gib") + .required(false) + .value_parser(value_parser!(u8)) + .help("Set a fixed memory limit for query"), + ) + .arg( + // RowGroupSize controls the number of rows present in one row group + // More rows = better compression but HIGHER Memory consumption during read/write + // 1048576 is the default value for DataFusion + Arg::new(Self::ROW_GROUP_SIZE) + .long(Self::ROW_GROUP_SIZE) + .env("P_PARQUET_ROW_GROUP_SIZE") + .value_name("NUMBER") + .required(false) + .default_value("1048576") + .value_parser(value_parser!(usize)) + .help("Number of rows in a row group"), + ).arg( + Arg::new(Self::MODE) + .long(Self::MODE) + .env("P_MODE") + .value_name("STRING") + .required(false) + .default_value("all") + .value_parser([ + "query", + "ingest", + "all"]) + .help("Mode of operation"), + ) + .arg( + Arg::new(Self::INGESTOR_ENDPOINT) + .long(Self::INGESTOR_ENDPOINT) + .env("P_INGESTOR_ENDPOINT") + .value_name("URL") + .required(false) + .help("URL to connect to this specific ingestor. Default is the address of the server.") + ) + .arg( + Arg::new(Self::PARQUET_COMPRESSION_ALGO) + .long(Self::PARQUET_COMPRESSION_ALGO) + .env("P_PARQUET_COMPRESSION_ALGO") + .value_name("[UNCOMPRESSED, SNAPPY, GZIP, LZO, BROTLI, LZ4, ZSTD]") + .required(false) + .default_value("lz4") + .value_parser([ + "uncompressed", + "snappy", + "gzip", + "lzo", + "brotli", + "lz4", + "zstd"]) + .help("Parquet compression algorithm"), + ) + .arg( + Arg::new(Self::HOT_TIER_PATH) + .long(Self::HOT_TIER_PATH) + .env("P_HOT_TIER_DIR") + .value_name("DIR") + .value_parser(validation::canonicalize_path) + .help("Local path on this device to be used for hot tier data") + .next_line_help(true), + ) + .arg( + Arg::new(Self::MAX_DISK_USAGE) + .long(Self::MAX_DISK_USAGE) + .env("P_MAX_DISK_USAGE_PERCENT") + .value_name("percentage") + .default_value("80.0") + .value_parser(validation::validate_disk_usage) + .help("Maximum allowed disk usage in percentage e.g 90.0 for 90%") + .next_line_help(true), + ) + .arg( + Arg::new(Self::MS_CLARITY_TAG) + .long(Self::MS_CLARITY_TAG) + .env("P_MS_CLARITY_TAG") + .value_name("STRING") + .required(false) + .help("Tag for MS Clarity"), + ).arg( + Arg::new(Self::CONNECTOR_NAME) + .long(Self::CONNECTOR_NAME) + .env("P_CONNECTOR_NAME") + .required(false) + .help("Name of the connector") + ) + .arg( + Arg::new(Self::CONNECTOR_BUFFER_SIZE) + .long(Self::CONNECTOR_BUFFER_SIZE) + .env("P_CONNECTOR_BATCH_SIZE") + .value_parser(value_parser!(usize)) + .required(false) + .help("Buffer size for processing") + ) + .arg( + Arg::new(Self::CONNECTOR_BUFFER_TIMEOUT) + .long(Self::CONNECTOR_BUFFER_TIMEOUT) + .env("P_CONNECTOR_BUFFER_TIMEOUT") + .value_parser(value_parser!(u64)) + .required(false) + .help("Buffer timeout for processing") + ) + .arg( + Arg::new(Self::CONNECTOR_OFFSET_MODE) + .long(Self::CONNECTOR_OFFSET_MODE) + .required(false) + .env("P_CONNECTOR_OFFSET_MODE") + .value_parser(["earliest", "latest", "group"]) + .help("Offset mode: earliest, latest, or group") + ) + .arg( + Arg::new(Self::CONNECTOR_BAD_DATA_POLICY) + .long(Self::CONNECTOR_BAD_DATA_POLICY) + .required(false) + .env("P_CONNECTOR_BAD_DATA_POLICY") + .help("Bad data handling policy: skip, error") + ) + .arg( + Arg::new(Self::CONNECTOR_MAX_RETRIES) + .long(Self::CONNECTOR_MAX_RETRIES) + .env("P_CONNECTOR_MAX_RETRIES") + .required(false) + .value_parser(value_parser!(u32)) + .help("Maximum number of retries on errors") + ) + .arg( + Arg::new(Self::CONNECTOR_RETRY_INTERVAL_MS) + .long(Self::CONNECTOR_RETRY_INTERVAL_MS) + .env("P_CONNECTOR_RETRY_INTERVAL_MS") + .value_parser(value_parser!(u64)) + .required(false) + .help("Retry interval in milliseconds") + ) + .arg( + Arg::new(Self::CONNECTOR_METRICS_ENABLED) + .long(Self::CONNECTOR_METRICS_ENABLED) + .env("P_CONNECTOR_METRICS_ENABLED") + .value_parser(value_parser!(bool)) + .required(false) + .help("Enable or disable connector metrics") + ) + .arg( + Arg::new(Self::CONNECTOR_INSTANCE_ID) + .long(Self::CONNECTOR_INSTANCE_ID) + .required(false) + .env("P_CONNECTOR_INSTANCE_ID") + .help("Instance ID for the connector") + ) + + // ConsumerConfig arguments: + .arg( + Arg::new(Self::CONSUMER_GROUP_INSTANCE_ID) + .long(Self::CONSUMER_GROUP_INSTANCE_ID) + .required(false) + .env("P_CONSUMER_GROUP_INSTANCE_ID") + .help("Consumer group instance ID") + ) + .arg( + Arg::new(Self::CONSUMER_PARTITION_ASSIGNMENT_STRATEGY) + .long(Self::CONSUMER_PARTITION_ASSIGNMENT_STRATEGY) + .env("P_CONSUMER_PARTITION_ASSIGNMENT_STRATEGY") + .help("Partition assignment strategy") + .required(false) + ) + .arg( + Arg::new(Self::CONSUMER_SESSION_TIMEOUT_MS) + .long(Self::CONSUMER_SESSION_TIMEOUT_MS) + .env("P_CONSUMER_SESSION_TIMEOUT_MS") + .value_parser(value_parser!(u32)) + .help("Consumer session timeout in ms") + .required(false) + ) + .arg( + Arg::new(Self::CONSUMER_HEARTBEAT_INTERVAL_MS) + .long(Self::CONSUMER_HEARTBEAT_INTERVAL_MS) + .env("P_CONSUMER_HEARTBEAT_INTERVAL_MS") + .value_parser(value_parser!(u32)) + .help("Consumer heartbeat interval in ms") + .required(false) + ) + .arg( + Arg::new(Self::CONSUMER_MAX_POLL_INTERVAL_MS) + .long(Self::CONSUMER_MAX_POLL_INTERVAL_MS) + .env("P_CONSUMER_MAX_POLL_INTERVAL_MS") + .value_parser(value_parser!(u32)) + .help("Max poll interval in ms") + .required(false) + ) + .arg( + Arg::new(Self::CONSUMER_ENABLE_AUTO_OFFSET_STORE) + .long(Self::CONSUMER_ENABLE_AUTO_OFFSET_STORE) + .env("P_CONSUMER_ENABLE_AUTO_OFFSET_STORE") + .value_parser(value_parser!(bool)) + .help("Enable auto offset store") + .default_value("true") // Just for as few metrics + .required(false) + ) + .arg( + Arg::new(Self::CONSUMER_AUTO_OFFSET_RESET) + .long(Self::CONSUMER_AUTO_OFFSET_RESET) + .env("P_CONSUMER_AUTO_OFFSET_RESET") + .value_parser(["earliest", "latest", "none"]) + .help("Auto offset reset behavior") + ) + .arg( + Arg::new(Self::CONSUMER_FETCH_MIN_BYTES) + .long(Self::CONSUMER_FETCH_MIN_BYTES) + .env("P_CONSUMER_FETCH_MIN_BYTES") + .value_parser(value_parser!(u32)) + .help("Fetch min bytes") + ) + .arg( + Arg::new(Self::CONSUMER_FETCH_MAX_BYTES) + .long(Self::CONSUMER_FETCH_MAX_BYTES) + .env("P_CONSUMER_FETCH_MAX_BYTES") + .value_parser(value_parser!(u32)) + .help("Fetch max bytes") + ) + .arg( + Arg::new(Self::CONSUMER_FETCH_MAX_WAIT_MS) + .long(Self::CONSUMER_FETCH_MAX_WAIT_MS) + .env("P_CONSUMER_FETCH_MAX_WAIT_MS") + .value_parser(value_parser!(u32)) + .help("Fetch max wait in ms") + ) + .arg( + Arg::new(Self::CONSUMER_MAX_PARTITION_FETCH_BYTES) + .long(Self::CONSUMER_MAX_PARTITION_FETCH_BYTES) + .env("P_CONSUMER_MAX_PARTITION_FETCH_BYTES") + .value_parser(value_parser!(u32)) + .help("Max partition fetch bytes") + ) + .arg( + Arg::new(Self::CONSUMER_QUEUED_MIN_MESSAGES) + .long(Self::CONSUMER_QUEUED_MIN_MESSAGES) + .env("P_CONSUMER_QUEUED_MIN_MESSAGES") + .value_parser(value_parser!(u32)) + .help("Queued min messages") + ) + .arg( + Arg::new(Self::CONSUMER_QUEUED_MAX_MESSAGES_KBYTES) + .long(Self::CONSUMER_QUEUED_MAX_MESSAGES_KBYTES) + .env("P_CONSUMER_QUEUED_MAX_MESSAGES_KBYTES") + .value_parser(value_parser!(u32)) + .help("Queued max messages kbytes") + ) + .arg( + Arg::new(Self::CONSUMER_ENABLE_PARTITION_EOF) + .long(Self::CONSUMER_ENABLE_PARTITION_EOF) + .env("P_CONSUMER_ENABLE_PARTITION_EOF") + .value_parser(value_parser!(bool)) + .help("Enable partition EOF") + ) + .arg( + Arg::new(Self::CONSUMER_CHECK_CRCS) + .long(Self::CONSUMER_CHECK_CRCS) + .env("P_CONSUMER_CHECK_CRCS") + .value_parser(value_parser!(bool)) + .help("Check CRCs") + ) + .arg( + Arg::new(Self::CONSUMER_ISOLATION_LEVEL) + .long(Self::CONSUMER_ISOLATION_LEVEL) + .env("P_CONSUMER_ISOLATION_LEVEL") + .value_parser(["read_uncommitted", "read_committed"]) + .help("Isolation level") + ) + .arg( + Arg::new(Self::CONSUMER_FETCH_MESSAGE_MAX_BYTES) + .long(Self::CONSUMER_FETCH_MESSAGE_MAX_BYTES) + .env("P_CONSUMER_FETCH_MESSAGE_MAX_BYTES") + .help("Fetch message max bytes (string)") + ) + .arg( + Arg::new(Self::CONSUMER_STATS_INTERVAL_MS) + .long(Self::CONSUMER_STATS_INTERVAL_MS) + .env("P_CONSUMER_STATS_INTERVAL_MS") + .value_parser(value_parser!(u64)) + .help("Consumer stats interval ms") + ) + .arg( + Arg::new(Self::KAFKA_TOPICS) + .long(Self::KAFKA_TOPICS) + .env("P_KAFKA_TOPICS") + .help("Kafka topics to consume from.Comma seperated string") + ) + .arg( + Arg::new(Self::KAFKA_BOOTSTRAP_SERVERS) + .long(Self::KAFKA_BOOTSTRAP_SERVERS) + .env("P_KAFKA_BOOTSTRAP_SERVERS") + .help("Kafka bootstrap servers.") + ) + .arg( + Arg::new(Self::KAFKA_GROUP_ID) + .long(Self::KAFKA_GROUP_ID) + .required(false) + .env("P_KAFKA_GROUP_ID") + .help("Kafka consumer group ID.") + ) + .group( + ArgGroup::new("oidc") + .args([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) + .requires_all([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) + .multiple(true) + ) } } @@ -514,6 +780,126 @@ impl FromArgMatches for Cli { } fn update_from_arg_matches(&mut self, m: &clap::ArgMatches) -> Result<(), clap::Error> { + if matches!(self.mode, Mode::Query) { + self.connector_config = None; + } + + if let Some(topics) = m.get_one::(Cli::KAFKA_TOPICS).cloned() { + let bootstrap_servers = m + .get_one::(Cli::KAFKA_BOOTSTRAP_SERVERS) + .cloned() + .unwrap_or_default(); + let group_id = m + .get_one::(Cli::KAFKA_GROUP_ID) + .cloned() + .unwrap_or("parseable-default-group".to_string()); + + if topics.is_empty() { + return Err(clap::Error::raw( + clap::error::ErrorKind::MissingRequiredArgument, + "Kafka topics required in ingest/all mode.", + )); + } + if bootstrap_servers.is_empty() { + return Err(clap::Error::raw( + clap::error::ErrorKind::MissingRequiredArgument, + "Kafka bootstrap servers required in ingest/all mode.", + )); + } + + let offset_mode = match m + .get_one::(Cli::CONNECTOR_OFFSET_MODE) + .map(|s| s.as_str()) + { + Some("earliest") => SourceOffset::Earliest, + Some("latest") => SourceOffset::Latest, + Some("group") | None => SourceOffset::Group, + _ => SourceOffset::Latest, + }; + + let buffer_size = m + .get_one::(Cli::CONNECTOR_BUFFER_SIZE) + .cloned() + .unwrap_or(10000); + let buffer_timeout = m + .get_one::(Cli::CONNECTOR_BUFFER_TIMEOUT) + .cloned() + .unwrap_or(5000); + + let max_retries = m + .get_one::(Cli::CONNECTOR_MAX_RETRIES) + .cloned() + .unwrap_or(20); + let retry_interval_ms = m + .get_one::(Cli::CONNECTOR_RETRY_INTERVAL_MS) + .cloned() + .unwrap_or(10000); + let metrics_enabled = m + .get_one::(Cli::CONNECTOR_METRICS_ENABLED) + .cloned() + .unwrap_or(true); + let connector_name = m + .get_one::(Cli::CONNECTOR_NAME) + .cloned() + .unwrap_or_else(|| "parseable-connectors".to_string()); + let instance_id = m + .get_one::(Cli::CONNECTOR_INSTANCE_ID) + .cloned() + .unwrap_or_else(|| "parseable-connectors".to_string()); + + let bad_data_policy = m.get_one::(Cli::CONNECTOR_BAD_DATA_POLICY).cloned(); + let bad_data = match bad_data_policy.as_deref() { + Some("drop") => Some(BadData::Drop {}), + Some("fail") => Some(BadData::Fail {}), + Some("dlt") => Some(BadData::Dlt {}), + _ => None, + }; + + let auto_offset_reset = m + .get_one::(Cli::CONSUMER_AUTO_OFFSET_RESET) + .cloned() + .unwrap_or_else(|| "earliest".to_string()); + + let mut consumer = ConsumerConfig::default(); + consumer.group_id = group_id.clone(); + consumer.auto_offset_reset = auto_offset_reset; + + let topics: Vec = topics.split(",").map(|t| t.to_owned()).collect(); + let topics_clone = topics.to_vec(); + + let kafka_config = KafkaConfig::builder() + .bootstrap_servers(bootstrap_servers) + .topic(topics_clone) + .with_consumer(consumer) + .build() + .map_err(|e| { + clap::Error::raw(clap::error::ErrorKind::InvalidValue, e.to_string()) + })?; + + let mut connector_config = ConnectorConfig::builder() + .connector_type(ConnectorType::KafkaSource) + .name(connector_name) + .buffer_size(buffer_size) + .buffer_timeout(Duration::from_millis(buffer_timeout)) + .offset_mode(offset_mode) + .topic(topics) + .max_retries(max_retries) + .retry_interval(Duration::from_millis(retry_interval_ms)) + .metrics_enabled(metrics_enabled) + .kafka_config(kafka_config) + .instance_id(instance_id) + .build() + .map_err(|e| { + clap::Error::raw(clap::error::ErrorKind::InvalidValue, e.to_string()) + })?; + + connector_config.bad_data = bad_data; + + self.connector_config = Some(connector_config); + } else { + warn!("No Kafka topics provided"); + } + self.trino_catalog = m.get_one::(Self::TRINO_CATALOG_NAME).cloned(); self.trino_endpoint = m.get_one::(Self::TRINO_ENDPOINT).cloned(); self.trino_auth = m.get_one::(Self::TRINO_AUTHORIZATION).cloned(); diff --git a/src/connectors/common/config.rs b/src/connectors/common/config.rs new file mode 100644 index 000000000..3aed150a3 --- /dev/null +++ b/src/connectors/common/config.rs @@ -0,0 +1,164 @@ +use crate::connectors::common::types::ConnectorType; +use crate::connectors::common::{BadData, ConnectorError}; +use crate::connectors::kafka::config::{KafkaConfig, SourceOffset}; +use serde::{Deserialize, Serialize}; +use std::{time::Duration, vec}; + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct ConnectorConfig { + // Basic Configuration + pub connector_type: ConnectorType, + pub name: String, + + // Performance Configuration + pub buffer_size: usize, + pub buffer_timeout: Duration, + + // Topic/Stream Configuration + pub topics: Vec, + pub offset_mode: SourceOffset, + + // Error Handling + pub bad_data: Option, + pub max_retries: u32, + pub retry_interval: Duration, + + // Kafka-specific Configuration + #[serde(skip_serializing_if = "Option::is_none")] + pub kafka_config: Option, + + // Monitoring + pub metrics_enabled: bool, + pub instance_id: String, +} + +impl Default for ConnectorConfig { + fn default() -> Self { + Self { + connector_type: ConnectorType::KafkaSource, + name: String::from("parseable-connectors"), + buffer_size: 10000, + buffer_timeout: Duration::from_millis(500), + topics: vec![], + offset_mode: SourceOffset::Earliest, + bad_data: None, + max_retries: 3, + retry_interval: Duration::from_secs(5), + kafka_config: Some(KafkaConfig::default()), + metrics_enabled: true, + instance_id: String::from("parseable-connectors"), + } + } +} + +impl ConnectorConfig { + pub fn builder() -> ConnectorConfigBuilder { + ConnectorConfigBuilder::default() + } + + pub fn validate(&self) -> anyhow::Result<(), ConnectorError> { + if self.buffer_size == 0 { + return Err(ConnectorError::Config("Buffer size must be > 0".into())); + } + + if let Some(kafka_config) = &self.kafka_config { + self.validate_kafka_config(kafka_config)?; + } + + Ok(()) + } + + fn validate_kafka_config(&self, config: &KafkaConfig) -> Result<(), ConnectorError> { + if config.bootstrap_servers.is_empty() { + return Err(ConnectorError::Config("Bootstrap servers required".into())); + } + + if config.topics().is_empty() { + return Err(ConnectorError::Config("Topic name required".into())); + } + + Ok(()) + } +} + +#[derive(Default)] +pub struct ConnectorConfigBuilder { + config: ConnectorConfig, +} + +impl ConnectorConfigBuilder { + pub fn connector_type(mut self, connector_type: ConnectorType) -> Self { + self.config.connector_type = connector_type; + self + } + + pub fn name(mut self, name: impl Into) -> Self { + self.config.name = name.into(); + self + } + + pub fn buffer_size(mut self, buffer_size: usize) -> Self { + self.config.buffer_size = buffer_size; + self + } + + pub fn buffer_timeout(mut self, buffer_timeout: Duration) -> Self { + self.config.buffer_timeout = buffer_timeout; + self + } + + pub fn max_retries(mut self, max_retries: u32) -> Self { + self.config.max_retries = max_retries; + self + } + + pub fn instance_id(mut self, instance_id: String) -> Self { + self.config.instance_id = instance_id; + self + } + + pub fn retry_interval(mut self, retry_interval: Duration) -> Self { + self.config.retry_interval = retry_interval; + self + } + + pub fn metrics_enabled(mut self, metrics_enabled: bool) -> Self { + self.config.metrics_enabled = metrics_enabled; + self + } + + pub fn topic(mut self, topics: Vec) -> Self { + self.config.topics = topics; + self + } + + pub fn offset_mode(mut self, offset_mode: SourceOffset) -> Self { + self.config.offset_mode = offset_mode; + self + } + + pub fn kafka_config(mut self, kafka_config: KafkaConfig) -> Self { + self.config.kafka_config = Some(kafka_config); + self + } + + pub fn build(self) -> anyhow::Result { + let config = self.config; + config.validate()?; + Ok(config) + } +} + +#[cfg(test)] +mod tests { + use super::*; + #[test] + fn test_config_validation() { + let result = ConnectorConfig::builder() + .connector_type(ConnectorType::KafkaSource) + .buffer_size(0) + .build(); + + assert!(result.is_err()); + } +} diff --git a/src/connectors/common/mod.rs b/src/connectors/common/mod.rs new file mode 100644 index 000000000..b0474ffa5 --- /dev/null +++ b/src/connectors/common/mod.rs @@ -0,0 +1,31 @@ +use serde::{Deserialize, Serialize}; +pub mod config; +pub mod processor; +pub mod shutdown; +pub mod types; + +#[derive(Debug, thiserror::Error)] +pub enum ConnectorError { + #[error("Kafka error: {0}")] + Kafka(#[from] rdkafka::error::KafkaError), + #[error("Configuration error: {0}")] + Config(String), + #[error("Processing error: {0}")] + Processing(String), + #[error("Initialization error: {0}")] + Init(String), +} + +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq, Hash, PartialOrd)] +#[serde(rename_all = "snake_case")] +pub enum BadData { + Fail {}, + Drop {}, + Dlt {}, +} + +impl Default for BadData { + fn default() -> Self { + BadData::Drop {} + } +} diff --git a/src/connectors/common/processor.rs b/src/connectors/common/processor.rs new file mode 100644 index 000000000..5f1e07bdc --- /dev/null +++ b/src/connectors/common/processor.rs @@ -0,0 +1,11 @@ +use async_trait::async_trait; + +#[async_trait] +pub trait Processor: Send + Sync + Sized + 'static { + async fn process(&self, records: IN) -> anyhow::Result; + + #[allow(unused_variables)] + async fn post_stream(&self) -> anyhow::Result<()> { + Ok(()) + } +} diff --git a/src/connectors/common/shutdown.rs b/src/connectors/common/shutdown.rs new file mode 100644 index 000000000..ba8f169cb --- /dev/null +++ b/src/connectors/common/shutdown.rs @@ -0,0 +1,118 @@ +use tokio::sync::mpsc; +use tokio_util::sync::CancellationToken; +use tracing::{info, warn}; + +#[derive(Debug)] +pub struct Shutdown { + cancel_token: CancellationToken, + shutdown_complete_tx: mpsc::Sender<()>, + shutdown_complete_rx: Option>, +} + +impl Shutdown { + pub fn new() -> Self { + let cancel_token = CancellationToken::new(); + let (shutdown_complete_tx, shutdown_complete_rx) = mpsc::channel(1); + Self { + cancel_token, + shutdown_complete_tx, + shutdown_complete_rx: Some(shutdown_complete_rx), + } + } + + pub fn start(&self) { + self.cancel_token.cancel(); + } + + pub async fn recv(&self) { + self.cancel_token.cancelled().await; + } + + pub async fn signal_listener(&self) { + let ctrl_c_signal = tokio::signal::ctrl_c(); + #[cfg(unix)] + let mut sigterm_signal = + tokio::signal::unix::signal(tokio::signal::unix::SignalKind::terminate()).unwrap(); + #[cfg(unix)] + tokio::select! { + _ = ctrl_c_signal => {}, + _ = sigterm_signal.recv() => {} + } + #[cfg(windows)] + let _ = ctrl_c_signal.await; + + warn!("Shutdown signal received!"); + self.start(); + } + + pub async fn complete(self) { + drop(self.shutdown_complete_tx); + self.shutdown_complete_rx.unwrap().recv().await; + info!("Shutdown complete!") + } +} + +impl Default for Shutdown { + fn default() -> Self { + Self::new() + } +} + +impl Clone for Shutdown { + fn clone(&self) -> Self { + Self { + cancel_token: self.cancel_token.clone(), + shutdown_complete_tx: self.shutdown_complete_tx.clone(), + shutdown_complete_rx: None, + } + } +} + +#[cfg(test)] +mod tests { + use std::sync::{Arc, Mutex}; + + use super::*; + use tokio::time::Duration; + + #[tokio::test] + async fn test_shutdown_recv() { + let shutdown = Shutdown::new(); + let shutdown_clone = shutdown.clone(); + // receive shutdown task + let task = tokio::spawn(async move { + shutdown_clone.recv().await; + 1 + }); + // start shutdown task after 200 ms + tokio::spawn(async move { + tokio::time::sleep(Duration::from_millis(200)).await; + shutdown.start(); + }); + // if shutdown is not received within 5 seconds, fail test + let check_value = tokio::select! { + _ = tokio::time::sleep(Duration::from_secs(5)) => panic!("Shutdown not received within 5 seconds"), + v = task => v.unwrap(), + }; + assert_eq!(check_value, 1); + } + + #[tokio::test] + async fn test_shutdown_wait_for_complete() { + let shutdown = Shutdown::new(); + let shutdown_clone = shutdown.clone(); + let check_value: Arc> = Arc::new(Mutex::new(false)); + let check_value_clone = Arc::clone(&check_value); + // receive shutdown task + tokio::spawn(async move { + shutdown_clone.recv().await; + tokio::time::sleep(Duration::from_millis(200)).await; + let mut check: std::sync::MutexGuard<'_, bool> = check_value_clone.lock().unwrap(); + *check = true; + }); + shutdown.start(); + shutdown.complete().await; + let check = check_value.lock().unwrap(); + assert!(*check, "shutdown did not successfully wait for complete"); + } +} diff --git a/src/connectors/common/types/mod.rs b/src/connectors/common/types/mod.rs new file mode 100644 index 000000000..ee2085eab --- /dev/null +++ b/src/connectors/common/types/mod.rs @@ -0,0 +1,45 @@ +use serde::{Deserialize, Serialize}; +use std::fmt; +use std::fmt::{Display, Formatter}; +use std::hash::Hash; + +#[derive(Debug, Clone, Hash, Eq, PartialEq, Serialize, Deserialize)] +pub enum ConnectorType { + KafkaSource, +} + +impl Display for ConnectorType { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match self { + ConnectorType::KafkaSource => write!(f, "kafka_source"), + } + } +} + +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq, Hash, PartialOrd)] +#[serde(rename_all = "snake_case")] +pub enum ConnectionType { + Source, + Sink, +} + +impl Display for ConnectionType { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match self { + ConnectionType::Source => write!(f, "SOURCE"), + ConnectionType::Sink => write!(f, "SINK"), + } + } +} + +impl TryFrom for ConnectionType { + type Error = String; + + fn try_from(value: String) -> anyhow::Result { + match value.to_lowercase().as_str() { + "source" => Ok(ConnectionType::Source), + "sink" => Ok(ConnectionType::Sink), + _ => Err(format!("Invalid connection type: {}", value)), + } + } +} diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs new file mode 100644 index 000000000..367024372 --- /dev/null +++ b/src/connectors/kafka/config.rs @@ -0,0 +1,627 @@ +use anyhow::bail; +use rdkafka::Offset; +use rustls::pki_types::{CertificateDer, PrivateKeyDer}; +use rustls_pemfile::{self, Item}; +use serde::{Deserialize, Serialize}; +use std::fmt::Display; +use std::io::BufReader; +use std::sync::Arc; +use tracing::{debug, info}; + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct KafkaConfig { + // Common configuration + pub bootstrap_servers: String, + topics: Vec, + pub client_id: Option, + + // Component-specific configurations + #[serde(skip_serializing_if = "Option::is_none")] + pub consumer: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub producer: Option, + + // Security and advanced settings + #[serde(skip_serializing_if = "Option::is_none")] + pub security: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct ConsumerConfig { + // Consumer group configuration + pub group_id: String, + pub group_instance_id: Option, + pub partition_assignment_strategy: String, + + // Session handling + pub session_timeout_ms: u32, + pub heartbeat_interval_ms: u32, + pub max_poll_interval_ms: u32, + + // Offset management + pub enable_auto_commit: bool, + pub auto_commit_interval_ms: u32, + pub enable_auto_offset_store: bool, + pub auto_offset_reset: String, + + // Fetch configuration + pub fetch_min_bytes: u32, + pub fetch_max_bytes: u32, + pub fetch_max_wait_ms: u32, + pub max_partition_fetch_bytes: u32, + + // Queue configuration + pub queued_min_messages: u32, + pub queued_max_messages_kbytes: u32, + + // Processing configuration + pub enable_partition_eof: bool, + pub check_crcs: bool, + pub isolation_level: String, + pub fetch_message_max_bytes: String, + pub stats_interval_ms: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct ProducerConfig { + pub acks: String, + pub compression_type: String, + pub batch_size: u32, + pub linger_ms: u32, + pub delivery_timeout_ms: u32, + pub max_in_flight_requests_per_connection: u32, + pub max_request_size: u32, + pub enable_idempotence: bool, + pub transaction_timeout_ms: Option, + pub queue_buffering_max_messages: u32, + queue_buffering_max_ms: u32, + retry_backoff_ms: u32, + batch_num_messages: u32, + retries: u32, +} + +impl Default for ConsumerConfig { + fn default() -> Self { + Self { + group_id: "default-cg".to_string(), + group_instance_id: Some("default-cg-ii".to_string()), + // NOTE: cooperative-sticky does not work well in rdkafka when using manual commit. + // @see https://github.com/confluentinc/librdkafka/issues/4629 + // @see https://github.com/confluentinc/librdkafka/issues/4368 + partition_assignment_strategy: "roundrobin,range".to_string(), + session_timeout_ms: 60000, + heartbeat_interval_ms: 3000, + max_poll_interval_ms: 300000, + enable_auto_commit: false, + auto_commit_interval_ms: 5000, + enable_auto_offset_store: true, + auto_offset_reset: "earliest".to_string(), + fetch_min_bytes: 1, + fetch_max_bytes: 52428800, + fetch_max_wait_ms: 500, + max_partition_fetch_bytes: 1048576, + queued_min_messages: 100000, + queued_max_messages_kbytes: 65536, + enable_partition_eof: false, + check_crcs: false, + isolation_level: "read_committed".to_string(), + fetch_message_max_bytes: "1048576".to_string(), + stats_interval_ms: Some(10000), + } + } +} + +impl Default for ProducerConfig { + fn default() -> Self { + Self { + acks: "all".to_string(), + compression_type: "lz4".to_string(), + batch_size: 16384, // 16KB default batch size + linger_ms: 5, // Small latency for better batching + delivery_timeout_ms: 120000, // 2 minute delivery timeout + max_in_flight_requests_per_connection: 5, + max_request_size: 1048576, // 1MB max request size + enable_idempotence: true, // Ensure exactly-once delivery + transaction_timeout_ms: Some(60000), // 1 minute transaction timeout + queue_buffering_max_messages: 100000, // Producer queue size + queue_buffering_max_ms: 100, // Max time to wait before sending + retry_backoff_ms: 100, // Backoff time between retries + batch_num_messages: 10000, // Messages per batch + retries: 3, // Number of retries + } + } +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +#[warn(non_camel_case_types)] +#[serde(rename_all = "UPPERCASE")] +#[allow(non_camel_case_types)] +pub enum SecurityProtocol { + Plaintext, + SSL, + SASL_SSL, + SASL_PLAINTEXT, +} + +impl Display for SecurityProtocol { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let str = match self { + SecurityProtocol::Plaintext => "PLAINTEXT", + SecurityProtocol::SSL => "SSL", + SecurityProtocol::SASL_SSL => "SASL_SSL", + SecurityProtocol::SASL_PLAINTEXT => "SASL_PLAINTEXT", + } + .to_string(); + write!(f, "{}", str) + } +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct SecurityConfig { + pub protocol: SecurityProtocol, + #[serde(skip_serializing_if = "Option::is_none")] + pub ssl_config: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub sasl_config: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct SSLConfig { + pub ca_certificate_pem: String, + pub client_certificate_pem: String, + pub client_key_pem: String, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all = "UPPERCASE")] +pub enum SASLMechanism { + Plain, + ScramSha256, + ScramSha512, + GssAPI, +} + +impl Display for SASLMechanism { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let str = match self { + SASLMechanism::Plain => "PLAIN", + SASLMechanism::ScramSha256 => "SCRAM-SHA-256", + SASLMechanism::ScramSha512 => "SCRAM-SHA-512", + SASLMechanism::GssAPI => "GSSAPI", + } + .to_string(); + write!(f, "{}", str) + } +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct SASLConfig { + pub mechanism: SASLMechanism, + pub username: String, + pub password: String, + #[serde(skip_serializing_if = "Option::is_none")] + pub kerberos_service_name: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub kerberos_principal: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub kerberos_keytab: Option, +} + +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +#[serde(rename_all = "lowercase")] +pub enum SourceOffset { + Earliest, + Latest, + Group, +} + +impl SourceOffset { + pub fn get_offset(&self) -> Offset { + match self { + SourceOffset::Earliest => Offset::Beginning, + SourceOffset::Latest => Offset::End, + SourceOffset::Group => Offset::Stored, + } + } +} + +impl KafkaConfig { + pub fn new( + bootstrap_servers: String, + topics: Vec, + consumer_config: Option, + ) -> Self { + Self { + bootstrap_servers, + topics, + client_id: None, + consumer: consumer_config, + producer: None, + security: None, + } + } + + pub fn consumer_config(&self) -> rdkafka::ClientConfig { + let mut config = rdkafka::ClientConfig::new(); + config.set("bootstrap.servers", &self.bootstrap_servers); + + if let Some(client_id) = &self.client_id { + config + .set("client.id", format!("parseable-{}-ci", client_id)) + .set("client.rack", format!("parseable-{}-cr", client_id)); + } + + if let Some(consumer) = &self.consumer { + let enable_auto_commit = consumer.enable_auto_commit.to_string(); + let group_id = format!("parseable-{}-gi", &consumer.group_id); + info!("Setting group.id to {}", group_id); + config + .set("group.id", group_id) + .set("log_level", "7") + .set("enable.auto.commit", enable_auto_commit) + .set( + "enable.auto.offset.store", + consumer.enable_auto_offset_store.to_string(), + ) + .set("auto.offset.reset", &consumer.auto_offset_reset) + .set( + "partition.assignment.strategy", + &consumer.partition_assignment_strategy, + ) + .set( + "session.timeout.ms", + consumer.session_timeout_ms.to_string(), + ) + .set( + "heartbeat.interval.ms", + consumer.heartbeat_interval_ms.to_string(), + ) + .set( + "max.poll.interval.ms", + consumer.max_poll_interval_ms.to_string(), + ) + .set("fetch.min.bytes", consumer.fetch_min_bytes.to_string()) + .set("fetch.max.bytes", consumer.fetch_max_bytes.to_string()) + .set( + "fetch.message.max.bytes", + consumer.fetch_message_max_bytes.to_string(), + ) + .set( + "max.partition.fetch.bytes", + consumer.max_partition_fetch_bytes.to_string(), + ) + .set( + "queued.min.messages", + consumer.queued_min_messages.to_string(), + ) + .set( + "queued.max.messages.kbytes", + consumer.queued_max_messages_kbytes.to_string(), + ) + .set( + "enable.partition.eof", + consumer.enable_partition_eof.to_string(), + ) + .set("isolation.level", &consumer.isolation_level) + .set( + "statistics.interval.ms", + consumer.stats_interval_ms.unwrap_or(10000).to_string(), + ); + + if let Some(instance_id) = &consumer.group_instance_id { + config.set("group.instance.id", instance_id); + } + } + + self.apply_security_config(&mut config); + + info!("Consumer configuration: {:?}", config); + config + } + + pub fn producer_config(&self) -> rdkafka::config::ClientConfig { + let mut config = rdkafka::config::ClientConfig::new(); + config.set("bootstrap.servers", &self.bootstrap_servers); + + if let Some(client_id) = &self.client_id { + config + .set("client.id", format!("parseable-{}-ci", client_id)) + .set("client.rack", format!("parseable-{}-cr", client_id)); + } + + if let Some(producer_config) = &self.producer { + config + .set("acks", &producer_config.acks) + .set("compression.type", &producer_config.compression_type) + .set("batch.size", producer_config.batch_size.to_string()) + .set("linger.ms", producer_config.linger_ms.to_string()) + .set( + "delivery.timeout.ms", + producer_config.delivery_timeout_ms.to_string(), + ) + .set( + "max.in.flight.requests.per.connection", + producer_config + .max_in_flight_requests_per_connection + .to_string(), + ) + .set( + "max.request.size", + producer_config.max_request_size.to_string(), + ) + .set( + "enable.idempotence", + producer_config.enable_idempotence.to_string(), + ) + .set( + "batch.num.messages", + producer_config.batch_num_messages.to_string(), + ) + .set( + "queue.buffering.max.messages", + producer_config.queue_buffering_max_messages.to_string(), + ) + .set( + "queue.buffering.max.ms", + producer_config.queue_buffering_max_ms.to_string(), + ) + .set( + "retry.backoff.ms", + producer_config.retry_backoff_ms.to_string(), + ) + .set("retries", producer_config.retries.to_string()); + + if let Some(timeout) = producer_config.transaction_timeout_ms { + config.set("transaction.timeout.ms", timeout.to_string()); + } + } + + self.apply_security_config(&mut config); + + config + } + + fn apply_security_config(&self, config: &mut rdkafka::ClientConfig) { + let security = match &self.security { + Some(sec) => sec, + None => { + debug!("No security configuration provided, using PLAINTEXT"); + config.set("security.protocol", "plaintext"); + return; + } + }; + + config.set( + "security.protocol", + security.protocol.to_string().to_lowercase(), + ); + + if matches!( + security.protocol, + SecurityProtocol::SSL | SecurityProtocol::SASL_SSL + ) { + if let Some(ssl) = &security.ssl_config { + debug!("Applying SSL configuration"); + config + .set("ssl.ca.pem", &ssl.ca_certificate_pem) + .set("ssl.certificate.pem", &ssl.client_certificate_pem) + .set("ssl.key.pem", &ssl.client_key_pem); + } else { + panic!( + "SSL configuration required for {:?} protocol", + security.protocol + ); + } + } + + if matches!( + security.protocol, + SecurityProtocol::SASL_SSL | SecurityProtocol::SASL_PLAINTEXT + ) { + if let Some(sasl) = &security.sasl_config { + debug!( + "Applying SASL configuration with mechanism: {}", + sasl.mechanism.to_string() + ); + config + .set("sasl.mechanism", sasl.mechanism.to_string()) + .set("sasl.username", &sasl.username) + .set("sasl.password", &sasl.password); + + // Apply Kerberos-specific configuration if using GSSAPI + if matches!(sasl.mechanism, SASLMechanism::GssAPI) { + if let Some(service_name) = &sasl.kerberos_service_name { + config.set("sasl.kerberos.service.name", service_name); + } + if let Some(principal) = &sasl.kerberos_principal { + config.set("sasl.kerberos.principal", principal); + } + if let Some(keytab) = &sasl.kerberos_keytab { + config.set("sasl.kerberos.keytab", keytab); + } + } + } else { + panic!( + "SASL configuration required for {:?} protocol", + security.protocol + ); + } + } + } +} +impl Default for KafkaConfig { + fn default() -> Self { + Self { + // Common configuration with standard broker port + bootstrap_servers: "localhost:9092".to_string(), + topics: vec![], + client_id: None, // Let Kafka generate a unique client ID if not specified + + // Component-specific configurations with production-ready defaults + consumer: Some(ConsumerConfig::default()), + producer: Some(ProducerConfig::default()), + + // Security defaults to plaintext for development + // Production environments should explicitly configure security + security: Some(SecurityConfig { + protocol: SecurityProtocol::Plaintext, + ssl_config: None, + sasl_config: None, + }), + } + } +} + +impl KafkaConfig { + pub fn builder() -> KafkaConfigBuilder { + KafkaConfigBuilder::default() + } + + pub fn topics(&self) -> Vec<&str> { + self.topics.iter().map(|s| s.as_str()).collect() + } +} + +#[derive(Default, Debug)] +pub struct KafkaConfigBuilder { + config: KafkaConfig, +} + +impl KafkaConfigBuilder { + pub fn bootstrap_servers(mut self, servers: impl Into) -> Self { + self.config.bootstrap_servers = servers.into(); + self + } + + pub fn topic(mut self, topics: Vec) -> Self { + self.config.topics = topics; + self + } + + pub fn client_id(mut self, client_id: impl Into) -> Self { + self.config.client_id = Some(client_id.into()); + self + } + + pub fn with_consumer(mut self, consumer: ConsumerConfig) -> Self { + self.config.consumer = Some(consumer); + self + } + + pub fn with_producer(mut self, producer: ProducerConfig) -> Self { + self.config.producer = Some(producer); + self + } + + pub fn with_security(mut self, security: SecurityConfig) -> Self { + self.config.security = Some(security); + self + } + + pub fn build(self) -> anyhow::Result { + let config = self.config; + + if config.bootstrap_servers.is_empty() { + anyhow::bail!("bootstrap_servers cannot be empty"); + } + + Ok(config) + } +} + +#[derive(Debug, Clone)] +#[allow(dead_code)] +pub struct KafkaCertificates { + ca_certificate: Arc>, + client_certificate: Arc>, + client_key: Arc>, +} + +#[allow(dead_code)] +fn parse_first_certificate(pem: &str) -> anyhow::Result> { + let mut reader = BufReader::new(pem.as_bytes()); + let items = rustls_pemfile::read_all(&mut reader); + + for item in items.flatten() { + if let Item::X509Certificate(cert_data) = item { + return Ok(cert_data); + } + } + bail!("No certificate found in PEM") +} + +#[allow(dead_code)] +fn parse_first_private_key(pem: &str) -> anyhow::Result> { + let mut reader = BufReader::new(pem.as_bytes()); + let items = rustls_pemfile::read_all(&mut reader); + + for item in items { + if let Ok(Item::Pkcs1Key(key_data)) = item { + return Ok(key_data.into()); + } + if let Ok(Item::Pkcs8Key(key_data)) = item { + return Ok(key_data.into()); + } + if let Ok(Item::Sec1Key(key_data)) = item { + return Ok(key_data.into()); + } + } + + bail!("No private key found in PEM") +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_consumer_config() { + let consumer_config = ConsumerConfig { + group_id: "test-group".to_string(), + partition_assignment_strategy: "cooperative-sticky".to_string(), + ..ConsumerConfig::default() + }; + + let config = KafkaConfig::new( + "localhost:9092".to_string(), + vec!["test-topic".to_string()], + Some(consumer_config), + ); + + let rdkafka_config = config.consumer_config(); + assert_eq!(rdkafka_config.get("group.id"), Some("test-group")); + assert_eq!( + rdkafka_config.get("partition.assignment.strategy"), + Some("cooperative-sticky") + ); + } + + #[test] + fn test_default_kafka_config() { + let config = KafkaConfig::default(); + assert_eq!(config.bootstrap_servers, "localhost:9092"); + assert!(config.topics.is_empty()); + assert!(config.consumer.is_some()); + assert!(config.producer.is_some()); + + if let Some(producer) = config.producer { + assert_eq!(producer.acks, "all"); + assert!(producer.enable_idempotence); + assert_eq!(producer.compression_type, "snappy"); + } + } + + #[test] + fn test_kafka_config_builder() { + let config = KafkaConfig::builder() + .bootstrap_servers("kafka1:9092,kafka2:9092") + .topic(vec!["test-topic".to_string()]) + .client_id("test-client") + .build() + .unwrap(); + + assert_eq!(config.bootstrap_servers, "kafka1:9092,kafka2:9092"); + assert_eq!(config.topics.first().unwrap(), "test-topic"); + assert_eq!(config.client_id, Some("test-client".to_string())); + } +} diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs new file mode 100644 index 000000000..529bf40ee --- /dev/null +++ b/src/connectors/kafka/consumer.rs @@ -0,0 +1,169 @@ +use crate::connectors::common::shutdown::Shutdown; +use crate::connectors::kafka::partition_stream_queue::PartitionStreamReceiver; +use crate::connectors::kafka::state::StreamState; +use crate::connectors::kafka::{ + partition_stream_queue, ConsumerRecord, KafkaContext, StreamConsumer, TopicPartition, +}; +use futures_util::FutureExt; +use rdkafka::consumer::Consumer; +use rdkafka::Statistics; +use std::sync::Arc; +use tokio::sync::{mpsc, RwLock}; +use tokio_stream::wrappers::ReceiverStream; +use tracing::{error, info}; + +pub struct KafkaStreams { + consumer: Arc, + stream_state: Arc>, + statistics: Arc>, + shutdown_handle: Shutdown, +} + +impl KafkaStreams { + pub fn init( + context: KafkaContext, + stream_state: Arc>, + shutdown_handle: Shutdown, + ) -> anyhow::Result { + info!("Initializing KafkaStreams..."); + let consumer = KafkaStreams::create_consumer(context); + let statistics = Arc::new(std::sync::RwLock::new(Statistics::default())); + info!("KafkaStreams initialized successfully."); + + Ok(Self { + consumer, + stream_state, + statistics, + shutdown_handle, + }) + } + + pub fn consumer(&self) -> Arc { + Arc::clone(&self.consumer) + } + + pub fn statistics(&self) -> Arc> { + Arc::clone(&self.statistics) + } + + pub fn state(&self) -> Arc> { + Arc::clone(&self.stream_state) + } + + /// Manages Kafka partition streams manually due to limitations in `rust-rdkafka`'s `split_partition_queue`. + /// + /// This method continuously listens incoming Kafka messages, dynamically creating + /// or updating streams for each partition. It is implemented using a separate standard thread to avoid + /// potential deadlocks and long-running task issues encountered with `tokio::spawn`. + /// + /// Steps: + /// 1. Consumes Kafka messages in a loop, processes each message to identify the associated partition. + /// 2. Dynamically creates a new stream for untracked partitions, allowing for isolated processing. + /// 3. Updates existing streams when new messages arrive for already tracked partitions. + /// 4. Listens for shutdown signals and gracefully terminates all partition streams, unsubscribing the consumer. + /// + /// Limitations and References: + /// - Issues with `split_partition_queue` in rust-rdkafka: + /// - https://github.com/fede1024/rust-rdkafka/issues/535 + /// - https://github.com/confluentinc/librdkafka/issues/4059 + /// - https://github.com/fede1024/rust-rdkafka/issues/535 + /// - https://github.com/confluentinc/librdkafka/issues/4059 + /// - https://github.com/fede1024/rust-rdkafka/issues/654 + /// - https://github.com/fede1024/rust-rdkafka/issues/651 + /// - https://github.com/fede1024/rust-rdkafka/issues/604 + /// - https://github.com/fede1024/rust-rdkafka/issues/564 + /// + /// - Potential deadlocks and long-running task issues with `tokio::spawn`: + /// - Details on blocking vs. async design choices: + /// - https://ryhl.io/blog/async-what-is-blocking/ + /// + /// Returns: + /// A `ReceiverStream` that produces `PartitionStreamReceiver` for each active partition. + pub fn partitioned(&self) -> ReceiverStream { + let (stream_tx, stream_rx) = mpsc::channel(100); + let consumer = self.consumer(); + let stream_state = self.state(); + let tokio_handle = tokio::runtime::Handle::current(); + let shutdown_handle = self.shutdown_handle.clone(); + + std::thread::spawn(move || { + tokio_handle.block_on(async move { + loop { + tokio::select! { + result = consumer.recv() => { + match result { + Ok(msg) => { + let mut state = stream_state.write().await; + let tp = TopicPartition::from_kafka_msg(&msg); + let consumer_record = ConsumerRecord::from_borrowed_msg(msg); + let ps_tx = match state.get_partition_sender(&tp) { + Some(ps_tx) => ps_tx.clone(), + None => { + info!("Creating new stream for {:?}", tp); + let (ps_tx, ps_rx) = partition_stream_queue::bounded(10_000, tp.clone()); + state.insert_partition_sender(tp.clone(), ps_tx.clone()); + stream_tx.send(ps_rx).await.unwrap(); + ps_tx + } + }; + ps_tx.send(consumer_record).await; + } + Err(err) => { + error!("Cannot get message from kafka consumer! Cause {:?}", err); + break + }, + }; + }, + _ = shutdown_handle.recv() => { + info!("Gracefully stopping kafka partition streams!"); + let mut stream_state = stream_state.write().await; + stream_state.clear(); + consumer.unsubscribe(); + break; + }, + else => { + error!("KafkaStreams terminated!"); + break; + } + } + } + }) + }); + + ReceiverStream::new(stream_rx) + } + + fn create_consumer(context: KafkaContext) -> Arc { + info!("Creating Kafka consumer from configs {:#?}", context.config); + + let kafka_config = &context.config; + let consumer_config = kafka_config.consumer_config(); + info!("Consumer configs: {:#?}", &consumer_config); + + let consumer: StreamConsumer = consumer_config + .create_with_context(context.clone()) + .expect("Consumer creation failed"); + + if consumer.recv().now_or_never().is_some() { + panic!("Consumer should not have any messages"); + } + + let consumer = Arc::new(consumer); + + let topics = &kafka_config.topics(); + KafkaStreams::subscribe(&consumer, topics); + + consumer + } + + fn subscribe(consumer: &Arc, topics: &Vec<&str>) { + match consumer.subscribe(topics) { + Ok(_) => { + info!("Subscribed to topics: {:?}", topics); + } + Err(e) => { + error!("Error subscribing to topics: {:?} {:?}", topics, e); + } + }; + } +} diff --git a/src/connectors/kafka/metrics.rs b/src/connectors/kafka/metrics.rs new file mode 100644 index 000000000..3a7ea7e24 --- /dev/null +++ b/src/connectors/kafka/metrics.rs @@ -0,0 +1,37 @@ +use prometheus::core::{Collector, Desc}; +use prometheus::proto::MetricFamily; +use rdkafka::Statistics; +use std::sync::{Arc, RwLock}; + +#[derive(Debug)] +pub struct KafkaConsumerMetricsCollector { + stats: Arc>, +} + +impl KafkaConsumerMetricsCollector { + pub fn new(stats: Arc>) -> Self { + Self { stats } + } + + pub fn statistics(&self) -> Result { + match self.stats.read() { + Ok(stats) => Ok(stats.clone()), + Err(err) => Err(format!( + "Cannot get kafka statistics from RwLock. Error: {}", + err + )), + } + } +} + +impl Collector for KafkaConsumerMetricsCollector { + fn desc(&self) -> Vec<&Desc> { + //TODO: + vec![] + } + + fn collect(&self) -> Vec { + //TODO: encode metrics + vec![] + } +} diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs new file mode 100644 index 000000000..132d8dc98 --- /dev/null +++ b/src/connectors/kafka/mod.rs @@ -0,0 +1,231 @@ +use crate::connectors::kafka::config::KafkaConfig; +use derive_more::Constructor; +use rdkafka::consumer::{ConsumerContext, Rebalance}; +use rdkafka::error::KafkaResult; +use rdkafka::message::{BorrowedMessage, Headers}; +use rdkafka::producer::ProducerContext; +use rdkafka::topic_partition_list::TopicPartitionListElem; +use rdkafka::{ClientContext, Message, Offset, Statistics}; +use serde::{Deserialize, Serialize}; +use std::collections::HashMap; +use std::sync::{Arc, RwLock}; +use tokio::sync::mpsc; +use tokio::sync::mpsc::Receiver; +use tracing::{error, info, warn}; + +pub mod config; +pub mod consumer; +pub mod metrics; +mod partition_stream_queue; +pub mod processor; +pub mod rebalance_listener; +pub mod sink; +pub mod state; +#[allow(dead_code)] +type BaseConsumer = rdkafka::consumer::BaseConsumer; +#[allow(dead_code)] +type FutureProducer = rdkafka::producer::FutureProducer; +type StreamConsumer = rdkafka::consumer::StreamConsumer; + +#[derive(Clone, Debug)] +pub struct KafkaContext { + config: Arc, + statistics: Arc>, + rebalance_tx: mpsc::Sender, +} + +impl KafkaContext { + pub fn new(config: Arc) -> (Self, Receiver) { + let (rebalance_tx, rebalance_rx) = mpsc::channel(10); + let statistics = Arc::new(RwLock::new(Statistics::default())); + ( + Self { + config, + statistics, + rebalance_tx, + }, + rebalance_rx, + ) + } + + pub fn notify(&self, rebalance_event: RebalanceEvent) { + let rebalance_sender = self.rebalance_tx.clone(); + std::thread::spawn(move || { + info!("Sending RebalanceEvent to listener..."); + if rebalance_sender.blocking_send(rebalance_event).is_err() { + warn!("Rebalance event receiver is closed!"); + } + info!("Sent RebalanceEvent to lister."); + }); + } + + pub fn config(&self) -> Arc { + Arc::clone(&self.config) + } +} + +#[derive(Debug, Clone)] +pub enum RebalanceEvent { + Assign(TopicPartitionList), + Revoke(TopicPartitionList, std::sync::mpsc::Sender<()>), +} + +impl RebalanceEvent { + pub fn get_assignment(&self) -> &TopicPartitionList { + match self { + RebalanceEvent::Assign(tpl) => tpl, + RebalanceEvent::Revoke(tpl, _) => tpl, + } + } +} + +#[derive(Constructor, Serialize, Deserialize, Debug, Default, Clone, PartialEq, Eq, Hash)] +pub struct TopicPartition { + pub topic: String, + pub partition: i32, +} + +impl TopicPartition { + pub fn from_kafka_msg(msg: &BorrowedMessage) -> Self { + Self::new(msg.topic().to_owned(), msg.partition()) + } + + pub fn from_tp_elem(elem: &TopicPartitionListElem<'_>) -> Self { + Self::new(elem.topic().to_owned(), elem.partition()) + } +} + +#[derive(Constructor, Serialize, Deserialize, Debug, Default, Clone, PartialEq, Eq, Hash)] +pub struct TopicPartitionList { + pub tpl: Vec, +} + +impl TopicPartitionList { + pub fn from_rdkafka_tpl(tpl: &rdkafka::topic_partition_list::TopicPartitionList) -> Self { + let elements = tpl.elements(); + let mut tp_vec = Vec::with_capacity(elements.len()); + for ref element in elements { + let tp = TopicPartition::from_tp_elem(element); + tp_vec.push(tp); + } + Self::new(tp_vec) + } + + pub fn is_empty(&self) -> bool { + self.tpl.is_empty() + } +} + +#[derive(Constructor, Debug, Hash, Eq, PartialEq)] +pub struct ConsumerRecord { + pub payload: Option>, + pub key: Option>, + pub topic: String, + pub partition: i32, + pub offset: i64, + pub timestamp: Option, + //pub headers: Option>>, +} + +impl ConsumerRecord { + pub fn from_borrowed_msg(msg: BorrowedMessage) -> Self { + Self { + key: msg.key().map(|k| k.to_vec()), + payload: msg.payload().map(|p| p.to_vec()), + topic: msg.topic().to_owned(), + partition: msg.partition(), + offset: msg.offset(), + timestamp: msg.timestamp().to_millis(), + //headers: extract_headers(&msg), + } + } + + pub fn key_str(&self) -> String { + self.key + .clone() + .map(|k| String::from_utf8_lossy(k.as_ref()).to_string()) + .unwrap_or_else(|| String::from("null")) + } + + pub fn offset_to_commit(&self) -> KafkaResult { + let mut offset_to_commit = rdkafka::TopicPartitionList::new(); + offset_to_commit.add_partition_offset( + &self.topic, + self.partition, + Offset::Offset(self.offset + 1), + )?; + Ok(offset_to_commit) + } +} + +#[allow(unused)] +fn extract_headers(msg: &BorrowedMessage<'_>) -> Option>> { + msg.headers().map(|headers| { + headers + .iter() + .map(|header| { + ( + header.key.to_string(), + header.value.map(|v| String::from_utf8_lossy(v).to_string()), + ) + }) + .collect() + }) +} + +impl ConsumerContext for KafkaContext { + fn pre_rebalance( + &self, + _base_consumer: &rdkafka::consumer::BaseConsumer, + rebalance: &Rebalance<'_>, + ) { + info!("Running pre-rebalance with {:?}", rebalance); + match rebalance { + Rebalance::Revoke(tpl) => { + let (pq_waiter_tx, pq_waiter_rx) = std::sync::mpsc::channel(); + if pq_waiter_rx.recv().is_err() { + warn!("Queue termination sender dropped"); + } + let tpl = TopicPartitionList::from_rdkafka_tpl(tpl); + self.notify(RebalanceEvent::Revoke(tpl, pq_waiter_tx)); + } + Rebalance::Assign(tpl) => { + let tpl = TopicPartitionList::from_rdkafka_tpl(tpl); + self.notify(RebalanceEvent::Assign(tpl)); + } + + Rebalance::Error(err) => error!("Error occurred during rebalance {:?}", err), + }; + } + + fn post_rebalance( + &self, + _base_consumer: &rdkafka::consumer::BaseConsumer, + rebalance: &Rebalance<'_>, + ) { + info!("Running post-rebalance with {:?}", rebalance); + } +} + +impl ProducerContext for KafkaContext { + type DeliveryOpaque = (); + fn delivery( + &self, + _delivery_result: &rdkafka::message::DeliveryResult<'_>, + _delivery_opaque: Self::DeliveryOpaque, + ) { + } +} + +impl ClientContext for KafkaContext { + fn stats(&self, new_stats: Statistics) { + match self.statistics.write() { + Ok(mut stats) => { + *stats = new_stats; + } + Err(e) => { + error!("Cannot write to kafka statistics from RwLock. Error: {}", e) + } + }; + } +} diff --git a/src/connectors/kafka/partition_stream_queue.rs b/src/connectors/kafka/partition_stream_queue.rs new file mode 100644 index 000000000..732ede282 --- /dev/null +++ b/src/connectors/kafka/partition_stream_queue.rs @@ -0,0 +1,90 @@ +use crate::connectors::kafka::{ConsumerRecord, TopicPartition}; +use std::sync::Arc; +use tokio::sync::{mpsc, Notify}; +use tokio_stream::wrappers::ReceiverStream; +use tracing::info; + +#[derive(Clone)] +pub struct PartitionStreamSender { + inner: mpsc::Sender, + notify: Arc, +} + +impl PartitionStreamSender { + fn new(inner: mpsc::Sender, notify: Arc) -> Self { + Self { inner, notify } + } + + pub fn terminate(&self) { + self.notify.notify_waiters(); + } + + pub async fn send(&self, consumer_record: ConsumerRecord) { + self.inner.send(consumer_record).await.unwrap(); + } + + pub fn sender(&self) -> mpsc::Sender { + self.inner.clone() + } +} + +pub struct PartitionStreamReceiver { + inner: ReceiverStream, + topic_partition: TopicPartition, + notify: Arc, +} + +impl PartitionStreamReceiver { + fn new( + receiver: mpsc::Receiver, + topic_partition: TopicPartition, + notify: Arc, + ) -> Self { + Self { + inner: ReceiverStream::new(receiver), + topic_partition, + notify, + } + } + + /// Processes the stream with a provided callback and listens for termination. + /// + /// # Parameters + /// - `invoke`: A callback function that processes the `ReceiverStream`. + /// + /// # Behavior + /// - The callback runs until either the stream is completed or a termination signal is received. + pub async fn run_drain(self, f: F) + where + F: Fn(ReceiverStream) -> Fut, + Fut: futures_util::Future, + { + let notify = self.notify.clone(); + + tokio::select! { + _ = f(self.inner) => { + info!("PartitionStreamReceiver completed processing for {:?}.", self.topic_partition); + } + _ = notify.notified() => { + info!("Received termination signal for {:?}.", self.topic_partition); + } + } + } + + pub fn topic_partition(&self) -> &TopicPartition { + &self.topic_partition + } +} + +pub fn bounded( + size: usize, + topic_partition: TopicPartition, +) -> (PartitionStreamSender, PartitionStreamReceiver) { + let (tx, rx) = mpsc::channel(size); + let notify = Arc::new(Notify::new()); + + let sender = PartitionStreamSender::new(tx, notify.clone()); + let receiver = PartitionStreamReceiver::new(rx, topic_partition, notify); + + (sender, receiver) +} diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs new file mode 100644 index 000000000..129ff7cd9 --- /dev/null +++ b/src/connectors/kafka/processor.rs @@ -0,0 +1,154 @@ +use crate::connectors::common::processor::Processor; +use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; +use crate::event::format; +use crate::event::format::EventFormat; +use crate::handlers::http::ingest::create_stream_if_not_exists; +use crate::metadata::STREAM_INFO; +use crate::storage::StreamType; +use async_trait::async_trait; +use chrono::Utc; +use futures_util::StreamExt; +use rdkafka::consumer::{CommitMode, Consumer}; +use serde_json::Value; +use std::collections::HashMap; +use std::sync::Arc; +use std::time::Duration; +use tokio_stream::wrappers::ReceiverStream; +use tracing::{debug, warn}; +use tracing::{error, info}; + +#[derive(Default, Debug, Clone)] +pub struct ParseableSinkProcessor; + +impl ParseableSinkProcessor { + async fn deserialize( + &self, + consumer_record: &ConsumerRecord, + ) -> anyhow::Result> { + let stream_name = consumer_record.topic.as_str(); + + create_stream_if_not_exists(stream_name, &StreamType::UserDefined.to_string()).await?; + let schema = STREAM_INFO.schema_raw(stream_name)?; + + match &consumer_record.payload { + None => { + warn!( + "Skipping tombstone or empty payload in partition {} key {}", + consumer_record.partition, + consumer_record.key_str() + ); + Ok(None) + } + Some(payload) => { + let data: Value = serde_json::from_slice(payload.as_ref())?; + + let event = format::json::Event { + data, + tags: String::default(), + metadata: String::default(), + }; + + // TODO: Implement a buffer (e.g., a wrapper around [Box]) to optimize the creation of ParseableEvent by compacting the internal RecordBatch. + let (record_batch, is_first) = event.into_recordbatch(schema, None, None)?; + + let p_event = crate::event::Event { + rb: record_batch, + stream_name: stream_name.to_string(), + origin_format: "json", + origin_size: payload.len() as u64, + is_first_event: is_first, + parsed_timestamp: Utc::now().naive_utc(), + time_partition: None, + custom_partition_values: HashMap::new(), + stream_type: StreamType::UserDefined, + }; + + Ok(Some(p_event)) + } + } + } +} + +#[async_trait] +impl Processor, ()> for ParseableSinkProcessor { + async fn process(&self, records: Vec) -> anyhow::Result<()> { + let len = records.len(); + debug!("Processing {} records", len); + + for cr in records { + if let Some(event) = self.deserialize(&cr).await? { + event.process().await?; + } + } + + debug!("Processed {} records", len); + Ok(()) + } +} + +#[derive(Clone)] +pub struct StreamWorker

+where + P: Processor, ()>, +{ + processor: Arc

, + consumer: Arc, + buffer_size: usize, + buffer_timeout: Duration, +} + +impl

StreamWorker

+where + P: Processor, ()> + Send + Sync + 'static, +{ + pub fn new( + processor: Arc

, + consumer: Arc, + buffer_size: usize, + buffer_timeout: Duration, + ) -> Self { + Self { + processor, + consumer, + buffer_size, + buffer_timeout, + } + } + + pub async fn process_partition( + &self, + tp: TopicPartition, + record_stream: ReceiverStream, + ) -> anyhow::Result<()> { + info!("Started processing stream for {:?}", tp); + let chunked_stream = tokio_stream::StreamExt::chunks_timeout( + record_stream, + self.buffer_size, + self.buffer_timeout, + ); + + chunked_stream + .for_each_concurrent(None, |records| async { + if let Some(last_record) = records.iter().max_by_key(|r| r.offset) { + let tpl = last_record.offset_to_commit().unwrap(); + + if let Err(e) = self.processor.process(records).await { + error!("Failed to process records for {:?}: {:?}", tp, e); + } + + //CommitMode::Async race condition. + //@see https://github.com/confluentinc/librdkafka/issues/4534 + //@see https://github.com/confluentinc/librdkafka/issues/4059 + if let Err(e) = self.consumer.commit(&tpl, CommitMode::Sync) { + error!("Failed to commit offsets for {:?}: {:?}", tp, e); + } + } + }) + .await; + + info!("Finished processing stream for {:?}", tp); + self.processor.post_stream().await?; + + Ok(()) + } +} diff --git a/src/connectors/kafka/rebalance_listener.rs b/src/connectors/kafka/rebalance_listener.rs new file mode 100644 index 000000000..f35a4062f --- /dev/null +++ b/src/connectors/kafka/rebalance_listener.rs @@ -0,0 +1,65 @@ +use crate::connectors::common::shutdown::Shutdown; +use crate::connectors::kafka::state::StreamState; +use crate::connectors::kafka::RebalanceEvent; +use std::sync::Arc; +use tokio::sync::RwLock; +use tokio::{runtime::Handle, sync::mpsc::Receiver}; +use tracing::{info, warn}; + +pub struct RebalanceListener { + rebalance_rx: Receiver, + stream_state: Arc>, + shutdown_handle: Shutdown, +} + +impl RebalanceListener { + pub fn new( + rebalance_rx: Receiver, + stream_state: Arc>, + shutdown_handle: Shutdown, + ) -> Self { + Self { + rebalance_rx, + stream_state, + shutdown_handle, + } + } + + pub fn start(self) { + let mut rebalance_receiver = self.rebalance_rx; + let stream_state = self.stream_state.clone(); + let shutdown_handle = self.shutdown_handle.clone(); + let tokio_runtime_handle = Handle::current(); + + std::thread::spawn(move || { + tokio_runtime_handle.block_on(async move { + loop { + tokio::select! { + rebalance = rebalance_receiver.recv() => { + match rebalance { + Some(RebalanceEvent::Assign(tpl)) => info!("RebalanceEvent Assign: {:?}", tpl), + Some(RebalanceEvent::Revoke(tpl, callback)) => { + info!("RebalanceEvent Revoke: {:?}", tpl); + let mut stream_state = stream_state.write().await; + stream_state.terminate_partition_streams(tpl).await; + if let Err(err) = callback.send(()) { + warn!("Error during sending response to context. Cause: {:?}", err); + } + info!("Finished Rebalance Revoke"); + } + None => { + info!("Rebalance event sender is closed!"); + break + } + } + }, + _ = shutdown_handle.recv() => { + info!("Gracefully stopping rebalance listener!"); + break; + }, + } + } + }) + }); + } +} diff --git a/src/connectors/kafka/sink.rs b/src/connectors/kafka/sink.rs new file mode 100644 index 000000000..e9a57e997 --- /dev/null +++ b/src/connectors/kafka/sink.rs @@ -0,0 +1,68 @@ +use crate::connectors::common::processor::Processor; +use crate::connectors::kafka::consumer::KafkaStreams; +use crate::connectors::kafka::processor::StreamWorker; +use crate::connectors::kafka::ConsumerRecord; +use anyhow::Result; +use futures_util::StreamExt; +use std::sync::Arc; +use tokio::time::Duration; +use tracing::error; + +pub struct KafkaSinkConnector

+where + P: Processor, ()>, +{ + kafka_streams: KafkaStreams, + worker: Arc>, +} + +impl

KafkaSinkConnector

+where + P: Processor, ()> + Send + Sync + 'static, +{ + pub fn new( + kafka_streams: KafkaStreams, + processor: P, + buffer_size: usize, + buffer_timeout: Duration, + ) -> Self { + let worker = Arc::new(StreamWorker::new( + Arc::new(processor), + kafka_streams.consumer(), + buffer_size, + buffer_timeout, + )); + + Self { + kafka_streams, + worker, + } + } + + pub async fn run(self) -> Result<()> { + self.kafka_streams + .partitioned() + .map(|partition_queue| { + let worker = Arc::clone(&self.worker); + let tp = partition_queue.topic_partition().clone(); + tokio::spawn(async move { + partition_queue + .run_drain(|record_stream| async { + worker + .process_partition(tp.clone(), record_stream) + .await + .unwrap(); + }) + .await + }) + }) + .for_each_concurrent(None, |task| async { + if let Err(e) = task.await { + error!("Task failed: {:?}", e); + } + }) + .await; + + Ok(()) + } +} diff --git a/src/connectors/kafka/state.rs b/src/connectors/kafka/state.rs new file mode 100644 index 000000000..471b4dd34 --- /dev/null +++ b/src/connectors/kafka/state.rs @@ -0,0 +1,50 @@ +use crate::connectors::kafka::partition_stream_queue::PartitionStreamSender; +use crate::connectors::kafka::{TopicPartition, TopicPartitionList}; +use std::collections::HashMap; +use tracing::info; + +pub struct StreamState { + partition_senders: HashMap, +} + +impl StreamState { + pub fn new(capacity: usize) -> Self { + Self { + partition_senders: HashMap::with_capacity(capacity), + } + } + + pub fn insert_partition_sender( + &mut self, + tp: TopicPartition, + sender: PartitionStreamSender, + ) -> Option { + self.partition_senders.insert(tp, sender) + } + + pub fn get_partition_sender(&self, tp: &TopicPartition) -> Option<&PartitionStreamSender> { + self.partition_senders.get(tp) + } + + pub async fn terminate_partition_streams(&mut self, tpl: TopicPartitionList) { + info!("Terminating streams: {:?}", tpl); + + for tp in tpl.tpl { + if let Some(sender) = self.partition_senders.remove(&tp) { + info!("Terminating stream for {:?}", tp); + drop(sender.sender()); + sender.terminate(); + info!("Waiting for stream to finish for {:?}", tp); + } else { + info!("Stream already completed for {:?}", tp); + } + } + + info!("All streams terminated!"); + } + + pub fn clear(&mut self) { + info!("Clearing all stream states..."); + self.partition_senders.clear(); + } +} diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs new file mode 100644 index 000000000..6f31175de --- /dev/null +++ b/src/connectors/mod.rs @@ -0,0 +1,85 @@ +use crate::connectors::common::config::ConnectorConfig; +use crate::connectors::common::processor::Processor; +use crate::connectors::common::shutdown::Shutdown; +use crate::connectors::kafka::consumer::KafkaStreams; +use crate::connectors::kafka::metrics::KafkaConsumerMetricsCollector; +use crate::connectors::kafka::processor::ParseableSinkProcessor; +use crate::connectors::kafka::rebalance_listener::RebalanceListener; +use crate::connectors::kafka::sink::KafkaSinkConnector; +use crate::connectors::kafka::state::StreamState; +use crate::connectors::kafka::{ConsumerRecord, KafkaContext}; +use crate::metrics; +use crate::option::{Mode, CONFIG}; +use prometheus::Registry; +use std::sync::Arc; +use tokio::sync::RwLock; +use tracing::{info, warn}; + +pub mod common; +pub mod kafka; + +pub async fn init() -> anyhow::Result<()> { + if matches!(CONFIG.parseable.mode, Mode::Ingest | Mode::All) { + match CONFIG.parseable.connector_config.clone() { + Some(config) => { + let shutdown_handle = Shutdown::new(); + let prometheus = metrics::build_metrics_handler(); + let registry = prometheus.registry.clone(); + let processor = ParseableSinkProcessor; + + tokio::spawn({ + let shutdown_handle = shutdown_handle.clone(); + async move { + shutdown_handle.signal_listener().await; + info!("Connector received shutdown signal!"); + } + }); + + run_kafka2parseable(config, registry, processor, shutdown_handle).await? + } + None => { + warn!("Kafka connector configuration is missing. Skipping Kafka pipeline."); + } + } + } + + Ok(()) +} + +async fn run_kafka2parseable

( + config: ConnectorConfig, + registry: Registry, + processor: P, + shutdown_handle: Shutdown, +) -> anyhow::Result<()> +where + P: Processor, ()> + Send + Sync + 'static, +{ + let kafka_config = Arc::new(config.kafka_config.clone().unwrap_or_default()); + let (kafka_context, rebalance_rx) = KafkaContext::new(kafka_config); + + //TODO: fetch topics metadata from kafka then give dynamic value to StreamState + let stream_state = Arc::new(RwLock::new(StreamState::new(60))); + let rebalance_listener = RebalanceListener::new( + rebalance_rx, + Arc::clone(&stream_state), + shutdown_handle.clone(), + ); + + let kafka_streams = KafkaStreams::init(kafka_context, stream_state, shutdown_handle.clone())?; + + let stats = kafka_streams.statistics(); + registry.register(Box::new(KafkaConsumerMetricsCollector::new(stats)))?; + + let kafka_parseable_sink_connector = KafkaSinkConnector::new( + kafka_streams, + processor, + config.buffer_size, + config.buffer_timeout, + ); + + rebalance_listener.start(); + kafka_parseable_sink_connector.run().await?; + + Ok(()) +} diff --git a/src/lib.rs b/src/lib.rs index 140c32dcc..a19b53365 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -22,6 +22,7 @@ pub mod analytics; pub mod banner; mod catalog; mod cli; +pub mod connectors; mod event; pub mod handlers; pub mod hottier; diff --git a/src/main.rs b/src/main.rs index f9f2e5993..7cfc085f5 100644 --- a/src/main.rs +++ b/src/main.rs @@ -17,18 +17,18 @@ */ use parseable::{ - banner, + banner, connectors, option::{Mode, CONFIG}, rbac, storage, IngestServer, ParseableServer, QueryServer, Server, }; -use tracing_subscriber::EnvFilter; +use tracing::level_filters::LevelFilter; +use tracing_subscriber::layer::SubscriberExt; +use tracing_subscriber::util::SubscriberInitExt; +use tracing_subscriber::{fmt, EnvFilter, Registry}; #[actix_web::main] async fn main() -> anyhow::Result<()> { - tracing_subscriber::fmt() - .with_env_filter(EnvFilter::from_default_env()) - .compact() - .init(); + init_logger(LevelFilter::DEBUG); // these are empty ptrs so mem footprint should be minimal let server: Box = match CONFIG.parseable.mode { @@ -46,7 +46,27 @@ async fn main() -> anyhow::Result<()> { // keep metadata info in mem metadata.set_global(); - server.init().await?; + let parseable_server = server.init(); + let connectors_task = connectors::init(); + + tokio::try_join!(parseable_server, connectors_task)?; Ok(()) } + +pub fn init_logger(default_level: LevelFilter) { + let filter_layer = EnvFilter::try_from_default_env() + .unwrap_or_else(|_| EnvFilter::new(default_level.to_string())); + + let fmt_layer = fmt::layer() + .with_thread_names(true) + .with_thread_ids(true) + .with_line_number(true) + .with_timer(tracing_subscriber::fmt::time::UtcTime::rfc_3339()) + .compact(); + + Registry::default() + .with(filter_layer) + .with(fmt_layer) + .init(); +} diff --git a/src/metadata.rs b/src/metadata.rs index 5447ea796..341fd9aae 100644 --- a/src/metadata.rs +++ b/src/metadata.rs @@ -178,6 +178,20 @@ impl StreamInfo { }) } + pub fn schema_raw( + &self, + stream_name: &str, + ) -> Result>, MetadataError> { + let map = self.read().expect(LOCK_EXPECT); + + let schema = map + .get(stream_name) + .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) + .map(|metadata| metadata.schema.clone())?; + + Ok(schema) + } + pub fn set_alert(&self, stream_name: &str, alerts: Alerts) -> Result<(), MetadataError> { let mut map = self.write().expect(LOCK_EXPECT); map.get_mut(stream_name) From f085a79d01e650b932bf39d5f9e861aaa45fd029 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 04:56:11 +0100 Subject: [PATCH 02/65] fix conflicts --- Cargo.toml | 1 - src/kafka.rs | 279 --------------------------------------------------- src/lib.rs | 1 - 3 files changed, 281 deletions(-) delete mode 100644 src/kafka.rs diff --git a/Cargo.toml b/Cargo.toml index d9e087dcd..df676fb74 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -71,7 +71,6 @@ num_cpus = "1.15" once_cell = "1.17.1" prometheus = { version = "0.13", features = ["process"] } rand = "0.8.5" -rdkafka = {version = "0.36.2", default-features = false, features = ["tokio"]} regex = "1.7.3" relative-path = { version = "1.7", features = ["serde"] } reqwest = { version = "0.11.27", default-features = false, features = [ diff --git a/src/kafka.rs b/src/kafka.rs deleted file mode 100644 index ba740df8a..000000000 --- a/src/kafka.rs +++ /dev/null @@ -1,279 +0,0 @@ -/* - * Parseable Server (C) 2022 - 2024 Parseable, Inc. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -use arrow_schema::Field; -use chrono::Utc; -use futures_util::StreamExt; -use itertools::Itertools; -use rdkafka::config::ClientConfig; -use rdkafka::consumer::stream_consumer::StreamConsumer; -use rdkafka::consumer::Consumer; -use rdkafka::error::{KafkaError as NativeKafkaError, RDKafkaError}; -use rdkafka::message::BorrowedMessage; -use rdkafka::util::Timeout; -use rdkafka::{Message, TopicPartitionList}; -use serde::{Deserialize, Serialize}; -use std::num::ParseIntError; -use std::sync::Arc; -use std::{collections::HashMap, fmt::Debug}; -use tracing::{debug, error, info, warn}; - -use crate::option::CONFIG; -use crate::{ - event::{ - self, - error::EventError, - format::{self, EventFormat}, - }, - handlers::http::ingest::{create_stream_if_not_exists, PostError}, - metadata::{error::stream_info::MetadataError, STREAM_INFO}, - storage::StreamType, -}; - -#[derive(Debug, Deserialize, Serialize, Clone, Copy)] -pub enum SslProtocol { - Plaintext, - Ssl, - SaslPlaintext, - SaslSsl, -} - -#[allow(dead_code)] -#[derive(Debug, thiserror::Error)] -pub enum KafkaError { - #[error("Please set env var {0} (To use Kafka integration env vars P_KAFKA_TOPICS, P_KAFKA_HOST, and P_KAFKA_GROUP are mandatory)")] - NoVarError(&'static str), - - #[error("Kafka error {0}")] - NativeError(#[from] NativeKafkaError), - #[error("RDKafka error {0}")] - RDKError(#[from] RDKafkaError), - - #[error("Error parsing int {1} for environment variable {0}")] - ParseIntError(&'static str, ParseIntError), - #[error("Error parsing duration int {1} for environment variable {0}")] - ParseDurationError(&'static str, ParseIntError), - - #[error("Stream not found: #{0}")] - StreamNotFound(String), - #[error("Post error: #{0}")] - PostError(#[from] PostError), - #[error("Metadata error: #{0}")] - MetadataError(#[from] MetadataError), - #[error("Event error: #{0}")] - EventError(#[from] EventError), - #[error("JSON error: #{0}")] - JsonError(#[from] serde_json::Error), - #[error("Invalid group offset storage: #{0}")] - InvalidGroupOffsetStorage(String), - - #[error("Invalid SSL protocol: #{0}")] - InvalidSslProtocolError(String), - #[error("Invalid unicode for environment variable {0}")] - EnvNotUnicode(&'static str), - #[error("")] - DoNotPrintError, -} - -// // Commented out functions -// // Might come in handy later -// fn parse_auto_env(key: &'static str) -> Result, ::Err> -// where -// T: FromStr, -// { -// Ok(if let Ok(val) = env::var(key) { -// Some(val.parse::()?) -// } else { -// None -// }) -// } - -// fn handle_duration_env_prefix(key: &'static str) -> Result, ParseIntError> { -// if let Ok(raw_secs) = env::var(format!("{key}_S")) { -// Ok(Some(Duration::from_secs(u64::from_str(&raw_secs)?))) -// } else if let Ok(raw_secs) = env::var(format!("{key}_M")) { -// Ok(Some(Duration::from_secs(u64::from_str(&raw_secs)? * 60))) -// } else { -// Ok(None) -// } -// } - -// fn parse_i32_env(key: &'static str) -> Result, KafkaError> { -// parse_auto_env::(key).map_err(|raw| KafkaError::ParseIntError(key, raw)) -// } - -// fn parse_duration_env_prefixed(key_prefix: &'static str) -> Result, KafkaError> { -// handle_duration_env_prefix(key_prefix) -// .map_err(|raw| KafkaError::ParseDurationError(key_prefix, raw)) -// } - -fn setup_consumer() -> Result<(StreamConsumer, Vec), KafkaError> { - if let Some(topics) = &CONFIG.parseable.kafka_topics { - // topics can be a comma separated list of topics to subscribe to - let topics = topics.split(",").map(|v| v.to_owned()).collect_vec(); - - let host = if CONFIG.parseable.kafka_host.is_some() { - CONFIG.parseable.kafka_host.as_ref() - } else { - return Err(KafkaError::NoVarError("P_KAKFA_HOST")); - }; - - let group = if CONFIG.parseable.kafka_group.is_some() { - CONFIG.parseable.kafka_group.as_ref() - } else { - return Err(KafkaError::NoVarError("P_KAKFA_GROUP")); - }; - - let mut conf = ClientConfig::new(); - conf.set("bootstrap.servers", host.unwrap()); - conf.set("group.id", group.unwrap()); - - if let Some(val) = CONFIG.parseable.kafka_client_id.as_ref() { - conf.set("client.id", val); - } - - // if let Some(val) = get_flag_env_val("a")? { - // conf.set("api.version.request", val.to_string()); - // } - - if let Some(ssl_protocol) = CONFIG.parseable.kafka_security_protocol.as_ref() { - conf.set("security.protocol", serde_json::to_string(&ssl_protocol)?); - } - - let consumer: StreamConsumer = conf.create()?; - consumer.subscribe(&topics.iter().map(|v| v.as_str()).collect_vec())?; - - if let Some(vals_raw) = CONFIG.parseable.kafka_partitions.as_ref() { - // partitions is a comma separated pairs of topic:partitions - let mut topic_partition_pairs = Vec::new(); - let mut set = true; - for vals in vals_raw.split(",") { - let intermediate = vals.split(":").collect_vec(); - if intermediate.len() != 2 { - warn!( - "Value for P_KAFKA_PARTITIONS is incorrect! Skipping setting partitions!" - ); - set = false; - break; - } - topic_partition_pairs.push(intermediate); - } - - if set { - let mut parts = TopicPartitionList::new(); - for pair in topic_partition_pairs { - let topic = pair[0]; - match pair[1].parse::() { - Ok(partition) => { - parts.add_partition(topic, partition); - } - Err(_) => warn!("Skipping setting partition for topic- {topic}"), - } - } - consumer.seek_partitions(parts, Timeout::Never)?; - } - } - Ok((consumer, topics.clone())) - } else { - // if the user hasn't even set KAFKA_TOPICS - // then they probably don't want to use the integration - // send back the DoNotPrint error - Err(KafkaError::DoNotPrintError) - } -} - -fn resolve_schema(stream_name: &str) -> Result>, KafkaError> { - let hash_map = STREAM_INFO.read().unwrap(); - let raw = hash_map - .get(stream_name) - .ok_or_else(|| KafkaError::StreamNotFound(stream_name.to_owned()))?; - Ok(raw.schema.clone()) -} - -async fn ingest_message(msg: BorrowedMessage<'_>) -> Result<(), KafkaError> { - let Some(payload) = msg.payload() else { - debug!("No payload received"); - return Ok(()); - }; - - let msg = msg.detach(); - let stream_name = msg.topic(); - - // stream should get created only if there is an incoming event, not before that - create_stream_if_not_exists(stream_name, &StreamType::UserDefined.to_string()).await?; - - let schema = resolve_schema(stream_name)?; - let event = format::json::Event { - data: serde_json::from_slice(payload)?, - tags: String::default(), - metadata: String::default(), - }; - - let time_partition = STREAM_INFO.get_time_partition(stream_name)?; - let static_schema_flag = STREAM_INFO.get_static_schema_flag(stream_name)?; - - let (rb, is_first) = event - .into_recordbatch( - &schema, - static_schema_flag.as_ref(), - time_partition.as_ref(), - ) - .map_err(|err| KafkaError::PostError(PostError::CustomError(err.to_string())))?; - - event::Event { - rb, - stream_name: stream_name.to_string(), - origin_format: "json", - origin_size: payload.len() as u64, - is_first_event: is_first, - parsed_timestamp: Utc::now().naive_utc(), - time_partition: None, - custom_partition_values: HashMap::new(), - stream_type: StreamType::UserDefined, - } - .process() - .await?; - - Ok(()) -} - -pub async fn setup_integration() { - let (consumer, stream_names) = match setup_consumer() { - Ok(c) => c, - Err(err) => { - match err { - KafkaError::DoNotPrintError => { - debug!("P_KAFKA_TOPICS not set, skipping kafka integration"); - } - _ => { - error!("{err}"); - } - } - return; - } - }; - - info!("Setup kafka integration for {stream_names:?}"); - let mut stream = consumer.stream(); - - while let Ok(curr) = stream.next().await.unwrap() { - if let Err(err) = ingest_message(curr).await { - error!("Unable to ingest incoming kafka message- {err}") - } - } -} diff --git a/src/lib.rs b/src/lib.rs index 79ac614e8..bff6e2114 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -26,7 +26,6 @@ pub mod connectors; mod event; pub mod handlers; pub mod hottier; -pub mod kafka; mod livetail; mod metadata; pub mod metrics; From d32eae5d74cc5d1b6aae8209d0d6cb771dbfc845 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 05:56:18 +0100 Subject: [PATCH 03/65] update cli.rs --- src/cli.rs | 124 +++++++----------------------- src/connectors/kafka/processor.rs | 2 +- 2 files changed, 27 insertions(+), 99 deletions(-) diff --git a/src/cli.rs b/src/cli.rs index 9cb1ae840..9323d874a 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -50,15 +50,9 @@ pub struct Cli { pub domain_address: Option, /// The local staging path is used as a temporary landing point - /// for incoming events and local cache + /// for incoming events pub local_staging_path: PathBuf, - /// The local cache path is used for speeding up query on latest data - pub local_cache_path: Option, - - /// Size for local cache - pub local_cache_size: u64, - /// Username for the basic authentication on the server pub username: String, @@ -101,12 +95,6 @@ pub struct Cli { /// port use by airplane(flight query service) pub flight_port: u16, - /// to query cached data - pub query_cache_path: Option, - - /// Size for local cache - pub query_cache_size: u64, - /// CORS behaviour pub cors: bool, @@ -136,10 +124,6 @@ impl Cli { pub const ADDRESS: &'static str = "address"; pub const DOMAIN_URI: &'static str = "origin"; pub const STAGING: &'static str = "local-staging-path"; - pub const CACHE: &'static str = "cache-path"; - pub const QUERY_CACHE: &'static str = "query-cache-path"; - pub const QUERY_CACHE_SIZE: &'static str = "query-cache-size"; - pub const CACHE_SIZE: &'static str = "cache-size"; pub const USERNAME: &'static str = "username"; pub const PASSWORD: &'static str = "password"; pub const CHECK_UPDATE: &'static str = "check-update"; @@ -283,63 +267,25 @@ impl Cli { .value_parser(validation::canonicalize_path) .help("Local path on this device where all trusted certificates are located.") ) - .arg( - Arg::new(Self::ADDRESS) - .long(Self::ADDRESS) - .env("P_ADDR") - .value_name("ADDR:PORT") - .default_value("0.0.0.0:8000") - .value_parser(validation::socket_addr) - .help("Address and port for Parseable HTTP(s) server"), - ) - .arg( - Arg::new(Self::STAGING) - .long(Self::STAGING) - .env("P_STAGING_DIR") - .value_name("DIR") - .default_value("./staging") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used as landing point for incoming events") - .next_line_help(true), - ) - .arg( - Arg::new(Self::CACHE) - .long(Self::CACHE) - .env("P_CACHE_DIR") - .value_name("DIR") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used for caching data") - .next_line_help(true), - ) - .arg( - Arg::new(Self::CACHE_SIZE) - .long(Self::CACHE_SIZE) - .env("P_CACHE_SIZE") - .value_name("size") - .default_value("1GiB") - .value_parser(validation::cache_size) - .help("Maximum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") - .next_line_help(true), - ) - .arg( - Arg::new(Self::QUERY_CACHE) - .long(Self::QUERY_CACHE) - .env("P_QUERY_CACHE_DIR") - .value_name("DIR") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used for caching data") - .next_line_help(true), - ) - .arg( - Arg::new(Self::QUERY_CACHE_SIZE) - .long(Self::QUERY_CACHE_SIZE) - .env("P_QUERY_CACHE_SIZE") - .value_name("size") - .default_value("1GiB") - .value_parser(validation::cache_size) - .help("Maximum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") - .next_line_help(true), - ) + .arg( + Arg::new(Self::ADDRESS) + .long(Self::ADDRESS) + .env("P_ADDR") + .value_name("ADDR:PORT") + .default_value("0.0.0.0:8000") + .value_parser(validation::socket_addr) + .help("Address and port for Parseable HTTP(s) server"), + ) + .arg( + Arg::new(Self::STAGING) + .long(Self::STAGING) + .env("P_STAGING_DIR") + .value_name("DIR") + .default_value("./staging") + .value_parser(validation::canonicalize_path) + .help("Local path on this device to be used as landing point for incoming events") + .next_line_help(true), + ) .arg( Arg::new(Self::USERNAME) .long(Self::USERNAME) @@ -906,8 +852,6 @@ impl FromArgMatches for Cli { self.trino_schema = m.get_one::(Self::TRINO_SCHEMA).cloned(); self.trino_username = m.get_one::(Self::TRINO_USER_NAME).cloned(); - self.local_cache_path = m.get_one::(Self::CACHE).cloned(); - self.query_cache_path = m.get_one::(Self::QUERY_CACHE).cloned(); self.tls_cert_path = m.get_one::(Self::TLS_CERT).cloned(); self.tls_key_path = m.get_one::(Self::TLS_KEY).cloned(); self.trusted_ca_certs_path = m.get_one::(Self::TRUSTED_CA_CERTS_PATH).cloned(); @@ -927,14 +871,6 @@ impl FromArgMatches for Cli { .get_one::(Self::STAGING) .cloned() .expect("default value for staging"); - self.local_cache_size = m - .get_one::(Self::CACHE_SIZE) - .cloned() - .expect("default value for cache size"); - self.query_cache_size = m - .get_one(Self::QUERY_CACHE_SIZE) - .cloned() - .expect("default value for query cache size"); self.username = m .get_one::(Self::USERNAME) .cloned() @@ -977,20 +913,12 @@ impl FromArgMatches for Cli { .get_one::(Self::ROW_GROUP_SIZE) .cloned() .expect("default for row_group size"); - self.parquet_compression = match m - .get_one::(Self::PARQUET_COMPRESSION_ALGO) - .expect("default for compression algo") - .as_str() - { - "uncompressed" => Compression::UNCOMPRESSED, - "snappy" => Compression::SNAPPY, - "gzip" => Compression::GZIP, - "lzo" => Compression::LZO, - "brotli" => Compression::BROTLI, - "lz4" => Compression::LZ4, - "zstd" => Compression::ZSTD, - _ => unreachable!(), - }; + self.parquet_compression = serde_json::from_str(&format!( + "{:?}", + m.get_one::(Self::PARQUET_COMPRESSION_ALGO) + .expect("default for compression algo") + )) + .expect("unexpected compression algo"); let openid_client_id = m.get_one::(Self::OPENID_CLIENT_ID).cloned(); let openid_client_secret = m.get_one::(Self::OPENID_CLIENT_SECRET).cloned(); diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index 129ff7cd9..ebab67938 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -49,7 +49,7 @@ impl ParseableSinkProcessor { }; // TODO: Implement a buffer (e.g., a wrapper around [Box]) to optimize the creation of ParseableEvent by compacting the internal RecordBatch. - let (record_batch, is_first) = event.into_recordbatch(schema, None, None)?; + let (record_batch, is_first) = event.into_recordbatch(&schema, None, None)?; let p_event = crate::event::Event { rb: record_batch, From 693b9c9b19230ec30c3b6a19a10cfb642eafa2e3 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 05:59:57 +0100 Subject: [PATCH 04/65] remove unused fn from metadata.rs --- src/metadata.rs | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/src/metadata.rs b/src/metadata.rs index 76416d5bf..54bb658b8 100644 --- a/src/metadata.rs +++ b/src/metadata.rs @@ -164,20 +164,6 @@ impl StreamInfo { Ok(Arc::new(schema)) } - /// update the schema in the metadata - pub fn set_schema( - &self, - stream_name: &str, - schema: HashMap>, - ) -> Result<(), MetadataError> { - let mut map = self.write().expect(LOCK_EXPECT); - map.get_mut(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| { - metadata.schema = schema; - }) - } - pub fn schema_raw( &self, stream_name: &str, From 3cc6b0e22a1b5fb71192f49d9ce39483905ea17b Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 06:08:15 +0100 Subject: [PATCH 05/65] add Copyright --- src/connectors/common/config.rs | 18 ++++++++++++++++++ src/connectors/common/mod.rs | 18 ++++++++++++++++++ src/connectors/common/processor.rs | 18 ++++++++++++++++++ src/connectors/common/shutdown.rs | 18 ++++++++++++++++++ src/connectors/common/types/mod.rs | 18 ++++++++++++++++++ src/connectors/kafka/config.rs | 18 ++++++++++++++++++ src/connectors/kafka/consumer.rs | 18 ++++++++++++++++++ src/connectors/kafka/metrics.rs | 18 ++++++++++++++++++ src/connectors/kafka/mod.rs | 18 ++++++++++++++++++ src/connectors/kafka/partition_stream_queue.rs | 18 ++++++++++++++++++ src/connectors/kafka/processor.rs | 18 ++++++++++++++++++ src/connectors/kafka/rebalance_listener.rs | 18 ++++++++++++++++++ src/connectors/kafka/sink.rs | 18 ++++++++++++++++++ src/connectors/kafka/state.rs | 18 ++++++++++++++++++ src/connectors/mod.rs | 18 ++++++++++++++++++ 15 files changed, 270 insertions(+) diff --git a/src/connectors/common/config.rs b/src/connectors/common/config.rs index 3aed150a3..d0dd9a484 100644 --- a/src/connectors/common/config.rs +++ b/src/connectors/common/config.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::common::types::ConnectorType; use crate::connectors::common::{BadData, ConnectorError}; use crate::connectors::kafka::config::{KafkaConfig, SourceOffset}; diff --git a/src/connectors/common/mod.rs b/src/connectors/common/mod.rs index b0474ffa5..8600bc0c8 100644 --- a/src/connectors/common/mod.rs +++ b/src/connectors/common/mod.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use serde::{Deserialize, Serialize}; pub mod config; pub mod processor; diff --git a/src/connectors/common/processor.rs b/src/connectors/common/processor.rs index 5f1e07bdc..cce0fe1cc 100644 --- a/src/connectors/common/processor.rs +++ b/src/connectors/common/processor.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use async_trait::async_trait; #[async_trait] diff --git a/src/connectors/common/shutdown.rs b/src/connectors/common/shutdown.rs index ba8f169cb..95a06d505 100644 --- a/src/connectors/common/shutdown.rs +++ b/src/connectors/common/shutdown.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use tokio::sync::mpsc; use tokio_util::sync::CancellationToken; use tracing::{info, warn}; diff --git a/src/connectors/common/types/mod.rs b/src/connectors/common/types/mod.rs index ee2085eab..83bbe0a45 100644 --- a/src/connectors/common/types/mod.rs +++ b/src/connectors/common/types/mod.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use serde::{Deserialize, Serialize}; use std::fmt; use std::fmt::{Display, Formatter}; diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index 367024372..5e79f32a2 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use anyhow::bail; use rdkafka::Offset; use rustls::pki_types::{CertificateDer, PrivateKeyDer}; diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs index 529bf40ee..bb443bb02 100644 --- a/src/connectors/kafka/consumer.rs +++ b/src/connectors/kafka/consumer.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::common::shutdown::Shutdown; use crate::connectors::kafka::partition_stream_queue::PartitionStreamReceiver; use crate::connectors::kafka::state::StreamState; diff --git a/src/connectors/kafka/metrics.rs b/src/connectors/kafka/metrics.rs index 3a7ea7e24..0a70bcc87 100644 --- a/src/connectors/kafka/metrics.rs +++ b/src/connectors/kafka/metrics.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use prometheus::core::{Collector, Desc}; use prometheus::proto::MetricFamily; use rdkafka::Statistics; diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs index 132d8dc98..3c14dd4bb 100644 --- a/src/connectors/kafka/mod.rs +++ b/src/connectors/kafka/mod.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::kafka::config::KafkaConfig; use derive_more::Constructor; use rdkafka::consumer::{ConsumerContext, Rebalance}; diff --git a/src/connectors/kafka/partition_stream_queue.rs b/src/connectors/kafka/partition_stream_queue.rs index 732ede282..e99cd649e 100644 --- a/src/connectors/kafka/partition_stream_queue.rs +++ b/src/connectors/kafka/partition_stream_queue.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::kafka::{ConsumerRecord, TopicPartition}; use std::sync::Arc; use tokio::sync::{mpsc, Notify}; diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index ebab67938..9ec0ed9f3 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::common::processor::Processor; use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; use crate::event::format; diff --git a/src/connectors/kafka/rebalance_listener.rs b/src/connectors/kafka/rebalance_listener.rs index f35a4062f..40a452b99 100644 --- a/src/connectors/kafka/rebalance_listener.rs +++ b/src/connectors/kafka/rebalance_listener.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::common::shutdown::Shutdown; use crate::connectors::kafka::state::StreamState; use crate::connectors::kafka::RebalanceEvent; diff --git a/src/connectors/kafka/sink.rs b/src/connectors/kafka/sink.rs index e9a57e997..e257c4abf 100644 --- a/src/connectors/kafka/sink.rs +++ b/src/connectors/kafka/sink.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::common::processor::Processor; use crate::connectors::kafka::consumer::KafkaStreams; use crate::connectors::kafka::processor::StreamWorker; diff --git a/src/connectors/kafka/state.rs b/src/connectors/kafka/state.rs index 471b4dd34..cc91bc9d8 100644 --- a/src/connectors/kafka/state.rs +++ b/src/connectors/kafka/state.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::kafka::partition_stream_queue::PartitionStreamSender; use crate::connectors::kafka::{TopicPartition, TopicPartitionList}; use std::collections::HashMap; diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 6f31175de..1507bbba2 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::common::config::ConnectorConfig; use crate::connectors::common::processor::Processor; use crate::connectors::common::shutdown::Shutdown; From 12f035863678c481d4d5bb6fd40df66f306d9a8a Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 06:22:44 +0100 Subject: [PATCH 06/65] fix deepsource issues --- src/cli.rs | 6 +++--- src/connectors/common/config.rs | 3 ++- src/connectors/common/mod.rs | 15 --------------- src/connectors/common/shutdown.rs | 6 +++--- src/connectors/common/types/mod.rs | 9 +++++++++ src/connectors/kafka/mod.rs | 8 ++++---- src/connectors/mod.rs | 2 +- 7 files changed, 22 insertions(+), 27 deletions(-) diff --git a/src/cli.rs b/src/cli.rs index 9323d874a..89ca3bf92 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -23,8 +23,8 @@ use tracing::warn; use url::Url; use crate::connectors::common::config::ConnectorConfig; +use crate::connectors::common::types::BadData; use crate::connectors::common::types::ConnectorType; -use crate::connectors::common::BadData; use crate::connectors::kafka::config::{ConsumerConfig, KafkaConfig, SourceOffset}; use crate::{ oidc::{self, OpenidConfig}, @@ -738,7 +738,7 @@ impl FromArgMatches for Cli { let group_id = m .get_one::(Cli::KAFKA_GROUP_ID) .cloned() - .unwrap_or("parseable-default-group".to_string()); + .unwrap_or_else(|| "parseable-default-group".to_string()); if topics.is_empty() { return Err(clap::Error::raw( @@ -807,7 +807,7 @@ impl FromArgMatches for Cli { .unwrap_or_else(|| "earliest".to_string()); let mut consumer = ConsumerConfig::default(); - consumer.group_id = group_id.clone(); + consumer.group_id = group_id; consumer.auto_offset_reset = auto_offset_reset; let topics: Vec = topics.split(",").map(|t| t.to_owned()).collect(); diff --git a/src/connectors/common/config.rs b/src/connectors/common/config.rs index d0dd9a484..4cabe12b1 100644 --- a/src/connectors/common/config.rs +++ b/src/connectors/common/config.rs @@ -16,8 +16,9 @@ * */ +use crate::connectors::common::types::BadData; use crate::connectors::common::types::ConnectorType; -use crate::connectors::common::{BadData, ConnectorError}; +use crate::connectors::common::ConnectorError; use crate::connectors::kafka::config::{KafkaConfig, SourceOffset}; use serde::{Deserialize, Serialize}; use std::{time::Duration, vec}; diff --git a/src/connectors/common/mod.rs b/src/connectors/common/mod.rs index 8600bc0c8..ac995c5a1 100644 --- a/src/connectors/common/mod.rs +++ b/src/connectors/common/mod.rs @@ -16,7 +16,6 @@ * */ -use serde::{Deserialize, Serialize}; pub mod config; pub mod processor; pub mod shutdown; @@ -33,17 +32,3 @@ pub enum ConnectorError { #[error("Initialization error: {0}")] Init(String), } - -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq, Hash, PartialOrd)] -#[serde(rename_all = "snake_case")] -pub enum BadData { - Fail {}, - Drop {}, - Dlt {}, -} - -impl Default for BadData { - fn default() -> Self { - BadData::Drop {} - } -} diff --git a/src/connectors/common/shutdown.rs b/src/connectors/common/shutdown.rs index 95a06d505..ce3cb5084 100644 --- a/src/connectors/common/shutdown.rs +++ b/src/connectors/common/shutdown.rs @@ -72,7 +72,7 @@ impl Shutdown { impl Default for Shutdown { fn default() -> Self { - Self::new() + Shutdown::new() } } @@ -95,7 +95,7 @@ mod tests { #[tokio::test] async fn test_shutdown_recv() { - let shutdown = Shutdown::new(); + let shutdown = Shutdown::default(); let shutdown_clone = shutdown.clone(); // receive shutdown task let task = tokio::spawn(async move { @@ -117,7 +117,7 @@ mod tests { #[tokio::test] async fn test_shutdown_wait_for_complete() { - let shutdown = Shutdown::new(); + let shutdown = Shutdown::default(); let shutdown_clone = shutdown.clone(); let check_value: Arc> = Arc::new(Mutex::new(false)); let check_value_clone = Arc::clone(&check_value); diff --git a/src/connectors/common/types/mod.rs b/src/connectors/common/types/mod.rs index 83bbe0a45..7ff6fe4b1 100644 --- a/src/connectors/common/types/mod.rs +++ b/src/connectors/common/types/mod.rs @@ -61,3 +61,12 @@ impl TryFrom for ConnectionType { } } } + +#[derive(Default, Serialize, Deserialize, Clone, Debug, PartialEq, Eq, Hash, PartialOrd)] +#[serde(rename_all = "snake_case")] +pub enum BadData { + Fail, + #[default] + Drop, + Dlt, // TODO: Implement DLT +} diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs index 3c14dd4bb..a7033ef84 100644 --- a/src/connectors/kafka/mod.rs +++ b/src/connectors/kafka/mod.rs @@ -159,10 +159,10 @@ impl ConsumerRecord { } pub fn key_str(&self) -> String { - self.key - .clone() - .map(|k| String::from_utf8_lossy(k.as_ref()).to_string()) - .unwrap_or_else(|| String::from("null")) + self.key.clone().map_or_else( + || String::from("null"), + |k| String::from_utf8_lossy(k.as_ref()).to_string(), + ) } pub fn offset_to_commit(&self) -> KafkaResult { diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 1507bbba2..e2d7f11fb 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -40,7 +40,7 @@ pub async fn init() -> anyhow::Result<()> { if matches!(CONFIG.parseable.mode, Mode::Ingest | Mode::All) { match CONFIG.parseable.connector_config.clone() { Some(config) => { - let shutdown_handle = Shutdown::new(); + let shutdown_handle = Shutdown::default(); let prometheus = metrics::build_metrics_handler(); let registry = prometheus.registry.clone(); let processor = ParseableSinkProcessor; From 0f6ca53545b1c956c817a55a3fb7e44b95d4989a Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 06:25:09 +0100 Subject: [PATCH 07/65] fix deepsource issues on shutdown.rs --- src/connectors/common/shutdown.rs | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/src/connectors/common/shutdown.rs b/src/connectors/common/shutdown.rs index ce3cb5084..03517a326 100644 --- a/src/connectors/common/shutdown.rs +++ b/src/connectors/common/shutdown.rs @@ -28,16 +28,6 @@ pub struct Shutdown { } impl Shutdown { - pub fn new() -> Self { - let cancel_token = CancellationToken::new(); - let (shutdown_complete_tx, shutdown_complete_rx) = mpsc::channel(1); - Self { - cancel_token, - shutdown_complete_tx, - shutdown_complete_rx: Some(shutdown_complete_rx), - } - } - pub fn start(&self) { self.cancel_token.cancel(); } @@ -72,7 +62,13 @@ impl Shutdown { impl Default for Shutdown { fn default() -> Self { - Shutdown::new() + let cancel_token = CancellationToken::new(); + let (shutdown_complete_tx, shutdown_complete_rx) = mpsc::channel(1); + Self { + cancel_token, + shutdown_complete_tx, + shutdown_complete_rx: Some(shutdown_complete_rx), + } } } From 58cc468594cafb8449504f803b73326dfca97471 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 21 Dec 2024 11:58:20 +0100 Subject: [PATCH 08/65] Add .idea to .gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 57ea8e65e..af88a4ac0 100644 --- a/.gitignore +++ b/.gitignore @@ -14,3 +14,4 @@ parseable parseable_* parseable-env-secret cache +.idea From aff48a202dd2c161bfc1abfcddebb0802baf2ba7 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sun, 22 Dec 2024 10:21:15 +0100 Subject: [PATCH 09/65] Add Kafka cluster setup to docker-compose files and refactor Dockerfile for rdkafka dependencies. Implement retrying for consumer.rcv() fn to handle temporary Kafka unavailability. --- Cargo.lock | 162 ++++++++++------ Cargo.toml | 7 +- Dockerfile | 20 +- docker-compose-distributed-test.yaml | 161 +++++++++++++++- docker-compose-local.yaml | 41 ++++ docker-compose-test.yaml | 178 ++++++++++++++++-- scripts/Dockerfile | 28 +++ scripts/kafka_log_stream_generator.py | 161 +++++++++++----- src/connectors/kafka/config.rs | 10 +- src/connectors/kafka/consumer.rs | 164 ++++++++++++---- src/connectors/kafka/mod.rs | 2 +- ...on_stream_queue.rs => partition_stream.rs} | 0 src/connectors/kafka/processor.rs | 5 +- src/connectors/kafka/sink.rs | 8 +- src/connectors/kafka/state.rs | 2 +- 15 files changed, 767 insertions(+), 182 deletions(-) create mode 100644 docker-compose-local.yaml create mode 100644 scripts/Dockerfile rename src/connectors/kafka/{partition_stream_queue.rs => partition_stream.rs} (100%) diff --git a/Cargo.lock b/Cargo.lock index 15ca371a8..feaacee8f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -81,7 +81,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e01ed3140b2f8d422c68afa1ed2e85d996ea619c988ac834d255db32138655cb" dependencies = [ "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -218,7 +218,7 @@ dependencies = [ "actix-router", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -248,7 +248,7 @@ dependencies = [ "pin-project", "prometheus", "quanta", - "thiserror", + "thiserror 1.0.64", ] [[package]] @@ -712,7 +712,7 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -723,7 +723,7 @@ checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -794,6 +794,17 @@ dependencies = [ "tower-service", ] +[[package]] +name = "backon" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba5289ec98f68f28dd809fd601059e6aa908bb8f6108620930828283d4ee23d7" +dependencies = [ + "fastrand 2.0.2", + "gloo-timers", + "tokio", +] + [[package]] name = "backtrace" version = "0.3.71" @@ -923,7 +934,7 @@ dependencies = [ "serde_json", "serde_repr", "serde_urlencoded", - "thiserror", + "thiserror 1.0.64", "tokio", "tokio-util", "tower-service", @@ -1040,7 +1051,7 @@ dependencies = [ "semver", "serde", "serde_json", - "thiserror", + "thiserror 1.0.64", ] [[package]] @@ -1156,7 +1167,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -1265,9 +1276,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.6" +version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" +checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" [[package]] name = "cpufeatures" @@ -1395,7 +1406,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -1406,7 +1417,7 @@ checksum = "a668eda54683121533a393014d8692171709ff57a7d61f187b6e782719f8933f" dependencies = [ "darling_core", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -1836,7 +1847,7 @@ checksum = "67e77553c4162a157adbf834ebae5b415acbecbeafc7a74b0e886657506a7611" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -1849,7 +1860,7 @@ dependencies = [ "proc-macro2", "quote", "rustc_version", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -1871,7 +1882,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -2084,7 +2095,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -2158,6 +2169,18 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" +[[package]] +name = "gloo-timers" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbb143cf96099802033e0d4f4963b19fd2e0b728bcf076cd9cf7f6634f092994" +dependencies = [ + "futures-channel", + "futures-core", + "js-sys", + "wasm-bindgen", +] + [[package]] name = "h2" version = "0.3.26" @@ -2748,9 +2771,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.153" +version = "0.2.169" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c198f91728a82281a64e1f4f9eeb25d82cb32a5de251c6bd1b5154d63a8e7bd" +checksum = "b5aba8db14291edd000dfcc4d620c7ebfb122c613afb886ca8803fa4e128a20a" [[package]] name = "libm" @@ -3108,7 +3131,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -3179,7 +3202,7 @@ dependencies = [ "reqwest 0.12.8", "serde", "serde_json", - "thiserror", + "thiserror 1.0.64", "url", "validator", ] @@ -3314,7 +3337,7 @@ dependencies = [ "regex", "regex-syntax 0.8.5", "structmeta", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -3345,6 +3368,7 @@ dependencies = [ "arrow-schema", "arrow-select", "async-trait", + "backon", "base64 0.22.0", "byteorder", "bytes", @@ -3402,7 +3426,7 @@ dependencies = [ "sysinfo", "testcontainers", "testcontainers-modules", - "thiserror", + "thiserror 2.0.9", "thread-priority", "tokio", "tokio-stream", @@ -3531,7 +3555,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -3571,7 +3595,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5ac2cf0f2e4f42b49f5ffd07dae8d746508ef7526c13940e5f524012ae6c6550" dependencies = [ "proc-macro2", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -3609,9 +3633,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.86" +version = "1.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77" +checksum = "37d3544b3f2748c54e147655edb5025752e2303145b5aefb3c3ea2c78b973bb0" dependencies = [ "unicode-ident", ] @@ -3643,7 +3667,7 @@ dependencies = [ "parking_lot", "procfs", "protobuf", - "thiserror", + "thiserror 1.0.64", ] [[package]] @@ -3685,7 +3709,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.79", + "syn 2.0.91", "tempfile", ] @@ -3699,7 +3723,7 @@ dependencies = [ "itertools 0.13.0", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -3756,7 +3780,7 @@ dependencies = [ "rustc-hash", "rustls 0.23.13", "socket2", - "thiserror", + "thiserror 1.0.64", "tokio", "tracing", ] @@ -3773,7 +3797,7 @@ dependencies = [ "rustc-hash", "rustls 0.23.13", "slab", - "thiserror", + "thiserror 1.0.64", "tinyvec", "tracing", ] @@ -3922,9 +3946,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.11.0" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38200e5ee88914975b69f657f0801b6f6dccafd44fd9326302a4aaeecfacb1d8" +checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" dependencies = [ "aho-corasick", "memchr", @@ -4107,7 +4131,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.79", + "syn 2.0.91", "unicode-ident", ] @@ -4385,7 +4409,7 @@ checksum = "e88edab869b01783ba905e7d0153f9fc1a6505a96e4ad3018011eedb838566d9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4408,7 +4432,7 @@ checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4459,7 +4483,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4590,7 +4614,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4633,7 +4657,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4668,7 +4692,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta-derive", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4679,7 +4703,7 @@ checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4701,7 +4725,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -4722,9 +4746,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.79" +version = "2.0.91" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89132cd0bf050864e1d38dc3bbc07a0eb8e7530af26344d3d2bbbef83499f590" +checksum = "d53cbcb5a243bd33b7858b1d7f4aca2153490815872d86d955d6ea29f743c035" dependencies = [ "proc-macro2", "quote", @@ -4748,9 +4772,9 @@ dependencies = [ [[package]] name = "sysinfo" -version = "0.31.4" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355dbe4f8799b304b05e1b0f05fc59b2a18d36645cf169607da45bde2f69a1be" +checksum = "948512566b1895f93b1592c7574baeb2de842f224f2aab158799ecadb8ebbb46" dependencies = [ "core-foundation-sys", "libc", @@ -4814,7 +4838,7 @@ dependencies = [ "serde", "serde_json", "serde_with", - "thiserror", + "thiserror 1.0.64", "tokio", "tokio-stream", "tokio-tar", @@ -4837,7 +4861,16 @@ version = "1.0.64" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d50af8abc119fb8bb6dbabcfa89656f46f84aa0ac7688088608076ad2b459a84" dependencies = [ - "thiserror-impl", + "thiserror-impl 1.0.64", +] + +[[package]] +name = "thiserror" +version = "2.0.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f072643fd0190df67a8bab670c20ef5d8737177d6ac6b2e9a236cb096206b2cc" +dependencies = [ + "thiserror-impl 2.0.9", ] [[package]] @@ -4848,7 +4881,18 @@ checksum = "08904e7672f5eb876eaaf87e0ce17857500934f4981c4a0ab2b4aa98baac7fc3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", +] + +[[package]] +name = "thiserror-impl" +version = "2.0.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b50fa271071aae2e6ee85f842e2e28ba8cd2c5fb67f11fcb1fd70b276f9e7d4" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.91", ] [[package]] @@ -4969,7 +5013,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -5214,7 +5258,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -5346,7 +5390,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f4e71ddbefed856d5881821d6ada4e606bbb91fd332296963ed596e2ad2100f3" dependencies = [ "libc", - "thiserror", + "thiserror 1.0.64", "windows 0.52.0", ] @@ -5421,7 +5465,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -5514,7 +5558,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", "wasm-bindgen-shared", ] @@ -5548,7 +5592,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -5687,7 +5731,7 @@ checksum = "9107ddc059d5b6fbfbffdfa7a7fe3e22a226def0b2608f72e9d552763d3e1ad7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -5698,7 +5742,7 @@ checksum = "29bee4b38ea3cde66011baa44dba677c432a78593e202392d1e9070cf2a7fca7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -6016,7 +6060,7 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.79", + "syn 2.0.91", ] [[package]] @@ -6038,7 +6082,7 @@ dependencies = [ "flate2", "indexmap 2.5.0", "memchr", - "thiserror", + "thiserror 1.0.64", "zopfli", ] diff --git a/Cargo.toml b/Cargo.toml index df676fb74..de979e891 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,6 +34,7 @@ mime = "0.3.17" rdkafka = { version = "0.37", features = ["cmake-build", "tracing", "sasl", "ssl", "libz-static"] } testcontainers = "0.23" testcontainers-modules = { version = "0.11", features = ["kafka"] } +backon = "1.3" ### other dependencies anyhow = { version = "1.0", features = ["backtrace"] } @@ -71,7 +72,7 @@ num_cpus = "1.15" once_cell = "1.17.1" prometheus = { version = "0.13", features = ["process"] } rand = "0.8.5" -regex = "1.7.3" +regex = "1.11.1" relative-path = { version = "1.7", features = ["serde"] } reqwest = { version = "0.11.27", default-features = false, features = [ "rustls-tls", @@ -83,8 +84,8 @@ semver = "1.0" serde = { version = "1.0", features = ["rc", "derive"] } serde_json = "1.0" static-files = "0.2" -sysinfo = "0.31.4" -thiserror = "1.0.64" +sysinfo = "0.33.0" +thiserror = "2.0.9" thread-priority = "1.0.0" tokio = { version = "1.42", default-features = false, features = [ "sync", diff --git a/Dockerfile b/Dockerfile index cfd88348b..ba0657b81 100644 --- a/Dockerfile +++ b/Dockerfile @@ -21,6 +21,15 @@ LABEL maintainer="Parseable Team " LABEL org.opencontainers.image.vendor="Parseable Inc" LABEL org.opencontainers.image.licenses="AGPL-3.0" +RUN apt-get update && \ + apt-get install --no-install-recommends -y \ + cmake \ + librdkafka-dev \ + ca-certificates \ + libsasl2-dev \ + libssl-dev && \ + rm -rf /var/lib/apt/lists/* + WORKDIR /parseable COPY . . RUN cargo build --release @@ -30,7 +39,16 @@ FROM gcr.io/distroless/cc-debian12:latest WORKDIR /parseable -# Copy the static shell into base image. +# Copy the Parseable binary from builder COPY --from=builder /parseable/target/release/parseable /usr/bin/parseable +# Copy only the libraries that binary needs since kafka is statically linked +COPY --from=builder /usr/lib/x86_64-linux-gnu/libsasl2.so.2 /usr/lib/x86_64-linux-gnu/ +COPY --from=builder /usr/lib/x86_64-linux-gnu/libssl.so.3 /usr/lib/x86_64-linux-gnu/ +COPY --from=builder /usr/lib/x86_64-linux-gnu/libcrypto.so.3 /usr/lib/x86_64-linux-gnu/ + +# Copy CA certificates +COPY --from=builder /etc/ssl/certs/ca-certificates.crt /etc/ssl/certs/ + + CMD ["/usr/bin/parseable"] diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index 06cfd585b..147b9f848 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -1,6 +1,6 @@ -version: "3.7" networks: parseable-internal: + services: # minio minio: @@ -18,7 +18,7 @@ services: ports: - 9000:9000 healthcheck: - test: ["CMD", "curl", "-f", "http://localhost:9000/minio/health/live"] + test: [ "CMD", "curl", "-f", "http://localhost:9000/minio/health/live" ] interval: 15s timeout: 20s retries: 5 @@ -29,9 +29,10 @@ services: build: context: . dockerfile: Dockerfile - command: ["parseable", "s3-store"] + platform: linux/amd64 + command: [ "parseable", "s3-store" ] ports: - - 8000:8000 + - "8000:8000" environment: - P_S3_URL=http://minio:9000 - P_S3_ACCESS_KEY=parseable @@ -47,7 +48,7 @@ services: networks: - parseable-internal healthcheck: - test: ["CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness"] + test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] interval: 15s timeout: 20s retries: 5 @@ -63,7 +64,8 @@ services: build: context: . dockerfile: Dockerfile - command: ["parseable", "s3-store"] + platform: linux/amd64 + command: [ "parseable", "s3-store" ] ports: - 8000 environment: @@ -79,16 +81,23 @@ services: - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=ingest - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 + - P_KAFKA_TOPICS=dist-test-logs-stream + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - P_KAFKA_GROUP_ID=parseable-kafka-sink-connector + # additional settings like security, tuning, etc. networks: - parseable-internal healthcheck: - test: ["CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness"] + test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] interval: 15s timeout: 20s retries: 5 depends_on: - parseable-query - minio + - kafka-0 + - kafka-1 + - kafka-2 deploy: restart_policy: condition: on-failure @@ -126,3 +135,141 @@ services: condition: on-failure delay: 20s max_attempts: 3 + + kafka-0: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=0 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_0_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-1: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=1 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_1_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-2: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=2 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_2_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-ui: + platform: linux/amd64 + image: provectuslabs/kafka-ui:latest + ports: + - "8080:8080" + depends_on: + - kafka-0 + - kafka-1 + - kafka-2 + environment: + KAFKA_CLUSTERS_0_NAME: dist-test + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 + KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + DYNAMIC_CONFIG_ENABLED: "true" + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + kafka-log-generator: + build: + context: ./scripts + dockerfile: Dockerfile + environment: + - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - KAFKA_TOPIC=dist-test-logs-stream + - LOG_RATE=500 + - TOTAL_LOGS=100000 + depends_on: + - kafka-0 + - kafka-1 + - kafka-2 + networks: + - parseable-internal + restart: "no" + +volumes: + kafka_0_data: + driver: local + kafka_1_data: + driver: local + kafka_2_data: + driver: local + diff --git a/docker-compose-local.yaml b/docker-compose-local.yaml new file mode 100644 index 000000000..c44283dd3 --- /dev/null +++ b/docker-compose-local.yaml @@ -0,0 +1,41 @@ +services: + kafka: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092:9092" + - "29092:29092" + volumes: + - "kafka_data:/bitnami" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=0 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka:9093 + # Listeners for internal and external communication + - KAFKA_CFG_LISTENERS=PLAINTEXT://0.0.0.0:9092,PLAINTEXT_INTERNAL://0.0.0.0:29092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://localhost:9092,PLAINTEXT_INTERNAL://kafka:29092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,PLAINTEXT_INTERNAL:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT_INTERNAL + + kafka-ui: + platform: linux/amd64 + image: provectuslabs/kafka-ui:latest + ports: + - "8080:8080" + depends_on: + - kafka + environment: + KAFKA_CLUSTERS_0_NAME: local + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka:29092 + KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + DYNAMIC_CONFIG_ENABLED: "true" + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + +volumes: + kafka_data: + driver: local diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index 59b323c78..e34c867ab 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -1,5 +1,3 @@ -version: "3.7" - networks: parseable-internal: @@ -17,7 +15,7 @@ services: - MINIO_ROOT_PASSWORD=supersecret - MINIO_UPDATE=off ports: - - 9000 + - "9000:9000" healthcheck: test: [ "CMD", "curl", "-f", "http://localhost:9000/minio/health/live" ] interval: 15s @@ -30,9 +28,10 @@ services: build: context: . dockerfile: Dockerfile - command: ["parseable", "s3-store"] + platform: linux/amd64 + command: [ "parseable", "s3-store" ] ports: - - 8000 + - "8000:8000" environment: - P_S3_URL=http://minio:9000 - P_S3_ACCESS_KEY=parseable @@ -44,15 +43,22 @@ services: - P_PASSWORD=parseableadmin - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - networks: - - parseable-internal + - P_KAFKA_TOPICS=test-logs-stream + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - P_KAFKA_GROUP_ID=parseable-kafka-sink-connector + # additional settings like security, tuning, etc. + depends_on: + - minio + - kafka-0 + - kafka-1 + - kafka-2 healthcheck: test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] interval: 15s timeout: 20s retries: 5 - depends_on: - - minio + networks: + - parseable-internal deploy: restart_policy: condition: on-failure @@ -61,13 +67,163 @@ services: quest: image: ghcr.io/parseablehq/quest:main - command: ["load", "http://parseable:8000", "parseableadmin", "parseableadmin", "20", "10", "5m", "minio:9000", "parseable", "supersecret", "parseable"] + platform: linux/amd64 + command: [ + "load", + "http://parseable:8000", + "parseableadmin", + "parseableadmin", + "20", + "10", + "5m", + "minio:9000", + "parseable", + "supersecret", + "parseable" + ] + depends_on: + - parseable + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + kafka-0: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=0 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_0_data:/bitnami/kafka networks: - parseable-internal + healthcheck: + test: ["CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1"] + interval: 10s + timeout: 5s + retries: 5 + + kafka-1: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=1 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_1_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: ["CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1"] + interval: 10s + timeout: 5s + retries: 5 + + kafka-2: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=2 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_2_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: ["CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1"] + interval: 10s + timeout: 5s + retries: 5 + + kafka-ui: + platform: linux/amd64 + image: provectuslabs/kafka-ui:latest + ports: + - "8080:8080" depends_on: - - parseable + - kafka-0 + - kafka-1 + - kafka-2 + environment: + KAFKA_CLUSTERS_0_NAME: test + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 + KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + DYNAMIC_CONFIG_ENABLED: "true" + networks: + - parseable-internal deploy: restart_policy: condition: on-failure delay: 20s max_attempts: 3 + + kafka-log-generator: + build: + context: ./scripts + dockerfile: Dockerfile + environment: + - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - KAFKA_TOPIC=test-logs-stream + - LOG_RATE=500 + - TOTAL_LOGS=100000 + depends_on: + - kafka-0 + - kafka-1 + - kafka-2 + networks: + - parseable-internal + restart: "no" + +volumes: + kafka_0_data: + driver: local + kafka_1_data: + driver: local + kafka_2_data: + driver: local diff --git a/scripts/Dockerfile b/scripts/Dockerfile new file mode 100644 index 000000000..0ad2d62a1 --- /dev/null +++ b/scripts/Dockerfile @@ -0,0 +1,28 @@ +# Parseable Server (C) 2022 - 2024 Parseable, Inc. +# +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as +# published by the Free Software Foundation, either version 3 of the +# License, or (at your option) any later version. +# +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. +# +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +FROM python:3.13-slim-bookworm + +RUN apt-get update && apt-get install -y --no-install-recommends \ + gcc \ + librdkafka-dev \ + && rm -rf /var/lib/apt/lists/* + +RUN pip install confluent-kafka + +WORKDIR /app +COPY kafka_log_stream_generator.py /app/ + +CMD ["python", "/app/kafka_log_stream_generator.py"] diff --git a/scripts/kafka_log_stream_generator.py b/scripts/kafka_log_stream_generator.py index 93eed25b3..4f1c487bb 100644 --- a/scripts/kafka_log_stream_generator.py +++ b/scripts/kafka_log_stream_generator.py @@ -1,64 +1,104 @@ -import json +import os +import sys import time +import json +import logging from datetime import datetime, timezone from random import choice, randint from uuid import uuid4 + from confluent_kafka import Producer +from confluent_kafka.admin import AdminClient +from confluent_kafka.cimpl import NewTopic + +logging.basicConfig( + level=logging.INFO, + format="%(asctime)s [%(levelname)s] %(message)s", + handlers=[ + logging.StreamHandler(sys.stdout) # Log to stdout + ] +) + +logger = logging.getLogger(__name__) + +KAFKA_BROKERS = os.getenv("KAFKA_BROKERS", "localhost:9092") +KAFKA_TOPIC = os.getenv("KAFKA_TOPIC", "local-logs-stream") +NUM_PARTITIONS = int(os.getenv("NUM_PARTITIONS", "6")) # Default partitions +REPLICATION_FACTOR = int(os.getenv("REPLICATION_FACTOR", "3")) # Default RF +TOTAL_LOGS = int(os.getenv("TOTAL_LOGS", "100000")) # Total logs to produce +LOG_RATE = int(os.getenv("LOG_RATE", "500")) # Logs per second +REPORT_EVERY = 10_000 # Progress report frequency + +producer_conf = { + "bootstrap.servers": KAFKA_BROKERS, + "queue.buffering.max.messages": 200_000, + "queue.buffering.max.ms": 100, # Up to 100ms linger + "batch.num.messages": 10_000, + "compression.type": "lz4", # Compression (lz4, snappy, zstd, gzip) + "message.send.max.retries": 3, + "reconnect.backoff.ms": 100, + "reconnect.backoff.max.ms": 3600000, + # "acks": "all", # Safer but can reduce throughput if replication is slow +} -# Configuration -config = { - "kafka_broker": "localhost:9092", # Replace with your Kafka broker address - "kafka_topic": "log-stream", # Replace with your Kafka topic name - "log_rate": 500, # Logs per second - "log_template": { - "timestamp": "", # Timestamp will be added dynamically - "correlation_id": "", # Unique identifier for tracing requests - "level": "INFO", # Log level (e.g., INFO, ERROR, DEBUG) - "message": "", # Main log message to be dynamically set - "pod": { - "name": "example-pod", # Kubernetes pod name - "namespace": "default", # Kubernetes namespace - "node": "node-01" # Kubernetes node name - }, - "request": { - "method": "", # HTTP method - "path": "", # HTTP request path - "remote_address": "" # IP address of the client - }, - "response": { - "status_code": 200, # HTTP response status code - "latency_ms": 0 # Latency in milliseconds - }, - "metadata": { - "container_id": "", # Container ID - "image": "example/image:1.0", # Docker image - "environment": "prod" # Environment (e.g., dev, staging, prod) - } - } +admin_client = AdminClient({"bootstrap.servers": KAFKA_BROKERS}) +producer = Producer(producer_conf) + +LOG_TEMPLATE = { + "timestamp": "", + "correlation_id": "", + "level": "INFO", + "message": "", + "pod": {"name": "", "namespace": "", "node": ""}, + "request": {"method": "", "path": "", "remote_address": ""}, + "response": {"status_code": 200, "latency_ms": 0}, + "metadata": {"container_id": "", "image": "", "environment": ""}, } -producer = Producer({"bootstrap.servers": config["kafka_broker"]}) + +def create_topic(topic_name, num_partitions, replication_factor): + new_topic = NewTopic( + topic=topic_name, + num_partitions=num_partitions, + replication_factor=replication_factor + ) + + logger.info(f"Creating topic '{topic_name}' with {num_partitions} partitions and RF {replication_factor}...") + fs = admin_client.create_topics([new_topic]) + + for topic, f in fs.items(): + try: + f.result() + logger.info(f"Topic '{topic}' created successfully.") + except Exception as e: + if "TopicExistsError" in str(e): + logger.warning(f"Topic '{topic}' already exists.") + else: + logger.error(f"Failed to create topic '{topic}': {e}") def delivery_report(err, msg): - if err is not None: - print(f"Delivery failed for message {msg.key()}: {err}") + if err: + logger.error(f"Delivery failed for message {msg.key()}: {err}") else: - print(f"Message delivered to {msg.topic()} [{msg.partition()}]") + logger.debug(f"Message delivered to {msg.topic()} [{msg.partition()}]") def generate_log(): - log = config["log_template"].copy() + log = LOG_TEMPLATE.copy() + + # Timestamp & correlation log["timestamp"] = datetime.now(timezone.utc).isoformat() log["correlation_id"] = str(uuid4()) + # Random level/message levels = ["INFO", "WARNING", "ERROR", "DEBUG"] messages = [ "Received incoming HTTP request", "Processed request successfully", "Failed to process request", "Request timeout encountered", - "Service unavailable" + "Service unavailable", ] log["level"] = choice(levels) log["message"] = choice(messages) @@ -69,55 +109,72 @@ def generate_log(): log["request"] = { "method": choice(methods), "path": choice(paths), - "remote_address": f"192.168.1.{randint(1, 255)}" + "remote_address": f"192.168.1.{randint(1, 255)}", } # Populate response fields log["response"] = { "status_code": choice([200, 201, 400, 401, 403, 404, 500]), - "latency_ms": randint(10, 1000) + "latency_ms": randint(10, 1000), } # Populate pod and metadata fields log["pod"] = { "name": f"pod-{randint(1, 100)}", "namespace": choice(["default", "kube-system", "production", "staging"]), - "node": f"node-{randint(1, 10)}" + "node": f"node-{randint(1, 10)}", } log["metadata"] = { "container_id": f"container-{randint(1000, 9999)}", "image": f"example/image:{randint(1, 5)}.0", - "environment": choice(["dev", "staging", "prod"]) + "environment": choice(["dev", "staging", "prod"]), } return log def main(): + logger.info("Starting rate-limited log producer...") + create_topic(KAFKA_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR) + logger.info(f"Broker: {KAFKA_BROKERS}, Topic: {KAFKA_TOPIC}, Rate: {LOG_RATE} logs/sec, Total Logs: {TOTAL_LOGS}") + + start_time = time.time() + try: - while True: - # Generate log message - log_message = generate_log() - log_json = json.dumps(log_message) + for i in range(TOTAL_LOGS): + log_data = generate_log() + log_str = json.dumps(log_data) # Send to Kafka producer.produce( - config["kafka_topic"], - value=log_json, + topic=KAFKA_TOPIC, + value=log_str, callback=delivery_report ) - # Flush the producer to ensure delivery - producer.flush() + if (i + 1) % REPORT_EVERY == 0: + logger.info(f"{i + 1} messages produced. Flushing producer...") + producer.flush() + + # Sleep to maintain the logs/second rate + time.sleep(1 / LOG_RATE) - # Wait based on the log rate - time.sleep(1 / config["log_rate"]) except KeyboardInterrupt: - print("Stopped log generation.") + logger.warning("Interrupted by user! Flushing remaining messages...") + producer.flush() + + except Exception as e: + logger.error(f"An error occurred: {e}") + finally: + logger.info("Flushing producer...") producer.flush() + elapsed = time.time() - start_time + logger.info(f"DONE! Produced {TOTAL_LOGS} log messages in {elapsed:.2f} seconds.") + logger.info(f"Effective rate: ~{TOTAL_LOGS / elapsed:,.0f} logs/sec") + if __name__ == "__main__": main() diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index 5e79f32a2..a92cf71c6 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -261,7 +261,10 @@ impl KafkaConfig { pub fn consumer_config(&self) -> rdkafka::ClientConfig { let mut config = rdkafka::ClientConfig::new(); - config.set("bootstrap.servers", &self.bootstrap_servers); + config + .set("bootstrap.servers", &self.bootstrap_servers) + .set("reconnect.backoff.ms", "100") + .set("reconnect.backoff.max.ms", "3600000"); if let Some(client_id) = &self.client_id { config @@ -339,7 +342,10 @@ impl KafkaConfig { pub fn producer_config(&self) -> rdkafka::config::ClientConfig { let mut config = rdkafka::config::ClientConfig::new(); - config.set("bootstrap.servers", &self.bootstrap_servers); + config + .set("bootstrap.servers", &self.bootstrap_servers) + .set("reconnect.backoff.ms", "100") + .set("reconnect.backoff.max.ms", "3600000"); if let Some(client_id) = &self.client_id { config diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs index bb443bb02..5c9ba3b7b 100644 --- a/src/connectors/kafka/consumer.rs +++ b/src/connectors/kafka/consumer.rs @@ -17,13 +17,16 @@ */ use crate::connectors::common::shutdown::Shutdown; -use crate::connectors::kafka::partition_stream_queue::PartitionStreamReceiver; +use crate::connectors::kafka::partition_stream::{PartitionStreamReceiver, PartitionStreamSender}; use crate::connectors::kafka::state::StreamState; use crate::connectors::kafka::{ - partition_stream_queue, ConsumerRecord, KafkaContext, StreamConsumer, TopicPartition, + partition_stream, ConsumerRecord, KafkaContext, StreamConsumer, TopicPartition, }; +use backon::{ExponentialBuilder, Retryable}; use futures_util::FutureExt; use rdkafka::consumer::Consumer; +use rdkafka::error::KafkaError; +use rdkafka::message::BorrowedMessage; use rdkafka::Statistics; use std::sync::Arc; use tokio::sync::{mpsc, RwLock}; @@ -81,10 +84,9 @@ impl KafkaStreams { /// 4. Listens for shutdown signals and gracefully terminates all partition streams, unsubscribing the consumer. /// /// Limitations and References: - /// - Issues with `split_partition_queue` in rust-rdkafka: + /// - Issues with `split_partition_queue` in rust-rdkafka: /// - https://github.com/fede1024/rust-rdkafka/issues/535 /// - https://github.com/confluentinc/librdkafka/issues/4059 - /// - https://github.com/fede1024/rust-rdkafka/issues/535 /// - https://github.com/confluentinc/librdkafka/issues/4059 /// - https://github.com/fede1024/rust-rdkafka/issues/654 /// - https://github.com/fede1024/rust-rdkafka/issues/651 @@ -106,51 +108,135 @@ impl KafkaStreams { std::thread::spawn(move || { tokio_handle.block_on(async move { + let retry_policy = ExponentialBuilder::default().with_max_times(5000); + loop { - tokio::select! { - result = consumer.recv() => { - match result { - Ok(msg) => { - let mut state = stream_state.write().await; - let tp = TopicPartition::from_kafka_msg(&msg); - let consumer_record = ConsumerRecord::from_borrowed_msg(msg); - let ps_tx = match state.get_partition_sender(&tp) { - Some(ps_tx) => ps_tx.clone(), - None => { - info!("Creating new stream for {:?}", tp); - let (ps_tx, ps_rx) = partition_stream_queue::bounded(10_000, tp.clone()); - state.insert_partition_sender(tp.clone(), ps_tx.clone()); - stream_tx.send(ps_rx).await.unwrap(); - ps_tx - } - }; - ps_tx.send(consumer_record).await; - } - Err(err) => { - error!("Cannot get message from kafka consumer! Cause {:?}", err); - break - }, - }; - }, - _ = shutdown_handle.recv() => { - info!("Gracefully stopping kafka partition streams!"); - let mut stream_state = stream_state.write().await; - stream_state.clear(); - consumer.unsubscribe(); - break; - }, - else => { - error!("KafkaStreams terminated!"); + let result = KafkaStreams::process_consumer_messages( + &consumer, + &stream_state, + &stream_tx, + &shutdown_handle, + &retry_policy, + ) + .await; + + match result { + Err(e) => { + error!( + "Partitioned processing encountered a critical error: {:?}", + e + ); break; } + Ok(..) => {} } } - }) + }); }); ReceiverStream::new(stream_rx) } + async fn process_consumer_messages( + consumer: &Arc, + stream_state: &RwLock, + stream_tx: &mpsc::Sender, + shutdown_handle: &Shutdown, + retry_policy: &ExponentialBuilder, + ) -> anyhow::Result<()> { + tokio::select! { + result = KafkaStreams::receive_with_retry(consumer, retry_policy) => match result { + Ok(msg) => KafkaStreams::handle_message(msg, stream_state, stream_tx).await, + Err(err) => { + anyhow::bail!("Unrecoverable error occurred while receiving Kafka message: {:?}", err); + }, + }, + _ = shutdown_handle.recv() => { + KafkaStreams::handle_shutdown(consumer, stream_state).await; + Ok(()) + }, + else => { + error!("KafkaStreams terminated unexpectedly!"); + Ok(()) + } + } + } + + async fn receive_with_retry<'a>( + consumer: &'a Arc, + retry_policy: &'a ExponentialBuilder, + ) -> Result, KafkaError> { + let recv_fn = || consumer.recv(); + + recv_fn + .retry(retry_policy.clone()) + .sleep(tokio::time::sleep) + .notify(|err, dur| { + tracing::warn!( + "Retrying message reception due to error: {:?}. Waiting for {:?}...", + err, + dur + ); + }) + .await + } + + /// Handle individual Kafka message and route it to the proper partition stream + async fn handle_message( + msg: BorrowedMessage<'_>, + stream_state: &RwLock, + stream_tx: &mpsc::Sender, + ) -> anyhow::Result<()> { + let mut state = stream_state.write().await; + let tp = TopicPartition::from_kafka_msg(&msg); + let consumer_record = ConsumerRecord::from_borrowed_msg(msg); + + let partition_stream_tx = + KafkaStreams::get_or_create_partition_stream(&mut state, stream_tx, tp).await; + partition_stream_tx.send(consumer_record).await; + + Ok(()) + } + + async fn get_or_create_partition_stream( + state: &mut StreamState, + stream_tx: &mpsc::Sender, + tp: TopicPartition, + ) -> PartitionStreamSender { + if let Some(ps_tx) = state.get_partition_sender(&tp) { + ps_tx.clone() + } else { + Self::create_new_partition_stream(state, stream_tx, tp).await + } + } + + async fn create_new_partition_stream( + state: &mut StreamState, + stream_tx: &mpsc::Sender, + tp: TopicPartition, + ) -> PartitionStreamSender { + info!("Creating new stream for {:?}", tp); + + let (ps_tx, ps_rx) = partition_stream::bounded(100_000, tp.clone()); + state.insert_partition_sender(tp.clone(), ps_tx.clone()); + + if let Err(e) = stream_tx.send(ps_rx).await { + error!( + "Failed to send partition stream receiver for {:?}: {:?}", + tp, e + ); + } + + ps_tx + } + + async fn handle_shutdown(consumer: &Arc, stream_state: &RwLock) { + info!("Gracefully stopping kafka partition streams!"); + let mut state = stream_state.write().await; + state.clear(); + consumer.unsubscribe(); + } + fn create_consumer(context: KafkaContext) -> Arc { info!("Creating Kafka consumer from configs {:#?}", context.config); diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs index a7033ef84..5b8f82974 100644 --- a/src/connectors/kafka/mod.rs +++ b/src/connectors/kafka/mod.rs @@ -34,7 +34,7 @@ use tracing::{error, info, warn}; pub mod config; pub mod consumer; pub mod metrics; -mod partition_stream_queue; +mod partition_stream; pub mod processor; pub mod rebalance_listener; pub mod sink; diff --git a/src/connectors/kafka/partition_stream_queue.rs b/src/connectors/kafka/partition_stream.rs similarity index 100% rename from src/connectors/kafka/partition_stream_queue.rs rename to src/connectors/kafka/partition_stream.rs diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index 9ec0ed9f3..4813e3c21 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -32,8 +32,7 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; use tokio_stream::wrappers::ReceiverStream; -use tracing::{debug, warn}; -use tracing::{error, info}; +use tracing::{debug, error, warn}; #[derive(Default, Debug, Clone)] pub struct ParseableSinkProcessor; @@ -138,7 +137,6 @@ where tp: TopicPartition, record_stream: ReceiverStream, ) -> anyhow::Result<()> { - info!("Started processing stream for {:?}", tp); let chunked_stream = tokio_stream::StreamExt::chunks_timeout( record_stream, self.buffer_size, @@ -164,7 +162,6 @@ where }) .await; - info!("Finished processing stream for {:?}", tp); self.processor.post_stream().await?; Ok(()) diff --git a/src/connectors/kafka/sink.rs b/src/connectors/kafka/sink.rs index e257c4abf..e2a117ee3 100644 --- a/src/connectors/kafka/sink.rs +++ b/src/connectors/kafka/sink.rs @@ -24,7 +24,7 @@ use anyhow::Result; use futures_util::StreamExt; use std::sync::Arc; use tokio::time::Duration; -use tracing::error; +use tracing::{error, info}; pub struct KafkaSinkConnector

where @@ -66,12 +66,16 @@ where tokio::spawn(async move { partition_queue .run_drain(|record_stream| async { + info!("Starting task for partition: {:?}", tp); + worker .process_partition(tp.clone(), record_stream) .await .unwrap(); }) - .await + .await; + + info!("Task completed for partition: {:?}", tp); }) }) .for_each_concurrent(None, |task| async { diff --git a/src/connectors/kafka/state.rs b/src/connectors/kafka/state.rs index cc91bc9d8..ca0904d4d 100644 --- a/src/connectors/kafka/state.rs +++ b/src/connectors/kafka/state.rs @@ -16,7 +16,7 @@ * */ -use crate::connectors::kafka::partition_stream_queue::PartitionStreamSender; +use crate::connectors::kafka::partition_stream::PartitionStreamSender; use crate::connectors::kafka::{TopicPartition, TopicPartitionList}; use std::collections::HashMap; use tracing::info; From 9ce10310c91327afc8e3990cc00d37a994180980 Mon Sep 17 00:00:00 2001 From: hippalus Date: Mon, 23 Dec 2024 23:43:42 +0100 Subject: [PATCH 10/65] feat(metrics): add KafkaMetricsCollector for Prometheus integration . Implement KafkaMetricsCollector to collect and expose Kafka client and broker metrics. Refactor ParseableServer.init(..) and connectors::init(..). --- Cargo.lock | 318 +---------- Cargo.toml | 2 - src/connectors/kafka/config.rs | 9 +- src/connectors/kafka/consumer.rs | 19 +- src/connectors/kafka/metrics.rs | 693 ++++++++++++++++++++++- src/connectors/mod.rs | 7 +- src/handlers/http/modal/ingest_server.rs | 9 +- src/handlers/http/modal/mod.rs | 4 +- src/handlers/http/modal/query_server.rs | 10 +- src/handlers/http/modal/server.rs | 8 +- src/main.rs | 8 +- 11 files changed, 714 insertions(+), 373 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index feaacee8f..7c8db9933 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -903,56 +903,6 @@ dependencies = [ "generic-array", ] -[[package]] -name = "bollard" -version = "0.17.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d41711ad46fda47cd701f6908e59d1bd6b9a2b7464c0d0aeab95c6d37096ff8a" -dependencies = [ - "base64 0.22.0", - "bollard-stubs", - "bytes", - "futures-core", - "futures-util", - "hex", - "home", - "http 1.1.0", - "http-body-util", - "hyper 1.4.1", - "hyper-named-pipe", - "hyper-rustls 0.27.3", - "hyper-util", - "hyperlocal", - "log", - "pin-project-lite", - "rustls 0.23.13", - "rustls-native-certs 0.7.3", - "rustls-pemfile 2.1.2", - "rustls-pki-types", - "serde", - "serde_derive", - "serde_json", - "serde_repr", - "serde_urlencoded", - "thiserror 1.0.64", - "tokio", - "tokio-util", - "tower-service", - "url", - "winapi", -] - -[[package]] -name = "bollard-stubs" -version = "1.45.0-rc.26.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d7c5415e3a6bc6d3e99eff6268e488fd4ee25e7b28c10f08fa6760bd9de16e4" -dependencies = [ - "serde", - "serde_repr", - "serde_with", -] - [[package]] name = "brotli" version = "6.0.0" @@ -1836,7 +1786,6 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b42b6fa04a440b495c8b04d0e71b707c585f83cb9cb28cf8cd0d976c315e31b4" dependencies = [ "powerfmt", - "serde", ] [[package]] @@ -1885,17 +1834,6 @@ dependencies = [ "syn 2.0.91", ] -[[package]] -name = "docker_credential" -version = "1.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31951f49556e34d90ed28342e1df7e1cb7a229c4cab0aecc627b5d91edd41d07" -dependencies = [ - "base64 0.21.7", - "serde", - "serde_json", -] - [[package]] name = "duct" version = "0.13.7" @@ -1939,17 +1877,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "etcetera" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "136d1b5283a1ab77bd9257427ffd09d8667ced0570b6f938942bc7568ed5b943" -dependencies = [ - "cfg-if", - "home", - "windows-sys 0.48.0", -] - [[package]] name = "fastrand" version = "1.9.0" @@ -1965,18 +1892,6 @@ version = "2.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "658bd65b1cf4c852a3cc96f18a8ce7b5640f6b703f905c7d74532294c2a63984" -[[package]] -name = "filetime" -version = "0.2.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35c0522e981e68cbfa8c3f978441a5f34b30b96e146b33cd3359176b50fe8586" -dependencies = [ - "cfg-if", - "libc", - "libredox", - "windows-sys 0.59.0", -] - [[package]] name = "fixedbitset" version = "0.4.2" @@ -2289,15 +2204,6 @@ version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" -[[package]] -name = "home" -version = "0.5.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" -dependencies = [ - "windows-sys 0.52.0", -] - [[package]] name = "hostname" version = "0.4.0" @@ -2453,21 +2359,6 @@ dependencies = [ "want", ] -[[package]] -name = "hyper-named-pipe" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73b7d8abf35697b81a825e386fc151e0d503e8cb5fcb93cc8669c376dfd6f278" -dependencies = [ - "hex", - "hyper 1.4.1", - "hyper-util", - "pin-project-lite", - "tokio", - "tower-service", - "winapi", -] - [[package]] name = "hyper-rustls" version = "0.24.2" @@ -2493,7 +2384,7 @@ dependencies = [ "hyper 1.4.1", "hyper-util", "rustls 0.23.13", - "rustls-native-certs 0.8.0", + "rustls-native-certs", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", @@ -2534,21 +2425,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "hyperlocal" -version = "0.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "986c5ce3b994526b3cd75578e62554abd09f0899d6206de48b3e96ab34ccc8c7" -dependencies = [ - "hex", - "http-body-util", - "hyper 1.4.1", - "hyper-util", - "pin-project-lite", - "tokio", - "tower-service", -] - [[package]] name = "iana-time-zone" version = "0.1.60" @@ -2602,7 +2478,6 @@ checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" dependencies = [ "autocfg", "hashbrown 0.12.3", - "serde", ] [[package]] @@ -2613,7 +2488,6 @@ checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" dependencies = [ "equivalent", "hashbrown 0.14.5", - "serde", ] [[package]] @@ -2781,17 +2655,6 @@ version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" -[[package]] -name = "libredox" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" -dependencies = [ - "bitflags 2.5.0", - "libc", - "redox_syscall 0.5.7", -] - [[package]] name = "libz-sys" version = "1.1.20" @@ -3274,7 +3137,7 @@ checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.4.1", + "redox_syscall", "smallvec", "windows-targets 0.48.5", ] @@ -3315,31 +3178,6 @@ dependencies = [ "zstd-sys", ] -[[package]] -name = "parse-display" -version = "0.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "914a1c2265c98e2446911282c6ac86d8524f495792c38c5bd884f80499c7538a" -dependencies = [ - "parse-display-derive", - "regex", - "regex-syntax 0.8.5", -] - -[[package]] -name = "parse-display-derive" -version = "0.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ae7800a4c974efd12df917266338e79a7a74415173caf7e70aa0a0707345281" -dependencies = [ - "proc-macro2", - "quote", - "regex", - "regex-syntax 0.8.5", - "structmeta", - "syn 2.0.91", -] - [[package]] name = "parse-zoneinfo" version = "0.3.0" @@ -3424,8 +3262,6 @@ dependencies = [ "sha2", "static-files", "sysinfo", - "testcontainers", - "testcontainers-modules", "thiserror 2.0.9", "thread-priority", "tokio", @@ -3917,15 +3753,6 @@ dependencies = [ "sasl2-sys", ] -[[package]] -name = "redox_syscall" -version = "0.3.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" -dependencies = [ - "bitflags 1.3.2", -] - [[package]] name = "redox_syscall" version = "0.4.1" @@ -3935,15 +3762,6 @@ dependencies = [ "bitflags 1.3.2", ] -[[package]] -name = "redox_syscall" -version = "0.5.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" -dependencies = [ - "bitflags 2.5.0", -] - [[package]] name = "regex" version = "1.11.1" @@ -4070,7 +3888,7 @@ dependencies = [ "pin-project-lite", "quinn", "rustls 0.23.13", - "rustls-native-certs 0.8.0", + "rustls-native-certs", "rustls-pemfile 2.1.2", "rustls-pki-types", "serde", @@ -4224,19 +4042,6 @@ dependencies = [ "zeroize", ] -[[package]] -name = "rustls-native-certs" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" -dependencies = [ - "openssl-probe", - "rustls-pemfile 2.1.2", - "rustls-pki-types", - "schannel", - "security-framework", -] - [[package]] name = "rustls-native-certs" version = "0.8.0" @@ -4456,36 +4261,6 @@ dependencies = [ "serde", ] -[[package]] -name = "serde_with" -version = "3.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ad483d2ab0149d5a5ebcd9972a3852711e0153d863bf5a5d0391d28883c4a20" -dependencies = [ - "base64 0.22.0", - "chrono", - "hex", - "indexmap 1.9.3", - "indexmap 2.5.0", - "serde", - "serde_derive", - "serde_json", - "serde_with_macros", - "time", -] - -[[package]] -name = "serde_with_macros" -version = "3.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65569b702f41443e8bc8bbb1c5779bd0450bbe723b56198980e80ec45780bce2" -dependencies = [ - "darling", - "proc-macro2", - "quote", - "syn 2.0.91", -] - [[package]] name = "sha1" version = "0.10.6" @@ -4683,29 +4458,6 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" -[[package]] -name = "structmeta" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e1575d8d40908d70f6fd05537266b90ae71b15dbbe7a8b7dffa2b759306d329" -dependencies = [ - "proc-macro2", - "quote", - "structmeta-derive", - "syn 2.0.91", -] - -[[package]] -name = "structmeta-derive" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.91", -] - [[package]] name = "strum" version = "0.26.2" @@ -4817,44 +4569,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "testcontainers" -version = "0.23.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f40cc2bd72e17f328faf8ca7687fe337e61bccd8acf9674fa78dd3792b045e1" -dependencies = [ - "async-trait", - "bollard", - "bollard-stubs", - "bytes", - "docker_credential", - "either", - "etcetera", - "futures", - "log", - "memchr", - "parse-display", - "pin-project-lite", - "serde", - "serde_json", - "serde_with", - "thiserror 1.0.64", - "tokio", - "tokio-stream", - "tokio-tar", - "tokio-util", - "url", -] - -[[package]] -name = "testcontainers-modules" -version = "0.11.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "064a2677e164cad39ef3c1abddb044d5a25c49d27005804563d8c4227aac8bd0" -dependencies = [ - "testcontainers", -] - [[package]] name = "thiserror" version = "1.0.64" @@ -5059,21 +4773,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "tokio-tar" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d5714c010ca3e5c27114c1cdeb9d14641ace49874aa5626d7149e47aedace75" -dependencies = [ - "filetime", - "futures-core", - "libc", - "redox_syscall 0.3.5", - "tokio", - "tokio-stream", - "xattr", -] - [[package]] name = "tokio-util" version = "0.7.10" @@ -6017,17 +5716,6 @@ dependencies = [ "windows-sys 0.48.0", ] -[[package]] -name = "xattr" -version = "1.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8da84f1a25939b27f6820d92aed108f83ff920fdf11a7b19366c27c4cda81d4f" -dependencies = [ - "libc", - "linux-raw-sys 0.4.13", - "rustix 0.38.34", -] - [[package]] name = "xxhash-rust" version = "0.8.10" diff --git a/Cargo.toml b/Cargo.toml index de979e891..d05998077 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,8 +32,6 @@ mime = "0.3.17" ### connectors dependencies rdkafka = { version = "0.37", features = ["cmake-build", "tracing", "sasl", "ssl", "libz-static"] } -testcontainers = "0.23" -testcontainers-modules = { version = "0.11", features = ["kafka"] } backon = "1.3" ### other dependencies diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index a92cf71c6..4ecc1e967 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -479,7 +479,7 @@ impl Default for KafkaConfig { // Common configuration with standard broker port bootstrap_servers: "localhost:9092".to_string(), topics: vec![], - client_id: None, // Let Kafka generate a unique client ID if not specified + client_id: Some("parseable-connect".to_string()), // Component-specific configurations with production-ready defaults consumer: Some(ConsumerConfig::default()), @@ -613,7 +613,10 @@ mod tests { ); let rdkafka_config = config.consumer_config(); - assert_eq!(rdkafka_config.get("group.id"), Some("test-group")); + assert_eq!( + rdkafka_config.get("group.id"), + Some("parseable-test-group-gi") + ); assert_eq!( rdkafka_config.get("partition.assignment.strategy"), Some("cooperative-sticky") @@ -631,7 +634,7 @@ mod tests { if let Some(producer) = config.producer { assert_eq!(producer.acks, "all"); assert!(producer.enable_idempotence); - assert_eq!(producer.compression_type, "snappy"); + assert_eq!(producer.compression_type, "lz4"); } } diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs index 5c9ba3b7b..d81f2ed82 100644 --- a/src/connectors/kafka/consumer.rs +++ b/src/connectors/kafka/consumer.rs @@ -47,8 +47,8 @@ impl KafkaStreams { shutdown_handle: Shutdown, ) -> anyhow::Result { info!("Initializing KafkaStreams..."); + let statistics = Arc::clone(&context.statistics); let consumer = KafkaStreams::create_consumer(context); - let statistics = Arc::new(std::sync::RwLock::new(Statistics::default())); info!("KafkaStreams initialized successfully."); Ok(Self { @@ -120,15 +120,12 @@ impl KafkaStreams { ) .await; - match result { - Err(e) => { - error!( - "Partitioned processing encountered a critical error: {:?}", - e - ); - break; - } - Ok(..) => {} + if let Err(e) = result { + error!( + "Partitioned processing encountered a critical error: {:?}", + e + ); + break; } } }); @@ -169,7 +166,7 @@ impl KafkaStreams { let recv_fn = || consumer.recv(); recv_fn - .retry(retry_policy.clone()) + .retry(*retry_policy) .sleep(tokio::time::sleep) .notify(|err, dur| { tracing::warn!( diff --git a/src/connectors/kafka/metrics.rs b/src/connectors/kafka/metrics.rs index 0a70bcc87..7cc11ff5b 100644 --- a/src/connectors/kafka/metrics.rs +++ b/src/connectors/kafka/metrics.rs @@ -17,39 +17,694 @@ */ use prometheus::core::{Collector, Desc}; -use prometheus::proto::MetricFamily; +use prometheus::{ + proto, Histogram, HistogramOpts, HistogramVec, IntCounter, IntCounterVec, IntGauge, + IntGaugeVec, Opts, +}; use rdkafka::Statistics; use std::sync::{Arc, RwLock}; #[derive(Debug)] -pub struct KafkaConsumerMetricsCollector { +pub struct KafkaMetricsCollector { stats: Arc>, + descs: Vec, + + // Core client metrics + msg_cnt: IntGauge, + msg_size: IntGauge, + msg_max: IntGauge, + msg_size_max: IntGauge, + metadata_cache_cnt: IntGauge, + tx: IntCounter, + tx_bytes: IntCounter, + rx: IntCounter, + rx_bytes: IntCounter, + txmsgs: IntCounter, + txmsg_bytes: IntCounter, + rxmsgs: IntCounter, + rxmsg_bytes: IntCounter, + + // Broker metrics + broker_state_cnt: IntGauge, + broker_outbuf_cnt: IntGauge, + broker_outbuf_msg_cnt: IntGauge, + broker_waitresp_cnt: IntGauge, + broker_waitresp_msg_cnt: IntGauge, + broker_tx: IntCounter, + broker_tx_bytes: IntCounter, + broker_tx_errs: IntCounter, + broker_tx_retries: IntCounter, + broker_req_timeouts: IntCounter, + broker_rx: IntCounter, + broker_rx_bytes: IntCounter, + broker_rx_errs: IntCounter, + broker_rx_corrid_errs: IntCounter, + broker_rx_partial: IntCounter, + broker_connects: IntCounter, + broker_disconnects: IntCounter, + broker_int_latency: Histogram, + broker_outbuf_latency: Histogram, + broker_rtt: Histogram, + broker_throttle: Histogram, + + // Topic metrics + topic_metadata_age: IntGaugeVec, + topic_batchsize: HistogramVec, + topic_batchcnt: HistogramVec, + + // Partition metrics with labels + partition_msgq_cnt: IntGaugeVec, + partition_msgq_bytes: IntGaugeVec, + partition_xmit_msgq_cnt: IntGaugeVec, + partition_xmit_msgq_bytes: IntGaugeVec, + partition_fetchq_cnt: IntGaugeVec, + partition_fetchq_size: IntGaugeVec, + partition_query_offset: IntGaugeVec, + partition_next_offset: IntGaugeVec, + partition_app_offset: IntGaugeVec, + partition_stored_offset: IntGaugeVec, + partition_committed_offset: IntGaugeVec, + partition_eof_offset: IntGaugeVec, + partition_lo_offset: IntGaugeVec, + partition_hi_offset: IntGaugeVec, + partition_consumer_lag: IntGaugeVec, + partition_consumer_lag_stored: IntGaugeVec, + partition_txmsgs: IntCounterVec, + partition_txbytes: IntCounterVec, + partition_rxmsgs: IntCounterVec, + partition_rxbytes: IntCounterVec, + partition_msgs: IntCounterVec, + partition_rx_ver_drops: IntCounterVec, + partition_msgs_inflight: IntGaugeVec, + + // Consumer group metrics + cgrp_rebalance_cnt: IntCounter, + cgrp_rebalance_age: IntGauge, + cgrp_assignment_size: IntGauge, + + // Exactly once semantics metrics + eos_epoch_cnt: IntCounter, + eos_producer_id: IntGauge, + eos_producer_epoch: IntGauge, } -impl KafkaConsumerMetricsCollector { - pub fn new(stats: Arc>) -> Self { - Self { stats } - } +impl KafkaMetricsCollector { + pub fn new(stats: Arc>) -> anyhow::Result { + let mut descs = Vec::new(); + + fn create_gauge_vec( + name: &str, + help: &str, + labels: &[&str], + descs: &mut Vec, + ) -> IntGaugeVec { + let gauge = IntGaugeVec::new(Opts::new(name, help), labels).unwrap(); + descs.extend(gauge.clone().desc().into_iter().cloned()); + gauge + } - pub fn statistics(&self) -> Result { - match self.stats.read() { - Ok(stats) => Ok(stats.clone()), - Err(err) => Err(format!( - "Cannot get kafka statistics from RwLock. Error: {}", - err - )), + fn create_counter_vec( + name: &str, + help: &str, + labels: &[&str], + descs: &mut Vec, + ) -> IntCounterVec { + let counter = IntCounterVec::new(Opts::new(name, help), labels).unwrap(); + descs.extend(counter.clone().desc().into_iter().cloned()); + counter } + + fn create_histogram_vec( + name: &str, + help: &str, + labels: &[&str], + descs: &mut Vec, + ) -> HistogramVec { + let histogram = HistogramVec::new(HistogramOpts::new(name, help), labels).unwrap(); + descs.extend(histogram.clone().desc().into_iter().cloned()); + histogram + } + + let topic_labels = &["topic"]; + let partition_labels = &["topic", "partition"]; + + let collector = KafkaMetricsCollector { + stats: stats.clone(), + descs: descs.clone(), + // Core metrics + msg_cnt: IntGauge::new( + "kafka_msg_cnt", + "Current number of messages in producer queues", + )?, + msg_size: IntGauge::new( + "kafka_msg_size", + "Current total size of messages in producer queues", + )?, + msg_max: IntGauge::new( + "kafka_msg_max", + "Maximum number of messages allowed in producer queues", + )?, + msg_size_max: IntGauge::new( + "kafka_msg_size_max", + "Maximum total size of messages allowed in producer queues", + )?, + metadata_cache_cnt: IntGauge::new( + "kafka_metadata_cache_cnt", + "Number of topics in metadata cache", + )?, + tx: IntCounter::new("kafka_tx_total", "Total number of transmissions")?, + tx_bytes: IntCounter::new("kafka_tx_bytes_total", "Total number of bytes transmitted")?, + rx: IntCounter::new("kafka_rx_total", "Total number of receptions")?, + rx_bytes: IntCounter::new("kafka_rx_bytes_total", "Total number of bytes received")?, + txmsgs: IntCounter::new("kafka_txmsgs_total", "Total number of messages transmitted")?, + txmsg_bytes: IntCounter::new( + "kafka_txmsg_bytes_total", + "Total number of message bytes transmitted", + )?, + rxmsgs: IntCounter::new("kafka_rxmsgs_total", "Total number of messages received")?, + rxmsg_bytes: IntCounter::new( + "kafka_rxmsg_bytes_total", + "Total number of message bytes received", + )?, + + // Broker metrics + broker_state_cnt: IntGauge::new("kafka_broker_state", "Broker connection state")?, + broker_outbuf_cnt: IntGauge::new( + "kafka_broker_outbuf_cnt", + "Number of requests awaiting transmission", + )?, + broker_outbuf_msg_cnt: IntGauge::new( + "kafka_broker_outbuf_msg_cnt", + "Number of messages awaiting transmission", + )?, + broker_waitresp_cnt: IntGauge::new( + "kafka_broker_waitresp_cnt", + "Number of requests in-flight", + )?, + broker_waitresp_msg_cnt: IntGauge::new( + "kafka_broker_waitresp_msg_cnt", + "Number of messages in-flight", + )?, + broker_tx: IntCounter::new("kafka_broker_tx_total", "Total broker transmissions")?, + broker_tx_bytes: IntCounter::new( + "kafka_broker_tx_bytes_total", + "Total broker bytes transmitted", + )?, + broker_tx_errs: IntCounter::new( + "kafka_broker_tx_errs_total", + "Total broker transmission errors", + )?, + broker_tx_retries: IntCounter::new( + "kafka_broker_tx_retries_total", + "Total broker transmission retries", + )?, + broker_req_timeouts: IntCounter::new( + "kafka_broker_req_timeouts_total", + "Total broker request timeouts", + )?, + broker_rx: IntCounter::new("kafka_broker_rx_total", "Total broker receptions")?, + broker_rx_bytes: IntCounter::new( + "kafka_broker_rx_bytes_total", + "Total broker bytes received", + )?, + broker_rx_errs: IntCounter::new( + "kafka_broker_rx_errs_total", + "Total broker reception errors", + )?, + broker_rx_corrid_errs: IntCounter::new( + "kafka_broker_rx_corrid_errs_total", + "Total broker correlation ID errors", + )?, + broker_rx_partial: IntCounter::new( + "kafka_broker_rx_partial_total", + "Total broker partial message sets", + )?, + broker_connects: IntCounter::new( + "kafka_broker_connects_total", + "Total broker connection attempts", + )?, + broker_disconnects: IntCounter::new( + "kafka_broker_disconnects_total", + "Total broker disconnections", + )?, + broker_int_latency: Histogram::with_opts(HistogramOpts::new( + "kafka_broker_int_latency", + "Internal broker latency", + ))?, + broker_outbuf_latency: Histogram::with_opts(HistogramOpts::new( + "kafka_broker_outbuf_latency", + "Outbuf latency", + ))?, + broker_rtt: Histogram::with_opts(HistogramOpts::new( + "kafka_broker_rtt", + "Broker round-trip time", + ))?, + + broker_throttle: Histogram::with_opts(HistogramOpts::new( + "kafka_broker_throttle", + "Broker throttle time", + ))?, + // Topic metrics with labels + topic_metadata_age: create_gauge_vec( + "kafka_topic_metadata_age", + "Age of topic metadata", + topic_labels, + &mut descs, + ), + topic_batchsize: create_histogram_vec( + "kafka_topic_batchsize", + "Topic batch sizes", + topic_labels, + &mut descs, + ), + topic_batchcnt: create_histogram_vec( + "kafka_topic_batchcnt", + "Topic batch counts", + topic_labels, + &mut descs, + ), + + // Partition metrics with labels + partition_msgq_cnt: create_gauge_vec( + "kafka_partition_msgq_cnt", + "Messages in partition queue", + partition_labels, + &mut descs, + ), + partition_msgq_bytes: create_gauge_vec( + "kafka_partition_msgq_bytes", + "Bytes in partition queue", + partition_labels, + &mut descs, + ), + partition_xmit_msgq_cnt: create_gauge_vec( + "kafka_partition_xmit_msgq_cnt", + "Messages in partition transmit queue", + partition_labels, + &mut descs, + ), + partition_xmit_msgq_bytes: create_gauge_vec( + "kafka_partition_xmit_msgq_bytes", + "Bytes in partition transmit queue", + partition_labels, + &mut descs, + ), + partition_fetchq_cnt: create_gauge_vec( + "kafka_partition_fetchq_cnt", + "Messages in partition fetch queue", + partition_labels, + &mut descs, + ), + partition_fetchq_size: create_gauge_vec( + "kafka_partition_fetchq_size", + "Size of partition fetch queue", + partition_labels, + &mut descs, + ), + partition_query_offset: create_gauge_vec( + "kafka_partition_query_offset", + "Current partition query offset", + partition_labels, + &mut descs, + ), + partition_next_offset: create_gauge_vec( + "kafka_partition_next_offset", + "Next partition offset", + partition_labels, + &mut descs, + ), + partition_app_offset: create_gauge_vec( + "kafka_partition_app_offset", + "Application partition offset", + partition_labels, + &mut descs, + ), + partition_stored_offset: create_gauge_vec( + "kafka_partition_stored_offset", + "Stored partition offset", + partition_labels, + &mut descs, + ), + partition_committed_offset: create_gauge_vec( + "kafka_partition_committed_offset", + "Committed partition offset", + partition_labels, + &mut descs, + ), + partition_eof_offset: create_gauge_vec( + "kafka_partition_eof_offset", + "EOF partition offset", + partition_labels, + &mut descs, + ), + partition_lo_offset: create_gauge_vec( + "kafka_partition_lo_offset", + "Low watermark partition offset", + partition_labels, + &mut descs, + ), + partition_hi_offset: create_gauge_vec( + "kafka_partition_hi_offset", + "High watermark partition offset", + partition_labels, + &mut descs, + ), + partition_consumer_lag: create_gauge_vec( + "kafka_partition_consumer_lag", + "Consumer lag", + partition_labels, + &mut descs, + ), + partition_consumer_lag_stored: create_gauge_vec( + "kafka_partition_consumer_lag_stored", + "Stored consumer lag", + partition_labels, + &mut descs, + ), + partition_txmsgs: create_counter_vec( + "kafka_partition_txmsgs_total", + "Total partition messages transmitted", + partition_labels, + &mut descs, + ), + partition_txbytes: create_counter_vec( + "kafka_partition_txbytes_total", + "Total partition bytes transmitted", + partition_labels, + &mut descs, + ), + partition_rxmsgs: create_counter_vec( + "kafka_partition_rxmsgs_total", + "Total partition messages received", + partition_labels, + &mut descs, + ), + partition_rxbytes: create_counter_vec( + "kafka_partition_rxbytes_total", + "Total partition bytes received", + partition_labels, + &mut descs, + ), + partition_msgs: create_counter_vec( + "kafka_partition_msgs_total", + "Total partition messages", + partition_labels, + &mut descs, + ), + partition_rx_ver_drops: create_counter_vec( + "kafka_partition_rx_ver_drops_total", + "Total partition version drops", + partition_labels, + &mut descs, + ), + partition_msgs_inflight: create_gauge_vec( + "kafka_partition_msgs_inflight", + "Messages in flight", + partition_labels, + &mut descs, + ), + cgrp_rebalance_cnt: IntCounter::new("kafka_cgrp_rebalance_total", "Total rebalances")?, + cgrp_rebalance_age: IntGauge::new("kafka_cgrp_rebalance_age", "Rebalance age")?, + cgrp_assignment_size: IntGauge::new("kafka_cgrp_assignment_size", "Assignment size")?, + + // Exactly once semantics metrics + eos_epoch_cnt: IntCounter::new("kafka_eos_epoch_total", "Total number of epochs")?, + eos_producer_id: IntGauge::new("kafka_eos_producer_id", "Producer ID")?, + eos_producer_epoch: IntGauge::new("kafka_eos_producer_epoch", "Producer epoch")?, + }; + + let mut collector = collector; + collector.descs = descs.clone(); + + Ok(collector) } } -impl Collector for KafkaConsumerMetricsCollector { +impl Collector for KafkaMetricsCollector { fn desc(&self) -> Vec<&Desc> { - //TODO: - vec![] + self.descs.iter().collect() + } + + fn collect(&self) -> Vec { + let stats = match self.stats.read() { + Ok(stats) => stats, + Err(_) => return vec![], + }; + + // Core metrics + let mut mfs = Vec::new(); + self.msg_cnt.set(stats.msg_cnt as i64); + self.msg_size.set(stats.msg_size as i64); + self.msg_max.set(stats.msg_max as i64); + self.msg_size_max.set(stats.msg_size_max as i64); + self.metadata_cache_cnt.set(stats.metadata_cache_cnt); + self.tx.inc_by(stats.tx as u64); + self.tx_bytes.inc_by(stats.tx_bytes as u64); + self.rx.inc_by(stats.rx as u64); + self.rx_bytes.inc_by(stats.rx_bytes as u64); + self.txmsgs.inc_by(stats.txmsgs as u64); + self.txmsg_bytes.inc_by(stats.txmsg_bytes as u64); + self.rxmsgs.inc_by(stats.rxmsgs as u64); + self.rxmsg_bytes.inc_by(stats.rxmsg_bytes as u64); + + mfs.extend(self.msg_cnt.collect()); + mfs.extend(self.msg_size.collect()); + mfs.extend(self.msg_max.collect()); + mfs.extend(self.msg_size_max.collect()); + mfs.extend(self.metadata_cache_cnt.collect()); + mfs.extend(self.tx.collect()); + mfs.extend(self.tx_bytes.collect()); + mfs.extend(self.rx.collect()); + mfs.extend(self.rx_bytes.collect()); + mfs.extend(self.txmsgs.collect()); + mfs.extend(self.txmsg_bytes.collect()); + mfs.extend(self.rxmsgs.collect()); + mfs.extend(self.rxmsg_bytes.collect()); + + // Broker metrics + for (_broker_id, broker) in stats.brokers.iter() { + self.broker_state_cnt.set(match broker.state.as_str() { + "UP" => 1, + "DOWN" => 0, + _ => -1, + }); + + self.broker_outbuf_cnt.set(broker.outbuf_cnt); + self.broker_outbuf_msg_cnt.set(broker.outbuf_msg_cnt); + self.broker_waitresp_cnt.set(broker.waitresp_cnt); + self.broker_waitresp_msg_cnt.set(broker.waitresp_msg_cnt); + + self.broker_tx.inc_by(broker.tx); + self.broker_tx_bytes.inc_by(broker.txbytes); + self.broker_tx_errs.inc_by(broker.txerrs); + self.broker_tx_retries.inc_by(broker.txretries); + self.broker_req_timeouts.inc_by(broker.req_timeouts); + self.broker_rx.inc_by(broker.rx); + self.broker_rx_bytes.inc_by(broker.rxbytes); + self.broker_rx_errs.inc_by(broker.rxerrs); + self.broker_rx_corrid_errs.inc_by(broker.rxcorriderrs); + self.broker_rx_partial.inc_by(broker.rxpartial); + + if let Some(connects) = broker.connects { + self.broker_connects.inc_by(connects as u64); + } + if let Some(disconnects) = broker.disconnects { + self.broker_disconnects.inc_by(disconnects as u64); + } + + // Latency metrics + if let Some(ref latency) = broker.int_latency { + self.broker_int_latency.observe(latency.avg as f64); + } + if let Some(ref latency) = broker.outbuf_latency { + self.broker_outbuf_latency.observe(latency.avg as f64); + } + if let Some(ref rtt) = broker.rtt { + self.broker_rtt.observe(rtt.avg as f64); + } + if let Some(ref throttle) = broker.throttle { + self.broker_throttle.observe(throttle.avg as f64); + } + } + + mfs.extend(self.broker_state_cnt.collect()); + mfs.extend(self.broker_outbuf_cnt.collect()); + mfs.extend(self.broker_outbuf_msg_cnt.collect()); + mfs.extend(self.broker_waitresp_cnt.collect()); + mfs.extend(self.broker_waitresp_msg_cnt.collect()); + mfs.extend(self.broker_tx.collect()); + mfs.extend(self.broker_tx_bytes.collect()); + mfs.extend(self.broker_tx_errs.collect()); + mfs.extend(self.broker_tx_retries.collect()); + mfs.extend(self.broker_req_timeouts.collect()); + mfs.extend(self.broker_rx.collect()); + mfs.extend(self.broker_rx_bytes.collect()); + mfs.extend(self.broker_rx_errs.collect()); + mfs.extend(self.broker_rx_corrid_errs.collect()); + mfs.extend(self.broker_rx_partial.collect()); + mfs.extend(self.broker_connects.collect()); + mfs.extend(self.broker_disconnects.collect()); + mfs.extend(self.broker_int_latency.collect()); + mfs.extend(self.broker_outbuf_latency.collect()); + mfs.extend(self.broker_rtt.collect()); + mfs.extend(self.broker_throttle.collect()); + + // Topic and partition metrics with labels + for (topic_name, topic) in stats.topics.iter() { + self.topic_metadata_age + .with_label_values(&[topic_name]) + .set(topic.metadata_age); + self.topic_batchsize + .with_label_values(&[topic_name]) + .observe(topic.batchsize.avg as f64); + self.topic_batchcnt + .with_label_values(&[topic_name]) + .observe(topic.batchcnt.avg as f64); + + for (partition_id, partition) in topic.partitions.iter() { + let labels = &[topic_name.as_str(), &partition_id.to_string()]; + self.partition_msgq_cnt + .with_label_values(labels) + .set(partition.msgq_cnt); + self.partition_msgq_bytes + .with_label_values(labels) + .set(partition.msgq_bytes as i64); + self.partition_xmit_msgq_cnt + .with_label_values(labels) + .set(partition.xmit_msgq_cnt); + self.partition_xmit_msgq_bytes + .with_label_values(labels) + .set(partition.xmit_msgq_bytes as i64); + self.partition_fetchq_cnt + .with_label_values(labels) + .set(partition.fetchq_cnt); + self.partition_fetchq_size + .with_label_values(labels) + .set(partition.fetchq_size as i64); + self.partition_query_offset + .with_label_values(labels) + .set(partition.query_offset); + self.partition_next_offset + .with_label_values(labels) + .set(partition.next_offset); + self.partition_app_offset + .with_label_values(labels) + .set(partition.app_offset); + self.partition_stored_offset + .with_label_values(labels) + .set(partition.stored_offset); + self.partition_committed_offset + .with_label_values(labels) + .set(partition.committed_offset); + self.partition_eof_offset + .with_label_values(labels) + .set(partition.eof_offset); + self.partition_lo_offset + .with_label_values(labels) + .set(partition.lo_offset); + self.partition_hi_offset + .with_label_values(labels) + .set(partition.hi_offset); + self.partition_consumer_lag + .with_label_values(labels) + .set(partition.consumer_lag); + self.partition_consumer_lag_stored + .with_label_values(labels) + .set(partition.consumer_lag_stored); + self.partition_txmsgs + .with_label_values(labels) + .inc_by(partition.txmsgs); + self.partition_txbytes + .with_label_values(labels) + .inc_by(partition.txbytes); + self.partition_rxmsgs + .with_label_values(labels) + .inc_by(partition.rxmsgs); + self.partition_rxbytes + .with_label_values(labels) + .inc_by(partition.rxbytes); + self.partition_msgs + .with_label_values(labels) + .inc_by(partition.msgs); + self.partition_rx_ver_drops + .with_label_values(labels) + .inc_by(partition.rx_ver_drops); + self.partition_msgs_inflight + .with_label_values(labels) + .set(partition.msgs_inflight); + } + } + + mfs.extend(self.topic_metadata_age.collect()); + mfs.extend(self.topic_batchsize.collect()); + mfs.extend(self.topic_batchcnt.collect()); + mfs.extend(self.partition_msgq_cnt.collect()); + mfs.extend(self.partition_msgq_bytes.collect()); + mfs.extend(self.partition_xmit_msgq_cnt.collect()); + mfs.extend(self.partition_xmit_msgq_bytes.collect()); + mfs.extend(self.partition_fetchq_cnt.collect()); + mfs.extend(self.partition_fetchq_size.collect()); + mfs.extend(self.partition_query_offset.collect()); + mfs.extend(self.partition_next_offset.collect()); + mfs.extend(self.partition_app_offset.collect()); + mfs.extend(self.partition_stored_offset.collect()); + mfs.extend(self.partition_committed_offset.collect()); + mfs.extend(self.partition_eof_offset.collect()); + mfs.extend(self.partition_lo_offset.collect()); + mfs.extend(self.partition_hi_offset.collect()); + mfs.extend(self.partition_consumer_lag.collect()); + mfs.extend(self.partition_consumer_lag_stored.collect()); + mfs.extend(self.partition_txmsgs.collect()); + mfs.extend(self.partition_txbytes.collect()); + mfs.extend(self.partition_rxmsgs.collect()); + mfs.extend(self.partition_rxbytes.collect()); + mfs.extend(self.partition_msgs.collect()); + mfs.extend(self.partition_rx_ver_drops.collect()); + mfs.extend(self.partition_msgs_inflight.collect()); + + // Consumer group metrics + if let Some(ref cgrp) = stats.cgrp { + self.cgrp_rebalance_cnt.inc_by(cgrp.rebalance_cnt as u64); + self.cgrp_rebalance_age.set(cgrp.rebalance_age); + self.cgrp_assignment_size.set(cgrp.assignment_size as i64); + } + + mfs.extend(self.cgrp_rebalance_cnt.collect()); + mfs.extend(self.cgrp_rebalance_age.collect()); + mfs.extend(self.cgrp_assignment_size.collect()); + + // EOS metrics + if let Some(ref eos) = stats.eos { + self.eos_epoch_cnt.inc_by(eos.epoch_cnt as u64); + self.eos_producer_id.set(eos.producer_id); + self.eos_producer_epoch.set(eos.producer_epoch); + } + + mfs.extend(self.eos_epoch_cnt.collect()); + mfs.extend(self.eos_producer_id.collect()); + mfs.extend(self.eos_producer_epoch.collect()); + + mfs } +} + +#[cfg(test)] +mod tests { + use super::*; + use prometheus::Registry; + + #[test] + fn test_kafka_metrics_collector() { + let stats = Arc::new(RwLock::new(Statistics::default())); + let collector = KafkaMetricsCollector::new(stats).unwrap(); + + let descs = collector.desc(); + assert!(!descs.is_empty()); + + let mfs = collector.collect(); + assert!(!mfs.is_empty()); - fn collect(&self) -> Vec { - //TODO: encode metrics - vec![] + let registry = Registry::new(); + assert!(registry.register(Box::new(collector)).is_ok()); } } diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index e2d7f11fb..905203f97 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -20,14 +20,14 @@ use crate::connectors::common::config::ConnectorConfig; use crate::connectors::common::processor::Processor; use crate::connectors::common::shutdown::Shutdown; use crate::connectors::kafka::consumer::KafkaStreams; -use crate::connectors::kafka::metrics::KafkaConsumerMetricsCollector; +use crate::connectors::kafka::metrics::KafkaMetricsCollector; use crate::connectors::kafka::processor::ParseableSinkProcessor; use crate::connectors::kafka::rebalance_listener::RebalanceListener; use crate::connectors::kafka::sink::KafkaSinkConnector; use crate::connectors::kafka::state::StreamState; use crate::connectors::kafka::{ConsumerRecord, KafkaContext}; -use crate::metrics; use crate::option::{Mode, CONFIG}; +use actix_web_prometheus::PrometheusMetrics; use prometheus::Registry; use std::sync::Arc; use tokio::sync::RwLock; @@ -36,12 +36,11 @@ use tracing::{info, warn}; pub mod common; pub mod kafka; -pub async fn init() -> anyhow::Result<()> { +pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { if matches!(CONFIG.parseable.mode, Mode::Ingest | Mode::All) { match CONFIG.parseable.connector_config.clone() { Some(config) => { let shutdown_handle = Shutdown::default(); - let prometheus = metrics::build_metrics_handler(); let registry = prometheus.registry.clone(); let processor = ParseableSinkProcessor; diff --git a/src/handlers/http/modal/ingest_server.rs b/src/handlers/http/modal/ingest_server.rs index 69dab1c5d..bb8451d71 100644 --- a/src/handlers/http/modal/ingest_server.rs +++ b/src/handlers/http/modal/ingest_server.rs @@ -31,7 +31,6 @@ use crate::handlers::http::logstream; use crate::handlers::http::middleware::DisAllowRootUser; use crate::handlers::http::middleware::RouteExt; use crate::handlers::http::role; -use crate::metrics; use crate::migration; use crate::migration::metadata_migration::migrate_ingester_metadata; use crate::rbac::role::Action; @@ -46,6 +45,7 @@ use crate::{handlers::http::base_path, option::CONFIG}; use actix_web::web; use actix_web::web::resource; use actix_web::Scope; +use actix_web_prometheus::PrometheusMetrics; use anyhow::anyhow; use async_trait::async_trait; use base64::Engine; @@ -99,9 +99,8 @@ impl ParseableServer for IngestServer { } /// configure the server and start an instance to ingest data - async fn init(&self) -> anyhow::Result<()> { - let prometheus = metrics::build_metrics_handler(); - CONFIG.storage().register_store_metrics(&prometheus); + async fn init(&self, prometheus: &PrometheusMetrics) -> anyhow::Result<()> { + CONFIG.storage().register_store_metrics(prometheus); migration::run_migration(&CONFIG).await?; @@ -116,7 +115,7 @@ impl ParseableServer for IngestServer { self.set_ingestor_metadata().await?; // Ingestors shouldn't have to deal with OpenId auth flow - let app = self.start(prometheus, None); + let app = self.start(prometheus.clone(), None); tokio::pin!(app); loop { diff --git a/src/handlers/http/modal/mod.rs b/src/handlers/http/modal/mod.rs index 5080f2038..e831be829 100644 --- a/src/handlers/http/modal/mod.rs +++ b/src/handlers/http/modal/mod.rs @@ -65,8 +65,8 @@ pub trait ParseableServer { /// load metadata/configuration from persistence for previous sessions of parseable async fn load_metadata(&self) -> anyhow::Result>; - /// code that describes starting and setup procedures for each type of server - async fn init(&self) -> anyhow::Result<()>; + /// code that describes starting and setup procedures for each type of server with prometheus + async fn init(&self, prometheus: &PrometheusMetrics) -> anyhow::Result<()>; /// configure the server async fn start( diff --git a/src/handlers/http/modal/query_server.rs b/src/handlers/http/modal/query_server.rs index 792bb6571..4ace99df7 100644 --- a/src/handlers/http/modal/query_server.rs +++ b/src/handlers/http/modal/query_server.rs @@ -28,9 +28,10 @@ use crate::rbac::role::Action; use crate::sync; use crate::users::dashboards::DASHBOARDS; use crate::users::filters::FILTERS; -use crate::{analytics, metrics, migration, storage}; +use crate::{analytics, migration, storage}; use actix_web::web::{resource, ServiceConfig}; use actix_web::{web, Scope}; +use actix_web_prometheus::PrometheusMetrics; use async_trait::async_trait; use bytes::Bytes; use tracing::{error, info}; @@ -85,9 +86,8 @@ impl ParseableServer for QueryServer { } /// initialize the server, run migrations as needed and start an instance - async fn init(&self) -> anyhow::Result<()> { - let prometheus = metrics::build_metrics_handler(); - CONFIG.storage().register_store_metrics(&prometheus); + async fn init(&self, prometheus: &PrometheusMetrics) -> anyhow::Result<()> { + CONFIG.storage().register_store_metrics(prometheus); migration::run_migration(&CONFIG).await?; @@ -118,7 +118,7 @@ impl ParseableServer for QueryServer { sync::object_store_sync().await; tokio::spawn(airplane::server()); - let app = self.start(prometheus, CONFIG.parseable.openid.clone()); + let app = self.start(prometheus.clone(), CONFIG.parseable.openid.clone()); tokio::pin!(app); loop { diff --git a/src/handlers/http/modal/server.rs b/src/handlers/http/modal/server.rs index 6c0ec9fd8..c3554509a 100644 --- a/src/handlers/http/modal/server.rs +++ b/src/handlers/http/modal/server.rs @@ -38,6 +38,7 @@ use actix_web::web; use actix_web::web::resource; use actix_web::Resource; use actix_web::Scope; +use actix_web_prometheus::PrometheusMetrics; use actix_web_static_files::ResourceFiles; use async_trait::async_trait; use bytes::Bytes; @@ -96,9 +97,8 @@ impl ParseableServer for Server { } // configure the server and start an instance of the single server setup - async fn init(&self) -> anyhow::Result<()> { - let prometheus = metrics::build_metrics_handler(); - CONFIG.storage().register_store_metrics(&prometheus); + async fn init(&self, prometheus: &PrometheusMetrics) -> anyhow::Result<()> { + CONFIG.storage().register_store_metrics(prometheus); migration::run_migration(&CONFIG).await?; @@ -123,7 +123,7 @@ impl ParseableServer for Server { tokio::spawn(handlers::livetail::server()); tokio::spawn(handlers::airplane::server()); - let app = self.start(prometheus, CONFIG.parseable.openid.clone()); + let app = self.start(prometheus.clone(), CONFIG.parseable.openid.clone()); tokio::pin!(app); diff --git a/src/main.rs b/src/main.rs index 7cfc085f5..1455a6243 100644 --- a/src/main.rs +++ b/src/main.rs @@ -17,7 +17,7 @@ */ use parseable::{ - banner, connectors, + banner, connectors, metrics, option::{Mode, CONFIG}, rbac, storage, IngestServer, ParseableServer, QueryServer, Server, }; @@ -46,8 +46,10 @@ async fn main() -> anyhow::Result<()> { // keep metadata info in mem metadata.set_global(); - let parseable_server = server.init(); - let connectors_task = connectors::init(); + let prometheus = metrics::build_metrics_handler(); + + let parseable_server = server.init(&prometheus); + let connectors_task = connectors::init(&prometheus); tokio::try_join!(parseable_server, connectors_task)?; From d8d055822a4a3a9c23b05a328cd21131906af770 Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 24 Dec 2024 20:08:14 +0100 Subject: [PATCH 11/65] fix kafka metrics collector registeration --- src/connectors/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 905203f97..d72a15ca9 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -86,7 +86,7 @@ where let kafka_streams = KafkaStreams::init(kafka_context, stream_state, shutdown_handle.clone())?; let stats = kafka_streams.statistics(); - registry.register(Box::new(KafkaConsumerMetricsCollector::new(stats)))?; + registry.register(Box::new(KafkaMetricsCollector::new(stats)?))?; let kafka_parseable_sink_connector = KafkaSinkConnector::new( kafka_streams, From 4d13ee2d8215224cf9758f6fa02cda60d61b727b Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 26 Dec 2024 08:42:42 +0100 Subject: [PATCH 12/65] Refactor connector configurations to adapt parseable cli options. --- Cargo.toml | 2 +- docker-compose-distributed-test.yaml | 5 +- docker-compose-test.yaml | 11 +- scripts/kafka_log_stream_generator.py | 15 + src/cli.rs | 438 +------- src/connectors/common/config.rs | 238 ++--- src/connectors/common/mod.rs | 1 - src/connectors/common/types/mod.rs | 72 -- src/connectors/kafka/config.rs | 1348 +++++++++++++++---------- src/connectors/kafka/consumer.rs | 11 +- src/connectors/kafka/metrics.rs | 123 ++- src/connectors/kafka/mod.rs | 12 + src/connectors/mod.rs | 27 +- src/main.rs | 2 +- src/option.rs | 7 + 15 files changed, 1053 insertions(+), 1259 deletions(-) delete mode 100644 src/connectors/common/types/mod.rs diff --git a/Cargo.toml b/Cargo.toml index d05998077..c00fd62f5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -46,7 +46,7 @@ bzip2 = { version = "*", features = ["static"] } cookie = "0.18.1" chrono = "0.4" chrono-humanize = "0.2" -clap = { version = "4.1", default-features = false, features = [ +clap = { version = "4.5", default-features = false, features = [ "std", "color", "help", diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index 147b9f848..7db668690 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -65,7 +65,7 @@ services: context: . dockerfile: Dockerfile platform: linux/amd64 - command: [ "parseable", "s3-store" ] + command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] ports: - 8000 environment: @@ -81,9 +81,8 @@ services: - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=ingest - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 - - P_KAFKA_TOPICS=dist-test-logs-stream + - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - - P_KAFKA_GROUP_ID=parseable-kafka-sink-connector # additional settings like security, tuning, etc. networks: - parseable-internal diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index e34c867ab..f562727be 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -29,7 +29,7 @@ services: context: . dockerfile: Dockerfile platform: linux/amd64 - command: [ "parseable", "s3-store" ] + command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] ports: - "8000:8000" environment: @@ -43,9 +43,8 @@ services: - P_PASSWORD=parseableadmin - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - - P_KAFKA_TOPICS=test-logs-stream + - P_KAFKA_CONSUMER_TOPICS=test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - - P_KAFKA_GROUP_ID=parseable-kafka-sink-connector # additional settings like security, tuning, etc. depends_on: - minio @@ -116,7 +115,7 @@ services: networks: - parseable-internal healthcheck: - test: ["CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1"] + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] interval: 10s timeout: 5s retries: 5 @@ -146,7 +145,7 @@ services: networks: - parseable-internal healthcheck: - test: ["CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1"] + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] interval: 10s timeout: 5s retries: 5 @@ -176,7 +175,7 @@ services: networks: - parseable-internal healthcheck: - test: ["CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1"] + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] interval: 10s timeout: 5s retries: 5 diff --git a/scripts/kafka_log_stream_generator.py b/scripts/kafka_log_stream_generator.py index 4f1c487bb..eab4bbe88 100644 --- a/scripts/kafka_log_stream_generator.py +++ b/scripts/kafka_log_stream_generator.py @@ -1,3 +1,18 @@ +# Parseable Server (C) 2022 - 2024 Parseable, Inc. +# +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as +# published by the Free Software Foundation, either version 3 of the +# License, or (at your option) any later version. +# +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. +# +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + import os import sys import time diff --git a/src/cli.rs b/src/cli.rs index 89ca3bf92..1f1e9f4b6 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -16,16 +16,12 @@ * */ -use clap::{value_parser, Arg, ArgGroup, Command, FromArgMatches}; +use clap::{value_parser, Arg, ArgGroup, Command, CommandFactory, FromArgMatches}; use std::path::PathBuf; -use std::time::Duration; -use tracing::warn; + use url::Url; use crate::connectors::common::config::ConnectorConfig; -use crate::connectors::common::types::BadData; -use crate::connectors::common::types::ConnectorType; -use crate::connectors::kafka::config::{ConsumerConfig, KafkaConfig, SourceOffset}; use crate::{ oidc::{self, OpenidConfig}, option::{validation, Compression, Mode}, @@ -112,8 +108,6 @@ pub struct Cli { pub trino_auth: Option, pub trino_schema: Option, pub trino_catalog: Option, - //Connectors config - pub connector_config: Option, } impl Cli { @@ -155,46 +149,6 @@ impl Cli { pub const TRINO_AUTHORIZATION: &'static str = "p-trino-authorization"; pub const TRINO_SCHEMA: &'static str = "p-trino-schema"; - // ConnectorConfig arguments - pub const CONNECTOR_NAME: &'static str = "connector-name"; - pub const CONNECTOR_BUFFER_SIZE: &'static str = "connector-buffer-size"; - pub const CONNECTOR_BUFFER_TIMEOUT: &'static str = "connector-buffer-timeout"; - pub const CONNECTOR_OFFSET_MODE: &'static str = "connector-offset-mode"; // earliest, latest, group - pub const CONNECTOR_BAD_DATA_POLICY: &'static str = "connector-bad-data-policy"; // e.g. "drop", "fail" , "dlt" - pub const CONNECTOR_MAX_RETRIES: &'static str = "connector-max-retries"; - pub const CONNECTOR_RETRY_INTERVAL_MS: &'static str = "connector-retry-interval-ms"; - pub const CONNECTOR_METRICS_ENABLED: &'static str = "connector-metrics-enabled"; - pub const CONNECTOR_INSTANCE_ID: &'static str = "connector-instance-id"; - - // ConsumerConfig arguments - pub const CONSUMER_GROUP_INSTANCE_ID: &'static str = "consumer-group-instance-id"; - pub const CONSUMER_PARTITION_ASSIGNMENT_STRATEGY: &'static str = - "consumer-partition-assignment-strategy"; - pub const CONSUMER_SESSION_TIMEOUT_MS: &'static str = "consumer-session-timeout-ms"; - pub const CONSUMER_HEARTBEAT_INTERVAL_MS: &'static str = "consumer-heartbeat-interval-ms"; - pub const CONSUMER_MAX_POLL_INTERVAL_MS: &'static str = "consumer-max-poll-interval-ms"; - pub const CONSUMER_ENABLE_AUTO_COMMIT: &'static str = "consumer-enable-auto-commit"; - pub const CONSUMER_AUTO_COMMIT_INTERVAL_MS: &'static str = "consumer-auto-commit-interval-ms"; - pub const CONSUMER_ENABLE_AUTO_OFFSET_STORE: &'static str = "consumer-enable-auto-offset-store"; - pub const CONSUMER_AUTO_OFFSET_RESET: &'static str = "consumer-auto-offset-reset"; - pub const CONSUMER_FETCH_MIN_BYTES: &'static str = "consumer-fetch-min-bytes"; - pub const CONSUMER_FETCH_MAX_BYTES: &'static str = "consumer-fetch-max-bytes"; - pub const CONSUMER_FETCH_MAX_WAIT_MS: &'static str = "consumer-fetch-max-wait-ms"; - pub const CONSUMER_MAX_PARTITION_FETCH_BYTES: &'static str = - "consumer-max-partition-fetch-bytes"; - pub const CONSUMER_QUEUED_MIN_MESSAGES: &'static str = "consumer-queued-min-messages"; - pub const CONSUMER_QUEUED_MAX_MESSAGES_KBYTES: &'static str = - "consumer-queued-max-messages-kbytes"; - pub const CONSUMER_ENABLE_PARTITION_EOF: &'static str = "consumer-enable-partition-eof"; - pub const CONSUMER_CHECK_CRCS: &'static str = "consumer-check-crcs"; - pub const CONSUMER_ISOLATION_LEVEL: &'static str = "consumer-isolation-level"; - pub const CONSUMER_FETCH_MESSAGE_MAX_BYTES: &'static str = "consumer-fetch-message-max-bytes"; - pub const CONSUMER_STATS_INTERVAL_MS: &'static str = "consumer-stats-interval-ms"; - - pub const KAFKA_TOPICS: &'static str = "kafka-topics"; - pub const KAFKA_BOOTSTRAP_SERVERS: &'static str = "kafka-bootstrap-servers"; - pub const KAFKA_GROUP_ID: &'static str = "kafka-group-id"; - pub fn local_stream_data_path(&self, stream_name: &str) -> PathBuf { self.local_staging_path.join(stream_name) } @@ -207,7 +161,8 @@ impl Cli { } pub fn create_cli_command_with_clap(name: &'static str) -> Command { - Command::new(name).next_line_help(false) + let mut command = Command::new(name) + .next_line_help(false) .arg( Arg::new(Self::TRINO_ENDPOINT) .long(Self::TRINO_ENDPOINT) @@ -267,25 +222,25 @@ impl Cli { .value_parser(validation::canonicalize_path) .help("Local path on this device where all trusted certificates are located.") ) - .arg( - Arg::new(Self::ADDRESS) - .long(Self::ADDRESS) - .env("P_ADDR") - .value_name("ADDR:PORT") - .default_value("0.0.0.0:8000") - .value_parser(validation::socket_addr) - .help("Address and port for Parseable HTTP(s) server"), - ) - .arg( - Arg::new(Self::STAGING) - .long(Self::STAGING) - .env("P_STAGING_DIR") - .value_name("DIR") - .default_value("./staging") - .value_parser(validation::canonicalize_path) - .help("Local path on this device to be used as landing point for incoming events") - .next_line_help(true), - ) + .arg( + Arg::new(Self::ADDRESS) + .long(Self::ADDRESS) + .env("P_ADDR") + .value_name("ADDR:PORT") + .default_value("0.0.0.0:8000") + .value_parser(validation::socket_addr) + .help("Address and port for Parseable HTTP(s) server"), + ) + .arg( + Arg::new(Self::STAGING) + .long(Self::STAGING) + .env("P_STAGING_DIR") + .value_name("DIR") + .default_value("./staging") + .value_parser(validation::canonicalize_path) + .help("Local path on this device to be used as landing point for incoming events") + .next_line_help(true), + ) .arg( Arg::new(Self::USERNAME) .long(Self::USERNAME) @@ -416,7 +371,7 @@ impl Cli { .arg( // RowGroupSize controls the number of rows present in one row group // More rows = better compression but HIGHER Memory consumption during read/write - // 1048576 is the default value for DataFusion + // 1048576 is the default value for DataFusion Arg::new(Self::ROW_GROUP_SIZE) .long(Self::ROW_GROUP_SIZE) .env("P_PARQUET_ROW_GROUP_SIZE") @@ -489,232 +444,17 @@ impl Cli { .value_name("STRING") .required(false) .help("Tag for MS Clarity"), - ).arg( - Arg::new(Self::CONNECTOR_NAME) - .long(Self::CONNECTOR_NAME) - .env("P_CONNECTOR_NAME") - .required(false) - .help("Name of the connector") - ) - .arg( - Arg::new(Self::CONNECTOR_BUFFER_SIZE) - .long(Self::CONNECTOR_BUFFER_SIZE) - .env("P_CONNECTOR_BATCH_SIZE") - .value_parser(value_parser!(usize)) - .required(false) - .help("Buffer size for processing") - ) - .arg( - Arg::new(Self::CONNECTOR_BUFFER_TIMEOUT) - .long(Self::CONNECTOR_BUFFER_TIMEOUT) - .env("P_CONNECTOR_BUFFER_TIMEOUT") - .value_parser(value_parser!(u64)) - .required(false) - .help("Buffer timeout for processing") - ) - .arg( - Arg::new(Self::CONNECTOR_OFFSET_MODE) - .long(Self::CONNECTOR_OFFSET_MODE) - .required(false) - .env("P_CONNECTOR_OFFSET_MODE") - .value_parser(["earliest", "latest", "group"]) - .help("Offset mode: earliest, latest, or group") - ) - .arg( - Arg::new(Self::CONNECTOR_BAD_DATA_POLICY) - .long(Self::CONNECTOR_BAD_DATA_POLICY) - .required(false) - .env("P_CONNECTOR_BAD_DATA_POLICY") - .help("Bad data handling policy: skip, error") - ) - .arg( - Arg::new(Self::CONNECTOR_MAX_RETRIES) - .long(Self::CONNECTOR_MAX_RETRIES) - .env("P_CONNECTOR_MAX_RETRIES") - .required(false) - .value_parser(value_parser!(u32)) - .help("Maximum number of retries on errors") - ) - .arg( - Arg::new(Self::CONNECTOR_RETRY_INTERVAL_MS) - .long(Self::CONNECTOR_RETRY_INTERVAL_MS) - .env("P_CONNECTOR_RETRY_INTERVAL_MS") - .value_parser(value_parser!(u64)) - .required(false) - .help("Retry interval in milliseconds") - ) - .arg( - Arg::new(Self::CONNECTOR_METRICS_ENABLED) - .long(Self::CONNECTOR_METRICS_ENABLED) - .env("P_CONNECTOR_METRICS_ENABLED") - .value_parser(value_parser!(bool)) - .required(false) - .help("Enable or disable connector metrics") - ) - .arg( - Arg::new(Self::CONNECTOR_INSTANCE_ID) - .long(Self::CONNECTOR_INSTANCE_ID) - .required(false) - .env("P_CONNECTOR_INSTANCE_ID") - .help("Instance ID for the connector") - ) - - // ConsumerConfig arguments: - .arg( - Arg::new(Self::CONSUMER_GROUP_INSTANCE_ID) - .long(Self::CONSUMER_GROUP_INSTANCE_ID) - .required(false) - .env("P_CONSUMER_GROUP_INSTANCE_ID") - .help("Consumer group instance ID") - ) - .arg( - Arg::new(Self::CONSUMER_PARTITION_ASSIGNMENT_STRATEGY) - .long(Self::CONSUMER_PARTITION_ASSIGNMENT_STRATEGY) - .env("P_CONSUMER_PARTITION_ASSIGNMENT_STRATEGY") - .help("Partition assignment strategy") - .required(false) - ) - .arg( - Arg::new(Self::CONSUMER_SESSION_TIMEOUT_MS) - .long(Self::CONSUMER_SESSION_TIMEOUT_MS) - .env("P_CONSUMER_SESSION_TIMEOUT_MS") - .value_parser(value_parser!(u32)) - .help("Consumer session timeout in ms") - .required(false) - ) - .arg( - Arg::new(Self::CONSUMER_HEARTBEAT_INTERVAL_MS) - .long(Self::CONSUMER_HEARTBEAT_INTERVAL_MS) - .env("P_CONSUMER_HEARTBEAT_INTERVAL_MS") - .value_parser(value_parser!(u32)) - .help("Consumer heartbeat interval in ms") - .required(false) - ) - .arg( - Arg::new(Self::CONSUMER_MAX_POLL_INTERVAL_MS) - .long(Self::CONSUMER_MAX_POLL_INTERVAL_MS) - .env("P_CONSUMER_MAX_POLL_INTERVAL_MS") - .value_parser(value_parser!(u32)) - .help("Max poll interval in ms") - .required(false) - ) - .arg( - Arg::new(Self::CONSUMER_ENABLE_AUTO_OFFSET_STORE) - .long(Self::CONSUMER_ENABLE_AUTO_OFFSET_STORE) - .env("P_CONSUMER_ENABLE_AUTO_OFFSET_STORE") - .value_parser(value_parser!(bool)) - .help("Enable auto offset store") - .default_value("true") // Just for as few metrics - .required(false) - ) - .arg( - Arg::new(Self::CONSUMER_AUTO_OFFSET_RESET) - .long(Self::CONSUMER_AUTO_OFFSET_RESET) - .env("P_CONSUMER_AUTO_OFFSET_RESET") - .value_parser(["earliest", "latest", "none"]) - .help("Auto offset reset behavior") - ) - .arg( - Arg::new(Self::CONSUMER_FETCH_MIN_BYTES) - .long(Self::CONSUMER_FETCH_MIN_BYTES) - .env("P_CONSUMER_FETCH_MIN_BYTES") - .value_parser(value_parser!(u32)) - .help("Fetch min bytes") - ) - .arg( - Arg::new(Self::CONSUMER_FETCH_MAX_BYTES) - .long(Self::CONSUMER_FETCH_MAX_BYTES) - .env("P_CONSUMER_FETCH_MAX_BYTES") - .value_parser(value_parser!(u32)) - .help("Fetch max bytes") - ) - .arg( - Arg::new(Self::CONSUMER_FETCH_MAX_WAIT_MS) - .long(Self::CONSUMER_FETCH_MAX_WAIT_MS) - .env("P_CONSUMER_FETCH_MAX_WAIT_MS") - .value_parser(value_parser!(u32)) - .help("Fetch max wait in ms") - ) - .arg( - Arg::new(Self::CONSUMER_MAX_PARTITION_FETCH_BYTES) - .long(Self::CONSUMER_MAX_PARTITION_FETCH_BYTES) - .env("P_CONSUMER_MAX_PARTITION_FETCH_BYTES") - .value_parser(value_parser!(u32)) - .help("Max partition fetch bytes") - ) - .arg( - Arg::new(Self::CONSUMER_QUEUED_MIN_MESSAGES) - .long(Self::CONSUMER_QUEUED_MIN_MESSAGES) - .env("P_CONSUMER_QUEUED_MIN_MESSAGES") - .value_parser(value_parser!(u32)) - .help("Queued min messages") - ) - .arg( - Arg::new(Self::CONSUMER_QUEUED_MAX_MESSAGES_KBYTES) - .long(Self::CONSUMER_QUEUED_MAX_MESSAGES_KBYTES) - .env("P_CONSUMER_QUEUED_MAX_MESSAGES_KBYTES") - .value_parser(value_parser!(u32)) - .help("Queued max messages kbytes") - ) - .arg( - Arg::new(Self::CONSUMER_ENABLE_PARTITION_EOF) - .long(Self::CONSUMER_ENABLE_PARTITION_EOF) - .env("P_CONSUMER_ENABLE_PARTITION_EOF") - .value_parser(value_parser!(bool)) - .help("Enable partition EOF") - ) - .arg( - Arg::new(Self::CONSUMER_CHECK_CRCS) - .long(Self::CONSUMER_CHECK_CRCS) - .env("P_CONSUMER_CHECK_CRCS") - .value_parser(value_parser!(bool)) - .help("Check CRCs") - ) - .arg( - Arg::new(Self::CONSUMER_ISOLATION_LEVEL) - .long(Self::CONSUMER_ISOLATION_LEVEL) - .env("P_CONSUMER_ISOLATION_LEVEL") - .value_parser(["read_uncommitted", "read_committed"]) - .help("Isolation level") - ) - .arg( - Arg::new(Self::CONSUMER_FETCH_MESSAGE_MAX_BYTES) - .long(Self::CONSUMER_FETCH_MESSAGE_MAX_BYTES) - .env("P_CONSUMER_FETCH_MESSAGE_MAX_BYTES") - .help("Fetch message max bytes (string)") - ) - .arg( - Arg::new(Self::CONSUMER_STATS_INTERVAL_MS) - .long(Self::CONSUMER_STATS_INTERVAL_MS) - .env("P_CONSUMER_STATS_INTERVAL_MS") - .value_parser(value_parser!(u64)) - .help("Consumer stats interval ms") - ) - .arg( - Arg::new(Self::KAFKA_TOPICS) - .long(Self::KAFKA_TOPICS) - .env("P_KAFKA_TOPICS") - .help("Kafka topics to consume from.Comma seperated string") - ) - .arg( - Arg::new(Self::KAFKA_BOOTSTRAP_SERVERS) - .long(Self::KAFKA_BOOTSTRAP_SERVERS) - .env("P_KAFKA_BOOTSTRAP_SERVERS") - .help("Kafka bootstrap servers.") - ) - .arg( - Arg::new(Self::KAFKA_GROUP_ID) - .long(Self::KAFKA_GROUP_ID) - .required(false) - .env("P_KAFKA_GROUP_ID") - .help("Kafka consumer group ID.") ) .group( ArgGroup::new("oidc") .args([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) .requires_all([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) .multiple(true) - ) + ); + + command = command.subcommand(ConnectorConfig::command()); + + command } } @@ -726,126 +466,6 @@ impl FromArgMatches for Cli { } fn update_from_arg_matches(&mut self, m: &clap::ArgMatches) -> Result<(), clap::Error> { - if matches!(self.mode, Mode::Query) { - self.connector_config = None; - } - - if let Some(topics) = m.get_one::(Cli::KAFKA_TOPICS).cloned() { - let bootstrap_servers = m - .get_one::(Cli::KAFKA_BOOTSTRAP_SERVERS) - .cloned() - .unwrap_or_default(); - let group_id = m - .get_one::(Cli::KAFKA_GROUP_ID) - .cloned() - .unwrap_or_else(|| "parseable-default-group".to_string()); - - if topics.is_empty() { - return Err(clap::Error::raw( - clap::error::ErrorKind::MissingRequiredArgument, - "Kafka topics required in ingest/all mode.", - )); - } - if bootstrap_servers.is_empty() { - return Err(clap::Error::raw( - clap::error::ErrorKind::MissingRequiredArgument, - "Kafka bootstrap servers required in ingest/all mode.", - )); - } - - let offset_mode = match m - .get_one::(Cli::CONNECTOR_OFFSET_MODE) - .map(|s| s.as_str()) - { - Some("earliest") => SourceOffset::Earliest, - Some("latest") => SourceOffset::Latest, - Some("group") | None => SourceOffset::Group, - _ => SourceOffset::Latest, - }; - - let buffer_size = m - .get_one::(Cli::CONNECTOR_BUFFER_SIZE) - .cloned() - .unwrap_or(10000); - let buffer_timeout = m - .get_one::(Cli::CONNECTOR_BUFFER_TIMEOUT) - .cloned() - .unwrap_or(5000); - - let max_retries = m - .get_one::(Cli::CONNECTOR_MAX_RETRIES) - .cloned() - .unwrap_or(20); - let retry_interval_ms = m - .get_one::(Cli::CONNECTOR_RETRY_INTERVAL_MS) - .cloned() - .unwrap_or(10000); - let metrics_enabled = m - .get_one::(Cli::CONNECTOR_METRICS_ENABLED) - .cloned() - .unwrap_or(true); - let connector_name = m - .get_one::(Cli::CONNECTOR_NAME) - .cloned() - .unwrap_or_else(|| "parseable-connectors".to_string()); - let instance_id = m - .get_one::(Cli::CONNECTOR_INSTANCE_ID) - .cloned() - .unwrap_or_else(|| "parseable-connectors".to_string()); - - let bad_data_policy = m.get_one::(Cli::CONNECTOR_BAD_DATA_POLICY).cloned(); - let bad_data = match bad_data_policy.as_deref() { - Some("drop") => Some(BadData::Drop {}), - Some("fail") => Some(BadData::Fail {}), - Some("dlt") => Some(BadData::Dlt {}), - _ => None, - }; - - let auto_offset_reset = m - .get_one::(Cli::CONSUMER_AUTO_OFFSET_RESET) - .cloned() - .unwrap_or_else(|| "earliest".to_string()); - - let mut consumer = ConsumerConfig::default(); - consumer.group_id = group_id; - consumer.auto_offset_reset = auto_offset_reset; - - let topics: Vec = topics.split(",").map(|t| t.to_owned()).collect(); - let topics_clone = topics.to_vec(); - - let kafka_config = KafkaConfig::builder() - .bootstrap_servers(bootstrap_servers) - .topic(topics_clone) - .with_consumer(consumer) - .build() - .map_err(|e| { - clap::Error::raw(clap::error::ErrorKind::InvalidValue, e.to_string()) - })?; - - let mut connector_config = ConnectorConfig::builder() - .connector_type(ConnectorType::KafkaSource) - .name(connector_name) - .buffer_size(buffer_size) - .buffer_timeout(Duration::from_millis(buffer_timeout)) - .offset_mode(offset_mode) - .topic(topics) - .max_retries(max_retries) - .retry_interval(Duration::from_millis(retry_interval_ms)) - .metrics_enabled(metrics_enabled) - .kafka_config(kafka_config) - .instance_id(instance_id) - .build() - .map_err(|e| { - clap::Error::raw(clap::error::ErrorKind::InvalidValue, e.to_string()) - })?; - - connector_config.bad_data = bad_data; - - self.connector_config = Some(connector_config); - } else { - warn!("No Kafka topics provided"); - } - self.trino_catalog = m.get_one::(Self::TRINO_CATALOG_NAME).cloned(); self.trino_endpoint = m.get_one::(Self::TRINO_ENDPOINT).cloned(); self.trino_auth = m.get_one::(Self::TRINO_AUTHORIZATION).cloned(); diff --git a/src/connectors/common/config.rs b/src/connectors/common/config.rs index 4cabe12b1..0fbe9b483 100644 --- a/src/connectors/common/config.rs +++ b/src/connectors/common/config.rs @@ -1,183 +1,95 @@ -/* - * Parseable Server (C) 2022 - 2024 Parseable, Inc. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -use crate::connectors::common::types::BadData; -use crate::connectors::common::types::ConnectorType; -use crate::connectors::common::ConnectorError; -use crate::connectors::kafka::config::{KafkaConfig, SourceOffset}; -use serde::{Deserialize, Serialize}; -use std::{time::Duration, vec}; - -#[derive(Debug, Clone, Serialize, Deserialize)] +use crate::connectors::kafka::config::KafkaConfig; +use clap::{ArgMatches, FromArgMatches, Parser, Subcommand, ValueEnum}; +use std::fmt; +use std::str::FromStr; + +#[derive(Debug, Clone, Parser)] +#[command(name = "connectors", about = "Configure Parseable connectors")] +#[group(id = "connector-config")] pub struct ConnectorConfig { - // Basic Configuration - pub connector_type: ConnectorType, - pub name: String, - - // Performance Configuration - pub buffer_size: usize, - pub buffer_timeout: Duration, - - // Topic/Stream Configuration - pub topics: Vec, - pub offset_mode: SourceOffset, - - // Error Handling - pub bad_data: Option, - pub max_retries: u32, - pub retry_interval: Duration, - - // Kafka-specific Configuration - #[serde(skip_serializing_if = "Option::is_none")] - pub kafka_config: Option, - - // Monitoring + #[command(subcommand)] + pub connectors: Connectors, + + #[arg( + value_enum, + long = "bad-data-policy", + required = false, + default_value_t = BadData::Drop, + env = "P_CONNECTOR_BAD_DATA_POLICY", + help = "Policy for handling bad data" + )] + pub bad_data: BadData, + + #[arg( + long = "metrics-enabled", + env = "P_CONNECTOR_METRICS_ENABLED", + required = false, + default_value_t = true, + help = "Enable metrics collection" + )] pub metrics_enabled: bool, - pub instance_id: String, } -impl Default for ConnectorConfig { - fn default() -> Self { - Self { - connector_type: ConnectorType::KafkaSource, - name: String::from("parseable-connectors"), - buffer_size: 10000, - buffer_timeout: Duration::from_millis(500), - topics: vec![], - offset_mode: SourceOffset::Earliest, - bad_data: None, - max_retries: 3, - retry_interval: Duration::from_secs(5), - kafka_config: Some(KafkaConfig::default()), - metrics_enabled: true, - instance_id: String::from("parseable-connectors"), - } - } +#[derive(Debug, Clone, Subcommand)] +pub enum Connectors { + #[command( + name = "kafka-sink", + about = "Configure Kafka Sink", + next_help_heading = "KAFKA OPTIONS" + )] + KafkaSink(#[command(flatten)] KafkaConfig), + // KinesisSink, + // PulsarSink, etc. } -impl ConnectorConfig { - pub fn builder() -> ConnectorConfigBuilder { - ConnectorConfigBuilder::default() - } - - pub fn validate(&self) -> anyhow::Result<(), ConnectorError> { - if self.buffer_size == 0 { - return Err(ConnectorError::Config("Buffer size must be > 0".into())); - } - - if let Some(kafka_config) = &self.kafka_config { - self.validate_kafka_config(kafka_config)?; +impl fmt::Display for Connectors { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Connectors::KafkaSink(_) => write!(f, "KafkaSink"), } - - Ok(()) - } - - fn validate_kafka_config(&self, config: &KafkaConfig) -> Result<(), ConnectorError> { - if config.bootstrap_servers.is_empty() { - return Err(ConnectorError::Config("Bootstrap servers required".into())); - } - - if config.topics().is_empty() { - return Err(ConnectorError::Config("Topic name required".into())); - } - - Ok(()) } } -#[derive(Default)] -pub struct ConnectorConfigBuilder { - config: ConnectorConfig, +#[derive(ValueEnum, Default, Clone, Debug, PartialEq, Eq, Hash)] +pub enum BadData { + Fail, + #[default] + Drop, + Dlt, //TODO: Implement Dead Letter Topic support when needed } -impl ConnectorConfigBuilder { - pub fn connector_type(mut self, connector_type: ConnectorType) -> Self { - self.config.connector_type = connector_type; - self - } +impl FromStr for BadData { + type Err = String; - pub fn name(mut self, name: impl Into) -> Self { - self.config.name = name.into(); - self - } - - pub fn buffer_size(mut self, buffer_size: usize) -> Self { - self.config.buffer_size = buffer_size; - self - } - - pub fn buffer_timeout(mut self, buffer_timeout: Duration) -> Self { - self.config.buffer_timeout = buffer_timeout; - self - } - - pub fn max_retries(mut self, max_retries: u32) -> Self { - self.config.max_retries = max_retries; - self - } - - pub fn instance_id(mut self, instance_id: String) -> Self { - self.config.instance_id = instance_id; - self - } - - pub fn retry_interval(mut self, retry_interval: Duration) -> Self { - self.config.retry_interval = retry_interval; - self - } - - pub fn metrics_enabled(mut self, metrics_enabled: bool) -> Self { - self.config.metrics_enabled = metrics_enabled; - self - } - - pub fn topic(mut self, topics: Vec) -> Self { - self.config.topics = topics; - self - } - - pub fn offset_mode(mut self, offset_mode: SourceOffset) -> Self { - self.config.offset_mode = offset_mode; - self - } - - pub fn kafka_config(mut self, kafka_config: KafkaConfig) -> Self { - self.config.kafka_config = Some(kafka_config); - self + fn from_str(s: &str) -> Result { + match s.to_lowercase().as_str() { + "drop" => Ok(BadData::Drop), + "fail" => Ok(BadData::Fail), + "dlt" => Ok(BadData::Dlt), + _ => Err(format!("Invalid bad data policy: {}", s)), + } } +} - pub fn build(self) -> anyhow::Result { - let config = self.config; - config.validate()?; - Ok(config) +impl Default for ConnectorConfig { + fn default() -> Self { + ConnectorConfig { + bad_data: BadData::Drop, + metrics_enabled: true, + connectors: Connectors::KafkaSink(KafkaConfig::default()), + } } } -#[cfg(test)] -mod tests { - use super::*; - #[test] - fn test_config_validation() { - let result = ConnectorConfig::builder() - .connector_type(ConnectorType::KafkaSource) - .buffer_size(0) - .build(); - - assert!(result.is_err()); +impl ConnectorConfig { + pub fn from(matches: &ArgMatches) -> Option { + matches + .subcommand_matches("connectors") + .and_then(|connector_matches| { + match ConnectorConfig::from_arg_matches(connector_matches) { + Ok(config) => Some(config), + Err(err) => err.exit(), + } + }) } } diff --git a/src/connectors/common/mod.rs b/src/connectors/common/mod.rs index ac995c5a1..f9bf8b137 100644 --- a/src/connectors/common/mod.rs +++ b/src/connectors/common/mod.rs @@ -19,7 +19,6 @@ pub mod config; pub mod processor; pub mod shutdown; -pub mod types; #[derive(Debug, thiserror::Error)] pub enum ConnectorError { diff --git a/src/connectors/common/types/mod.rs b/src/connectors/common/types/mod.rs deleted file mode 100644 index 7ff6fe4b1..000000000 --- a/src/connectors/common/types/mod.rs +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Parseable Server (C) 2022 - 2024 Parseable, Inc. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -use serde::{Deserialize, Serialize}; -use std::fmt; -use std::fmt::{Display, Formatter}; -use std::hash::Hash; - -#[derive(Debug, Clone, Hash, Eq, PartialEq, Serialize, Deserialize)] -pub enum ConnectorType { - KafkaSource, -} - -impl Display for ConnectorType { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - match self { - ConnectorType::KafkaSource => write!(f, "kafka_source"), - } - } -} - -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq, Hash, PartialOrd)] -#[serde(rename_all = "snake_case")] -pub enum ConnectionType { - Source, - Sink, -} - -impl Display for ConnectionType { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - match self { - ConnectionType::Source => write!(f, "SOURCE"), - ConnectionType::Sink => write!(f, "SINK"), - } - } -} - -impl TryFrom for ConnectionType { - type Error = String; - - fn try_from(value: String) -> anyhow::Result { - match value.to_lowercase().as_str() { - "source" => Ok(ConnectionType::Source), - "sink" => Ok(ConnectionType::Sink), - _ => Err(format!("Invalid connection type: {}", value)), - } - } -} - -#[derive(Default, Serialize, Deserialize, Clone, Debug, PartialEq, Eq, Hash, PartialOrd)] -#[serde(rename_all = "snake_case")] -pub enum BadData { - Fail, - #[default] - Drop, - Dlt, // TODO: Implement DLT -} diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index 4ecc1e967..de2a824ec 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -1,654 +1,954 @@ -/* - * Parseable Server (C) 2022 - 2024 Parseable, Inc. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -use anyhow::bail; -use rdkafka::Offset; -use rustls::pki_types::{CertificateDer, PrivateKeyDer}; -use rustls_pemfile::{self, Item}; +use clap::{Args, ValueEnum}; +use rdkafka::{ClientConfig, Offset}; use serde::{Deserialize, Serialize}; -use std::fmt::Display; -use std::io::BufReader; -use std::sync::Arc; -use tracing::{debug, info}; +use std::path::PathBuf; +use std::time::Duration; -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(ValueEnum, Debug, Clone)] +pub enum SourceOffset { + Earliest, + Latest, + Group, +} + +impl SourceOffset { + pub fn get_offset(&self) -> Offset { + match self { + SourceOffset::Earliest => Offset::Beginning, + SourceOffset::Latest => Offset::End, + SourceOffset::Group => Offset::Stored, + } + } +} + +#[derive(Debug, Clone, Args)] +#[command(name = "kafka-config", about = "Configure Kafka connector settings")] +#[group(id = "kafka")] pub struct KafkaConfig { - // Common configuration + #[arg( + long = "bootstrap-servers", + env = "P_KAFKA_BOOTSTRAP_SERVERS", + value_name = "bootstrap-servers", + required = true, + help = "Comma-separated list of Kafka bootstrap servers" + )] pub bootstrap_servers: String, - topics: Vec, - pub client_id: Option, - // Component-specific configurations - #[serde(skip_serializing_if = "Option::is_none")] + #[arg( + long = "client-id", + env = "P_KAFKA_CLIENT_ID", + required = false, + default_value_t = String::from("parseable-connect"), + value_name = "client_id", + help = "Client ID for Kafka connection" + )] + pub client_id: String, + + #[command(flatten)] + #[group(id = "consumer", required = false)] pub consumer: Option, - #[serde(skip_serializing_if = "Option::is_none")] + + #[command(flatten)] + #[group(id = "producer", required = false)] pub producer: Option, - // Security and advanced settings - #[serde(skip_serializing_if = "Option::is_none")] + #[command(flatten)] + #[group(id = "security", required = false)] pub security: Option, } -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(Debug, Clone, Args)] pub struct ConsumerConfig { - // Consumer group configuration + #[arg( + long = "consumer-topics", + env = "P_KAFKA_CONSUMER_TOPICS", + value_name = "consumer-topics", + required = true, + value_delimiter = ',', + help = "Comma-separated list of topics" + )] + pub topics: Vec, + + #[arg( + long = "consumer-group-id", + env = "P_KAFKA_CONSUMER_GROUP_ID", + value_name = "id", + required = false, + default_value_t = String::from("parseable-connect-cg"), + help = "Consumer group ID" + )] pub group_id: String, - pub group_instance_id: Option, + + // uses per partition stream micro-batch buffer size + #[arg( + long = "buffer-size", + env = "P_KAFKA_CONSUMER_BUFFER_SIZE", + value_name = "size", + required = false, + default_value_t = 10000, + help = "Size of the buffer for batching records" + )] + pub buffer_size: usize, + + // uses per partition stream micro-batch buffer timeout + #[clap( + value_parser = humantime::parse_duration, + default_value= "10000ms", + long = "buffer-timeout", + env = "P_KAFKA_CONSUMER_BUFFER_TIMEOUT", + value_name = "timeout_ms", + required = false, + help = "Timeout for buffer flush in milliseconds" + )] + pub buffer_timeout: Duration, + + #[arg( + long = "consumer-group-instance-id", + required = false, + env = "P_KAFKA_CONSUMER_GROUP_INSTANCE_ID", + default_value_t = format!("parseable-connect-cg-ii-{}", rand::random::()).to_string(), + help = "Group instance ID for static membership" + )] + pub group_instance_id: String, + + #[arg( + long = "consumer-partition-strategy", + env = "P_KAFKA_CONSUMER_PARTITION_STRATEGY", + required = false, + default_value_t = String::from("roundrobin,range"), + help = "Partition assignment strategy" + )] pub partition_assignment_strategy: String, - // Session handling + #[arg( + long = "consumer-session-timeout", + env = "P_KAFKA_CONSUMER_SESSION_TIMEOUT", + required = false, + default_value_t = 60000, + help = "Session timeout in milliseconds" + )] pub session_timeout_ms: u32, + + #[arg( + long = "consumer-heartbeat-interval", + env = "P_KAFKA_CONSUMER_HEARTBEAT_INTERVAL", + required = false, + default_value_t = 3000, + help = "Heartbeat interval in milliseconds" + )] pub heartbeat_interval_ms: u32, + + #[arg( + long = "consumer-max-poll-interval", + env = "P_KAFKA_CONSUMER_MAX_POLL_INTERVAL", + required = false, + default_value_t = 300000, + help = "Maximum poll interval in milliseconds" + )] pub max_poll_interval_ms: u32, - // Offset management - pub enable_auto_commit: bool, - pub auto_commit_interval_ms: u32, + #[arg( + long = "consumer-enable-auto-offset-store", + env = "P_KAFKA_CONSUMER_ENABLE_AUTO_OFFSET_STORE", + required = false, + default_value_t = true, + help = "Enable auto offset store" + )] pub enable_auto_offset_store: bool, - pub auto_offset_reset: String, - // Fetch configuration + #[clap( + value_enum, + long = "consumer-auto-offset-reset", + required = false, + env = "P_KAFKA_CONSUMER_AUTO_OFFSET_RESET", + default_value_t = SourceOffset::Earliest, + help = "Auto offset reset behavior" + )] + pub auto_offset_reset: SourceOffset, + + #[arg( + long = "consumer-fetch-min-bytes", + env = "P_KAFKA_CONSUMER_FETCH_MIN_BYTES", + default_value_t = 1, + required = false, + help = "Minimum bytes to fetch" + )] pub fetch_min_bytes: u32, + + #[arg( + long = "consumer-fetch-max-bytes", + env = "P_KAFKA_CONSUMER_FETCH_MAX_BYTES", + default_value_t = 52428800, + required = false, + help = "Maximum bytes to fetch" + )] pub fetch_max_bytes: u32, + + #[arg( + long = "consumer-fetch-max-wait", + env = "P_KAFKA_CONSUMER_FETCH_MAX_WAIT", + default_value_t = 500, + required = false, + help = "Maximum wait time for fetch in milliseconds" + )] pub fetch_max_wait_ms: u32, + + #[arg( + long = "consumer-max-partition-fetch-bytes", + env = "P_KAFKA_CONSUMER_MAX_PARTITION_FETCH_BYTES", + required = false, + default_value_t = 1048576, + help = "Maximum bytes to fetch per partition" + )] pub max_partition_fetch_bytes: u32, - // Queue configuration + #[arg( + long = "consumer-queued-min-messages", + env = "P_KAFKA_CONSUMER_QUEUED_MIN_MESSAGES", + required = false, + default_value_t = 100000, + help = "Minimum messages to queue" + )] pub queued_min_messages: u32, + + #[arg( + long = "consumer-queued-max-messages-kbytes", + env = "P_KAFKA_CONSUMER_QUEUED_MAX_MESSAGES_KBYTES", + required = false, + default_value_t = 65536, + help = "Maximum message queue size in KBytes" + )] pub queued_max_messages_kbytes: u32, - // Processing configuration + #[arg( + long = "consumer-enable-partition-eof", + env = "P_KAFKA_CONSUMER_ENABLE_PARTITION_EOF", + required = false, + default_value_t = false, + help = "Enable partition EOF" + )] pub enable_partition_eof: bool, + + #[arg( + long = "consumer-check-crcs", + env = "P_KAFKA_CONSUMER_CHECK_CRCS", + required = false, + default_value_t = false, + help = "Check CRCs on messages" + )] pub check_crcs: bool, + + #[arg( + long = "consumer-isolation-level", + env = "P_KAFKA_CONSUMER_ISOLATION_LEVEL", + required = false, + default_value_t = String::from("read_committed"), + help = "Transaction isolation level" + )] pub isolation_level: String, - pub fetch_message_max_bytes: String, - pub stats_interval_ms: Option, + + #[arg( + long = "consumer-fetch-message-max-bytes", + env = "P_KAFKA_CONSUMER_FETCH_MESSAGE_MAX_BYTES", + required = false, + default_value_t = 1048576, + help = "Maximum bytes per message" + )] + pub fetch_message_max_bytes: u64, + + #[arg( + long = "consumer-stats-interval", + env = "P_KAFKA_CONSUMER_STATS_INTERVAL", + required = false, + default_value_t = 10000, + help = "Statistics interval in milliseconds" + )] + pub stats_interval_ms: u64, } -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(Debug, Clone, Args)] pub struct ProducerConfig { + #[arg( + long = "producer-acks", + env = "P_KAFKA_PRODUCER_ACKS", + required = false, + default_value_t = String::from("all"), + value_parser = ["0", "1", "all"], + help = "Number of acknowledgments the producer requires" + )] pub acks: String, + + #[arg( + long = "producer-compression-type", + env = "P_KAFKA_PRODUCER_COMPRESSION_TYPE", + required = false, + default_value_t= String::from("lz4"), + value_parser = ["none", "gzip", "snappy", "lz4", "zstd"], + help = "Compression type for messages" + )] pub compression_type: String, + + #[arg( + long = "producer-batch-size", + env = "P_KAFKA_PRODUCER_BATCH_SIZE", + required = false, + default_value_t = 16384, + help = "Maximum size of a request in bytes" + )] pub batch_size: u32, + + #[arg( + long = "producer-linger-ms", + env = "P_KAFKA_PRODUCER_LINGER_MS", + required = false, + default_value_t = 5, + help = "Delay to wait for more messages in the same batch" + )] pub linger_ms: u32, - pub delivery_timeout_ms: u32, + + #[arg( + long = "producer-message-timeout-ms", + env = "P_KAFKA_PRODUCER_MESSAGE_TIMEOUT_MS", + required = false, + default_value_t = 120000, + help = "Local message timeout" + )] + pub message_timeout_ms: u32, + + #[arg( + long = "producer-max-inflight", + env = "P_KAFKA_PRODUCER_MAX_INFLIGHT", + required = false, + default_value_t = 5, + help = "Maximum number of in-flight requests per connection" + )] pub max_in_flight_requests_per_connection: u32, - pub max_request_size: u32, - pub enable_idempotence: bool, - pub transaction_timeout_ms: Option, - pub queue_buffering_max_messages: u32, - queue_buffering_max_ms: u32, - retry_backoff_ms: u32, - batch_num_messages: u32, - retries: u32, -} -impl Default for ConsumerConfig { - fn default() -> Self { - Self { - group_id: "default-cg".to_string(), - group_instance_id: Some("default-cg-ii".to_string()), - // NOTE: cooperative-sticky does not work well in rdkafka when using manual commit. - // @see https://github.com/confluentinc/librdkafka/issues/4629 - // @see https://github.com/confluentinc/librdkafka/issues/4368 - partition_assignment_strategy: "roundrobin,range".to_string(), - session_timeout_ms: 60000, - heartbeat_interval_ms: 3000, - max_poll_interval_ms: 300000, - enable_auto_commit: false, - auto_commit_interval_ms: 5000, - enable_auto_offset_store: true, - auto_offset_reset: "earliest".to_string(), - fetch_min_bytes: 1, - fetch_max_bytes: 52428800, - fetch_max_wait_ms: 500, - max_partition_fetch_bytes: 1048576, - queued_min_messages: 100000, - queued_max_messages_kbytes: 65536, - enable_partition_eof: false, - check_crcs: false, - isolation_level: "read_committed".to_string(), - fetch_message_max_bytes: "1048576".to_string(), - stats_interval_ms: Some(10000), - } - } -} + #[arg( + long = "producer-message-max-bytes", + env = "P_KAFKA_PRODUCER_MESSAGE_MAX_BYTES", + required = false, + default_value_t = 1048576, + help = "Maximum size of a message in bytes" + )] + pub message_max_bytes: u32, + + #[arg( + long = "producer-enable-idempotence", + env = "P_KAFKA_PRODUCER_ENABLE_IDEMPOTENCE", + required = false, + default_value_t = true, + help = "Enable idempotent producer" + )] + pub enable_idempotence: bool, -impl Default for ProducerConfig { - fn default() -> Self { - Self { - acks: "all".to_string(), - compression_type: "lz4".to_string(), - batch_size: 16384, // 16KB default batch size - linger_ms: 5, // Small latency for better batching - delivery_timeout_ms: 120000, // 2 minute delivery timeout - max_in_flight_requests_per_connection: 5, - max_request_size: 1048576, // 1MB max request size - enable_idempotence: true, // Ensure exactly-once delivery - transaction_timeout_ms: Some(60000), // 1 minute transaction timeout - queue_buffering_max_messages: 100000, // Producer queue size - queue_buffering_max_ms: 100, // Max time to wait before sending - retry_backoff_ms: 100, // Backoff time between retries - batch_num_messages: 10000, // Messages per batch - retries: 3, // Number of retries - } - } -} + #[arg( + long = "producer-transaction-timeout-ms", + env = "P_KAFKA_PRODUCER_TRANSACTION_TIMEOUT_MS", + required = false, + default_value_t = 60000, + help = "Transaction timeout" + )] + pub transaction_timeout_ms: u64, + + #[arg( + long = "producer-buffer-memory", + env = "P_KAFKA_PRODUCER_BUFFER_MEMORY", + required = false, + default_value_t = 33554432, + help = "Total bytes of memory the producer can use" + )] + pub buffer_memory: u32, + + #[arg( + long = "producer-retry-backoff-ms", + env = "P_KAFKA_PRODUCER_RETRY_BACKOFF_MS", + required = false, + default_value_t = 100, + help = "Time to wait before retrying a failed request" + )] + pub retry_backoff_ms: u32, + + #[arg( + long = "producer-request-timeout-ms", + env = "P_KAFKA_PRODUCER_REQUEST_TIMEOUT_MS", + required = false, + default_value_t = 30000, + help = "Time to wait for a response from brokers" + )] + pub request_timeout_ms: u32, + + #[arg( + long = "producer-queue-buffering-max-messages", + env = "P_KAFKA_PRODUCER_QUEUE_BUFFERING_MAX_MESSAGES", + required = false, + default_value_t = 100000, + help = "Maximum number of messages allowed on the producer queue" + )] + pub queue_buffering_max_messages: u32, -#[derive(Debug, Clone, Serialize, Deserialize)] -#[warn(non_camel_case_types)] -#[serde(rename_all = "UPPERCASE")] -#[allow(non_camel_case_types)] -pub enum SecurityProtocol { - Plaintext, - SSL, - SASL_SSL, - SASL_PLAINTEXT, -} + #[arg( + long = "producer-queue-buffering-max-kbytes", + env = "P_KAFKA_PRODUCER_QUEUE_BUFFERING_MAX_KBYTES", + required = false, + default_value_t = 1048576, + help = "Maximum total message size sum allowed on the producer queue" + )] + pub queue_buffering_max_kbytes: u32, + + #[arg( + long = "producer-delivery-timeout-ms", + env = "P_KAFKA_PRODUCER_DELIVERY_TIMEOUT_MS", + required = false, + default_value_t = 120000, + help = "Maximum time to report success or failure after send" + )] + pub delivery_timeout_ms: u32, -impl Display for SecurityProtocol { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let str = match self { - SecurityProtocol::Plaintext => "PLAINTEXT", - SecurityProtocol::SSL => "SSL", - SecurityProtocol::SASL_SSL => "SASL_SSL", - SecurityProtocol::SASL_PLAINTEXT => "SASL_PLAINTEXT", - } - .to_string(); - write!(f, "{}", str) - } + #[arg( + long = "producer-max-retries", + env = "P_KAFKA_PRODUCER_MAX_RETRIES", + required = false, + default_value_t = 2147483647, + help = "Maximum number of retries per message" + )] + pub max_retries: u32, + + #[arg( + long = "producer-retry-backoff-max-ms", + env = "P_KAFKA_PRODUCER_RETRY_BACKOFF_MAX_MS", + required = false, + default_value_t = 1000, + help = "Maximum back-off time between retries" + )] + pub retry_backoff_max_ms: u32, } -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(Debug, Clone, Args)] pub struct SecurityConfig { + #[clap( + value_enum, + long = "security-protocol", + env = "P_KAFKA_SECURITY_PROTOCOL", + required = false, + default_value_t = SecurityProtocol::Plaintext, + help = "Security protocol" + )] pub protocol: SecurityProtocol, - #[serde(skip_serializing_if = "Option::is_none")] - pub ssl_config: Option, - #[serde(skip_serializing_if = "Option::is_none")] - pub sasl_config: Option, -} -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct SSLConfig { - pub ca_certificate_pem: String, - pub client_certificate_pem: String, - pub client_key_pem: String, -} + // SSL Configuration + #[arg( + long = "ssl-ca-location", + env = "P_KAFKA_SSL_CA_LOCATION", + required = false, + help = "CA certificate file path" + )] + pub ssl_ca_location: Option, + + #[arg( + long = "ssl-certificate-location", + env = "P_KAFKA_SSL_CERTIFICATE_LOCATION", + required = false, + help = "Client certificate file path" + )] + pub ssl_certificate_location: Option, + + #[arg( + long = "ssl-key-location", + env = "P_KAFKA_SSL_KEY_LOCATION", + required = false, + help = "Client key file path" + )] + pub ssl_key_location: Option, + + // SASL Configuration + #[arg( + value_enum, + long = "sasl-mechanism", + env = "P_KAFKA_SASL_MECHANISM", + required = false, + help = "SASL mechanism" + )] + pub sasl_mechanism: Option, + + #[arg( + long = "sasl-username", + env = "P_KAFKA_SASL_USERNAME", + required = false, + help = "SASL username" + )] + pub sasl_username: Option, + + #[arg( + long = "sasl-password", + env = "P_KAFKA_SASL_PASSWORD", + required = false, + help = "SASL password" + )] + pub sasl_password: Option, + + #[arg( + long = "ssl-key-password", + env = "P_KAFKA_SSL_KEY_PASSWORD", + required = false, + help = "SSL key password" + )] + pub ssl_key_password: Option, + + // Kerberos configuration fields + #[arg( + long = "kerberos-service-name", + env = "P_KAFKA_KERBEROS_SERVICE_NAME", + required = false, + help = "Kerberos service name" + )] + pub kerberos_service_name: Option, -#[derive(Debug, Clone, Serialize, Deserialize)] -#[serde(rename_all = "UPPERCASE")] -pub enum SASLMechanism { - Plain, - ScramSha256, - ScramSha512, - GssAPI, -} + #[arg( + long = "kerberos-principal", + env = "P_KAFKA_KERBEROS_PRINCIPAL", + required = false, + help = "Kerberos principal" + )] + pub kerberos_principal: Option, -impl Display for SASLMechanism { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let str = match self { - SASLMechanism::Plain => "PLAIN", - SASLMechanism::ScramSha256 => "SCRAM-SHA-256", - SASLMechanism::ScramSha512 => "SCRAM-SHA-512", - SASLMechanism::GssAPI => "GSSAPI", - } - .to_string(); - write!(f, "{}", str) - } + #[arg( + long = "kerberos-keytab", + env = "P_KAFKA_KERBEROS_KEYTAB", + required = false, + help = "Path to Kerberos keytab file" + )] + pub kerberos_keytab: Option, } -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct SASLConfig { - pub mechanism: SASLMechanism, - pub username: String, - pub password: String, - #[serde(skip_serializing_if = "Option::is_none")] - pub kerberos_service_name: Option, - #[serde(skip_serializing_if = "Option::is_none")] - pub kerberos_principal: Option, - #[serde(skip_serializing_if = "Option::is_none")] - pub kerberos_keytab: Option, -} +impl KafkaConfig { + pub fn to_rdkafka_consumer_config(&self) -> ClientConfig { + let mut config = ClientConfig::new(); -#[derive(Debug, Clone, Copy, Serialize, Deserialize)] -#[serde(rename_all = "lowercase")] -pub enum SourceOffset { - Earliest, - Latest, - Group, -} + // Basic configuration + config + .set("bootstrap.servers", &self.bootstrap_servers) + .set("client.id", &self.client_id); -impl SourceOffset { - pub fn get_offset(&self) -> Offset { - match self { - SourceOffset::Earliest => Offset::Beginning, - SourceOffset::Latest => Offset::End, - SourceOffset::Group => Offset::Stored, + // Consumer configuration + if let Some(consumer) = &self.consumer { + consumer.apply_to_config(&mut config); } - } -} -impl KafkaConfig { - pub fn new( - bootstrap_servers: String, - topics: Vec, - consumer_config: Option, - ) -> Self { - Self { - bootstrap_servers, - topics, - client_id: None, - consumer: consumer_config, - producer: None, - security: None, + // Security configuration + if let Some(security) = &self.security { + security.apply_to_config(&mut config); + } else { + config.set("security.protocol", "PLAINTEXT"); } + + config } - pub fn consumer_config(&self) -> rdkafka::ClientConfig { - let mut config = rdkafka::ClientConfig::new(); + pub fn to_rdkafka_producer_config(&self) -> ClientConfig { + let mut config = ClientConfig::new(); + + // Basic configuration config .set("bootstrap.servers", &self.bootstrap_servers) - .set("reconnect.backoff.ms", "100") - .set("reconnect.backoff.max.ms", "3600000"); + .set("client.id", &self.client_id); - if let Some(client_id) = &self.client_id { - config - .set("client.id", format!("parseable-{}-ci", client_id)) - .set("client.rack", format!("parseable-{}-cr", client_id)); + // Producer configuration + if let Some(producer) = &self.producer { + producer.apply_to_config(&mut config); + } + + // Security configuration + if let Some(security) = &self.security { + security.apply_to_config(&mut config); + } else { + config.set("security.protocol", "PLAINTEXT"); + } + + config + } + + pub fn consumer(&self) -> Option<&ConsumerConfig> { + self.consumer.as_ref() + } + + pub fn producer(&self) -> Option<&ProducerConfig> { + self.producer.as_ref() + } + + pub fn security(&self) -> Option<&SecurityConfig> { + self.security.as_ref() + } + + pub fn validate(&self) -> anyhow::Result<()> { + if self.bootstrap_servers.is_empty() { + anyhow::bail!("Bootstrap servers must not be empty"); } if let Some(consumer) = &self.consumer { - let enable_auto_commit = consumer.enable_auto_commit.to_string(); - let group_id = format!("parseable-{}-gi", &consumer.group_id); - info!("Setting group.id to {}", group_id); - config - .set("group.id", group_id) - .set("log_level", "7") - .set("enable.auto.commit", enable_auto_commit) - .set( - "enable.auto.offset.store", - consumer.enable_auto_offset_store.to_string(), - ) - .set("auto.offset.reset", &consumer.auto_offset_reset) - .set( - "partition.assignment.strategy", - &consumer.partition_assignment_strategy, - ) - .set( - "session.timeout.ms", - consumer.session_timeout_ms.to_string(), - ) - .set( - "heartbeat.interval.ms", - consumer.heartbeat_interval_ms.to_string(), - ) - .set( - "max.poll.interval.ms", - consumer.max_poll_interval_ms.to_string(), - ) - .set("fetch.min.bytes", consumer.fetch_min_bytes.to_string()) - .set("fetch.max.bytes", consumer.fetch_max_bytes.to_string()) - .set( - "fetch.message.max.bytes", - consumer.fetch_message_max_bytes.to_string(), - ) - .set( - "max.partition.fetch.bytes", - consumer.max_partition_fetch_bytes.to_string(), - ) - .set( - "queued.min.messages", - consumer.queued_min_messages.to_string(), - ) - .set( - "queued.max.messages.kbytes", - consumer.queued_max_messages_kbytes.to_string(), - ) - .set( - "enable.partition.eof", - consumer.enable_partition_eof.to_string(), - ) - .set("isolation.level", &consumer.isolation_level) - .set( - "statistics.interval.ms", - consumer.stats_interval_ms.unwrap_or(10000).to_string(), - ); + consumer.validate()?; + } - if let Some(instance_id) = &consumer.group_instance_id { - config.set("group.instance.id", instance_id); - } + if let Some(producer) = &self.producer { + producer.validate()?; } - self.apply_security_config(&mut config); + if let Some(security) = &self.security { + security.validate()?; + } - info!("Consumer configuration: {:?}", config); + Ok(()) + } +} + +impl ConsumerConfig { + pub fn validate(&self) -> anyhow::Result<()> { + if self.group_id.is_empty() { + anyhow::bail!("Consumer group ID must not be empty"); + } + Ok(()) + } + + fn apply_to_config(&self, config: &mut ClientConfig) { config + .set("group.id", &self.group_id) + .set( + "partition.assignment.strategy", + &self.partition_assignment_strategy, + ) + .set("session.timeout.ms", self.session_timeout_ms.to_string()) + .set( + "heartbeat.interval.ms", + self.heartbeat_interval_ms.to_string(), + ) + .set( + "max.poll.interval.ms", + self.max_poll_interval_ms.to_string(), + ) + .set("enable.auto.commit", "false") + .set("fetch.min.bytes", self.fetch_min_bytes.to_string()) + .set("fetch.max.bytes", self.fetch_max_bytes.to_string()) + .set( + "max.partition.fetch.bytes", + self.max_partition_fetch_bytes.to_string(), + ) + .set("isolation.level", self.isolation_level.to_string()) + .set("group.instance.id", self.group_instance_id.to_string()) + .set("statistics.interval.ms", self.stats_interval_ms.to_string()); + } + + pub fn topics(&self) -> Vec<&str> { + self.topics.iter().map(|t| t.as_str()).collect() } +} - pub fn producer_config(&self) -> rdkafka::config::ClientConfig { - let mut config = rdkafka::config::ClientConfig::new(); +impl ProducerConfig { + fn apply_to_config(&self, config: &mut ClientConfig) { config - .set("bootstrap.servers", &self.bootstrap_servers) - .set("reconnect.backoff.ms", "100") - .set("reconnect.backoff.max.ms", "3600000"); + .set("acks", self.acks.to_string()) + .set("compression.type", self.compression_type.to_string()) + .set("batch.size", self.batch_size.to_string()) + .set("linger.ms", self.linger_ms.to_string()) + .set("enable.idempotence", self.enable_idempotence.to_string()) + .set( + "max.in.flight.requests.per.connection", + self.max_in_flight_requests_per_connection.to_string(), + ) + .set("delivery.timeout.ms", self.delivery_timeout_ms.to_string()) + .set("retry.backoff.ms", self.retry_backoff_ms.to_string()) + .set( + "transaction.timeout.ms", + self.transaction_timeout_ms.to_string(), + ) + .set("request.timeout.ms", self.request_timeout_ms.to_string()) + .set("max.retries", self.max_retries.to_string()) + .set( + "retry.backoff.max.ms", + self.retry_backoff_max_ms.to_string(), + ) + .set("buffer.memory", self.buffer_memory.to_string()) + .set("message.timeout.ms", self.message_timeout_ms.to_string()) + .set("message.max.bytes", self.message_max_bytes.to_string()); + } - if let Some(client_id) = &self.client_id { - config - .set("client.id", format!("parseable-{}-ci", client_id)) - .set("client.rack", format!("parseable-{}-cr", client_id)); + fn validate(&self) -> anyhow::Result<()> { + if self.batch_size == 0 { + anyhow::bail!("Batch size must be greater than 0"); } - if let Some(producer_config) = &self.producer { - config - .set("acks", &producer_config.acks) - .set("compression.type", &producer_config.compression_type) - .set("batch.size", producer_config.batch_size.to_string()) - .set("linger.ms", producer_config.linger_ms.to_string()) - .set( - "delivery.timeout.ms", - producer_config.delivery_timeout_ms.to_string(), - ) - .set( - "max.in.flight.requests.per.connection", - producer_config - .max_in_flight_requests_per_connection - .to_string(), - ) - .set( - "max.request.size", - producer_config.max_request_size.to_string(), - ) - .set( - "enable.idempotence", - producer_config.enable_idempotence.to_string(), - ) - .set( - "batch.num.messages", - producer_config.batch_num_messages.to_string(), - ) - .set( - "queue.buffering.max.messages", - producer_config.queue_buffering_max_messages.to_string(), - ) - .set( - "queue.buffering.max.ms", - producer_config.queue_buffering_max_ms.to_string(), - ) - .set( - "retry.backoff.ms", - producer_config.retry_backoff_ms.to_string(), - ) - .set("retries", producer_config.retries.to_string()); - - if let Some(timeout) = producer_config.transaction_timeout_ms { - config.set("transaction.timeout.ms", timeout.to_string()); - } + if self.linger_ms > self.delivery_timeout_ms { + anyhow::bail!("Linger time cannot be greater than delivery timeout"); } - self.apply_security_config(&mut config); - - config + Ok(()) } +} - fn apply_security_config(&self, config: &mut rdkafka::ClientConfig) { - let security = match &self.security { - Some(sec) => sec, - None => { - debug!("No security configuration provided, using PLAINTEXT"); - config.set("security.protocol", "plaintext"); - return; - } - }; - - config.set( - "security.protocol", - security.protocol.to_string().to_lowercase(), - ); +impl SecurityConfig { + fn apply_to_config(&self, config: &mut ClientConfig) { + // Set security protocol + config.set("security.protocol", self.protocol.to_string()); + // Configure SSL if enabled if matches!( - security.protocol, - SecurityProtocol::SSL | SecurityProtocol::SASL_SSL + self.protocol, + SecurityProtocol::Ssl | SecurityProtocol::SaslSsl ) { - if let Some(ssl) = &security.ssl_config { - debug!("Applying SSL configuration"); - config - .set("ssl.ca.pem", &ssl.ca_certificate_pem) - .set("ssl.certificate.pem", &ssl.client_certificate_pem) - .set("ssl.key.pem", &ssl.client_key_pem); - } else { - panic!( - "SSL configuration required for {:?} protocol", - security.protocol + if let Some(ref path) = self.ssl_ca_location { + config.set("ssl.ca.location", path.to_string_lossy().to_string()); + } + if let Some(ref path) = self.ssl_certificate_location { + config.set( + "ssl.certificate.location", + path.to_string_lossy().to_string(), ); } + if let Some(ref path) = self.ssl_key_location { + config.set("ssl.key.location", path.to_string_lossy().to_string()); + } + if let Some(ref password) = self.ssl_key_password { + config.set("ssl.key.password", password); + } } + // Configure SASL if enabled if matches!( - security.protocol, - SecurityProtocol::SASL_SSL | SecurityProtocol::SASL_PLAINTEXT + self.protocol, + SecurityProtocol::SaslSsl | SecurityProtocol::SaslPlaintext ) { - if let Some(sasl) = &security.sasl_config { - debug!( - "Applying SASL configuration with mechanism: {}", - sasl.mechanism.to_string() - ); - config - .set("sasl.mechanism", sasl.mechanism.to_string()) - .set("sasl.username", &sasl.username) - .set("sasl.password", &sasl.password); - - // Apply Kerberos-specific configuration if using GSSAPI - if matches!(sasl.mechanism, SASLMechanism::GssAPI) { - if let Some(service_name) = &sasl.kerberos_service_name { - config.set("sasl.kerberos.service.name", service_name); - } - if let Some(principal) = &sasl.kerberos_principal { - config.set("sasl.kerberos.principal", principal); - } - if let Some(keytab) = &sasl.kerberos_keytab { - config.set("sasl.kerberos.keytab", keytab); - } - } - } else { - panic!( - "SASL configuration required for {:?} protocol", - security.protocol - ); + if let Some(ref mechanism) = self.sasl_mechanism { + config.set("sasl.mechanism", mechanism.to_string()); + } + if let Some(ref username) = self.sasl_username { + config.set("sasl.username", username); + } + if let Some(ref password) = self.sasl_password { + config.set("sasl.password", password); } - } - } -} -impl Default for KafkaConfig { - fn default() -> Self { - Self { - // Common configuration with standard broker port - bootstrap_servers: "localhost:9092".to_string(), - topics: vec![], - client_id: Some("parseable-connect".to_string()), - // Component-specific configurations with production-ready defaults - consumer: Some(ConsumerConfig::default()), - producer: Some(ProducerConfig::default()), + // Configure Kerberos settings if using GSSAPI + if matches!(self.sasl_mechanism, Some(SaslMechanism::Gssapi)) { + if let Some(ref service) = self.kerberos_service_name { + config.set("sasl.kerberos.service.name", service); + } + if let Some(ref principal) = self.kerberos_principal { + config.set("sasl.kerberos.principal", principal); + } + if let Some(ref keytab) = self.kerberos_keytab { + config.set("sasl.kerberos.keytab", keytab.to_string_lossy().to_string()); + } + } - // Security defaults to plaintext for development - // Production environments should explicitly configure security - security: Some(SecurityConfig { - protocol: SecurityProtocol::Plaintext, - ssl_config: None, - sasl_config: None, - }), + // TODO: Implement OAuthBearer mechanism for SASL when needed. This depends on the vendor (on-prem, Confluent Kafka, AWS MSK, etc.). } } -} -impl KafkaConfig { - pub fn builder() -> KafkaConfigBuilder { - KafkaConfigBuilder::default() - } + fn validate(&self) -> anyhow::Result<()> { + match self.protocol { + SecurityProtocol::Ssl | SecurityProtocol::SaslSsl => { + if self.ssl_ca_location.is_none() { + anyhow::bail!("CA certificate location is required for SSL"); + } + if self.ssl_certificate_location.is_none() { + anyhow::bail!("Client certificate location is required for SSL"); + } + if self.ssl_key_location.is_none() { + anyhow::bail!("Client key location is required for SSL"); + } + } + SecurityProtocol::SaslPlaintext => { + if self.sasl_mechanism.is_none() { + anyhow::bail!("SASL mechanism is required when SASL is enabled"); + } + if self.sasl_username.is_none() || self.sasl_password.is_none() { + anyhow::bail!("SASL username and password are required"); + } - pub fn topics(&self) -> Vec<&str> { - self.topics.iter().map(|s| s.as_str()).collect() + if matches!(self.sasl_mechanism, Some(SaslMechanism::Gssapi)) + && self.kerberos_service_name.is_none() + { + anyhow::bail!("Kerberos service name is required for GSSAPI"); + } + } + SecurityProtocol::Plaintext => {} // No validation needed for PLAINTEXT + } + Ok(()) } } -#[derive(Default, Debug)] -pub struct KafkaConfigBuilder { - config: KafkaConfig, +#[derive(ValueEnum, Debug, Clone, Serialize, Deserialize)] +pub enum SecurityProtocol { + Plaintext, + Ssl, + SaslSsl, + SaslPlaintext, } -impl KafkaConfigBuilder { - pub fn bootstrap_servers(mut self, servers: impl Into) -> Self { - self.config.bootstrap_servers = servers.into(); - self - } +impl std::str::FromStr for SecurityProtocol { + type Err = String; - pub fn topic(mut self, topics: Vec) -> Self { - self.config.topics = topics; - self + fn from_str(s: &str) -> Result { + match s.to_uppercase().as_str() { + "PLAINTEXT" => Ok(SecurityProtocol::Plaintext), + "SSL" => Ok(SecurityProtocol::Ssl), + "SASL_SSL" => Ok(SecurityProtocol::SaslSsl), + "SASL_PLAINTEXT" => Ok(SecurityProtocol::SaslPlaintext), + _ => Err(format!("Invalid security protocol: {}", s)), + } } +} - pub fn client_id(mut self, client_id: impl Into) -> Self { - self.config.client_id = Some(client_id.into()); - self +impl std::fmt::Display for SecurityProtocol { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + SecurityProtocol::Plaintext => write!(f, "PLAINTEXT"), + SecurityProtocol::Ssl => write!(f, "SSL"), + SecurityProtocol::SaslSsl => write!(f, "SASL_SSL"), + SecurityProtocol::SaslPlaintext => write!(f, "SASL_PLAINTEXT"), + } } +} - pub fn with_consumer(mut self, consumer: ConsumerConfig) -> Self { - self.config.consumer = Some(consumer); - self - } +#[derive(ValueEnum, Debug, Clone, Serialize, Deserialize)] +pub enum SaslMechanism { + Plain, + ScramSha256, + ScramSha512, + Gssapi, + OAuthBearer, +} - pub fn with_producer(mut self, producer: ProducerConfig) -> Self { - self.config.producer = Some(producer); - self +impl Default for KafkaConfig { + fn default() -> Self { + Self { + // Common configuration with standard broker port + bootstrap_servers: "localhost:9092".to_string(), + client_id: "parseable-connect".to_string(), + // Component-specific configurations with production-ready defaults + consumer: Some(ConsumerConfig::default()), + producer: Some(ProducerConfig::default()), + // Security configuration with plaintext protocol + security: Some(SecurityConfig::default()), + } } +} - pub fn with_security(mut self, security: SecurityConfig) -> Self { - self.config.security = Some(security); - self +impl std::fmt::Display for SaslMechanism { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + SaslMechanism::Plain => write!(f, "PLAIN"), + SaslMechanism::ScramSha256 => write!(f, "SCRAM-SHA-256"), + SaslMechanism::ScramSha512 => write!(f, "SCRAM-SHA-512"), + SaslMechanism::Gssapi => write!(f, "GSSAPI"), + SaslMechanism::OAuthBearer => write!(f, "OAUTHBEARER"), + } } - - pub fn build(self) -> anyhow::Result { - let config = self.config; - - if config.bootstrap_servers.is_empty() { - anyhow::bail!("bootstrap_servers cannot be empty"); +} +impl Default for ProducerConfig { + fn default() -> Self { + Self { + acks: "all".to_string(), + compression_type: "lz4".to_string(), + batch_size: 16384, // 16KB default batch size + linger_ms: 5, // Small latency for better batching + delivery_timeout_ms: 120000, // 2 minute delivery timeout + max_retries: 20, + max_in_flight_requests_per_connection: 5, + message_max_bytes: 1048576, // 1MB maximum message size + enable_idempotence: true, // Ensure exactly-once delivery + transaction_timeout_ms: 60000, // 1 minute transaction timeout + queue_buffering_max_messages: 100000, // Producer queue size + retry_backoff_ms: 100, // Backoff time between retries + message_timeout_ms: 120000, // 2 minute message timeout + buffer_memory: 33554432, // 32MB buffer memory + request_timeout_ms: 60000, // 60 second request timeout + queue_buffering_max_kbytes: 1048576, // 1MB maximum queue size + retry_backoff_max_ms: 1000, // Maximum backoff time between retries } - - Ok(config) } } -#[derive(Debug, Clone)] -#[allow(dead_code)] -pub struct KafkaCertificates { - ca_certificate: Arc>, - client_certificate: Arc>, - client_key: Arc>, +impl Default for ConsumerConfig { + fn default() -> Self { + Self { + topics: vec![], + group_id: "parseable-connect-cg".to_string(), + buffer_size: 10_000, + buffer_timeout: Duration::from_millis(5000), + group_instance_id: "parseable-cg-ii".to_string(), + // NOTE: cooperative-sticky does not work well in rdkafka when using manual commit. + // @see https://github.com/confluentinc/librdkafka/issues/4629 + // @see https://github.com/confluentinc/librdkafka/issues/4368 + partition_assignment_strategy: "roundrobin,range".to_string(), + session_timeout_ms: 60000, + heartbeat_interval_ms: 3000, + max_poll_interval_ms: 300000, + enable_auto_offset_store: true, + auto_offset_reset: SourceOffset::Earliest, + fetch_min_bytes: 1, + fetch_max_bytes: 52428800, + fetch_max_wait_ms: 500, + max_partition_fetch_bytes: 1048576, + queued_min_messages: 100000, + queued_max_messages_kbytes: 65536, + enable_partition_eof: false, + check_crcs: false, + isolation_level: "read_committed".to_string(), + fetch_message_max_bytes: 1048576, + stats_interval_ms: 10000, + } + } } -#[allow(dead_code)] -fn parse_first_certificate(pem: &str) -> anyhow::Result> { - let mut reader = BufReader::new(pem.as_bytes()); - let items = rustls_pemfile::read_all(&mut reader); - - for item in items.flatten() { - if let Item::X509Certificate(cert_data) = item { - return Ok(cert_data); +impl Default for SecurityConfig { + fn default() -> Self { + Self { + protocol: SecurityProtocol::Plaintext, + ssl_ca_location: None, + ssl_certificate_location: None, + ssl_key_location: None, + sasl_mechanism: None, + sasl_username: None, + sasl_password: None, + ssl_key_password: None, + kerberos_service_name: None, + kerberos_principal: None, + kerberos_keytab: None, } } - bail!("No certificate found in PEM") } -#[allow(dead_code)] -fn parse_first_private_key(pem: &str) -> anyhow::Result> { - let mut reader = BufReader::new(pem.as_bytes()); - let items = rustls_pemfile::read_all(&mut reader); - - for item in items { - if let Ok(Item::Pkcs1Key(key_data)) = item { - return Ok(key_data.into()); - } - if let Ok(Item::Pkcs8Key(key_data)) = item { - return Ok(key_data.into()); - } - if let Ok(Item::Sec1Key(key_data)) = item { - return Ok(key_data.into()); +impl std::str::FromStr for SaslMechanism { + type Err = String; + + fn from_str(s: &str) -> Result { + match s.to_uppercase().as_str() { + "PLAIN" => Ok(SaslMechanism::Plain), + "SCRAM-SHA-256" => Ok(SaslMechanism::ScramSha256), + "SCRAM-SHA-512" => Ok(SaslMechanism::ScramSha512), + "GSSAPI" => Ok(SaslMechanism::Gssapi), + "OAUTHBEARER" => Ok(SaslMechanism::OAuthBearer), + _ => Err(format!("Invalid SASL mechanism: {}", s)), } } +} - bail!("No private key found in PEM") +#[derive(Debug, Clone, Serialize, Deserialize)] +pub enum Acks { + None, + Leader, + All, } -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_consumer_config() { - let consumer_config = ConsumerConfig { - group_id: "test-group".to_string(), - partition_assignment_strategy: "cooperative-sticky".to_string(), - ..ConsumerConfig::default() - }; - - let config = KafkaConfig::new( - "localhost:9092".to_string(), - vec!["test-topic".to_string()], - Some(consumer_config), - ); - - let rdkafka_config = config.consumer_config(); - assert_eq!( - rdkafka_config.get("group.id"), - Some("parseable-test-group-gi") - ); - assert_eq!( - rdkafka_config.get("partition.assignment.strategy"), - Some("cooperative-sticky") - ); - } +impl std::str::FromStr for Acks { + type Err = String; - #[test] - fn test_default_kafka_config() { - let config = KafkaConfig::default(); - assert_eq!(config.bootstrap_servers, "localhost:9092"); - assert!(config.topics.is_empty()); - assert!(config.consumer.is_some()); - assert!(config.producer.is_some()); - - if let Some(producer) = config.producer { - assert_eq!(producer.acks, "all"); - assert!(producer.enable_idempotence); - assert_eq!(producer.compression_type, "lz4"); + fn from_str(s: &str) -> Result { + match s { + "0" => Ok(Acks::None), + "1" => Ok(Acks::Leader), + "all" => Ok(Acks::All), + _ => Err(format!("Invalid acks value: {}", s)), } } - - #[test] - fn test_kafka_config_builder() { - let config = KafkaConfig::builder() - .bootstrap_servers("kafka1:9092,kafka2:9092") - .topic(vec!["test-topic".to_string()]) - .client_id("test-client") - .build() - .unwrap(); - - assert_eq!(config.bootstrap_servers, "kafka1:9092,kafka2:9092"); - assert_eq!(config.topics.first().unwrap(), "test-topic"); - assert_eq!(config.client_id, Some("test-client".to_string())); - } } diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs index d81f2ed82..31c72f703 100644 --- a/src/connectors/kafka/consumer.rs +++ b/src/connectors/kafka/consumer.rs @@ -238,7 +238,7 @@ impl KafkaStreams { info!("Creating Kafka consumer from configs {:#?}", context.config); let kafka_config = &context.config; - let consumer_config = kafka_config.consumer_config(); + let consumer_config = kafka_config.to_rdkafka_consumer_config(); info!("Consumer configs: {:#?}", &consumer_config); let consumer: StreamConsumer = consumer_config @@ -250,14 +250,17 @@ impl KafkaStreams { } let consumer = Arc::new(consumer); + let topics = kafka_config + .consumer() + .expect("Consumer config is missing") + .topics(); - let topics = &kafka_config.topics(); - KafkaStreams::subscribe(&consumer, topics); + KafkaStreams::subscribe(&consumer, &topics); consumer } - fn subscribe(consumer: &Arc, topics: &Vec<&str>) { + fn subscribe(consumer: &Arc, topics: &[&str]) { match consumer.subscribe(topics) { Ok(_) => { info!("Subscribed to topics: {:?}", topics); diff --git a/src/connectors/kafka/metrics.rs b/src/connectors/kafka/metrics.rs index 7cc11ff5b..774ae4779 100644 --- a/src/connectors/kafka/metrics.rs +++ b/src/connectors/kafka/metrics.rs @@ -112,43 +112,10 @@ impl KafkaMetricsCollector { pub fn new(stats: Arc>) -> anyhow::Result { let mut descs = Vec::new(); - fn create_gauge_vec( - name: &str, - help: &str, - labels: &[&str], - descs: &mut Vec, - ) -> IntGaugeVec { - let gauge = IntGaugeVec::new(Opts::new(name, help), labels).unwrap(); - descs.extend(gauge.clone().desc().into_iter().cloned()); - gauge - } - - fn create_counter_vec( - name: &str, - help: &str, - labels: &[&str], - descs: &mut Vec, - ) -> IntCounterVec { - let counter = IntCounterVec::new(Opts::new(name, help), labels).unwrap(); - descs.extend(counter.clone().desc().into_iter().cloned()); - counter - } - - fn create_histogram_vec( - name: &str, - help: &str, - labels: &[&str], - descs: &mut Vec, - ) -> HistogramVec { - let histogram = HistogramVec::new(HistogramOpts::new(name, help), labels).unwrap(); - descs.extend(histogram.clone().desc().into_iter().cloned()); - histogram - } - let topic_labels = &["topic"]; let partition_labels = &["topic", "partition"]; - let collector = KafkaMetricsCollector { + let mut collector = KafkaMetricsCollector { stats: stats.clone(), descs: descs.clone(), // Core metrics @@ -265,19 +232,19 @@ impl KafkaMetricsCollector { "Broker throttle time", ))?, // Topic metrics with labels - topic_metadata_age: create_gauge_vec( + topic_metadata_age: Self::create_gauge_vec( "kafka_topic_metadata_age", "Age of topic metadata", topic_labels, &mut descs, ), - topic_batchsize: create_histogram_vec( + topic_batchsize: Self::create_histogram_vec( "kafka_topic_batchsize", "Topic batch sizes", topic_labels, &mut descs, ), - topic_batchcnt: create_histogram_vec( + topic_batchcnt: Self::create_histogram_vec( "kafka_topic_batchcnt", "Topic batch counts", topic_labels, @@ -285,139 +252,139 @@ impl KafkaMetricsCollector { ), // Partition metrics with labels - partition_msgq_cnt: create_gauge_vec( + partition_msgq_cnt: Self::create_gauge_vec( "kafka_partition_msgq_cnt", "Messages in partition queue", partition_labels, &mut descs, ), - partition_msgq_bytes: create_gauge_vec( + partition_msgq_bytes: Self::create_gauge_vec( "kafka_partition_msgq_bytes", "Bytes in partition queue", partition_labels, &mut descs, ), - partition_xmit_msgq_cnt: create_gauge_vec( + partition_xmit_msgq_cnt: Self::create_gauge_vec( "kafka_partition_xmit_msgq_cnt", "Messages in partition transmit queue", partition_labels, &mut descs, ), - partition_xmit_msgq_bytes: create_gauge_vec( + partition_xmit_msgq_bytes: Self::create_gauge_vec( "kafka_partition_xmit_msgq_bytes", "Bytes in partition transmit queue", partition_labels, &mut descs, ), - partition_fetchq_cnt: create_gauge_vec( + partition_fetchq_cnt: Self::create_gauge_vec( "kafka_partition_fetchq_cnt", "Messages in partition fetch queue", partition_labels, &mut descs, ), - partition_fetchq_size: create_gauge_vec( + partition_fetchq_size: Self::create_gauge_vec( "kafka_partition_fetchq_size", "Size of partition fetch queue", partition_labels, &mut descs, ), - partition_query_offset: create_gauge_vec( + partition_query_offset: Self::create_gauge_vec( "kafka_partition_query_offset", "Current partition query offset", partition_labels, &mut descs, ), - partition_next_offset: create_gauge_vec( + partition_next_offset: Self::create_gauge_vec( "kafka_partition_next_offset", "Next partition offset", partition_labels, &mut descs, ), - partition_app_offset: create_gauge_vec( + partition_app_offset: Self::create_gauge_vec( "kafka_partition_app_offset", "Application partition offset", partition_labels, &mut descs, ), - partition_stored_offset: create_gauge_vec( + partition_stored_offset: Self::create_gauge_vec( "kafka_partition_stored_offset", "Stored partition offset", partition_labels, &mut descs, ), - partition_committed_offset: create_gauge_vec( + partition_committed_offset: Self::create_gauge_vec( "kafka_partition_committed_offset", "Committed partition offset", partition_labels, &mut descs, ), - partition_eof_offset: create_gauge_vec( + partition_eof_offset: Self::create_gauge_vec( "kafka_partition_eof_offset", "EOF partition offset", partition_labels, &mut descs, ), - partition_lo_offset: create_gauge_vec( + partition_lo_offset: Self::create_gauge_vec( "kafka_partition_lo_offset", "Low watermark partition offset", partition_labels, &mut descs, ), - partition_hi_offset: create_gauge_vec( + partition_hi_offset: Self::create_gauge_vec( "kafka_partition_hi_offset", "High watermark partition offset", partition_labels, &mut descs, ), - partition_consumer_lag: create_gauge_vec( + partition_consumer_lag: Self::create_gauge_vec( "kafka_partition_consumer_lag", "Consumer lag", partition_labels, &mut descs, ), - partition_consumer_lag_stored: create_gauge_vec( + partition_consumer_lag_stored: Self::create_gauge_vec( "kafka_partition_consumer_lag_stored", "Stored consumer lag", partition_labels, &mut descs, ), - partition_txmsgs: create_counter_vec( + partition_txmsgs: Self::create_counter_vec( "kafka_partition_txmsgs_total", "Total partition messages transmitted", partition_labels, &mut descs, ), - partition_txbytes: create_counter_vec( + partition_txbytes: Self::create_counter_vec( "kafka_partition_txbytes_total", "Total partition bytes transmitted", partition_labels, &mut descs, ), - partition_rxmsgs: create_counter_vec( + partition_rxmsgs: Self::create_counter_vec( "kafka_partition_rxmsgs_total", "Total partition messages received", partition_labels, &mut descs, ), - partition_rxbytes: create_counter_vec( + partition_rxbytes: Self::create_counter_vec( "kafka_partition_rxbytes_total", "Total partition bytes received", partition_labels, &mut descs, ), - partition_msgs: create_counter_vec( + partition_msgs: Self::create_counter_vec( "kafka_partition_msgs_total", "Total partition messages", partition_labels, &mut descs, ), - partition_rx_ver_drops: create_counter_vec( + partition_rx_ver_drops: Self::create_counter_vec( "kafka_partition_rx_ver_drops_total", "Total partition version drops", partition_labels, &mut descs, ), - partition_msgs_inflight: create_gauge_vec( + partition_msgs_inflight: Self::create_gauge_vec( "kafka_partition_msgs_inflight", "Messages in flight", partition_labels, @@ -433,11 +400,43 @@ impl KafkaMetricsCollector { eos_producer_epoch: IntGauge::new("kafka_eos_producer_epoch", "Producer epoch")?, }; - let mut collector = collector; - collector.descs = descs.clone(); + collector.descs = descs; Ok(collector) } + + fn create_gauge_vec( + name: &str, + help: &str, + labels: &[&str], + descs: &mut Vec, + ) -> IntGaugeVec { + let gauge = IntGaugeVec::new(Opts::new(name, help), labels).unwrap(); + descs.extend(gauge.clone().desc().into_iter().cloned()); + gauge + } + + fn create_counter_vec( + name: &str, + help: &str, + labels: &[&str], + descs: &mut Vec, + ) -> IntCounterVec { + let counter = IntCounterVec::new(Opts::new(name, help), labels).unwrap(); + descs.extend(counter.clone().desc().into_iter().cloned()); + counter + } + + fn create_histogram_vec( + name: &str, + help: &str, + labels: &[&str], + descs: &mut Vec, + ) -> HistogramVec { + let histogram = HistogramVec::new(HistogramOpts::new(name, help), labels).unwrap(); + descs.extend(histogram.clone().desc().into_iter().cloned()); + histogram + } } impl Collector for KafkaMetricsCollector { diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs index 5b8f82974..bd2edcba3 100644 --- a/src/connectors/kafka/mod.rs +++ b/src/connectors/kafka/mod.rs @@ -18,6 +18,7 @@ use crate::connectors::kafka::config::KafkaConfig; use derive_more::Constructor; +use rdkafka::client::OAuthToken; use rdkafka::consumer::{ConsumerContext, Rebalance}; use rdkafka::error::KafkaResult; use rdkafka::message::{BorrowedMessage, Headers}; @@ -26,6 +27,7 @@ use rdkafka::topic_partition_list::TopicPartitionListElem; use rdkafka::{ClientContext, Message, Offset, Statistics}; use serde::{Deserialize, Serialize}; use std::collections::HashMap; +use std::error::Error; use std::sync::{Arc, RwLock}; use tokio::sync::mpsc; use tokio::sync::mpsc::Receiver; @@ -236,6 +238,9 @@ impl ProducerContext for KafkaContext { } impl ClientContext for KafkaContext { + // TODO: when implementing OAuth, set this to true + const ENABLE_REFRESH_OAUTH_TOKEN: bool = false; + fn stats(&self, new_stats: Statistics) { match self.statistics.write() { Ok(mut stats) => { @@ -246,4 +251,11 @@ impl ClientContext for KafkaContext { } }; } + + fn generate_oauth_token( + &self, + _oauthbearer_config: Option<&str>, + ) -> Result> { + todo!("Implement OAuth token generation") + } } diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index d72a15ca9..0dfe2e3a5 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -16,9 +16,10 @@ * */ -use crate::connectors::common::config::ConnectorConfig; +use crate::connectors::common::config::Connectors; use crate::connectors::common::processor::Processor; use crate::connectors::common::shutdown::Shutdown; +use crate::connectors::kafka::config::KafkaConfig; use crate::connectors::kafka::consumer::KafkaStreams; use crate::connectors::kafka::metrics::KafkaMetricsCollector; use crate::connectors::kafka::processor::ParseableSinkProcessor; @@ -30,6 +31,7 @@ use crate::option::{Mode, CONFIG}; use actix_web_prometheus::PrometheusMetrics; use prometheus::Registry; use std::sync::Arc; +use std::time::Duration; use tokio::sync::RwLock; use tracing::{info, warn}; @@ -38,8 +40,8 @@ pub mod kafka; pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { if matches!(CONFIG.parseable.mode, Mode::Ingest | Mode::All) { - match CONFIG.parseable.connector_config.clone() { - Some(config) => { + match CONFIG.connector_config.clone() { + Some(connectors) => { let shutdown_handle = Shutdown::default(); let registry = prometheus.registry.clone(); let processor = ParseableSinkProcessor; @@ -51,8 +53,11 @@ pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { info!("Connector received shutdown signal!"); } }); - - run_kafka2parseable(config, registry, processor, shutdown_handle).await? + match connectors.connectors { + Connectors::KafkaSink(config) => { + run_kafka2parseable(config, registry, processor, shutdown_handle).await? + } + } } None => { warn!("Kafka connector configuration is missing. Skipping Kafka pipeline."); @@ -64,7 +69,7 @@ pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { } async fn run_kafka2parseable

( - config: ConnectorConfig, + config: KafkaConfig, registry: Registry, processor: P, shutdown_handle: Shutdown, @@ -72,7 +77,7 @@ async fn run_kafka2parseable

( where P: Processor, ()> + Send + Sync + 'static, { - let kafka_config = Arc::new(config.kafka_config.clone().unwrap_or_default()); + let kafka_config = Arc::new(config.clone()); let (kafka_context, rebalance_rx) = KafkaContext::new(kafka_config); //TODO: fetch topics metadata from kafka then give dynamic value to StreamState @@ -88,12 +93,8 @@ where let stats = kafka_streams.statistics(); registry.register(Box::new(KafkaMetricsCollector::new(stats)?))?; - let kafka_parseable_sink_connector = KafkaSinkConnector::new( - kafka_streams, - processor, - config.buffer_size, - config.buffer_timeout, - ); + let kafka_parseable_sink_connector = + KafkaSinkConnector::new(kafka_streams, processor, 10000, Duration::from_millis(5000)); rebalance_listener.start(); kafka_parseable_sink_connector.run().await?; diff --git a/src/main.rs b/src/main.rs index 1455a6243..f747ddc1c 100644 --- a/src/main.rs +++ b/src/main.rs @@ -47,7 +47,7 @@ async fn main() -> anyhow::Result<()> { metadata.set_global(); let prometheus = metrics::build_metrics_handler(); - + let parseable_server = server.init(&prometheus); let connectors_task = connectors::init(&prometheus); diff --git a/src/option.rs b/src/option.rs index 0ead3ea0f..57e8db65d 100644 --- a/src/option.rs +++ b/src/option.rs @@ -17,6 +17,7 @@ */ use crate::cli::Cli; +use crate::connectors::common::config::ConnectorConfig; use crate::storage::object_storage::parseable_json_path; use crate::storage::{ AzureBlobConfig, FSConfig, ObjectStorageError, ObjectStorageProvider, S3Config, @@ -30,6 +31,7 @@ use serde::{Deserialize, Serialize}; use std::env; use std::path::PathBuf; use std::sync::Arc; + pub const MIN_CACHE_SIZE_BYTES: u64 = 1073741824; // 1 GiB pub const JOIN_COMMUNITY: &str = @@ -41,6 +43,7 @@ pub struct Config { pub parseable: Cli, storage: Arc, pub storage_name: &'static str, + pub connector_config: Option, } impl Config { @@ -98,6 +101,7 @@ parseable [command] --help parseable: cli, storage: Arc::new(storage), storage_name: "drive", + connector_config: ConnectorConfig::from(m), } } Some(("s3-store", m)) => { @@ -114,6 +118,7 @@ parseable [command] --help parseable: cli, storage: Arc::new(storage), storage_name: "s3", + connector_config: ConnectorConfig::from(m), } } Some(("blob-store", m)) => { @@ -130,6 +135,7 @@ parseable [command] --help parseable: cli, storage: Arc::new(storage), storage_name: "blob_store", + connector_config: ConnectorConfig::from(m), } } _ => unreachable!(), @@ -222,6 +228,7 @@ fn create_parseable_cli_command() -> Command { .mut_arg(Cli::PASSWORD, |arg| { arg.required(false).default_value(Cli::DEFAULT_PASSWORD) }); + let s3 = Cli::create_cli_command_with_clap("s3-store"); let s3 = ::augment_args_for_update(s3); From d26d4de3ed69bbdfda7507d9236452e31cf0b931 Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 26 Dec 2024 10:28:53 +0100 Subject: [PATCH 13/65] Refactor metrics.rs to reduce cyclomatic complexity. --- src/connectors/common/config.rs | 14 +- src/connectors/kafka/metrics.rs | 967 +++++++++++++++++++------------- 2 files changed, 568 insertions(+), 413 deletions(-) diff --git a/src/connectors/common/config.rs b/src/connectors/common/config.rs index 0fbe9b483..e9db3c346 100644 --- a/src/connectors/common/config.rs +++ b/src/connectors/common/config.rs @@ -14,20 +14,11 @@ pub struct ConnectorConfig { value_enum, long = "bad-data-policy", required = false, - default_value_t = BadData::Drop, + default_value_t = BadData::Fail, env = "P_CONNECTOR_BAD_DATA_POLICY", help = "Policy for handling bad data" )] pub bad_data: BadData, - - #[arg( - long = "metrics-enabled", - env = "P_CONNECTOR_METRICS_ENABLED", - required = false, - default_value_t = true, - help = "Enable metrics collection" - )] - pub metrics_enabled: bool, } #[derive(Debug, Clone, Subcommand)] @@ -52,8 +43,8 @@ impl fmt::Display for Connectors { #[derive(ValueEnum, Default, Clone, Debug, PartialEq, Eq, Hash)] pub enum BadData { - Fail, #[default] + Fail, Drop, Dlt, //TODO: Implement Dead Letter Topic support when needed } @@ -75,7 +66,6 @@ impl Default for ConnectorConfig { fn default() -> Self { ConnectorConfig { bad_data: BadData::Drop, - metrics_enabled: true, connectors: Connectors::KafkaSink(KafkaConfig::default()), } } diff --git a/src/connectors/kafka/metrics.rs b/src/connectors/kafka/metrics.rs index 774ae4779..45a5f52d4 100644 --- a/src/connectors/kafka/metrics.rs +++ b/src/connectors/kafka/metrics.rs @@ -28,8 +28,16 @@ use std::sync::{Arc, RwLock}; pub struct KafkaMetricsCollector { stats: Arc>, descs: Vec, + core_metrics: CoreMetrics, + broker_metrics: BrokerMetrics, + topic_metrics: TopicMetrics, + partition_metrics: PartitionMetrics, + consumer_metrics: ConsumerGroupMetrics, + eos_metrics: EosMetrics, +} - // Core client metrics +#[derive(Debug)] +struct CoreMetrics { msg_cnt: IntGauge, msg_size: IntGauge, msg_max: IntGauge, @@ -43,82 +51,84 @@ pub struct KafkaMetricsCollector { txmsg_bytes: IntCounter, rxmsgs: IntCounter, rxmsg_bytes: IntCounter, +} - // Broker metrics - broker_state_cnt: IntGauge, - broker_outbuf_cnt: IntGauge, - broker_outbuf_msg_cnt: IntGauge, - broker_waitresp_cnt: IntGauge, - broker_waitresp_msg_cnt: IntGauge, - broker_tx: IntCounter, - broker_tx_bytes: IntCounter, - broker_tx_errs: IntCounter, - broker_tx_retries: IntCounter, - broker_req_timeouts: IntCounter, - broker_rx: IntCounter, - broker_rx_bytes: IntCounter, - broker_rx_errs: IntCounter, - broker_rx_corrid_errs: IntCounter, - broker_rx_partial: IntCounter, - broker_connects: IntCounter, - broker_disconnects: IntCounter, - broker_int_latency: Histogram, - broker_outbuf_latency: Histogram, - broker_rtt: Histogram, - broker_throttle: Histogram, - - // Topic metrics - topic_metadata_age: IntGaugeVec, - topic_batchsize: HistogramVec, - topic_batchcnt: HistogramVec, - - // Partition metrics with labels - partition_msgq_cnt: IntGaugeVec, - partition_msgq_bytes: IntGaugeVec, - partition_xmit_msgq_cnt: IntGaugeVec, - partition_xmit_msgq_bytes: IntGaugeVec, - partition_fetchq_cnt: IntGaugeVec, - partition_fetchq_size: IntGaugeVec, - partition_query_offset: IntGaugeVec, - partition_next_offset: IntGaugeVec, - partition_app_offset: IntGaugeVec, - partition_stored_offset: IntGaugeVec, - partition_committed_offset: IntGaugeVec, - partition_eof_offset: IntGaugeVec, - partition_lo_offset: IntGaugeVec, - partition_hi_offset: IntGaugeVec, - partition_consumer_lag: IntGaugeVec, - partition_consumer_lag_stored: IntGaugeVec, - partition_txmsgs: IntCounterVec, - partition_txbytes: IntCounterVec, - partition_rxmsgs: IntCounterVec, - partition_rxbytes: IntCounterVec, - partition_msgs: IntCounterVec, - partition_rx_ver_drops: IntCounterVec, - partition_msgs_inflight: IntGaugeVec, - - // Consumer group metrics - cgrp_rebalance_cnt: IntCounter, - cgrp_rebalance_age: IntGauge, - cgrp_assignment_size: IntGauge, - - // Exactly once semantics metrics - eos_epoch_cnt: IntCounter, - eos_producer_id: IntGauge, - eos_producer_epoch: IntGauge, +#[derive(Debug)] +struct BrokerMetrics { + state_cnt: IntGauge, + outbuf_cnt: IntGauge, + outbuf_msg_cnt: IntGauge, + waitresp_cnt: IntGauge, + waitresp_msg_cnt: IntGauge, + tx: IntCounter, + tx_bytes: IntCounter, + tx_errs: IntCounter, + tx_retries: IntCounter, + req_timeouts: IntCounter, + rx: IntCounter, + rx_bytes: IntCounter, + rx_errs: IntCounter, + rx_corrid_errs: IntCounter, + rx_partial: IntCounter, + connects: IntCounter, + disconnects: IntCounter, + int_latency: Histogram, + outbuf_latency: Histogram, + rtt: Histogram, + throttle: Histogram, } -impl KafkaMetricsCollector { - pub fn new(stats: Arc>) -> anyhow::Result { - let mut descs = Vec::new(); +#[derive(Debug)] +struct TopicMetrics { + metadata_age: IntGaugeVec, + batchsize: HistogramVec, + batchcnt: HistogramVec, +} - let topic_labels = &["topic"]; - let partition_labels = &["topic", "partition"]; +#[derive(Debug)] +struct PartitionMetrics { + msgq_cnt: IntGaugeVec, + msgq_bytes: IntGaugeVec, + xmit_msgq_cnt: IntGaugeVec, + xmit_msgq_bytes: IntGaugeVec, + fetchq_cnt: IntGaugeVec, + fetchq_size: IntGaugeVec, + query_offset: IntGaugeVec, + next_offset: IntGaugeVec, + app_offset: IntGaugeVec, + stored_offset: IntGaugeVec, + committed_offset: IntGaugeVec, + eof_offset: IntGaugeVec, + lo_offset: IntGaugeVec, + hi_offset: IntGaugeVec, + consumer_lag: IntGaugeVec, + consumer_lag_stored: IntGaugeVec, + txmsgs: IntCounterVec, + txbytes: IntCounterVec, + rxmsgs: IntCounterVec, + rxbytes: IntCounterVec, + msgs: IntCounterVec, + rx_ver_drops: IntCounterVec, + msgs_inflight: IntGaugeVec, +} + +#[derive(Debug)] +struct ConsumerGroupMetrics { + rebalance_cnt: IntCounter, + rebalance_age: IntGauge, + assignment_size: IntGauge, +} + +#[derive(Debug)] +struct EosMetrics { + epoch_cnt: IntCounter, + producer_id: IntGauge, + producer_epoch: IntGauge, +} - let mut collector = KafkaMetricsCollector { - stats: stats.clone(), - descs: descs.clone(), - // Core metrics +impl CoreMetrics { + fn new() -> anyhow::Result { + Ok(Self { msg_cnt: IntGauge::new( "kafka_msg_cnt", "Current number of messages in producer queues", @@ -153,256 +163,610 @@ impl KafkaMetricsCollector { "kafka_rxmsg_bytes_total", "Total number of message bytes received", )?, + }) + } +} - // Broker metrics - broker_state_cnt: IntGauge::new("kafka_broker_state", "Broker connection state")?, - broker_outbuf_cnt: IntGauge::new( +impl BrokerMetrics { + fn new() -> anyhow::Result { + Ok(Self { + state_cnt: IntGauge::new("kafka_broker_state", "Broker connection state")?, + outbuf_cnt: IntGauge::new( "kafka_broker_outbuf_cnt", "Number of requests awaiting transmission", )?, - broker_outbuf_msg_cnt: IntGauge::new( + outbuf_msg_cnt: IntGauge::new( "kafka_broker_outbuf_msg_cnt", "Number of messages awaiting transmission", )?, - broker_waitresp_cnt: IntGauge::new( + waitresp_cnt: IntGauge::new( "kafka_broker_waitresp_cnt", "Number of requests in-flight", )?, - broker_waitresp_msg_cnt: IntGauge::new( + waitresp_msg_cnt: IntGauge::new( "kafka_broker_waitresp_msg_cnt", "Number of messages in-flight", )?, - broker_tx: IntCounter::new("kafka_broker_tx_total", "Total broker transmissions")?, - broker_tx_bytes: IntCounter::new( + tx: IntCounter::new("kafka_broker_tx_total", "Total broker transmissions")?, + tx_bytes: IntCounter::new( "kafka_broker_tx_bytes_total", "Total broker bytes transmitted", )?, - broker_tx_errs: IntCounter::new( + tx_errs: IntCounter::new( "kafka_broker_tx_errs_total", "Total broker transmission errors", )?, - broker_tx_retries: IntCounter::new( + tx_retries: IntCounter::new( "kafka_broker_tx_retries_total", "Total broker transmission retries", )?, - broker_req_timeouts: IntCounter::new( + req_timeouts: IntCounter::new( "kafka_broker_req_timeouts_total", "Total broker request timeouts", )?, - broker_rx: IntCounter::new("kafka_broker_rx_total", "Total broker receptions")?, - broker_rx_bytes: IntCounter::new( + rx: IntCounter::new("kafka_broker_rx_total", "Total broker receptions")?, + rx_bytes: IntCounter::new( "kafka_broker_rx_bytes_total", "Total broker bytes received", )?, - broker_rx_errs: IntCounter::new( + rx_errs: IntCounter::new( "kafka_broker_rx_errs_total", "Total broker reception errors", )?, - broker_rx_corrid_errs: IntCounter::new( + rx_corrid_errs: IntCounter::new( "kafka_broker_rx_corrid_errs_total", "Total broker correlation ID errors", )?, - broker_rx_partial: IntCounter::new( + rx_partial: IntCounter::new( "kafka_broker_rx_partial_total", "Total broker partial message sets", )?, - broker_connects: IntCounter::new( + connects: IntCounter::new( "kafka_broker_connects_total", "Total broker connection attempts", )?, - broker_disconnects: IntCounter::new( + disconnects: IntCounter::new( "kafka_broker_disconnects_total", "Total broker disconnections", )?, - broker_int_latency: Histogram::with_opts(HistogramOpts::new( + int_latency: Histogram::with_opts(HistogramOpts::new( "kafka_broker_int_latency", "Internal broker latency", ))?, - broker_outbuf_latency: Histogram::with_opts(HistogramOpts::new( + outbuf_latency: Histogram::with_opts(HistogramOpts::new( "kafka_broker_outbuf_latency", "Outbuf latency", ))?, - broker_rtt: Histogram::with_opts(HistogramOpts::new( + rtt: Histogram::with_opts(HistogramOpts::new( "kafka_broker_rtt", "Broker round-trip time", ))?, - - broker_throttle: Histogram::with_opts(HistogramOpts::new( + throttle: Histogram::with_opts(HistogramOpts::new( "kafka_broker_throttle", "Broker throttle time", ))?, - // Topic metrics with labels - topic_metadata_age: Self::create_gauge_vec( + }) + } +} + +impl TopicMetrics { + fn new(labels: &[&str], descs: &mut Vec) -> Self { + Self { + metadata_age: KafkaMetricsCollector::create_gauge_vec( "kafka_topic_metadata_age", "Age of topic metadata", - topic_labels, - &mut descs, + labels, + descs, ), - topic_batchsize: Self::create_histogram_vec( + batchsize: KafkaMetricsCollector::create_histogram_vec( "kafka_topic_batchsize", "Topic batch sizes", - topic_labels, - &mut descs, + labels, + descs, ), - topic_batchcnt: Self::create_histogram_vec( + batchcnt: KafkaMetricsCollector::create_histogram_vec( "kafka_topic_batchcnt", "Topic batch counts", - topic_labels, - &mut descs, + labels, + descs, ), + } + } +} - // Partition metrics with labels - partition_msgq_cnt: Self::create_gauge_vec( +impl PartitionMetrics { + fn new(labels: &[&str], descs: &mut Vec) -> Self { + Self { + msgq_cnt: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_msgq_cnt", "Messages in partition queue", - partition_labels, - &mut descs, + labels, + descs, ), - partition_msgq_bytes: Self::create_gauge_vec( + msgq_bytes: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_msgq_bytes", "Bytes in partition queue", - partition_labels, - &mut descs, + labels, + descs, ), - partition_xmit_msgq_cnt: Self::create_gauge_vec( + xmit_msgq_cnt: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_xmit_msgq_cnt", "Messages in partition transmit queue", - partition_labels, - &mut descs, + labels, + descs, ), - partition_xmit_msgq_bytes: Self::create_gauge_vec( + xmit_msgq_bytes: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_xmit_msgq_bytes", "Bytes in partition transmit queue", - partition_labels, - &mut descs, + labels, + descs, ), - partition_fetchq_cnt: Self::create_gauge_vec( + fetchq_cnt: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_fetchq_cnt", "Messages in partition fetch queue", - partition_labels, - &mut descs, + labels, + descs, ), - partition_fetchq_size: Self::create_gauge_vec( + fetchq_size: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_fetchq_size", "Size of partition fetch queue", - partition_labels, - &mut descs, + labels, + descs, ), - partition_query_offset: Self::create_gauge_vec( + query_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_query_offset", "Current partition query offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_next_offset: Self::create_gauge_vec( + next_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_next_offset", "Next partition offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_app_offset: Self::create_gauge_vec( + app_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_app_offset", "Application partition offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_stored_offset: Self::create_gauge_vec( + stored_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_stored_offset", "Stored partition offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_committed_offset: Self::create_gauge_vec( + committed_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_committed_offset", "Committed partition offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_eof_offset: Self::create_gauge_vec( + eof_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_eof_offset", "EOF partition offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_lo_offset: Self::create_gauge_vec( + lo_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_lo_offset", "Low watermark partition offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_hi_offset: Self::create_gauge_vec( + hi_offset: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_hi_offset", "High watermark partition offset", - partition_labels, - &mut descs, + labels, + descs, ), - partition_consumer_lag: Self::create_gauge_vec( + consumer_lag: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_consumer_lag", "Consumer lag", - partition_labels, - &mut descs, + labels, + descs, ), - partition_consumer_lag_stored: Self::create_gauge_vec( + consumer_lag_stored: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_consumer_lag_stored", "Stored consumer lag", - partition_labels, - &mut descs, + labels, + descs, ), - partition_txmsgs: Self::create_counter_vec( + txmsgs: KafkaMetricsCollector::create_counter_vec( "kafka_partition_txmsgs_total", "Total partition messages transmitted", - partition_labels, - &mut descs, + labels, + descs, ), - partition_txbytes: Self::create_counter_vec( + txbytes: KafkaMetricsCollector::create_counter_vec( "kafka_partition_txbytes_total", "Total partition bytes transmitted", - partition_labels, - &mut descs, + labels, + descs, ), - partition_rxmsgs: Self::create_counter_vec( + rxmsgs: KafkaMetricsCollector::create_counter_vec( "kafka_partition_rxmsgs_total", "Total partition messages received", - partition_labels, - &mut descs, + labels, + descs, ), - partition_rxbytes: Self::create_counter_vec( + rxbytes: KafkaMetricsCollector::create_counter_vec( "kafka_partition_rxbytes_total", "Total partition bytes received", - partition_labels, - &mut descs, + labels, + descs, ), - partition_msgs: Self::create_counter_vec( + msgs: KafkaMetricsCollector::create_counter_vec( "kafka_partition_msgs_total", "Total partition messages", - partition_labels, - &mut descs, + labels, + descs, ), - partition_rx_ver_drops: Self::create_counter_vec( + rx_ver_drops: KafkaMetricsCollector::create_counter_vec( "kafka_partition_rx_ver_drops_total", "Total partition version drops", - partition_labels, - &mut descs, + labels, + descs, ), - partition_msgs_inflight: Self::create_gauge_vec( + msgs_inflight: KafkaMetricsCollector::create_gauge_vec( "kafka_partition_msgs_inflight", "Messages in flight", - partition_labels, - &mut descs, + labels, + descs, ), - cgrp_rebalance_cnt: IntCounter::new("kafka_cgrp_rebalance_total", "Total rebalances")?, - cgrp_rebalance_age: IntGauge::new("kafka_cgrp_rebalance_age", "Rebalance age")?, - cgrp_assignment_size: IntGauge::new("kafka_cgrp_assignment_size", "Assignment size")?, - - // Exactly once semantics metrics - eos_epoch_cnt: IntCounter::new("kafka_eos_epoch_total", "Total number of epochs")?, - eos_producer_id: IntGauge::new("kafka_eos_producer_id", "Producer ID")?, - eos_producer_epoch: IntGauge::new("kafka_eos_producer_epoch", "Producer epoch")?, - }; + } + } + + fn collect_metrics( + &self, + topic_name: &str, + partition_id: &i32, + partition: &rdkafka::statistics::Partition, + ) -> Vec { + let mut mfs = Vec::new(); + let labels = &[topic_name, &partition_id.to_string()]; + + self.set_gauges(labels, partition); + self.set_counters(labels, partition); + + self.collect_all_metrics(&mut mfs); + + mfs + } + + fn set_gauges(&self, labels: &[&str], partition: &rdkafka::statistics::Partition) { + self.msgq_cnt + .with_label_values(labels) + .set(partition.msgq_cnt); + self.msgq_bytes + .with_label_values(labels) + .set(partition.msgq_bytes as i64); + self.xmit_msgq_cnt + .with_label_values(labels) + .set(partition.xmit_msgq_cnt); + self.xmit_msgq_bytes + .with_label_values(labels) + .set(partition.xmit_msgq_bytes as i64); + self.fetchq_cnt + .with_label_values(labels) + .set(partition.fetchq_cnt); + self.fetchq_size + .with_label_values(labels) + .set(partition.fetchq_size as i64); + self.query_offset + .with_label_values(labels) + .set(partition.query_offset); + self.next_offset + .with_label_values(labels) + .set(partition.next_offset); + self.app_offset + .with_label_values(labels) + .set(partition.app_offset); + self.stored_offset + .with_label_values(labels) + .set(partition.stored_offset); + self.committed_offset + .with_label_values(labels) + .set(partition.committed_offset); + self.eof_offset + .with_label_values(labels) + .set(partition.eof_offset); + self.lo_offset + .with_label_values(labels) + .set(partition.lo_offset); + self.hi_offset + .with_label_values(labels) + .set(partition.hi_offset); + self.consumer_lag + .with_label_values(labels) + .set(partition.consumer_lag); + self.consumer_lag_stored + .with_label_values(labels) + .set(partition.consumer_lag_stored); + self.msgs_inflight + .with_label_values(labels) + .set(partition.msgs_inflight); + } + + fn set_counters(&self, labels: &[&str], partition: &rdkafka::statistics::Partition) { + self.txmsgs + .with_label_values(labels) + .inc_by(partition.txmsgs); + self.txbytes + .with_label_values(labels) + .inc_by(partition.txbytes); + self.rxmsgs + .with_label_values(labels) + .inc_by(partition.rxmsgs); + self.rxbytes + .with_label_values(labels) + .inc_by(partition.rxbytes); + self.msgs.with_label_values(labels).inc_by(partition.msgs); + self.rx_ver_drops + .with_label_values(labels) + .inc_by(partition.rx_ver_drops); + } + + fn collect_all_metrics(&self, mfs: &mut Vec) { + // Collect gauges + mfs.extend(self.msgq_cnt.collect()); + mfs.extend(self.msgq_bytes.collect()); + mfs.extend(self.xmit_msgq_cnt.collect()); + mfs.extend(self.xmit_msgq_bytes.collect()); + mfs.extend(self.fetchq_cnt.collect()); + mfs.extend(self.fetchq_size.collect()); + mfs.extend(self.query_offset.collect()); + mfs.extend(self.next_offset.collect()); + mfs.extend(self.app_offset.collect()); + mfs.extend(self.stored_offset.collect()); + mfs.extend(self.committed_offset.collect()); + mfs.extend(self.eof_offset.collect()); + mfs.extend(self.lo_offset.collect()); + mfs.extend(self.hi_offset.collect()); + mfs.extend(self.consumer_lag.collect()); + mfs.extend(self.consumer_lag_stored.collect()); + mfs.extend(self.msgs_inflight.collect()); + + // Collect counters + mfs.extend(self.txmsgs.collect()); + mfs.extend(self.txbytes.collect()); + mfs.extend(self.rxmsgs.collect()); + mfs.extend(self.rxbytes.collect()); + mfs.extend(self.msgs.collect()); + mfs.extend(self.rx_ver_drops.collect()); + } +} - collector.descs = descs; +impl BrokerMetrics { + fn collect_metrics(&self, broker: &rdkafka::statistics::Broker) -> Vec { + let mut mfs = Vec::new(); + + self.state_cnt.set(match broker.state.as_str() { + "UP" => 1, + "DOWN" => 0, + _ => -1, + }); + + self.set_gauges(broker); + self.set_counters(broker); + self.set_latency_metrics(broker); + self.collect_all_metrics(&mut mfs); + + mfs + } - Ok(collector) + fn set_gauges(&self, broker: &rdkafka::statistics::Broker) { + self.outbuf_cnt.set(broker.outbuf_cnt); + self.outbuf_msg_cnt.set(broker.outbuf_msg_cnt); + self.waitresp_cnt.set(broker.waitresp_cnt); + self.waitresp_msg_cnt.set(broker.waitresp_msg_cnt); + } + + fn set_counters(&self, broker: &rdkafka::statistics::Broker) { + self.tx.inc_by(broker.tx); + self.tx_bytes.inc_by(broker.txbytes); + self.tx_errs.inc_by(broker.txerrs); + self.tx_retries.inc_by(broker.txretries); + self.req_timeouts.inc_by(broker.req_timeouts); + self.rx.inc_by(broker.rx); + self.rx_bytes.inc_by(broker.rxbytes); + self.rx_errs.inc_by(broker.rxerrs); + self.rx_corrid_errs.inc_by(broker.rxcorriderrs); + self.rx_partial.inc_by(broker.rxpartial); + + if let Some(connects) = broker.connects { + self.connects.inc_by(connects as u64); + } + if let Some(disconnects) = broker.disconnects { + self.disconnects.inc_by(disconnects as u64); + } + } + + fn set_latency_metrics(&self, broker: &rdkafka::statistics::Broker) { + if let Some(ref latency) = broker.int_latency { + self.int_latency.observe(latency.avg as f64); + } + if let Some(ref latency) = broker.outbuf_latency { + self.outbuf_latency.observe(latency.avg as f64); + } + if let Some(ref rtt) = broker.rtt { + self.rtt.observe(rtt.avg as f64); + } + if let Some(ref throttle) = broker.throttle { + self.throttle.observe(throttle.avg as f64); + } + } + + fn collect_all_metrics(&self, mfs: &mut Vec) { + mfs.extend(self.state_cnt.collect()); + mfs.extend(self.outbuf_cnt.collect()); + mfs.extend(self.outbuf_msg_cnt.collect()); + mfs.extend(self.waitresp_cnt.collect()); + mfs.extend(self.waitresp_msg_cnt.collect()); + mfs.extend(self.tx.collect()); + mfs.extend(self.tx_bytes.collect()); + mfs.extend(self.tx_errs.collect()); + mfs.extend(self.tx_retries.collect()); + mfs.extend(self.req_timeouts.collect()); + mfs.extend(self.rx.collect()); + mfs.extend(self.rx_bytes.collect()); + mfs.extend(self.rx_errs.collect()); + mfs.extend(self.rx_corrid_errs.collect()); + mfs.extend(self.rx_partial.collect()); + mfs.extend(self.connects.collect()); + mfs.extend(self.disconnects.collect()); + mfs.extend(self.int_latency.collect()); + mfs.extend(self.outbuf_latency.collect()); + mfs.extend(self.rtt.collect()); + mfs.extend(self.throttle.collect()); + } +} + +impl TopicMetrics { + fn collect_metrics( + &self, + topic_name: &str, + topic: &rdkafka::statistics::Topic, + ) -> Vec { + let mut mfs = Vec::new(); + let labels = &[topic_name]; + + self.metadata_age + .with_label_values(labels) + .set(topic.metadata_age); + self.batchsize + .with_label_values(labels) + .observe(topic.batchsize.avg as f64); + self.batchcnt + .with_label_values(labels) + .observe(topic.batchcnt.avg as f64); + + mfs.extend(self.metadata_age.collect()); + mfs.extend(self.batchsize.collect()); + mfs.extend(self.batchcnt.collect()); + + mfs + } +} + +impl ConsumerGroupMetrics { + fn collect_metrics( + &self, + cgrp: &rdkafka::statistics::ConsumerGroup, + ) -> Vec { + let mut mfs = Vec::new(); + + self.rebalance_cnt.inc_by(cgrp.rebalance_cnt as u64); + self.rebalance_age.set(cgrp.rebalance_age); + self.assignment_size.set(cgrp.assignment_size as i64); + + mfs.extend(self.rebalance_cnt.collect()); + mfs.extend(self.rebalance_age.collect()); + mfs.extend(self.assignment_size.collect()); + + mfs + } +} + +impl EosMetrics { + fn collect_metrics( + &self, + eos: &rdkafka::statistics::ExactlyOnceSemantics, + ) -> Vec { + let mut mfs = Vec::new(); + + self.epoch_cnt.inc_by(eos.epoch_cnt as u64); + self.producer_id.set(eos.producer_id); + self.producer_epoch.set(eos.producer_epoch); + + mfs.extend(self.epoch_cnt.collect()); + mfs.extend(self.producer_id.collect()); + mfs.extend(self.producer_epoch.collect()); + + mfs + } +} + +impl CoreMetrics { + fn collect_metrics(&self, stats: &Statistics) -> Vec { + let mut mfs = Vec::new(); + + self.msg_cnt.set(stats.msg_cnt as i64); + self.msg_size.set(stats.msg_size as i64); + self.msg_max.set(stats.msg_max as i64); + self.msg_size_max.set(stats.msg_size_max as i64); + self.metadata_cache_cnt.set(stats.metadata_cache_cnt); + self.tx.inc_by(stats.tx as u64); + self.tx_bytes.inc_by(stats.tx_bytes as u64); + self.rx.inc_by(stats.rx as u64); + self.rx_bytes.inc_by(stats.rx_bytes as u64); + self.txmsgs.inc_by(stats.txmsgs as u64); + self.txmsg_bytes.inc_by(stats.txmsg_bytes as u64); + self.rxmsgs.inc_by(stats.rxmsgs as u64); + self.rxmsg_bytes.inc_by(stats.rxmsg_bytes as u64); + + mfs.extend(self.msg_cnt.collect()); + mfs.extend(self.msg_size.collect()); + mfs.extend(self.msg_max.collect()); + mfs.extend(self.msg_size_max.collect()); + mfs.extend(self.metadata_cache_cnt.collect()); + mfs.extend(self.tx.collect()); + mfs.extend(self.tx_bytes.collect()); + mfs.extend(self.rx.collect()); + mfs.extend(self.rx_bytes.collect()); + mfs.extend(self.txmsgs.collect()); + mfs.extend(self.txmsg_bytes.collect()); + mfs.extend(self.rxmsgs.collect()); + mfs.extend(self.rxmsg_bytes.collect()); + + mfs + } +} + +impl ConsumerGroupMetrics { + fn new() -> anyhow::Result { + Ok(Self { + rebalance_cnt: IntCounter::new("kafka_cgrp_rebalance_total", "Total rebalances")?, + rebalance_age: IntGauge::new("kafka_cgrp_rebalance_age", "Rebalance age")?, + assignment_size: IntGauge::new("kafka_cgrp_assignment_size", "Assignment size")?, + }) + } +} + +impl EosMetrics { + fn new() -> anyhow::Result { + Ok(Self { + epoch_cnt: IntCounter::new("kafka_eos_epoch_total", "Total number of epochs")?, + producer_id: IntGauge::new("kafka_eos_producer_id", "Producer ID")?, + producer_epoch: IntGauge::new("kafka_eos_producer_epoch", "Producer epoch")?, + }) + } +} + +impl KafkaMetricsCollector { + pub fn new(stats: Arc>) -> anyhow::Result { + let mut descs = Vec::new(); + let topic_labels = &["topic"]; + let partition_labels = &["topic", "partition"]; + + let core_metrics = CoreMetrics::new()?; + let broker_metrics = BrokerMetrics::new()?; + let topic_metrics = TopicMetrics::new(topic_labels, &mut descs); + let partition_metrics = PartitionMetrics::new(partition_labels, &mut descs); + let consumer_metrics = ConsumerGroupMetrics::new()?; + let eos_metrics = EosMetrics::new()?; + + Ok(KafkaMetricsCollector { + stats, + descs, + core_metrics, + broker_metrics, + topic_metrics, + partition_metrics, + consumer_metrics, + eos_metrics, + }) } fn create_gauge_vec( @@ -450,239 +814,40 @@ impl Collector for KafkaMetricsCollector { Err(_) => return vec![], }; - // Core metrics let mut mfs = Vec::new(); - self.msg_cnt.set(stats.msg_cnt as i64); - self.msg_size.set(stats.msg_size as i64); - self.msg_max.set(stats.msg_max as i64); - self.msg_size_max.set(stats.msg_size_max as i64); - self.metadata_cache_cnt.set(stats.metadata_cache_cnt); - self.tx.inc_by(stats.tx as u64); - self.tx_bytes.inc_by(stats.tx_bytes as u64); - self.rx.inc_by(stats.rx as u64); - self.rx_bytes.inc_by(stats.rx_bytes as u64); - self.txmsgs.inc_by(stats.txmsgs as u64); - self.txmsg_bytes.inc_by(stats.txmsg_bytes as u64); - self.rxmsgs.inc_by(stats.rxmsgs as u64); - self.rxmsg_bytes.inc_by(stats.rxmsg_bytes as u64); - mfs.extend(self.msg_cnt.collect()); - mfs.extend(self.msg_size.collect()); - mfs.extend(self.msg_max.collect()); - mfs.extend(self.msg_size_max.collect()); - mfs.extend(self.metadata_cache_cnt.collect()); - mfs.extend(self.tx.collect()); - mfs.extend(self.tx_bytes.collect()); - mfs.extend(self.rx.collect()); - mfs.extend(self.rx_bytes.collect()); - mfs.extend(self.txmsgs.collect()); - mfs.extend(self.txmsg_bytes.collect()); - mfs.extend(self.rxmsgs.collect()); - mfs.extend(self.rxmsg_bytes.collect()); + // Collect core metrics + mfs.extend(self.core_metrics.collect_metrics(&stats)); - // Broker metrics + // Collect broker metrics for (_broker_id, broker) in stats.brokers.iter() { - self.broker_state_cnt.set(match broker.state.as_str() { - "UP" => 1, - "DOWN" => 0, - _ => -1, - }); - - self.broker_outbuf_cnt.set(broker.outbuf_cnt); - self.broker_outbuf_msg_cnt.set(broker.outbuf_msg_cnt); - self.broker_waitresp_cnt.set(broker.waitresp_cnt); - self.broker_waitresp_msg_cnt.set(broker.waitresp_msg_cnt); - - self.broker_tx.inc_by(broker.tx); - self.broker_tx_bytes.inc_by(broker.txbytes); - self.broker_tx_errs.inc_by(broker.txerrs); - self.broker_tx_retries.inc_by(broker.txretries); - self.broker_req_timeouts.inc_by(broker.req_timeouts); - self.broker_rx.inc_by(broker.rx); - self.broker_rx_bytes.inc_by(broker.rxbytes); - self.broker_rx_errs.inc_by(broker.rxerrs); - self.broker_rx_corrid_errs.inc_by(broker.rxcorriderrs); - self.broker_rx_partial.inc_by(broker.rxpartial); - - if let Some(connects) = broker.connects { - self.broker_connects.inc_by(connects as u64); - } - if let Some(disconnects) = broker.disconnects { - self.broker_disconnects.inc_by(disconnects as u64); - } - - // Latency metrics - if let Some(ref latency) = broker.int_latency { - self.broker_int_latency.observe(latency.avg as f64); - } - if let Some(ref latency) = broker.outbuf_latency { - self.broker_outbuf_latency.observe(latency.avg as f64); - } - if let Some(ref rtt) = broker.rtt { - self.broker_rtt.observe(rtt.avg as f64); - } - if let Some(ref throttle) = broker.throttle { - self.broker_throttle.observe(throttle.avg as f64); - } + mfs.extend(self.broker_metrics.collect_metrics(broker)); } - mfs.extend(self.broker_state_cnt.collect()); - mfs.extend(self.broker_outbuf_cnt.collect()); - mfs.extend(self.broker_outbuf_msg_cnt.collect()); - mfs.extend(self.broker_waitresp_cnt.collect()); - mfs.extend(self.broker_waitresp_msg_cnt.collect()); - mfs.extend(self.broker_tx.collect()); - mfs.extend(self.broker_tx_bytes.collect()); - mfs.extend(self.broker_tx_errs.collect()); - mfs.extend(self.broker_tx_retries.collect()); - mfs.extend(self.broker_req_timeouts.collect()); - mfs.extend(self.broker_rx.collect()); - mfs.extend(self.broker_rx_bytes.collect()); - mfs.extend(self.broker_rx_errs.collect()); - mfs.extend(self.broker_rx_corrid_errs.collect()); - mfs.extend(self.broker_rx_partial.collect()); - mfs.extend(self.broker_connects.collect()); - mfs.extend(self.broker_disconnects.collect()); - mfs.extend(self.broker_int_latency.collect()); - mfs.extend(self.broker_outbuf_latency.collect()); - mfs.extend(self.broker_rtt.collect()); - mfs.extend(self.broker_throttle.collect()); - - // Topic and partition metrics with labels + // Collect topic and partition metrics for (topic_name, topic) in stats.topics.iter() { - self.topic_metadata_age - .with_label_values(&[topic_name]) - .set(topic.metadata_age); - self.topic_batchsize - .with_label_values(&[topic_name]) - .observe(topic.batchsize.avg as f64); - self.topic_batchcnt - .with_label_values(&[topic_name]) - .observe(topic.batchcnt.avg as f64); + mfs.extend(self.topic_metrics.collect_metrics(topic_name, topic)); + // Collect partition metrics for each topic for (partition_id, partition) in topic.partitions.iter() { - let labels = &[topic_name.as_str(), &partition_id.to_string()]; - self.partition_msgq_cnt - .with_label_values(labels) - .set(partition.msgq_cnt); - self.partition_msgq_bytes - .with_label_values(labels) - .set(partition.msgq_bytes as i64); - self.partition_xmit_msgq_cnt - .with_label_values(labels) - .set(partition.xmit_msgq_cnt); - self.partition_xmit_msgq_bytes - .with_label_values(labels) - .set(partition.xmit_msgq_bytes as i64); - self.partition_fetchq_cnt - .with_label_values(labels) - .set(partition.fetchq_cnt); - self.partition_fetchq_size - .with_label_values(labels) - .set(partition.fetchq_size as i64); - self.partition_query_offset - .with_label_values(labels) - .set(partition.query_offset); - self.partition_next_offset - .with_label_values(labels) - .set(partition.next_offset); - self.partition_app_offset - .with_label_values(labels) - .set(partition.app_offset); - self.partition_stored_offset - .with_label_values(labels) - .set(partition.stored_offset); - self.partition_committed_offset - .with_label_values(labels) - .set(partition.committed_offset); - self.partition_eof_offset - .with_label_values(labels) - .set(partition.eof_offset); - self.partition_lo_offset - .with_label_values(labels) - .set(partition.lo_offset); - self.partition_hi_offset - .with_label_values(labels) - .set(partition.hi_offset); - self.partition_consumer_lag - .with_label_values(labels) - .set(partition.consumer_lag); - self.partition_consumer_lag_stored - .with_label_values(labels) - .set(partition.consumer_lag_stored); - self.partition_txmsgs - .with_label_values(labels) - .inc_by(partition.txmsgs); - self.partition_txbytes - .with_label_values(labels) - .inc_by(partition.txbytes); - self.partition_rxmsgs - .with_label_values(labels) - .inc_by(partition.rxmsgs); - self.partition_rxbytes - .with_label_values(labels) - .inc_by(partition.rxbytes); - self.partition_msgs - .with_label_values(labels) - .inc_by(partition.msgs); - self.partition_rx_ver_drops - .with_label_values(labels) - .inc_by(partition.rx_ver_drops); - self.partition_msgs_inflight - .with_label_values(labels) - .set(partition.msgs_inflight); + mfs.extend(self.partition_metrics.collect_metrics( + topic_name, + partition_id, + partition, + )); } } - mfs.extend(self.topic_metadata_age.collect()); - mfs.extend(self.topic_batchsize.collect()); - mfs.extend(self.topic_batchcnt.collect()); - mfs.extend(self.partition_msgq_cnt.collect()); - mfs.extend(self.partition_msgq_bytes.collect()); - mfs.extend(self.partition_xmit_msgq_cnt.collect()); - mfs.extend(self.partition_xmit_msgq_bytes.collect()); - mfs.extend(self.partition_fetchq_cnt.collect()); - mfs.extend(self.partition_fetchq_size.collect()); - mfs.extend(self.partition_query_offset.collect()); - mfs.extend(self.partition_next_offset.collect()); - mfs.extend(self.partition_app_offset.collect()); - mfs.extend(self.partition_stored_offset.collect()); - mfs.extend(self.partition_committed_offset.collect()); - mfs.extend(self.partition_eof_offset.collect()); - mfs.extend(self.partition_lo_offset.collect()); - mfs.extend(self.partition_hi_offset.collect()); - mfs.extend(self.partition_consumer_lag.collect()); - mfs.extend(self.partition_consumer_lag_stored.collect()); - mfs.extend(self.partition_txmsgs.collect()); - mfs.extend(self.partition_txbytes.collect()); - mfs.extend(self.partition_rxmsgs.collect()); - mfs.extend(self.partition_rxbytes.collect()); - mfs.extend(self.partition_msgs.collect()); - mfs.extend(self.partition_rx_ver_drops.collect()); - mfs.extend(self.partition_msgs_inflight.collect()); - - // Consumer group metrics + // Collect consumer group metrics if let Some(ref cgrp) = stats.cgrp { - self.cgrp_rebalance_cnt.inc_by(cgrp.rebalance_cnt as u64); - self.cgrp_rebalance_age.set(cgrp.rebalance_age); - self.cgrp_assignment_size.set(cgrp.assignment_size as i64); + mfs.extend(self.consumer_metrics.collect_metrics(cgrp)); } - mfs.extend(self.cgrp_rebalance_cnt.collect()); - mfs.extend(self.cgrp_rebalance_age.collect()); - mfs.extend(self.cgrp_assignment_size.collect()); - - // EOS metrics + // Collect EOS metrics if let Some(ref eos) = stats.eos { - self.eos_epoch_cnt.inc_by(eos.epoch_cnt as u64); - self.eos_producer_id.set(eos.producer_id); - self.eos_producer_epoch.set(eos.producer_epoch); + mfs.extend(self.eos_metrics.collect_metrics(eos)); } - mfs.extend(self.eos_epoch_cnt.collect()); - mfs.extend(self.eos_producer_id.collect()); - mfs.extend(self.eos_producer_epoch.collect()); - mfs } } From 7604bc516bea3777f89844f4c0714ecb7257440a Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 26 Dec 2024 11:10:19 +0100 Subject: [PATCH 14/65] Refactor chunk size configuration --- src/connectors/kafka/config.rs | 24 +++++++++++++++++++++++- src/connectors/kafka/processor.rs | 26 +++++++++++++------------- src/connectors/kafka/sink.rs | 10 +--------- src/connectors/mod.rs | 4 +--- 4 files changed, 38 insertions(+), 26 deletions(-) diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index de2a824ec..c95777208 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -640,6 +640,28 @@ impl ConsumerConfig { pub fn topics(&self) -> Vec<&str> { self.topics.iter().map(|t| t.as_str()).collect() } + + pub fn buffer_config(&self) -> BufferConfig { + BufferConfig { + buffer_size: self.buffer_size, + buffer_timeout: self.buffer_timeout, + } + } +} + +#[derive(Clone, Debug)] +pub struct BufferConfig { + pub buffer_size: usize, + pub buffer_timeout: Duration, +} + +impl Default for BufferConfig { + fn default() -> Self { + Self { + buffer_size: 10000, + buffer_timeout: Duration::from_millis(10000), + } + } } impl ProducerConfig { @@ -874,7 +896,7 @@ impl Default for ConsumerConfig { topics: vec![], group_id: "parseable-connect-cg".to_string(), buffer_size: 10_000, - buffer_timeout: Duration::from_millis(5000), + buffer_timeout: Duration::from_millis(10000), group_instance_id: "parseable-cg-ii".to_string(), // NOTE: cooperative-sticky does not work well in rdkafka when using manual commit. // @see https://github.com/confluentinc/librdkafka/issues/4629 diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index 4813e3c21..c18ca1dc9 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -17,6 +17,7 @@ */ use crate::connectors::common::processor::Processor; +use crate::connectors::kafka::config::BufferConfig; use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; use crate::event::format; use crate::event::format::EventFormat; @@ -30,7 +31,6 @@ use rdkafka::consumer::{CommitMode, Consumer}; use serde_json::Value; use std::collections::HashMap; use std::sync::Arc; -use std::time::Duration; use tokio_stream::wrappers::ReceiverStream; use tracing::{debug, error, warn}; @@ -110,25 +110,25 @@ where { processor: Arc

, consumer: Arc, - buffer_size: usize, - buffer_timeout: Duration, + buffer_config: BufferConfig, } impl

StreamWorker

where P: Processor, ()> + Send + Sync + 'static, { - pub fn new( - processor: Arc

, - consumer: Arc, - buffer_size: usize, - buffer_timeout: Duration, - ) -> Self { + pub fn new(processor: Arc

, consumer: Arc) -> Self { + let buffer_config = consumer + .context() + .config() + .consumer() + .expect("Consumer config is missing") + .buffer_config(); + Self { processor, consumer, - buffer_size, - buffer_timeout, + buffer_config, } } @@ -139,8 +139,8 @@ where ) -> anyhow::Result<()> { let chunked_stream = tokio_stream::StreamExt::chunks_timeout( record_stream, - self.buffer_size, - self.buffer_timeout, + self.buffer_config.buffer_size, + self.buffer_config.buffer_timeout, ); chunked_stream diff --git a/src/connectors/kafka/sink.rs b/src/connectors/kafka/sink.rs index e2a117ee3..d1c02b71c 100644 --- a/src/connectors/kafka/sink.rs +++ b/src/connectors/kafka/sink.rs @@ -23,7 +23,6 @@ use crate::connectors::kafka::ConsumerRecord; use anyhow::Result; use futures_util::StreamExt; use std::sync::Arc; -use tokio::time::Duration; use tracing::{error, info}; pub struct KafkaSinkConnector

@@ -38,17 +37,10 @@ impl

KafkaSinkConnector

where P: Processor, ()> + Send + Sync + 'static, { - pub fn new( - kafka_streams: KafkaStreams, - processor: P, - buffer_size: usize, - buffer_timeout: Duration, - ) -> Self { + pub fn new(kafka_streams: KafkaStreams, processor: P) -> Self { let worker = Arc::new(StreamWorker::new( Arc::new(processor), kafka_streams.consumer(), - buffer_size, - buffer_timeout, )); Self { diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 0dfe2e3a5..5d8038cd5 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -31,7 +31,6 @@ use crate::option::{Mode, CONFIG}; use actix_web_prometheus::PrometheusMetrics; use prometheus::Registry; use std::sync::Arc; -use std::time::Duration; use tokio::sync::RwLock; use tracing::{info, warn}; @@ -93,8 +92,7 @@ where let stats = kafka_streams.statistics(); registry.register(Box::new(KafkaMetricsCollector::new(stats)?))?; - let kafka_parseable_sink_connector = - KafkaSinkConnector::new(kafka_streams, processor, 10000, Duration::from_millis(5000)); + let kafka_parseable_sink_connector = KafkaSinkConnector::new(kafka_streams, processor); rebalance_listener.start(); kafka_parseable_sink_connector.run().await?; From 3a0fbb0a5f8e918674d6e4e3dbc7d164c0f80604 Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 26 Dec 2024 11:14:26 +0100 Subject: [PATCH 15/65] use comment instead todo! macro --- src/connectors/kafka/mod.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs index bd2edcba3..48987d98b 100644 --- a/src/connectors/kafka/mod.rs +++ b/src/connectors/kafka/mod.rs @@ -256,6 +256,7 @@ impl ClientContext for KafkaContext { &self, _oauthbearer_config: Option<&str>, ) -> Result> { - todo!("Implement OAuth token generation") + // TODO Implement OAuth token generation when needed + Err("OAuth token generation is not implemented".into()) } } From 7f94f3a3e52b08f7719e93e206fdf7fb35c9e011 Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 26 Dec 2024 20:37:21 +0100 Subject: [PATCH 16/65] add license header --- src/connectors/common/config.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/connectors/common/config.rs b/src/connectors/common/config.rs index e9db3c346..ec05f7708 100644 --- a/src/connectors/common/config.rs +++ b/src/connectors/common/config.rs @@ -1,3 +1,21 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + use crate::connectors::kafka::config::KafkaConfig; use clap::{ArgMatches, FromArgMatches, Parser, Subcommand, ValueEnum}; use std::fmt; From bfb4071cfc46de8c65ce0f1264d979ae91ecdcaa Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 26 Dec 2024 20:56:04 +0100 Subject: [PATCH 17/65] cargo update --- Cargo.lock | 1709 +++++++++++++++++++++++++++++----------------------- 1 file changed, 957 insertions(+), 752 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7c8db9933..ce8b077a0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8,7 +8,7 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f7b0a21988c1bf877cf4759ef5ddaac04c1c9fe808c9142ecb78ba97d97a28a" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "bytes", "futures-core", "futures-sink", @@ -46,9 +46,9 @@ dependencies = [ "actix-tls", "actix-utils", "ahash", - "base64 0.22.0", - "bitflags 2.5.0", - "brotli", + "base64 0.22.1", + "bitflags 2.6.0", + "brotli 6.0.0", "bytes", "bytestring", "derive_more", @@ -101,9 +101,9 @@ dependencies = [ [[package]] name = "actix-rt" -version = "2.9.0" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28f32d40287d3f402ae0028a9d54bef51af15c8769492826a69d28f81893151d" +checksum = "24eda4e2a6e042aa4e55ac438a2ae052d3b5da0ecf83d7411e1a368946925208" dependencies = [ "futures-core", "tokio", @@ -111,16 +111,16 @@ dependencies = [ [[package]] name = "actix-server" -version = "2.3.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3eb13e7eef0423ea6eab0e59f6c72e7cb46d33691ad56a726b3cd07ddec2c2d4" +checksum = "7ca2549781d8dd6d75c40cf6b6051260a2cc2f3c62343d761a969a0640646894" dependencies = [ "actix-rt", "actix-service", "actix-utils", "futures-core", "futures-util", - "mio 0.8.11", + "mio", "socket2", "tokio", "tracing", @@ -223,13 +223,13 @@ dependencies = [ [[package]] name = "actix-web-httpauth" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d613edf08a42ccc6864c941d30fe14e1b676a77d16f1dbadc1174d065a0a775" +checksum = "456348ed9dcd72a13a1f4a660449fafdecee9ac8205552e286809eb5b0b29bd3" dependencies = [ "actix-utils", "actix-web", - "base64 0.21.7", + "base64 0.22.1", "futures-core", "futures-util", "log", @@ -248,7 +248,7 @@ dependencies = [ "pin-project", "prometheus", "quanta", - "thiserror 1.0.64", + "thiserror 1.0.69", ] [[package]] @@ -265,19 +265,13 @@ dependencies = [ [[package]] name = "addr2line" -version = "0.21.0" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" dependencies = [ "gimli", ] -[[package]] -name = "adler" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" - [[package]] name = "adler2" version = "2.0.0" @@ -324,9 +318,9 @@ dependencies = [ [[package]] name = "allocator-api2" -version = "0.2.18" +version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" +checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" [[package]] name = "android-tzdata" @@ -345,66 +339,67 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.13" +version = "0.6.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d96bd03f33fe50a863e394ee9718a706f988b9079b20c3784fb726e7678b62fb" +checksum = "8acc5369981196006228e28809f761875c0327210a891e941f4c683b3a99529b" dependencies = [ "anstyle", "anstyle-parse", "anstyle-query", "anstyle-wincon", "colorchoice", + "is_terminal_polyfill", "utf8parse", ] [[package]] name = "anstyle" -version = "1.0.6" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8901269c6307e8d93993578286ac0edf7f195079ffff5ebdeea6a59ffb7e36bc" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anstyle-parse" -version = "0.2.3" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c75ac65da39e5fe5ab759307499ddad880d724eed2f6ce5b5e8a26f4f387928c" +checksum = "3b2d16507662817a6a20a9ea92df6652ee4f94f914589377d69f3b21bc5798a9" dependencies = [ "utf8parse", ] [[package]] name = "anstyle-query" -version = "1.0.2" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e28923312444cdd728e4738b3f9c9cac739500909bb3d3c94b43551b16517648" +checksum = "79947af37f4177cfead1110013d678905c37501914fba0efea834c3fe9a8d60c" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "anstyle-wincon" -version = "3.0.2" +version = "3.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd54b81ec8d6180e24654d0b371ad22fc3dd083b6ff8ba325b72e00c87660a7" +checksum = "2109dbce0e72be3ec00bed26e6a7479ca384ad226efdd66db8fa2e3a38c83125" dependencies = [ "anstyle", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "anyhow" -version = "1.0.82" +version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f538837af36e6f6a9be0faa67f9a314f8119e4e4b5867c6ab40ed60360142519" +checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" dependencies = [ "backtrace", ] [[package]] name = "arbitrary" -version = "1.3.2" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d5a26814d8dcb93b0e5a0ff3c6d80a8843bafb21b39e8e18a6f05471870e110" +checksum = "dde20b3d026af13f561bdd0f15edf01fc734f0dafcedbaf42bba506a9517f223" dependencies = [ "derive_arbitrary", ] @@ -423,21 +418,21 @@ dependencies = [ [[package]] name = "arrayref" -version = "0.3.7" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" +checksum = "76a2e8124351fda1ef8aaaa3bbd7ebbcb486bbcd4225aca0aa0d84bb2db8fecb" [[package]] name = "arrayvec" -version = "0.7.4" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45aef0d9cf9a039bf6cd1acc451b137aca819977b0928dece52bd92811b640ba" +checksum = "c91839b07e474b3995035fd8ac33ee54f9c9ccbbb1ea33d9909c71bffdf1259d" dependencies = [ "arrow-arith", "arrow-array", @@ -456,9 +451,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03675e42d1560790f3524800e41403b40d0da1c793fe9528929fde06d8c7649a" +checksum = "855c57c4efd26722b044dcd3e348252560e3e0333087fb9f6479dc0bf744054f" dependencies = [ "arrow-array", "arrow-buffer", @@ -471,9 +466,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd2bf348cf9f02a5975c5962c7fa6dee107a2009a7b41ac5fb1a027e12dc033f" +checksum = "bd03279cea46569acf9295f6224fbc370c5df184b4d2ecfe97ccb131d5615a7f" dependencies = [ "ahash", "arrow-buffer", @@ -482,15 +477,15 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.2", "num", ] [[package]] name = "arrow-buffer" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3092e37715f168976012ce52273c3989b5793b0db5f06cbaa246be25e5f0924d" +checksum = "9e4a9b9b1d6d7117f6138e13bc4dd5daa7f94e671b70e8c9c4dc37b4f5ecfc16" dependencies = [ "bytes", "half", @@ -499,9 +494,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ce1018bb710d502f9db06af026ed3561552e493e989a79d0d0f5d9cf267a785" +checksum = "bc70e39916e60c5b7af7a8e2719e3ae589326039e1e863675a008bee5ffe90fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -509,7 +504,7 @@ dependencies = [ "arrow-schema", "arrow-select", "atoi", - "base64 0.22.0", + "base64 0.22.1", "chrono", "comfy-table", "half", @@ -520,9 +515,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd178575f45624d045e4ebee714e246a05d9652e41363ee3f57ec18cca97f740" +checksum = "789b2af43c1049b03a8d088ff6b2257cdcea1756cd76b174b1f2600356771b97" dependencies = [ "arrow-array", "arrow-buffer", @@ -539,9 +534,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e4ac0c4ee79150afe067dc4857154b3ee9c1cd52b5f40d59a77306d0ed18d65" +checksum = "e4e75edf21ffd53744a9b8e3ed11101f610e7ceb1a29860432824f1834a1f623" dependencies = [ "arrow-buffer", "arrow-schema", @@ -551,16 +546,16 @@ dependencies = [ [[package]] name = "arrow-flight" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b915fb36d935b969894d7909ad417c67ddeadebbbd57c3c168edf64721a37d31" +checksum = "4c09b331887a526f203f2123444792aee924632bd08b9940435070901075832e" dependencies = [ "arrow-array", "arrow-buffer", "arrow-cast", "arrow-ipc", "arrow-schema", - "base64 0.22.0", + "base64 0.22.1", "bytes", "futures", "paste", @@ -572,9 +567,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb307482348a1267f91b0912e962cd53440e5de0f7fb24c5f7b10da70b38c94a" +checksum = "d186a909dece9160bf8312f5124d797884f608ef5435a36d9d608e0b2a9bcbf8" dependencies = [ "arrow-array", "arrow-buffer", @@ -588,9 +583,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d24805ba326758effdd6f2cbdd482fcfab749544f21b134701add25b33f474e6" +checksum = "b66ff2fedc1222942d0bd2fd391cb14a85baa3857be95c9373179bd616753b85" dependencies = [ "arrow-array", "arrow-buffer", @@ -599,7 +594,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.5.0", + "indexmap 2.7.0", "lexical-core", "num", "serde", @@ -608,9 +603,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "644046c479d80ae8ed02a7f1e1399072ea344ca6a7b0e293ab2d5d9ed924aa3b" +checksum = "ece7b5bc1180e6d82d1a60e1688c199829e8842e38497563c3ab6ea813e527fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -623,9 +618,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a29791f8eb13b340ce35525b723f5f0df17ecb955599e11f65c2a94ab34e2efb" +checksum = "745c114c8f0e8ce211c83389270de6fbe96a9088a7b32c2a041258a443fe83ff" dependencies = [ "ahash", "arrow-array", @@ -637,18 +632,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c85320a3a2facf2b2822b57aa9d6d9d55edb8aee0b6b5d3b8df158e503d10858" +checksum = "b95513080e728e4cec37f1ff5af4f12c9688d47795d17cda80b6ec2cf74d4678" dependencies = [ "serde", ] [[package]] name = "arrow-select" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cc7e6b582e23855fd1625ce46e51647aa440c20ea2e71b1d748e0839dd73cba" +checksum = "8e415279094ea70323c032c6e739c48ad8d80e78a09bef7117b8718ad5bf3722" dependencies = [ "ahash", "arrow-array", @@ -660,9 +655,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0775b6567c66e56ded19b87a954b6b1beffbdd784ef95a3a2b03f59570c1d230" +checksum = "11d956cae7002eb8d83a27dbd34daaea1cf5b75852f0b84deb4d93a276e92bbf" dependencies = [ "arrow-array", "arrow-buffer", @@ -677,11 +672,11 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.8" +version = "0.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07dbbf24db18d609b1462965249abdf49129ccad073ec257da372adc83259c60" +checksum = "df895a515f70646414f4b45c0b79082783b80552b373a68283012928df56f522" dependencies = [ - "bzip2", + "bzip2 0.4.4", "flate2", "futures-core", "futures-io", @@ -695,9 +690,9 @@ dependencies = [ [[package]] name = "async-stream" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd56dd203fef61ac097dd65721a419ddccb106b2d2b70ba60a6b529f03961a51" +checksum = "0b5a71a6f37880a80d1d7f19efd781e4b5de42c88f0722cc13bcb6cc2cfe8476" dependencies = [ "async-stream-impl", "futures-core", @@ -706,9 +701,9 @@ dependencies = [ [[package]] name = "async-stream-impl" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" +checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", @@ -717,9 +712,9 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.82" +version = "0.1.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" +checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", @@ -743,22 +738,22 @@ checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" [[package]] name = "autocfg" -version = "1.2.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1fdabc7756949593fe60f30ec81974b613357de856987752631dea1e3394c80" +checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "axum" -version = "0.7.5" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a6c9af12842a67734c9a2e355436e5d03b22383ed60cf13cd0c18fbfe3dcbcf" +checksum = "edca88bc138befd0323b20752846e6587272d3b03b0343c8ea28a6f819e6e71f" dependencies = [ "async-trait", "axum-core", "bytes", "futures-util", - "http 1.1.0", - "http-body 1.0.0", + "http 1.2.0", + "http-body 1.0.1", "http-body-util", "itoa", "matchit", @@ -768,28 +763,28 @@ dependencies = [ "pin-project-lite", "rustversion", "serde", - "sync_wrapper 1.0.1", - "tower", + "sync_wrapper 1.0.2", + "tower 0.5.2", "tower-layer", "tower-service", ] [[package]] name = "axum-core" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e6b8ba012a258d63c9adfa28b9ddcf66149da6f986c5b5452e629d5ee64bf00" +checksum = "09f2bd6146b97ae3359fa0cc6d6b376d9539582c7b4220f041a33ec24c226199" dependencies = [ "async-trait", "bytes", "futures-util", - "http 1.1.0", - "http-body 1.0.0", + "http 1.2.0", + "http-body 1.0.1", "http-body-util", "mime", "pin-project-lite", "rustversion", - "sync_wrapper 1.0.1", + "sync_wrapper 1.0.2", "tower-layer", "tower-service", ] @@ -800,32 +795,26 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ba5289ec98f68f28dd809fd601059e6aa908bb8f6108620930828283d4ee23d7" dependencies = [ - "fastrand 2.0.2", + "fastrand 2.3.0", "gloo-timers", "tokio", ] [[package]] name = "backtrace" -version = "0.3.71" +version = "0.3.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b05800d2e817c8b3b4b54abd461726265fa9789ae34330622f2db9ee696f9d" +checksum = "8d82cb332cdfaed17ae235a638438ac4d4839913cc2af585c3c6746e8f8bee1a" dependencies = [ "addr2line", - "cc", "cfg-if", "libc", - "miniz_oxide 0.7.2", + "miniz_oxide", "object", "rustc-demangle", + "windows-targets 0.52.6", ] -[[package]] -name = "base64" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" - [[package]] name = "base64" version = "0.21.7" @@ -834,9 +823,9 @@ checksum = "9d297deb1925b89f2ccc13d7635fa0714f12c87adce1c75356b39ca9b7178567" [[package]] name = "base64" -version = "0.22.0" +version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9475866fec1451be56a3c2400fd081ff546538961565ccb5b7142cbd22bc7a51" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" [[package]] name = "base64ct" @@ -868,9 +857,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1" +checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de" [[package]] name = "blake2" @@ -883,9 +872,9 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.1" +version = "1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30cca6d3674597c30ddf2c587bf8d9d65c9a84d2326d941cc79c9842dfe0ef52" +checksum = "b8ee0c1824c4dea5b5f81736aff91bae041d2c07ee1192bec91054e10e3e601e" dependencies = [ "arrayref", "arrayvec", @@ -914,6 +903,17 @@ dependencies = [ "brotli-decompressor", ] +[[package]] +name = "brotli" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + [[package]] name = "brotli-decompressor" version = "4.0.1" @@ -938,15 +938,15 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.6.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" +checksum = "325918d6fe32f23b19878fe4b34794ae41fc19ddbe53b10571a4874d44ffd39b" [[package]] name = "bytestring" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74d80203ea6b29df88012294f62733de21cfeab47f17b41af3a38bc30a03ee72" +checksum = "e465647ae23b2823b0753f50decb2d5a86d2bb2cac04788fafd1f80e45378e5f" dependencies = [ "bytes", ] @@ -961,6 +961,16 @@ dependencies = [ "libc", ] +[[package]] +name = "bzip2" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bafdbf26611df8c14810e268ddceda071c297570a5fb360ceddf617fe417ef58" +dependencies = [ + "bzip2-sys", + "libc", +] + [[package]] name = "bzip2-sys" version = "0.1.11+1.0.8" @@ -974,18 +984,18 @@ dependencies = [ [[package]] name = "camino" -version = "1.1.6" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c59e92b5a388f549b863a7bea62612c09f24c8393560709a54558a9abdfb3b9c" +checksum = "8b96ec4966b5813e2c0507c1f86115c8c5abaadc3980879c3424042a02fd1ad3" dependencies = [ "serde", ] [[package]] name = "cargo-platform" -version = "0.1.8" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24b1f0365a6c6bb4020cd05806fd0d33c44d38046b8bd7f0e40814b9763cabfc" +checksum = "e35af189006b9c0f00a064685c727031e3ed2d8020f7ba284d78cc2671bd36ea" dependencies = [ "serde", ] @@ -1001,14 +1011,14 @@ dependencies = [ "semver", "serde", "serde_json", - "thiserror 1.0.64", + "thiserror 1.0.69", ] [[package]] name = "cargo_toml" -version = "0.20.1" +version = "0.20.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35a60491a82bdc0440640298990087ac1625e23c2feacd584eb33775903d5bb3" +checksum = "88da5a13c620b4ca0078845707ea9c3faf11edbc3ffd8497d11d686211cd1ac0" dependencies = [ "serde", "toml", @@ -1016,9 +1026,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.23" +version = "1.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3bbb537bb4a30b90362caddba8f360c0a56bc13d3a5570028e7197204cb54a17" +checksum = "c31a0499c1dc64f458ad13872de75c0eb7e3fdb0e67964610c914b034fc5956e" dependencies = [ "jobserver", "libc", @@ -1031,6 +1041,12 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" +[[package]] +name = "cfg_aliases" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" + [[package]] name = "change-detection" version = "1.2.0" @@ -1043,9 +1059,9 @@ dependencies = [ [[package]] name = "chrono" -version = "0.4.38" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" +checksum = "7e36cc9d416881d2e24f9a963be5fb1cd90966419ac844274161d10488b3e825" dependencies = [ "android-tzdata", "iana-time-zone", @@ -1067,9 +1083,9 @@ dependencies = [ [[package]] name = "chrono-tz" -version = "0.9.0" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93698b29de5e97ad0ae26447b344c482a7284c737d9ddc5f9e52b74a336671bb" +checksum = "cd6dd8046d00723a59a2f8c5f295c515b9bb9a331ee4f8f3d4dd49e428acd3b6" dependencies = [ "chrono", "chrono-tz-build", @@ -1078,20 +1094,19 @@ dependencies = [ [[package]] name = "chrono-tz-build" -version = "0.3.0" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c088aee841df9c3041febbb73934cfc39708749bf96dc827e3359cd39ef11b1" +checksum = "e94fea34d77a245229e7746bd2beb786cd2a896f306ff491fb8cecb3074b10a7" dependencies = [ "parse-zoneinfo", - "phf", "phf_codegen", ] [[package]] name = "clap" -version = "4.5.4" +version = "4.5.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90bc066a67923782aa8515dbaea16946c5bcc5addbd668bb80af688e53e548a0" +checksum = "3135e7ec2ef7b10c6ed8950f0f792ed96ee093fa088608f1c76e569722700c84" dependencies = [ "clap_builder", "clap_derive", @@ -1099,9 +1114,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.2" +version = "4.5.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae129e2e766ae0ec03484e609954119f123cc1fe650337e155d03b022f24f7b4" +checksum = "30582fc632330df2bd26877bde0c1f4470d57c582bbc070376afcd04d8cb4838" dependencies = [ "anstream", "anstyle", @@ -1110,11 +1125,11 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.4" +version = "4.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "528131438037fd55894f62d6e9f068b8f45ac57ffa77517819645d10aed04f64" +checksum = "4ac6a0c7b1a9e9a5186361f67dfa1b88213572f427fb9ab038efb2bd8c582dab" dependencies = [ - "heck 0.5.0", + "heck", "proc-macro2", "quote", "syn 2.0.91", @@ -1122,9 +1137,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.7.0" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98cc8fbded0c607b7ba9dd60cd98df59af97e84d24e49c8557331cfc26d301ce" +checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" [[package]] name = "clokwerk" @@ -1146,15 +1161,15 @@ dependencies = [ [[package]] name = "colorchoice" -version = "1.0.0" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" +checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" [[package]] name = "comfy-table" -version = "7.1.1" +version = "7.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" +checksum = "24f165e7b643266ea80cb858aed492ad9280e3e05ce24d4a99d7d7b889b6a4d9" dependencies = [ "strum", "strum_macros", @@ -1183,9 +1198,9 @@ dependencies = [ [[package]] name = "constant_time_eq" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" +checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" [[package]] name = "convert_case" @@ -1224,6 +1239,16 @@ dependencies = [ "libc", ] +[[package]] +name = "core-foundation" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b55271e5c8c478ad3f38ad24ef34923091e0548492a266d19b3c0b4d82574c63" +dependencies = [ + "core-foundation-sys", + "libc", +] + [[package]] name = "core-foundation-sys" version = "0.8.7" @@ -1232,9 +1257,9 @@ checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" [[package]] name = "cpufeatures" -version = "0.2.12" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" +checksum = "16b80225097f2e5ae4e7179dd2266824648f3e2f49d9134d584b76389d31c4c3" dependencies = [ "libc", ] @@ -1250,9 +1275,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.5" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" dependencies = [ "crossbeam-epoch", "crossbeam-utils", @@ -1269,9 +1294,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.20" +version = "0.8.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" [[package]] name = "crossterm" @@ -1279,11 +1304,11 @@ version = "0.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "829d955a0bb380ef178a640b91779e3987da38c9aea133b20614cfed8cdea9c6" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "crossterm_winapi", - "mio 1.0.2", + "mio", "parking_lot", - "rustix 0.38.34", + "rustix", "signal-hook", "signal-hook-mio", "winapi", @@ -1316,9 +1341,9 @@ dependencies = [ [[package]] name = "csv" -version = "1.3.0" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" +checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" dependencies = [ "csv-core", "itoa", @@ -1337,9 +1362,9 @@ dependencies = [ [[package]] name = "darling" -version = "0.20.8" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54e36fcd13ed84ffdfda6f5be89b31287cbb80c439841fe69e04841435464391" +checksum = "6f63b86c8a8826a49b8c21f08a2d07338eec8d900540f8630dc76284be802989" dependencies = [ "darling_core", "darling_macro", @@ -1347,9 +1372,9 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.20.8" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c2cf1c23a687a1feeb728783b993c4e1ad83d99f351801977dd809b48d0a70f" +checksum = "95133861a8032aaea082871032f5815eb9e98cef03fa916ab4500513994df9e5" dependencies = [ "fnv", "ident_case", @@ -1361,9 +1386,9 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.20.8" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a668eda54683121533a393014d8692171709ff57a7d61f187b6e782719f8933f" +checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", @@ -1386,15 +1411,15 @@ dependencies = [ [[package]] name = "data-encoding" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e962a19be5cfc3f3bf6dd8f61eb50107f356ad6270fbb3ed41476571db78be5" +checksum = "e8566979429cf69b49a5c740c60791108e86440e8be149bbea4fe54d2c32d6e2" [[package]] name = "datafusion" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee907b081e45e1d14e1f327e89ef134f91fcebad0bfc2dc229fa9f6044379682" +checksum = "dae5f2abc725737d6e87b6d348a5aa2d0a77e4cf873045f004546da946e6e619" dependencies = [ "ahash", "arrow", @@ -1404,7 +1429,7 @@ dependencies = [ "async-compression", "async-trait", "bytes", - "bzip2", + "bzip2 0.4.4", "chrono", "dashmap", "datafusion-catalog", @@ -1427,7 +1452,7 @@ dependencies = [ "glob", "half", "hashbrown 0.14.5", - "indexmap 2.5.0", + "indexmap 2.7.0", "itertools 0.13.0", "log", "num_cpus", @@ -1449,9 +1474,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c2b914f6e33c429af7d8696c72a47ed9225d7e2b82c747ebdfa2408ed53579f" +checksum = "998761705551f11ffa4ee692cc285b44eb1def6e0d28c4eaf5041b9e2810dc1e" dependencies = [ "arrow-schema", "async-trait", @@ -1464,9 +1489,9 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a84f8e76330c582a6b8ada0b2c599ca46cfe46b7585e458fc3f4092bc722a18" +checksum = "11986f191e88d950f10a5cc512a598afba27d92e04a0201215ad60785005115a" dependencies = [ "ahash", "arrow", @@ -1488,9 +1513,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf08cc30d92720d557df13bd5a5696213bd5ea0f38a866d8d85055d866fba774" +checksum = "694c9d7ea1b82f95768215c4cb5c2d5c613690624e832a7ee64be563139d582f" dependencies = [ "log", "tokio", @@ -1498,9 +1523,9 @@ dependencies = [ [[package]] name = "datafusion-execution" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86bc4183d5c45b9f068a6f351678a0d1eb1225181424542bb75db18ec280b822" +checksum = "30b4cedcd98151e0a297f34021b6b232ff0ebc0f2f18ea5e7446b5ebda99b1a1" dependencies = [ "arrow", "chrono", @@ -1519,9 +1544,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "202119ce58e4d103e37ae64aab40d4e574c97bdd2bea994bf307b175fcbfa74d" +checksum = "a8dd114dc0296cacaee98ad3165724529fcca9a65b2875abcd447b9cc02b2b74" dependencies = [ "ahash", "arrow", @@ -1541,9 +1566,9 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8b181ce8569216abb01ef3294aa16c0a40d7d39350c2ff01ede00f167a535f2" +checksum = "5d1ba2bb018218d9260bbd7de6a46a20f61b93d4911dba8aa07735625004c4fb" dependencies = [ "arrow", "datafusion-common", @@ -1552,13 +1577,13 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e4124b8066444e05a24472f852e94cf56546c0f4d92d00f018f207216902712" +checksum = "547cb780a4ac51fd8e52c0fb9188bc16cea4e35aebf6c454bda0b82a7a417304" dependencies = [ "arrow", "arrow-buffer", - "base64 0.22.0", + "base64 0.22.1", "blake2", "blake3", "chrono", @@ -1579,9 +1604,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b94acdac235ea21810150a89751617ef2db7e32eba27f54be48a81bde2bfe119" +checksum = "e68cf5aa7ebcac08bd04bb709a9a6d4963eafd227da62b628133bc509c40f5a0" dependencies = [ "ahash", "arrow", @@ -1600,9 +1625,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c9ea085bbf900bf16e2ca0f56fc56236b2e4f2e1a2cccb67bcd83c5ab4ad0ef" +checksum = "e2285d080dfecdfb8605b0ab2f1a41e2473208dc8e9bd6f5d1dbcfe97f517e6f" dependencies = [ "ahash", "arrow", @@ -1614,9 +1639,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c882e61665ed60c5ce9b061c1e587aeb8ae5ae4bcb5e5f2465139ab25328e0f" +checksum = "6b6ffbbb7cf7bf0c0e05eb6207023fef341cac83a593a5365a6fc83803c572a9" dependencies = [ "arrow", "arrow-array", @@ -1637,9 +1662,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98a354ce96df3ca6d025093adac9fd55ca09931c9b6f2630140721a95873fde4" +checksum = "6e78d30ebd6e9f74d4aeddec32744f5a18b5f9584591bc586fb5259c4848bac5" dependencies = [ "datafusion-common", "datafusion-expr", @@ -1649,9 +1674,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf677c74fb7b5a1899ef52709e4a70fff3ed80bdfb4bbe495909810e83d5f39" +checksum = "be172c44bf344df707e0c041fa3f41e6dc5fb0976f539c68bc442bca150ee58c" dependencies = [ "arrow", "async-trait", @@ -1660,7 +1685,7 @@ dependencies = [ "datafusion-expr", "datafusion-physical-expr", "hashbrown 0.14.5", - "indexmap 2.5.0", + "indexmap 2.7.0", "itertools 0.13.0", "log", "paste", @@ -1669,9 +1694,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30b077999f6eb6c43d6b25bc66332a3be2f693c382840f008dd763b8540f9530" +checksum = "43b86b7fa0b8161c49b0f005b0df193fc6d9b65ceec675f155422cda5d1583ca" dependencies = [ "ahash", "arrow", @@ -1680,7 +1705,7 @@ dependencies = [ "arrow-ord", "arrow-schema", "arrow-string", - "base64 0.22.0", + "base64 0.22.1", "chrono", "datafusion-common", "datafusion-execution", @@ -1691,7 +1716,7 @@ dependencies = [ "half", "hashbrown 0.14.5", "hex", - "indexmap 2.5.0", + "indexmap 2.7.0", "itertools 0.13.0", "log", "paste", @@ -1701,9 +1726,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dce847f885c2b13bbe29f5c8b7948797131aa470af6e16d2a94f4428b4f4f1bd" +checksum = "242ba8a26351d9ca16295814c46743b0d1b00ec372174bdfbba991d0953dd596" dependencies = [ "ahash", "arrow", @@ -1715,9 +1740,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d13238e3b9fdd62a4c18760bfef714bb990d1e1d3430e9f416aae4b3cfaa71af" +checksum = "25ca088eb904bf1cfc9c5e5653110c70a6eaba43164085a9d180b35b77ce3b8b" dependencies = [ "arrow-schema", "datafusion-common", @@ -1729,9 +1754,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faba6f55a7eaf0241d07d12c2640de52742646b10f754485d5192bdfe2c9ceae" +checksum = "4989a53b824abc759685eb643f4d604c2fc2fea4e2c309ac3473bea263ecbbeb" dependencies = [ "ahash", "arrow", @@ -1752,7 +1777,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap 2.5.0", + "indexmap 2.7.0", "itertools 0.13.0", "log", "once_cell", @@ -1764,9 +1789,9 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "42.0.0" +version = "42.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dad8d96a9b52e1aa24f9373696a815be828193efce7cb0bbd2140b6bb67d1819" +checksum = "66b9b75b9da10ed656073ac0553708f17eb8fa5a7b065ef9848914c93150ab9e" dependencies = [ "arrow", "arrow-array", @@ -1790,9 +1815,9 @@ dependencies = [ [[package]] name = "derive_arbitrary" -version = "1.3.2" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67e77553c4162a157adbf834ebae5b415acbecbeafc7a74b0e886657506a7611" +checksum = "30542c1ad912e0e3d22a1935c290e12e8a29d704a420177a31faad4a601a0800" dependencies = [ "proc-macro2", "quote", @@ -1854,9 +1879,9 @@ checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" [[package]] name = "encoding_rs" -version = "0.8.34" +version = "0.8.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b45de904aa0b010bce2ab45264d0631681847fa7b6f2eaa7dab7619943bc4f59" +checksum = "75030f3c4f45dafd7586dd6780965a8c7e8e285a5ecb86713e63a79c5b2766f3" dependencies = [ "cfg-if", ] @@ -1869,12 +1894,12 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.8" +version = "0.3.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a258e46cdc063eb8519c00b9fc845fc47bcfca4130e2f08e88665ceda8474245" +checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" dependencies = [ "libc", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -1888,9 +1913,9 @@ dependencies = [ [[package]] name = "fastrand" -version = "2.0.2" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "658bd65b1cf4c852a3cc96f18a8ce7b5640f6b703f905c7d74532294c2a63984" +checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "fixedbitset" @@ -1900,9 +1925,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flatbuffers" -version = "24.3.25" +version = "24.12.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" +checksum = "4f1baf0dbf96932ec9a3038d57900329c015b0bfb7b63d904f3bc27e2b02a096" dependencies = [ "bitflags 1.3.2", "rustc_version", @@ -1910,12 +1935,12 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.34" +version = "1.0.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1b589b4dc103969ad3cf85c950899926ec64300a1a46d76c03a6072957036f0" +checksum = "c936bfdafb507ebbf50b8074c54fa31c5be9a1e7e5f467dd659697041407d07c" dependencies = [ "crc32fast", - "miniz_oxide 0.8.0", + "miniz_oxide", ] [[package]] @@ -1941,9 +1966,9 @@ checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" [[package]] name = "futures" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" dependencies = [ "futures-channel", "futures-core", @@ -1972,9 +1997,9 @@ checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" [[package]] name = "futures-executor" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" dependencies = [ "futures-core", "futures-task", @@ -2061,9 +2086,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94b22e06ecb0110981051723910cbf0b5f5e09a2062dd7663334ee79a9d1286c" +checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" dependencies = [ "cfg-if", "js-sys", @@ -2074,9 +2099,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.28.1" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" +checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" @@ -2108,7 +2133,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.5.0", + "indexmap 2.7.0", "slab", "tokio", "tokio-util", @@ -2117,17 +2142,17 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.5" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa82e28a107a8cc405f0839610bdc9b15f1e25ec7d696aa5cf173edbcb1486ab" +checksum = "ccae279728d634d083c00f6099cb58f01cc99c145b84b8be2f6c74618d79922e" dependencies = [ "atomic-waker", "bytes", "fnv", "futures-core", "futures-sink", - "http 1.1.0", - "indexmap 2.5.0", + "http 1.2.0", + "indexmap 2.7.0", "slab", "tokio", "tokio-util", @@ -2170,6 +2195,12 @@ dependencies = [ "allocator-api2", ] +[[package]] +name = "hashbrown" +version = "0.15.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" + [[package]] name = "hashlru" version = "0.11.1" @@ -2180,12 +2211,6 @@ dependencies = [ "serde", ] -[[package]] -name = "heck" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" - [[package]] name = "heck" version = "0.5.0" @@ -2228,9 +2253,9 @@ dependencies = [ [[package]] name = "http" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21b9ddb458710bc376481b842f5da65cdf31522de232c1ca8146abce2a358258" +checksum = "f16ca2af56261c99fba8bac40a10251ce8188205a4c448fbb745a2e4daa76fea" dependencies = [ "bytes", "fnv", @@ -2239,11 +2264,11 @@ dependencies = [ [[package]] name = "http-auth-basic" -version = "0.3.3" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd2e17aacf7f4a2428def798e2ff4f4f883c0987bdaf47dd5c8bc027bc9f1ebc" +checksum = "0e0c088bddfd73005b09807131224ad12c302655436b1270c8346a3ae8aaa37a" dependencies = [ - "base64 0.13.1", + "base64 0.22.1", ] [[package]] @@ -2259,32 +2284,32 @@ dependencies = [ [[package]] name = "http-body" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cac85db508abc24a2e48553ba12a996e87244a0395ce011e62b37158745d643" +checksum = "1efedce1fb8e6913f23e0c92de8e62cd5b772a67e7b3946df930a62566c93184" dependencies = [ "bytes", - "http 1.1.0", + "http 1.2.0", ] [[package]] name = "http-body-util" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0475f8b2ac86659c21b64320d5d653f9efe42acd2a4e560073ec61a155a34f1d" +checksum = "793429d76616a256bcb62c2a2ec2bed781c8307e797e2598c50010f2bee2544f" dependencies = [ "bytes", - "futures-core", - "http 1.1.0", - "http-body 1.0.0", + "futures-util", + "http 1.2.0", + "http-body 1.0.1", "pin-project-lite", ] [[package]] name = "httparse" -version = "1.8.0" +version = "1.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" +checksum = "7d71d3574edd2771538b901e6549113b4006ece66150fb69c0fb6d9a2adae946" [[package]] name = "httpdate" @@ -2316,9 +2341,9 @@ dependencies = [ [[package]] name = "hyper" -version = "0.14.30" +version = "0.14.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a152ddd61dfaec7273fe8419ab357f33aee0d914c5f4efbf0d96fa749eea5ec9" +checksum = "41dfc780fdec9373c01bae43289ea34c972e40ee3c9f6b3c8801a35f35586ce7" dependencies = [ "bytes", "futures-channel", @@ -2340,16 +2365,16 @@ dependencies = [ [[package]] name = "hyper" -version = "1.4.1" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50dfd22e0e76d0f662d429a5f80fcaf3855009297eab6a0a9f8543834744ba05" +checksum = "256fb8d4bd6413123cc9d91832d78325c48ff41677595be797d90f42969beae0" dependencies = [ "bytes", "futures-channel", "futures-util", - "h2 0.4.5", - "http 1.1.0", - "http-body 1.0.0", + "h2 0.4.7", + "http 1.2.0", + "http-body 1.0.1", "httparse", "httpdate", "itoa", @@ -2367,7 +2392,7 @@ checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", "http 0.2.12", - "hyper 0.14.30", + "hyper 0.14.32", "rustls 0.21.12", "tokio", "tokio-rustls 0.24.1", @@ -2375,30 +2400,30 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.27.3" +version = "0.27.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08afdbb5c31130e3034af566421053ab03787c640246a446327f550d11bcb333" +checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2" dependencies = [ "futures-util", - "http 1.1.0", - "hyper 1.4.1", + "http 1.2.0", + "hyper 1.5.2", "hyper-util", - "rustls 0.23.13", + "rustls 0.23.20", "rustls-native-certs", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.0", + "tokio-rustls 0.26.1", "tower-service", - "webpki-roots 0.26.1", + "webpki-roots 0.26.7", ] [[package]] name = "hyper-timeout" -version = "0.5.1" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3203a961e5c83b6f5498933e78b6b263e208c197b63e9c6c53cc82ffd3f63793" +checksum = "2b90d566bffbce6a75bd8b09a05aa8c2cb1fabb6cb348f8840c9e4c90a0d83b0" dependencies = [ - "hyper 1.4.1", + "hyper 1.5.2", "hyper-util", "pin-project-lite", "tokio", @@ -2407,29 +2432,28 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.6" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ab92f4f49ee4fb4f997c784b7a2e0fa70050211e0b6a287f898c3c9785ca956" +checksum = "df2dcfbe0677734ab2f3ffa7fa7bfd4706bfdc1ef393f2ee30184aed67e631b4" dependencies = [ "bytes", "futures-channel", "futures-util", - "http 1.1.0", - "http-body 1.0.0", - "hyper 1.4.1", + "http 1.2.0", + "http-body 1.0.1", + "hyper 1.5.2", "pin-project-lite", "socket2", "tokio", - "tower", "tower-service", "tracing", ] [[package]] name = "iana-time-zone" -version = "0.1.60" +version = "0.1.61" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7ffbb5a1b541ea2561f8c41c087286cc091e21e556a4f09a8f6cbf17b69b141" +checksum = "235e081f3925a06703c2d0117ea8b91f042756fd6e7a6e5d901e8ca1a996b220" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -2448,6 +2472,124 @@ dependencies = [ "cc", ] +[[package]] +name = "icu_collections" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db2fa452206ebee18c4b5c2274dbf1de17008e874b4dc4f0aea9d01ca79e4526" +dependencies = [ + "displaydoc", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locid" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13acbb8371917fc971be86fc8057c41a64b521c184808a698c02acc242dbf637" +dependencies = [ + "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_locid_transform" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01d11ac35de8e40fdeda00d9e1e9d92525f3f9d887cdd7aa81d727596788b54e" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_locid_transform_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_locid_transform_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdc8ff3388f852bede6b579ad4e978ab004f139284d7b28715f773507b946f6e" + +[[package]] +name = "icu_normalizer" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19ce3e0da2ec68599d193c93d088142efd7f9c5d6fc9b803774855747dc6a84f" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_normalizer_data", + "icu_properties", + "icu_provider", + "smallvec", + "utf16_iter", + "utf8_iter", + "write16", + "zerovec", +] + +[[package]] +name = "icu_normalizer_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8cafbf7aa791e9b22bec55a167906f9e1215fd475cd22adfcf660e03e989516" + +[[package]] +name = "icu_properties" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93d6020766cfc6302c15dbbc9c8778c37e62c14427cb7f6e601d849e092aeef5" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_locid_transform", + "icu_properties_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_properties_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67a8effbc3dd3e4ba1afa8ad918d5684b8868b3b26500753effea8d2eed19569" + +[[package]] +name = "icu_provider" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ed421c8a8ef78d3e2dbc98a973be2f3770cb42b606e3ab18d6237c4dfde68d9" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_provider_macros", + "stable_deref_trait", + "tinystr", + "writeable", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_provider_macros" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.91", +] + [[package]] name = "ident_case" version = "1.0.1" @@ -2464,11 +2606,32 @@ dependencies = [ "unicode-normalization", ] +[[package]] +name = "idna" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "686f825264d630750a544639377bae737628043f20d38bbc029e8f29ea968a7e" +dependencies = [ + "idna_adapter", + "smallvec", + "utf8_iter", +] + +[[package]] +name = "idna_adapter" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "daca1df1c957320b2cf139ac61e7bd64fed304c5040df000a745aa1de3b4ef71" +dependencies = [ + "icu_normalizer", + "icu_properties", +] + [[package]] name = "impl-more" -version = "0.1.6" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "206ca75c9c03ba3d4ace2460e57b189f39f43de612c2f85836e65c929701bb2d" +checksum = "e8a5a9a0ff0086c7a148acb942baaabeadf9504d10400b5a05645853729b9cd2" [[package]] name = "indexmap" @@ -2482,19 +2645,19 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.5.0" +version = "2.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" +checksum = "62f822373a4fe84d4bb149bf54e584a7f4abec90e072ed49cda0edea5b95471f" dependencies = [ "equivalent", - "hashbrown 0.14.5", + "hashbrown 0.15.2", ] [[package]] name = "instant" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" +checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" dependencies = [ "cfg-if", "js-sys", @@ -2509,21 +2672,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] -name = "io-lifetimes" -version = "1.0.11" +name = "ipnet" +version = "2.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eae7b9aee968036d54dce06cebaefd919e4472e753296daccd6d344e3e2df0c2" -dependencies = [ - "hermit-abi", - "libc", - "windows-sys 0.48.0", -] +checksum = "ddc24109865250148c2e0f3d25d4f0f479571723792d3802153c60922a4fb708" [[package]] -name = "ipnet" -version = "2.9.0" +name = "is_terminal_polyfill" +version = "1.70.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" +checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" [[package]] name = "itertools" @@ -2545,25 +2703,26 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.11" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" +checksum = "d75a2a4b1b190afb6f5425f10f6a8f959d2ea0b9c2b1d79553551850539e4674" [[package]] name = "jobserver" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b099aaa34a9751c5bf0878add70444e1ed2dd73f347be99003d4577277de6e" +checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" dependencies = [ "libc", ] [[package]] name = "js-sys" -version = "0.3.69" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" +checksum = "6717b6b5b077764fb5966237269cb3c64edddde4b14ce42647430a78ced9e7b7" dependencies = [ + "once_cell", "wasm-bindgen", ] @@ -2575,15 +2734,15 @@ checksum = "d4345964bb142484797b161f473a503a434de77149dd8c7427788c6e13379388" [[package]] name = "lazy_static" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" [[package]] name = "lexical-core" -version = "0.8.5" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cde5de06e8d4c2faabc400238f9ae1c74d5412d03a7bd067645ccbc47070e46" +checksum = "b765c31809609075565a70b4b71402281283aeda7ecaf4818ac14a7b2ade8958" dependencies = [ "lexical-parse-float", "lexical-parse-integer", @@ -2594,9 +2753,9 @@ dependencies = [ [[package]] name = "lexical-parse-float" -version = "0.8.5" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "683b3a5ebd0130b8fb52ba0bdc718cc56815b6a097e28ae5a6997d0ad17dc05f" +checksum = "de6f9cb01fb0b08060209a057c048fcbab8717b4c1ecd2eac66ebfe39a65b0f2" dependencies = [ "lexical-parse-integer", "lexical-util", @@ -2605,9 +2764,9 @@ dependencies = [ [[package]] name = "lexical-parse-integer" -version = "0.8.6" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d0994485ed0c312f6d965766754ea177d07f9c00c9b82a5ee62ed5b47945ee9" +checksum = "72207aae22fc0a121ba7b6d479e42cbfea549af1479c3f3a4f12c70dd66df12e" dependencies = [ "lexical-util", "static_assertions", @@ -2615,18 +2774,18 @@ dependencies = [ [[package]] name = "lexical-util" -version = "0.8.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5255b9ff16ff898710eb9eb63cb39248ea8a5bb036bea8085b1a767ff6c4e3fc" +checksum = "5a82e24bf537fd24c177ffbbdc6ebcc8d54732c35b50a3f28cc3f4e4c949a0b3" dependencies = [ "static_assertions", ] [[package]] name = "lexical-write-float" -version = "0.8.5" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accabaa1c4581f05a3923d1b4cfd124c329352288b7b9da09e766b0668116862" +checksum = "c5afc668a27f460fb45a81a757b6bf2f43c2d7e30cb5a2dcd3abf294c78d62bd" dependencies = [ "lexical-util", "lexical-write-integer", @@ -2635,9 +2794,9 @@ dependencies = [ [[package]] name = "lexical-write-integer" -version = "0.8.5" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1b6f3d1f4422866b68192d62f77bc5c700bee84f3069f2469d7bc8c77852446" +checksum = "629ddff1a914a836fb245616a7888b62903aae58fa771e1d83943035efa0f978" dependencies = [ "lexical-util", "static_assertions", @@ -2651,9 +2810,9 @@ checksum = "b5aba8db14291edd000dfcc4d620c7ebfb122c613afb886ca8803fa4e128a20a" [[package]] name = "libm" -version = "0.2.8" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" +checksum = "8355be11b20d696c8f18f6cc018c4e372165b1fa8126cef092399c9951984ffa" [[package]] name = "libz-sys" @@ -2669,15 +2828,15 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.1.4" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f051f77a7c8e6957c0696eac88f26b0117e54f52d3fc682ab19397a8812846a4" +checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" [[package]] -name = "linux-raw-sys" -version = "0.4.13" +name = "litemap" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c" +checksum = "4ee93343901ab17bd981295f2cf0026d4ad018c7c31ba84549a4ddbb47a45104" [[package]] name = "local-channel" @@ -2698,9 +2857,9 @@ checksum = "4d873d7c67ce09b42110d801813efbc9364414e356be9935700d368351657487" [[package]] name = "lock_api" -version = "0.4.11" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" dependencies = [ "autocfg", "scopeguard", @@ -2714,9 +2873,9 @@ checksum = "9374ef4228402d4b7e403e5838cb880d9ee663314b0a900d5a6aabf0c213552e" [[package]] name = "log" -version = "0.4.21" +version = "0.4.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" [[package]] name = "lz4_flex" @@ -2792,9 +2951,9 @@ checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" [[package]] name = "mime_guess" -version = "2.0.4" +version = "2.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4192263c238a5f0d0c6bfd21f336a313a4ce1c450542449ca191bb657b4642ef" +checksum = "f7c44f8e672c00fe5308fa235f821cb4198414e1c77935c1ab6948d3fd78550e" dependencies = [ "mime", "unicase", @@ -2808,41 +2967,19 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d811f3e15f28568be3407c8e7fdb6514c1cda3cb30683f15b6a1a1dc4ea14a7" -dependencies = [ - "adler", -] - -[[package]] -name = "miniz_oxide" -version = "0.8.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" +checksum = "4ffbe83022cedc1d264172192511ae958937694cd57ce297164951b8b3568394" dependencies = [ "adler2", ] [[package]] name = "mio" -version = "0.8.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c" -dependencies = [ - "libc", - "log", - "wasi 0.11.0+wasi-snapshot-preview1", - "windows-sys 0.48.0", -] - -[[package]] -name = "mio" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80e04d1dcff3aae0704555fe5fee3bcfaf3d1fdf8a7e521d5b9d2b42acb52cec" +checksum = "2886843bf800fba2e3377cff24abf6379b4c4d5c6681eaf9ea5b0d15090450bd" dependencies = [ - "hermit-abi", "libc", "log", "wasi 0.11.0+wasi-snapshot-preview1", @@ -2886,9 +3023,9 @@ dependencies = [ [[package]] name = "num" -version = "0.4.2" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3135b08af27d103b0a51f2ae0f8632117b7b185ccf931445affa8df530576a41" +checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" dependencies = [ "num-bigint", "num-complex", @@ -2900,20 +3037,19 @@ dependencies = [ [[package]] name = "num-bigint" -version = "0.4.4" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ - "autocfg", "num-integer", "num-traits", ] [[package]] name = "num-complex" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23c6602fda94a57c990fe0df199a035d83576b496aa29f4e634a8ac6004e68a6" +checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" dependencies = [ "num-traits", ] @@ -2935,9 +3071,9 @@ dependencies = [ [[package]] name = "num-iter" -version = "0.1.44" +version = "0.1.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d869c01cc0c455284163fd0092f1f93835385ccab5a98a0dcc497b2f8bf055a9" +checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" dependencies = [ "autocfg", "num-integer", @@ -2946,11 +3082,10 @@ dependencies = [ [[package]] name = "num-rational" -version = "0.4.1" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0638a1c9d0a3c0914158145bc76cff373a75a627e6ecbfb71cbe6f453a5a19b0" +checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" dependencies = [ - "autocfg", "num-bigint", "num-integer", "num-traits", @@ -2958,9 +3093,9 @@ dependencies = [ [[package]] name = "num-traits" -version = "0.2.18" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da0df0e5185db44f69b44f26786fe401b6c293d1907744beaa7fa62b2e5a517a" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" dependencies = [ "autocfg", "libm", @@ -3008,33 +3143,34 @@ dependencies = [ [[package]] name = "object" -version = "0.32.2" +version = "0.36.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eb4c22c6154a1e759d7099f9ffad7cc5ef8245f9efbab4a41b92623079c82f3" +checksum = "3cfccb68961a56facde1163f9319e0d15743352344e7808a11795fb99698dcaf" dependencies = [ "async-trait", - "base64 0.22.0", + "base64 0.22.1", "bytes", "chrono", "futures", + "httparse", "humantime", - "hyper 1.4.1", + "hyper 1.5.2", "itertools 0.13.0", "md-5", "parking_lot", "percent-encoding", "quick-xml", "rand", - "reqwest 0.12.8", + "reqwest 0.12.10", "ring", "serde", "serde_json", @@ -3047,9 +3183,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.19.0" +version = "1.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" +checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" [[package]] name = "openid" @@ -3057,15 +3193,15 @@ version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "627898ab5b3fff5e5f1dc0e404bafdbb87a4337d815e86149f53640380946ccc" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "biscuit", "chrono", "lazy_static", "mime", - "reqwest 0.12.8", + "reqwest 0.12.10", "serde", "serde_json", - "thiserror 1.0.64", + "thiserror 1.0.69", "url", "validator", ] @@ -3115,15 +3251,15 @@ checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" [[package]] name = "parking" -version = "2.2.0" +version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae" +checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" -version = "0.12.1" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" dependencies = [ "lock_api", "parking_lot_core", @@ -3131,22 +3267,22 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.9" +version = "0.9.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" +checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" dependencies = [ "cfg-if", "libc", "redox_syscall", "smallvec", - "windows-targets 0.48.5", + "windows-targets 0.52.6", ] [[package]] name = "parquet" -version = "53.0.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0fbf928021131daaa57d334ca8e3904fe9ae22f73c56244fc7db9b04eedc3d8" +checksum = "2b449890367085eb65d7d3321540abc3d7babbd179ce31df0016e90719114191" dependencies = [ "ahash", "arrow-array", @@ -3156,14 +3292,14 @@ dependencies = [ "arrow-ipc", "arrow-schema", "arrow-select", - "base64 0.22.0", - "brotli", + "base64 0.22.1", + "brotli 7.0.0", "bytes", "chrono", "flate2", "futures", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.2", "lz4_flex", "num", "num-bigint", @@ -3180,9 +3316,9 @@ dependencies = [ [[package]] name = "parse-zoneinfo" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c705f256449c60da65e11ff6626e0c16a0a0b96aaa348de61376b249bc340f41" +checksum = "1f2a05b18d44e2957b88f96ba460715e295bc1d7510468a2f3d3b44535d26c24" dependencies = [ "regex", ] @@ -3207,10 +3343,10 @@ dependencies = [ "arrow-select", "async-trait", "backon", - "base64 0.22.0", + "base64 0.22.1", "byteorder", "bytes", - "bzip2", + "bzip2 0.5.0", "cargo_toml", "chrono", "chrono-humanize", @@ -3253,7 +3389,7 @@ dependencies = [ "reqwest 0.11.27", "rstest", "rustls 0.22.4", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.2.0", "semver", "serde", "serde_json", @@ -3269,7 +3405,7 @@ dependencies = [ "tokio-util", "tonic", "tonic-web", - "tower-http 0.6.1", + "tower-http 0.6.2", "tracing", "tracing-subscriber", "ulid", @@ -3328,12 +3464,12 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "petgraph" -version = "0.6.4" +version = "0.6.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" +checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset", - "indexmap 2.5.0", + "indexmap 2.7.0", ] [[package]] @@ -3376,18 +3512,18 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.5" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bf43b791c5b9e34c3d182969b4abb522f9343702850a2e57f460d00d09b4b3" +checksum = "be57f64e946e500c8ee36ef6331845d40a93055567ec57e8fae13efd33759b95" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.5" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" +checksum = "3c0f5fad0874fc7abcd4d750e76917eaebbecaa2c20bde22e1dbeeba8beb758c" dependencies = [ "proc-macro2", "quote", @@ -3396,9 +3532,9 @@ dependencies = [ [[package]] name = "pin-project-lite" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" +checksum = "915a1e146535de9163f3987b8944ed8cf49a18bb0056bcebcdcece385cece4ff" [[package]] name = "pin-utils" @@ -3408,9 +3544,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkg-config" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" +checksum = "953ec861398dccce10c670dfeaf3ec4911ca479e9c02154b3a215178c5f566f2" [[package]] name = "powerfmt" @@ -3420,15 +3556,18 @@ checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" [[package]] name = "ppv-lite86" -version = "0.2.17" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +dependencies = [ + "zerocopy", +] [[package]] name = "prettyplease" -version = "0.2.19" +version = "0.2.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ac2cf0f2e4f42b49f5ffd07dae8d746508ef7526c13940e5f524012ae6c6550" +checksum = "64d1ec885c64d0457d564db4ec299b2dae3f9c02808b8ad9c3a089c591b18033" dependencies = [ "proc-macro2", "syn 2.0.91", @@ -3478,22 +3617,32 @@ dependencies = [ [[package]] name = "procfs" -version = "0.14.2" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1de8dacb0873f77e6aefc6d71e044761fcc68060290f5b1089fcdf84626bb69" +checksum = "731e0d9356b0c25f16f33b5be79b1c57b562f141ebfcdb0ad8ac2c13a24293b4" dependencies = [ - "bitflags 1.3.2", - "byteorder", + "bitflags 2.6.0", "hex", "lazy_static", - "rustix 0.36.17", + "procfs-core", + "rustix", +] + +[[package]] +name = "procfs-core" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d3554923a69f4ce04c4a754260c338f505ce22642d3830e049a399fc2059a29" +dependencies = [ + "bitflags 2.6.0", + "hex", ] [[package]] name = "prometheus" -version = "0.13.3" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "449811d15fbdf5ceb5c1144416066429cf82316e2ec8ce0c1f6f8a02e7bbcf8c" +checksum = "3d33c28a30771f7f96db69893f78b857f7450d7e0237e9c8fc6427a81bae7ed1" dependencies = [ "cfg-if", "fnv", @@ -3503,7 +3652,7 @@ dependencies = [ "parking_lot", "procfs", "protobuf", - "thiserror 1.0.64", + "thiserror 1.0.69", ] [[package]] @@ -3520,9 +3669,9 @@ dependencies = [ [[package]] name = "prost" -version = "0.13.3" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b0487d90e047de87f984913713b85c601c05609aad5b0df4b4573fbf69aa13f" +checksum = "2c0fef6c4230e4ccf618a35c59d7ede15dea37de8427500f50aff708806e42ec" dependencies = [ "bytes", "prost-derive", @@ -3530,12 +3679,11 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.13.3" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c1318b19085f08681016926435853bbf7858f9c082d0999b80550ff5d9abe15" +checksum = "d0f3e5beed80eb580c68e2c600937ac2c4eedabdfd5ef1e5b7ea4f3fba84497b" dependencies = [ - "bytes", - "heck 0.5.0", + "heck", "itertools 0.13.0", "log", "multimap", @@ -3551,9 +3699,9 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.13.3" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9552f850d5f0964a4e4d0bf306459ac29323ddfbae05e35a7c0d35cb0803cc5" +checksum = "157c5a9d7ea5c2ed2d9fb8f495b64759f7816c7eaea54ba3978f0d63000162e3" dependencies = [ "anyhow", "itertools 0.13.0", @@ -3564,9 +3712,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.13.3" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4759aa0d3a6232fb8dbdb97b61de2c20047c68aca932c7ed76da9d788508d670" +checksum = "cc2f1e56baa61e93533aebc21af4d2134b70f66275e0fcdf3cbe43d77ff7e8fc" dependencies = [ "prost", ] @@ -3595,9 +3743,9 @@ dependencies = [ [[package]] name = "quick-xml" -version = "0.36.1" +version = "0.37.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96a05e2e8efddfa51a84ca47cec303fac86c8541b686d37cac5efc0e094417bc" +checksum = "f22f29bdff3987b4d8632ef95fd6424ec7e4e0a57e2f4fc63e489e75357f6a03" dependencies = [ "memchr", "serde", @@ -3605,57 +3753,61 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.5" +version = "0.11.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c7c5fdde3cdae7203427dc4f0a68fe0ed09833edc525a03456b153b79828684" +checksum = "62e96808277ec6f97351a2380e6c25114bc9e67037775464979f3037c92d05ef" dependencies = [ "bytes", "pin-project-lite", "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.13", + "rustls 0.23.20", "socket2", - "thiserror 1.0.64", + "thiserror 2.0.9", "tokio", "tracing", ] [[package]] name = "quinn-proto" -version = "0.11.8" +version = "0.11.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fadfaed2cd7f389d0161bb73eeb07b7b78f8691047a6f3e73caaeae55310a4a6" +checksum = "a2fe5ef3495d7d2e377ff17b1a8ce2ee2ec2a18cde8b6ad6619d65d0701c135d" dependencies = [ "bytes", + "getrandom", "rand", "ring", "rustc-hash", - "rustls 0.23.13", + "rustls 0.23.20", + "rustls-pki-types", "slab", - "thiserror 1.0.64", + "thiserror 2.0.9", "tinyvec", "tracing", + "web-time", ] [[package]] name = "quinn-udp" -version = "0.5.4" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bffec3605b73c6f1754535084a85229fa8a30f86014e6c81aeec4abb68b0285" +checksum = "1c40286217b4ba3a71d644d752e6a0b71f13f1b6a2c5311acfcbe0c2418ed904" dependencies = [ + "cfg_aliases", "libc", "once_cell", "socket2", "tracing", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "quote" -version = "1.0.37" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" +checksum = "0e4dccaaaf89514f546c693ddc140f729f958c247918a13380cccc6078391acc" dependencies = [ "proc-macro2", ] @@ -3755,11 +3907,11 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.4.1" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" dependencies = [ - "bitflags 1.3.2", + "bitflags 2.6.0", ] [[package]] @@ -3770,7 +3922,7 @@ checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.8", + "regex-automata 0.4.9", "regex-syntax 0.8.5", ] @@ -3785,9 +3937,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.8" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "368758f23274712b504848e9d5a6f010445cc8b87a7cdb4d7cbee666c1288da3" +checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" dependencies = [ "aho-corasick", "memchr", @@ -3835,7 +3987,7 @@ dependencies = [ "h2 0.3.26", "http 0.2.12", "http-body 0.4.6", - "hyper 0.14.30", + "hyper 0.14.32", "hyper-rustls 0.24.2", "ipnet", "js-sys", @@ -3864,20 +4016,20 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.8" +version = "0.12.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f713147fbe92361e52392c73b8c9e48c04c6625bce969ef54dc901e58e042a7b" +checksum = "3d3536321cfc54baa8cf3e273d5e1f63f889067829c4b410fcdbac8ca7b80994" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "bytes", "futures-core", "futures-util", - "h2 0.4.5", - "http 1.1.0", - "http-body 1.0.0", + "h2 0.4.7", + "http 1.2.0", + "http-body 1.0.1", "http-body-util", - "hyper 1.4.1", - "hyper-rustls 0.27.3", + "hyper 1.5.2", + "hyper-rustls 0.27.5", "hyper-util", "ipnet", "js-sys", @@ -3887,24 +4039,25 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.13", + "rustls 0.23.20", "rustls-native-certs", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.2.0", "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", - "sync_wrapper 1.0.1", + "sync_wrapper 1.0.2", "tokio", - "tokio-rustls 0.26.0", + "tokio-rustls 0.26.1", "tokio-util", + "tower 0.5.2", "tower-service", "url", "wasm-bindgen", "wasm-bindgen-futures", "wasm-streams", "web-sys", - "webpki-roots 0.26.1", + "webpki-roots 0.26.7", "windows-registry", ] @@ -3955,15 +4108,15 @@ dependencies = [ [[package]] name = "rustc-demangle" -version = "0.1.23" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" +checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" [[package]] name = "rustc-hash" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "583034fd73374156e66797ed8e5b0d5690409c9226b22d87cb7f19821c05d152" +checksum = "c7fb8039b3032c191086b10f11f319a6e99e1e82889c5cc6046f515c9db1d497" [[package]] name = "rustc_version" @@ -3976,29 +4129,15 @@ dependencies = [ [[package]] name = "rustix" -version = "0.36.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "305efbd14fde4139eb501df5f136994bb520b033fa9fbdce287507dc23b8c7ed" -dependencies = [ - "bitflags 1.3.2", - "errno", - "io-lifetimes", - "libc", - "linux-raw-sys 0.1.4", - "windows-sys 0.45.0", -] - -[[package]] -name = "rustix" -version = "0.38.34" +version = "0.38.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" +checksum = "f93dc38ecbab2eb790ff964bb77fa94faf256fd3e73285fd7ba0903b76bedb85" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "errno", "libc", - "linux-raw-sys 0.4.13", - "windows-sys 0.52.0", + "linux-raw-sys", + "windows-sys 0.59.0", ] [[package]] @@ -4029,9 +4168,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.13" +version = "0.23.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2dabaac7466917e566adb06783a81ca48944c6898a1b08b9374106dd671f4c8" +checksum = "5065c3f250cbd332cd894be57c40fa52387247659b14a2d6041d121547903b1b" dependencies = [ "log", "once_cell", @@ -4044,12 +4183,11 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcaf18a4f2be7326cd874a5fa579fae794320a0f388d365dca7e480e55f83f8a" +checksum = "7fcff2dd52b58a8d98a70243663a0d234c4e2b79235637849d15913394a247d3" dependencies = [ "openssl-probe", - "rustls-pemfile 2.1.2", "rustls-pki-types", "schannel", "security-framework", @@ -4066,19 +4204,21 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "2.1.2" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29993a25686778eb88d4189742cd713c9bce943bc54251a33509dc63cbacf73d" +checksum = "dce314e5fee3f39953d46bb63bb8a46d40c2f8fb7cc5a3b6cab2bde9721d6e50" dependencies = [ - "base64 0.22.0", "rustls-pki-types", ] [[package]] name = "rustls-pki-types" -version = "1.9.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e696e35370c65c9c541198af4543ccd580cf17fc25d8e05c5a242b202488c55" +checksum = "d2bf47e6ff922db3825eb750c4e2ff784c6ff8fb9e13046ef6a1d1c5401b0b37" +dependencies = [ + "web-time", +] [[package]] name = "rustls-webpki" @@ -4103,15 +4243,15 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.15" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80af6f9131f277a45a3fba6ce8e2258037bb0477a67e610d3c1fe046ab31de47" +checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" [[package]] name = "ryu" -version = "1.0.17" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e86697c916019a8588c99b5fac3cead74ec0b4b819707a682fd4d23fa0ce1ba1" +checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" [[package]] name = "same-file" @@ -4136,11 +4276,11 @@ dependencies = [ [[package]] name = "schannel" -version = "0.1.23" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbc91545643bcf3a0bbb6569265615222618bdf33ce4ffbbd13c4bbd4c093534" +checksum = "1f29ebaa345f945cec9fbbc532eb307f0fdad8161f281b6369539c8d84876b3d" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -4161,12 +4301,12 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.10.0" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "770452e37cad93e0a50d5abc3990d2bc351c36d0328f86cefec2f2fb206eaef6" +checksum = "81d3f8c9bfcc3cbb6b0179eb57042d75b1582bdc65c3cb95f3fa999509c03cbc" dependencies = [ - "bitflags 1.3.2", - "core-foundation", + "bitflags 2.6.0", + "core-foundation 0.10.0", "core-foundation-sys", "libc", "security-framework-sys", @@ -4174,9 +4314,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.10.0" +version = "2.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41f3cc463c0ef97e11c3461a9d3787412d30e8e7eb907c79180c4a57bf7c04ef" +checksum = "1863fd3768cd83c56a7f60faa4dc0d403f1b6df0a38c3c25f44b7894e45370d5" dependencies = [ "core-foundation-sys", "libc", @@ -4184,9 +4324,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.22" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92d43fe69e652f3df9bdc2b85b2854a0825b86e4fb76bc44d945137d053639ca" +checksum = "3cb6eb87a131f756572d7fb904f6e7b68633f09cca868c5df1c4b8d1a694bbba" dependencies = [ "serde", ] @@ -4199,18 +4339,18 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.198" +version = "1.0.216" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9846a40c979031340571da2545a4e5b7c4163bdae79b301d5f86d03979451fcc" +checksum = "0b9781016e935a97e8beecf0c933758c97a5520d32930e460142b4cd80c6338e" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.198" +version = "1.0.216" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88edab869b01783ba905e7d0153f9fc1a6505a96e4ad3018011eedb838566d9" +checksum = "46f859dbbf73865c6627ed570e78961cd3ac92407a2d117204c49232485da55e" dependencies = [ "proc-macro2", "quote", @@ -4219,12 +4359,13 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.116" +version = "1.0.134" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e17db7126d17feb94eb3fad46bf1a96b034e8aacbc2e775fe81505f8b0b2813" +checksum = "d00f4175c42ee48b15416f6193a959ba3a0d67fc699a0db9ad12df9f83991c7d" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.7.0", "itoa", + "memchr", "ryu", "serde", ] @@ -4274,9 +4415,9 @@ dependencies = [ [[package]] name = "sha1_smol" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae1a47186c03a32177042e55dbc5fd5aee900b8e0069a8d70fba96a9375cd012" +checksum = "bbfa15b3dddfee50a0fff136974b3e1bde555604ba463834a7eb7deb6417705d" [[package]] name = "sha2" @@ -4331,7 +4472,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "34db1a06d485c9142248b7a054f034b349b212551f3dfd19c94d45a754a217cd" dependencies = [ "libc", - "mio 1.0.2", + "mio", "signal-hook", ] @@ -4386,7 +4527,7 @@ version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "03c3c6b7927ffe7ecaa769ee0e3994da3b8cafc8f444578982c83ecb161af917" dependencies = [ - "heck 0.5.0", + "heck", "proc-macro2", "quote", "syn 2.0.91", @@ -4400,9 +4541,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "socket2" -version = "0.5.6" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05ffd9c0a93b7543e062e759284fcf5f5e3b098501104bfbdde4d404db792871" +checksum = "c970269d99b64e60ec3bd6ad27270092a5394c4e309314b18ae3fe575695fbe8" dependencies = [ "libc", "windows-sys 0.52.0", @@ -4435,11 +4576,17 @@ dependencies = [ "syn 2.0.91", ] +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + [[package]] name = "static-files" -version = "0.2.3" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64712ea1e3e140010e1d9605872ba205afa2ab5bd38191cc6ebd248ae1f6a06b" +checksum = "4e8590e848e1c53be9258210bcd4a8f4118e08988f03a4e2d63b62e4ad9f7ced" dependencies = [ "change-detection", "mime_guess", @@ -4454,26 +4601,26 @@ checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" [[package]] name = "strsim" -version = "0.10.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" [[package]] name = "strum" -version = "0.26.2" +version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" dependencies = [ "strum_macros", ] [[package]] name = "strum_macros" -version = "0.26.2" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6cf59daf282c0a494ba14fd21610a0325f9f90ec9d1231dea26bcb1d696c946" +checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" dependencies = [ - "heck 0.4.1", + "heck", "proc-macro2", "quote", "rustversion", @@ -4482,9 +4629,9 @@ dependencies = [ [[package]] name = "subtle" -version = "2.5.0" +version = "2.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "syn" @@ -4515,13 +4662,24 @@ checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" [[package]] name = "sync_wrapper" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" +checksum = "0bf256ce5efdfa370213c1dabab5935a12e49f2c58d15e9eac2870d3b4f27263" dependencies = [ "futures-core", ] +[[package]] +name = "synstructure" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.91", +] + [[package]] name = "sysinfo" version = "0.33.0" @@ -4543,7 +4701,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ba3a3adc5c275d719af8cb4272ea1c4a6d668a777f37e115f6d11ddbc1c8e0e7" dependencies = [ "bitflags 1.3.2", - "core-foundation", + "core-foundation 0.9.4", "system-configuration-sys", ] @@ -4559,23 +4717,24 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.10.1" +version = "3.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" +checksum = "28cce251fcbc87fac86a866eeb0d6c2d536fc16d06f184bb61aeae11aa4cee0c" dependencies = [ "cfg-if", - "fastrand 2.0.2", - "rustix 0.38.34", - "windows-sys 0.52.0", + "fastrand 2.3.0", + "once_cell", + "rustix", + "windows-sys 0.59.0", ] [[package]] name = "thiserror" -version = "1.0.64" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d50af8abc119fb8bb6dbabcfa89656f46f84aa0ac7688088608076ad2b459a84" +checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" dependencies = [ - "thiserror-impl 1.0.64", + "thiserror-impl 1.0.69", ] [[package]] @@ -4589,9 +4748,9 @@ dependencies = [ [[package]] name = "thiserror-impl" -version = "1.0.64" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08904e7672f5eb876eaaf87e0ce17857500934f4981c4a0ab2b4aa98baac7fc3" +checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", @@ -4611,11 +4770,11 @@ dependencies = [ [[package]] name = "thread-priority" -version = "1.0.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "599e8e829c2314b750ecade9309ecc6cf9a48c2e62fe25680b6c1d2172463ca3" +checksum = "cfe075d7053dae61ac5413a34ea7d4913b6e6207844fd726bdd858b37ff72bf5" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "cfg-if", "libc", "log", @@ -4646,9 +4805,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.36" +version = "0.3.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5dfd88e563464686c916c7e46e623e520ddc6d79fa6641390f2e3fa86e83e885" +checksum = "35e7868883861bd0e56d9ac6efcaaca0d6d5d82a2a7ec8209ff492c07cf37b21" dependencies = [ "deranged", "itoa", @@ -4669,9 +4828,9 @@ checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.18" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f252a68540fde3a3877aeea552b832b40ab9a69e318efd078774a01ddee1ccf" +checksum = "2834e6017e3e5e4b9834939793b282bc03b37a3336245fa820e35e233e2a85de" dependencies = [ "num-conv", "time-core", @@ -4686,11 +4845,21 @@ dependencies = [ "crunchy", ] +[[package]] +name = "tinystr" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9117f5d4db391c1cf6927e7bea3db74b9a1c1add8f7eda9ffd5364f40f57b82f" +dependencies = [ + "displaydoc", + "zerovec", +] + [[package]] name = "tinyvec" -version = "1.6.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +checksum = "022db8904dfa342efe721985167e9fcd16c29b226db4397ed752a761cfce81e8" dependencies = [ "tinyvec_macros", ] @@ -4710,7 +4879,7 @@ dependencies = [ "backtrace", "bytes", "libc", - "mio 1.0.2", + "mio", "parking_lot", "pin-project-lite", "signal-hook-registry", @@ -4753,20 +4922,19 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.0" +version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" +checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" dependencies = [ - "rustls 0.23.13", - "rustls-pki-types", + "rustls 0.23.20", "tokio", ] [[package]] name = "tokio-stream" -version = "0.1.16" +version = "0.1.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f4e6ce100d0eb49a2734f8c0812bcd324cf357d21810932c5df6b96ef2b86f1" +checksum = "eca58d7bba4a75707817a2c44174253f9236b2d5fbd055602e9d5c07c139a047" dependencies = [ "futures-core", "pin-project-lite", @@ -4775,23 +4943,22 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.10" +version = "0.7.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" +checksum = "d7fcaa8d55a2bdd6b83ace262b016eca0d79ee02818c5c1bcdf0305114081078" dependencies = [ "bytes", "futures-core", "futures-sink", "pin-project-lite", "tokio", - "tracing", ] [[package]] name = "toml" -version = "0.8.12" +version = "0.8.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9dd1545e8208b4a5af1aa9bbd0b4cf7e9ea08fabc5d0a5c67fcaafa17433aa3" +checksum = "a1ed1f98e3fdc28d6d910e6737ae6ab1a93bf1985935a1193e68f93eeb68d24e" dependencies = [ "serde", "serde_spanned", @@ -4814,7 +4981,7 @@ version = "0.22.22" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.7.0", "serde", "serde_spanned", "toml_datetime", @@ -4830,25 +4997,25 @@ dependencies = [ "async-stream", "async-trait", "axum", - "base64 0.22.0", + "base64 0.22.1", "bytes", "flate2", - "h2 0.4.5", - "http 1.1.0", - "http-body 1.0.0", + "h2 0.4.7", + "http 1.2.0", + "http-body 1.0.1", "http-body-util", - "hyper 1.4.1", + "hyper 1.5.2", "hyper-timeout", "hyper-util", "percent-encoding", "pin-project", "prost", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.2.0", "socket2", "tokio", - "tokio-rustls 0.26.0", + "tokio-rustls 0.26.1", "tokio-stream", - "tower", + "tower 0.4.13", "tower-layer", "tower-service", "tracing", @@ -4861,10 +5028,10 @@ version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5299dd20801ad736dccb4a5ea0da7376e59cd98f213bf1c3d478cf53f4834b58" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "bytes", - "http 1.1.0", - "http-body 1.0.0", + "http 1.2.0", + "http-body 1.0.1", "http-body-util", "pin-project", "tokio-stream", @@ -4895,16 +5062,31 @@ dependencies = [ "tracing", ] +[[package]] +name = "tower" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d039ad9159c98b70ecfd540b2573b97f7f52c3e8d9f8ad57a24b916a536975f9" +dependencies = [ + "futures-core", + "futures-util", + "pin-project-lite", + "sync_wrapper 1.0.2", + "tokio", + "tower-layer", + "tower-service", +] + [[package]] name = "tower-http" version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e9cd434a998747dd2c4276bc96ee2e0c7a2eadf3cae88e52be55a05fa9053f5" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "bytes", - "http 1.1.0", - "http-body 1.0.0", + "http 1.2.0", + "http-body 1.0.1", "http-body-util", "pin-project-lite", "tower-layer", @@ -4913,13 +5095,13 @@ dependencies = [ [[package]] name = "tower-http" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8437150ab6bbc8c5f0f519e3d5ed4aa883a83dd4cdd3d1b21f9482936046cb97" +checksum = "403fa3b783d4b626a8ad51d766ab03cb6d2dbfc46b1c5d4448395e6628dc9697" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "bytes", - "http 1.1.0", + "http 1.2.0", "pin-project-lite", "tower-layer", "tower-service", @@ -4933,9 +5115,9 @@ checksum = "121c2a6cda46980bb0fcd1647ffaf6cd3fc79a013de288782836f6df9c48780e" [[package]] name = "tower-service" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" +checksum = "8df9b6e13f2d32c91b9bd719c00d1958837bc7dec474d94952798cc8e69eeec3" [[package]] name = "tracing" @@ -4983,9 +5165,9 @@ dependencies = [ [[package]] name = "tracing-subscriber" -version = "0.3.18" +version = "0.3.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad0f048c97dbd9faa9b7df56362b8ebcaa52adb06b498c050d2f4e32f90a7a8b" +checksum = "e8189decb5ac0fa7bc8b96b7cb9b2701d60d48805aca84a238004d665fcc4008" dependencies = [ "matchers", "nu-ansi-term", @@ -5024,9 +5206,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "ulid" -version = "1.1.2" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34778c17965aa2a08913b57e1f34db9b4a63f5de31768b55bf20d2795f921259" +checksum = "04f903f293d11f31c0c29e4148f6dc0d033a7f80cebc0282bea147611667d289" dependencies = [ "getrandom", "rand", @@ -5036,45 +5218,42 @@ dependencies = [ [[package]] name = "unicase" -version = "2.7.0" +version = "2.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7d2d4dafb69621809a81864c9c1b864479e1235c0dd4e199924b9742439ed89" -dependencies = [ - "version_check", -] +checksum = "75b844d17643ee918803943289730bec8aac480150456169e647ed0b576ba539" [[package]] name = "unicode-bidi" -version = "0.3.15" +version = "0.3.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" +checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.13" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e91b56cd4cadaeb79bbf1a5645f6b4f8dc5bde8834ad5894a8db35fda9efa1fe" +checksum = "adb9e6ca4f869e1180728b7950e35922a7fc6397f7b641499e8f3ef06e50dc83" [[package]] name = "unicode-normalization" -version = "0.1.23" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" +checksum = "5033c97c4262335cded6d6fc3e5c18ab755e1a3dc96376350f3d8e9f009ad956" dependencies = [ "tinyvec", ] [[package]] name = "unicode-segmentation" -version = "1.11.0" +version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" [[package]] name = "unicode-width" -version = "0.1.11" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51733f11c9c4f72aa0c160008246859e340b00807569a0da0e7a1079b27ba85" +checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" [[package]] name = "untrusted" @@ -5084,49 +5263,60 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "uptime_lib" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4e71ddbefed856d5881821d6ada4e606bbb91fd332296963ed596e2ad2100f3" +checksum = "9e64b558561f12a171bbea5325c3f24f129db371adee1d7ae93b6e310bd69192" dependencies = [ "libc", - "thiserror 1.0.64", - "windows 0.52.0", + "thiserror 1.0.69", + "windows 0.57.0", ] [[package]] name = "ureq" -version = "2.9.6" +version = "2.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11f214ce18d8b2cbe84ed3aa6486ed3f5b285cf8d8fbdbce9f3f767a724adc35" +checksum = "02d1a66277ed75f640d608235660df48c8e3c19f3b4edb6a263315626cc3c01d" dependencies = [ - "base64 0.21.7", + "base64 0.22.1", "flate2", "log", "once_cell", - "rustls 0.22.4", + "rustls 0.23.20", "rustls-pki-types", - "rustls-webpki 0.102.8", "url", - "webpki-roots 0.26.1", + "webpki-roots 0.26.7", ] [[package]] name = "url" -version = "2.5.0" +version = "2.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" +checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" dependencies = [ "form_urlencoded", - "idna", + "idna 1.0.3", "percent-encoding", "serde", ] +[[package]] +name = "utf16_iter" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8232dd3cdaed5356e0f716d285e4b40b932ac434100fe9b7e0e8e935b9e6246" + +[[package]] +name = "utf8_iter" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" + [[package]] name = "utf8parse" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" @@ -5143,7 +5333,7 @@ version = "0.18.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "db79c75af171630a3148bd3e6d7c4f42b6a9a014c2945bc5ed0020cbb8d9478e" dependencies = [ - "idna", + "idna 0.5.0", "once_cell", "regex", "serde", @@ -5155,9 +5345,9 @@ dependencies = [ [[package]] name = "validator_derive" -version = "0.18.1" +version = "0.18.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55591299b7007f551ed1eb79a684af7672c19c3193fb9e0a31936987bb2438ec" +checksum = "df0bcf92720c40105ac4b2dda2a4ea3aa717d4d6a862cc217da653a4bd5c6b10" dependencies = [ "darling", "once_cell", @@ -5181,9 +5371,9 @@ checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" [[package]] name = "vergen" -version = "8.3.1" +version = "8.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e27d6bdd219887a9eadd19e1c34f32e47fa332301184935c6d9bca26f3cca525" +checksum = "2990d9ea5967266ea0ccf413a4aa5c42a93dbcfda9cb49a97de6931726b12566" dependencies = [ "anyhow", "cargo_metadata", @@ -5195,15 +5385,15 @@ dependencies = [ [[package]] name = "version_check" -version = "0.9.4" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" [[package]] name = "waker-fn" -version = "1.1.1" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3c4517f54858c779bbcbf228f4fca63d121bf85fbecb2dc578cdf4a39395690" +checksum = "317211a0dc0ceedd78fb2ca9a44aed3d7b9b26f81870d485c07122b4350673b7" [[package]] name = "walkdir" @@ -5238,23 +5428,23 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.92" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" +checksum = "a474f6281d1d70c17ae7aa6a613c87fce69a127e2624002df63dcb39d6cf6396" dependencies = [ "cfg-if", + "once_cell", "wasm-bindgen-macro", ] [[package]] name = "wasm-bindgen-backend" -version = "0.2.92" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" +checksum = "5f89bb38646b4f81674e8f5c3fb81b562be1fd936d84320f3264486418519c79" dependencies = [ "bumpalo", "log", - "once_cell", "proc-macro2", "quote", "syn 2.0.91", @@ -5263,21 +5453,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.42" +version = "0.4.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76bc14366121efc8dbb487ab05bcc9d346b3b5ec0eaa76e46594cabbe51762c0" +checksum = "38176d9b44ea84e9184eff0bc34cc167ed044f816accfe5922e54d84cf48eca2" dependencies = [ "cfg-if", "js-sys", + "once_cell", "wasm-bindgen", "web-sys", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.92" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" +checksum = "2cc6181fd9a7492eef6fef1f33961e3695e4579b9872a6f7c83aee556666d4fe" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -5285,9 +5476,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.92" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" +checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", @@ -5298,15 +5489,15 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.92" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" +checksum = "943aab3fdaaa029a6e0271b35ea10b72b943135afe9bffca82384098ad0e06a6" [[package]] name = "wasm-streams" -version = "0.4.0" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b65dc4c90b63b118468cf747d8bf3566c1913ef60be765b5730ead9e0a3ba129" +checksum = "15053d8d85c7eccdbefef60f06769760a563c7f0a9d6902a13d35c7800b0ad65" dependencies = [ "futures-util", "js-sys", @@ -5317,9 +5508,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.69" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" +checksum = "04dd7223427d52553d3702c004d3b2fe07c148165faa56313cb00211e31c12bc" dependencies = [ "js-sys", "wasm-bindgen", @@ -5343,9 +5534,9 @@ checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1" [[package]] name = "webpki-roots" -version = "0.26.1" +version = "0.26.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3de34ae270483955a94f4b21bdaaeb83d508bb84a01435f393818edb0012009" +checksum = "5d642ff16b7e79272ae451b7322067cdc17cadf68c23264be9d94a32319efe7e" dependencies = [ "rustls-pki-types", ] @@ -5368,11 +5559,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.7" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "134306a13c5647ad6453e8deaec55d3a44d6021970129e6188735e74bf546697" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -5483,15 +5674,6 @@ dependencies = [ "windows-targets 0.52.6", ] -[[package]] -name = "windows-sys" -version = "0.45.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" -dependencies = [ - "windows-targets 0.42.2", -] - [[package]] name = "windows-sys" version = "0.48.0" @@ -5519,21 +5701,6 @@ dependencies = [ "windows-targets 0.52.6", ] -[[package]] -name = "windows-targets" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" -dependencies = [ - "windows_aarch64_gnullvm 0.42.2", - "windows_aarch64_msvc 0.42.2", - "windows_i686_gnu 0.42.2", - "windows_i686_msvc 0.42.2", - "windows_x86_64_gnu 0.42.2", - "windows_x86_64_gnullvm 0.42.2", - "windows_x86_64_msvc 0.42.2", -] - [[package]] name = "windows-targets" version = "0.48.5" @@ -5565,12 +5732,6 @@ dependencies = [ "windows_x86_64_msvc 0.52.6", ] -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" - [[package]] name = "windows_aarch64_gnullvm" version = "0.48.5" @@ -5583,12 +5744,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" -[[package]] -name = "windows_aarch64_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" - [[package]] name = "windows_aarch64_msvc" version = "0.48.5" @@ -5601,12 +5756,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" -[[package]] -name = "windows_i686_gnu" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" - [[package]] name = "windows_i686_gnu" version = "0.48.5" @@ -5625,12 +5774,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" -[[package]] -name = "windows_i686_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" - [[package]] name = "windows_i686_msvc" version = "0.48.5" @@ -5643,12 +5786,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" -[[package]] -name = "windows_x86_64_gnu" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" - [[package]] name = "windows_x86_64_gnu" version = "0.48.5" @@ -5661,12 +5798,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" -[[package]] -name = "windows_x86_64_gnullvm" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" - [[package]] name = "windows_x86_64_gnullvm" version = "0.48.5" @@ -5679,12 +5810,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" -[[package]] -name = "windows_x86_64_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" - [[package]] name = "windows_x86_64_msvc" version = "0.48.5" @@ -5716,11 +5841,23 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "write16" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" + +[[package]] +name = "writeable" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" + [[package]] name = "xxhash-rust" -version = "0.8.10" +version = "0.8.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "927da81e25be1e1a2901d59b81b37dd2efd1fc9c9345a55007f09bf5a2d3ee03" +checksum = "d7d48f1b18be023c95e7b75f481cac649d74be7c507ff4a407c55cfb957f7934" [[package]] name = "xz2" @@ -5731,46 +5868,114 @@ dependencies = [ "lzma-sys", ] +[[package]] +name = "yoke" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "120e6aef9aa629e3d4f52dc8cc43a015c7724194c97dfaf45180d2daf2b77f40" +dependencies = [ + "serde", + "stable_deref_trait", + "yoke-derive", + "zerofrom", +] + +[[package]] +name = "yoke-derive" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.91", + "synstructure", +] + [[package]] name = "zerocopy" -version = "0.7.32" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" dependencies = [ + "byteorder", "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.32" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.91", +] + +[[package]] +name = "zerofrom" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" +checksum = "cff3ee08c995dee1859d998dea82f7374f2826091dd9cd47def953cae446cd2e" +dependencies = [ + "zerofrom-derive", +] + +[[package]] +name = "zerofrom-derive" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", "syn 2.0.91", + "synstructure", ] [[package]] name = "zeroize" -version = "1.7.0" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" + +[[package]] +name = "zerovec" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa2b893d79df23bfb12d5461018d408ea19dfafe76c2c7ef6d4eba614f8ff079" +dependencies = [ + "yoke", + "zerofrom", + "zerovec-derive", +] + +[[package]] +name = "zerovec-derive" +version = "0.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d" +checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.91", +] [[package]] name = "zip" -version = "2.2.0" +version = "2.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc5e4288ea4057ae23afc69a4472434a87a2495cafce6632fd1c4ec9f5cf3494" +checksum = "ae9c1ea7b3a5e1f4b922ff856a129881167511563dc219869afe3787fc0c1a45" dependencies = [ "arbitrary", "crc32fast", "crossbeam-utils", "displaydoc", "flate2", - "indexmap 2.5.0", + "indexmap 2.7.0", "memchr", - "thiserror 1.0.64", + "thiserror 2.0.9", "zopfli", ] @@ -5790,27 +5995,27 @@ dependencies = [ [[package]] name = "zstd" -version = "0.13.1" +version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d789b1514203a1120ad2429eae43a7bd32b90976a7bb8a05f7ec02fa88cc23a" +checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" dependencies = [ "zstd-safe", ] [[package]] name = "zstd-safe" -version = "7.1.0" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd99b45c6bc03a018c8b8a86025678c87e55526064e38f9df301989dce7ec0a" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" dependencies = [ "zstd-sys", ] [[package]] name = "zstd-sys" -version = "2.0.10+zstd.1.5.6" +version = "2.0.13+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" +checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" dependencies = [ "cc", "pkg-config", From 73a86593c0f906fdd0090723634ecd716cbb2dbc Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 26 Dec 2024 23:07:59 +0100 Subject: [PATCH 18/65] add resource limits for docker containers --- Dockerfile | 4 ++- docker-compose-distributed-test.yaml | 54 ++++++++++++++++++++++++---- docker-compose-test.yaml | 43 +++++++++++++++++++++- 3 files changed, 92 insertions(+), 9 deletions(-) diff --git a/Dockerfile b/Dockerfile index ba0657b81..361facc1e 100644 --- a/Dockerfile +++ b/Dockerfile @@ -14,7 +14,7 @@ # along with this program. If not, see . # build stage -FROM rust:1.77.1-bookworm as builder +FROM rust:1.83.0-bookworm as builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " @@ -24,8 +24,10 @@ LABEL org.opencontainers.image.licenses="AGPL-3.0" RUN apt-get update && \ apt-get install --no-install-recommends -y \ cmake \ + clang \ librdkafka-dev \ ca-certificates \ + build-essential \ libsasl2-dev \ libssl-dev && \ rm -rf /var/lib/apt/lists/* diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index 7db668690..97efb4a58 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -5,6 +5,12 @@ services: # minio minio: image: minio/minio:RELEASE.2023-02-10T18-48-39Z + deploy: + resources: + limits: + memory: 4G + reservations: + memory: 2G entrypoint: - sh - -euc @@ -55,6 +61,11 @@ services: depends_on: - minio deploy: + resources: + limits: + memory: 4G + reservations: + memory: 2G restart_policy: condition: on-failure delay: 20s @@ -98,6 +109,11 @@ services: - kafka-1 - kafka-2 deploy: + resources: + limits: + memory: 4G + reservations: + memory: 2G restart_policy: condition: on-failure delay: 20s @@ -137,6 +153,12 @@ services: kafka-0: image: docker.io/bitnami/kafka:3.9 + deploy: + resources: + limits: + memory: 2G + reservations: + memory: 1G ports: - "9092" environment: @@ -167,6 +189,12 @@ services: kafka-1: image: docker.io/bitnami/kafka:3.9 + deploy: + resources: + limits: + memory: 2G + reservations: + memory: 1G ports: - "9092" environment: @@ -197,6 +225,12 @@ services: kafka-2: image: docker.io/bitnami/kafka:3.9 + deploy: + resources: + limits: + memory: 2G + reservations: + memory: 1G ports: - "9092" environment: @@ -228,20 +262,20 @@ services: kafka-ui: platform: linux/amd64 image: provectuslabs/kafka-ui:latest - ports: - - "8080:8080" - depends_on: - - kafka-0 - - kafka-1 - - kafka-2 environment: KAFKA_CLUSTERS_0_NAME: dist-test KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 KAFKA_CLUSTERS_0_METRICS_PORT: 9101 DYNAMIC_CONFIG_ENABLED: "true" + JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC networks: - parseable-internal deploy: + resources: + limits: + memory: 512M + reservations: + memory: 256M restart_policy: condition: on-failure delay: 20s @@ -255,13 +289,19 @@ services: - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - KAFKA_TOPIC=dist-test-logs-stream - LOG_RATE=500 - - TOTAL_LOGS=100000 + - TOTAL_LOGS=50000 depends_on: - kafka-0 - kafka-1 - kafka-2 networks: - parseable-internal + deploy: + resources: + limits: + memory: 512M + reservations: + memory: 256M restart: "no" volumes: diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index f562727be..7005fc40c 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -4,6 +4,12 @@ networks: services: minio: image: minio/minio:RELEASE.2023-02-10T18-48-39Z + deploy: + resources: + limits: + memory: 4G + reservations: + memory: 2G entrypoint: - sh - -euc @@ -59,6 +65,11 @@ services: networks: - parseable-internal deploy: + resources: + limits: + memory: 4G + reservations: + memory: 3G restart_policy: condition: on-failure delay: 20s @@ -92,6 +103,12 @@ services: kafka-0: image: docker.io/bitnami/kafka:3.9 + deploy: + resources: + limits: + memory: 2G + reservations: + memory: 1G ports: - "9092" environment: @@ -122,6 +139,12 @@ services: kafka-1: image: docker.io/bitnami/kafka:3.9 + deploy: + resources: + limits: + memory: 2G + reservations: + memory: 1G ports: - "9092" environment: @@ -152,6 +175,12 @@ services: kafka-2: image: docker.io/bitnami/kafka:3.9 + deploy: + resources: + limits: + memory: 2G + reservations: + memory: 1G ports: - "9092" environment: @@ -194,9 +223,15 @@ services: KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 KAFKA_CLUSTERS_0_METRICS_PORT: 9101 DYNAMIC_CONFIG_ENABLED: "true" + JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC networks: - parseable-internal deploy: + resources: + limits: + memory: 512M + reservations: + memory: 256M restart_policy: condition: on-failure delay: 20s @@ -210,13 +245,19 @@ services: - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - KAFKA_TOPIC=test-logs-stream - LOG_RATE=500 - - TOTAL_LOGS=100000 + - TOTAL_LOGS=10000 depends_on: - kafka-0 - kafka-1 - kafka-2 networks: - parseable-internal + deploy: + resources: + limits: + memory: 512M + reservations: + memory: 256M restart: "no" volumes: From bb3b5cbf20801b3001094b646c97ad912e4e2d7c Mon Sep 17 00:00:00 2001 From: hippalus Date: Fri, 27 Dec 2024 19:09:53 +0100 Subject: [PATCH 19/65] scale down kafka broker to single node since OOM on integration test workflows --- docker-compose-distributed-test.yaml | 137 +++------------------ docker-compose-test.yaml | 126 ++----------------- scripts/kafka_log_stream_generator.py | 2 +- src/connectors/kafka/consumer.rs | 7 +- src/connectors/kafka/mod.rs | 7 +- src/connectors/kafka/rebalance_listener.rs | 2 + 6 files changed, 34 insertions(+), 247 deletions(-) diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index 97efb4a58..e628a86fc 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -5,12 +5,6 @@ services: # minio minio: image: minio/minio:RELEASE.2023-02-10T18-48-39Z - deploy: - resources: - limits: - memory: 4G - reservations: - memory: 2G entrypoint: - sh - -euc @@ -22,7 +16,7 @@ services: - MINIO_ROOT_PASSWORD=supersecret - MINIO_UPDATE=off ports: - - 9000:9000 + - "9000:9000" healthcheck: test: [ "CMD", "curl", "-f", "http://localhost:9000/minio/health/live" ] interval: 15s @@ -61,11 +55,6 @@ services: depends_on: - minio deploy: - resources: - limits: - memory: 4G - reservations: - memory: 2G restart_policy: condition: on-failure delay: 20s @@ -78,7 +67,7 @@ services: platform: linux/amd64 command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] ports: - - 8000 + - "8000" environment: - P_S3_URL=http://minio:9000 - P_S3_ACCESS_KEY=parseable @@ -93,7 +82,7 @@ services: - P_MODE=ingest - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092 # additional settings like security, tuning, etc. networks: - parseable-internal @@ -106,14 +95,7 @@ services: - parseable-query - minio - kafka-0 - - kafka-1 - - kafka-2 deploy: - resources: - limits: - memory: 4G - reservations: - memory: 2G restart_policy: condition: on-failure delay: 20s @@ -153,19 +135,13 @@ services: kafka-0: image: docker.io/bitnami/kafka:3.9 - deploy: - resources: - limits: - memory: 2G - reservations: - memory: 1G ports: - "9092" environment: # KRaft settings - KAFKA_CFG_NODE_ID=0 - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093 - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv # Listeners - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 @@ -174,8 +150,8 @@ services: - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=1 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=1 - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 volumes: - kafka_0_data:/bitnami/kafka @@ -187,95 +163,22 @@ services: timeout: 5s retries: 5 - kafka-1: - image: docker.io/bitnami/kafka:3.9 - deploy: - resources: - limits: - memory: 2G - reservations: - memory: 1G - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=1 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_1_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - - kafka-2: - image: docker.io/bitnami/kafka:3.9 - deploy: - resources: - limits: - memory: 2G - reservations: - memory: 1G - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=2 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_2_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - kafka-ui: platform: linux/amd64 image: provectuslabs/kafka-ui:latest environment: KAFKA_CLUSTERS_0_NAME: dist-test - KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 KAFKA_CLUSTERS_0_METRICS_PORT: 9101 DYNAMIC_CONFIG_ENABLED: "true" JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC networks: - parseable-internal + depends_on: + - kafka-0 + ports: + - "8080:8080" deploy: - resources: - limits: - memory: 512M - reservations: - memory: 256M restart_policy: condition: on-failure delay: 20s @@ -286,29 +189,17 @@ services: context: ./scripts dockerfile: Dockerfile environment: - - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - KAFKA_BROKERS=kafka-0:9092 - KAFKA_TOPIC=dist-test-logs-stream - - LOG_RATE=500 - - TOTAL_LOGS=50000 + - LOG_RATE=100 + - TOTAL_LOGS=20000 depends_on: - kafka-0 - - kafka-1 - - kafka-2 networks: - parseable-internal - deploy: - resources: - limits: - memory: 512M - reservations: - memory: 256M restart: "no" volumes: kafka_0_data: driver: local - kafka_1_data: - driver: local - kafka_2_data: - driver: local diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index 7005fc40c..203f14b7d 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -4,12 +4,6 @@ networks: services: minio: image: minio/minio:RELEASE.2023-02-10T18-48-39Z - deploy: - resources: - limits: - memory: 4G - reservations: - memory: 2G entrypoint: - sh - -euc @@ -50,13 +44,11 @@ services: - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - P_KAFKA_CONSUMER_TOPICS=test-logs-stream - - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092 # additional settings like security, tuning, etc. depends_on: - minio - kafka-0 - - kafka-1 - - kafka-2 healthcheck: test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] interval: 15s @@ -65,11 +57,6 @@ services: networks: - parseable-internal deploy: - resources: - limits: - memory: 4G - reservations: - memory: 3G restart_policy: condition: on-failure delay: 20s @@ -103,19 +90,13 @@ services: kafka-0: image: docker.io/bitnami/kafka:3.9 - deploy: - resources: - limits: - memory: 2G - reservations: - memory: 1G ports: - "9092" environment: # KRaft settings - KAFKA_CFG_NODE_ID=0 - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093 - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv # Listeners - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 @@ -124,8 +105,8 @@ services: - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=1 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=1 - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 volumes: - kafka_0_data:/bitnami/kafka @@ -137,78 +118,6 @@ services: timeout: 5s retries: 5 - kafka-1: - image: docker.io/bitnami/kafka:3.9 - deploy: - resources: - limits: - memory: 2G - reservations: - memory: 1G - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=1 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_1_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - - kafka-2: - image: docker.io/bitnami/kafka:3.9 - deploy: - resources: - limits: - memory: 2G - reservations: - memory: 1G - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=2 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_2_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - kafka-ui: platform: linux/amd64 image: provectuslabs/kafka-ui:latest @@ -216,22 +125,15 @@ services: - "8080:8080" depends_on: - kafka-0 - - kafka-1 - - kafka-2 environment: KAFKA_CLUSTERS_0_NAME: test - KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 KAFKA_CLUSTERS_0_METRICS_PORT: 9101 DYNAMIC_CONFIG_ENABLED: "true" JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC networks: - parseable-internal deploy: - resources: - limits: - memory: 512M - reservations: - memory: 256M restart_policy: condition: on-failure delay: 20s @@ -242,28 +144,16 @@ services: context: ./scripts dockerfile: Dockerfile environment: - - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - KAFKA_BROKERS=kafka-0:9092 - KAFKA_TOPIC=test-logs-stream - - LOG_RATE=500 - - TOTAL_LOGS=10000 + - LOG_RATE=100 + - TOTAL_LOGS=20000 depends_on: - kafka-0 - - kafka-1 - - kafka-2 networks: - parseable-internal - deploy: - resources: - limits: - memory: 512M - reservations: - memory: 256M restart: "no" volumes: kafka_0_data: driver: local - kafka_1_data: - driver: local - kafka_2_data: - driver: local diff --git a/scripts/kafka_log_stream_generator.py b/scripts/kafka_log_stream_generator.py index eab4bbe88..bc7ec86e3 100644 --- a/scripts/kafka_log_stream_generator.py +++ b/scripts/kafka_log_stream_generator.py @@ -39,7 +39,7 @@ KAFKA_BROKERS = os.getenv("KAFKA_BROKERS", "localhost:9092") KAFKA_TOPIC = os.getenv("KAFKA_TOPIC", "local-logs-stream") NUM_PARTITIONS = int(os.getenv("NUM_PARTITIONS", "6")) # Default partitions -REPLICATION_FACTOR = int(os.getenv("REPLICATION_FACTOR", "3")) # Default RF +REPLICATION_FACTOR = int(os.getenv("REPLICATION_FACTOR", "1")) # Default RF TOTAL_LOGS = int(os.getenv("TOTAL_LOGS", "100000")) # Total logs to produce LOG_RATE = int(os.getenv("LOG_RATE", "500")) # Logs per second REPORT_EVERY = 10_000 # Progress report frequency diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs index 31c72f703..d191ee520 100644 --- a/src/connectors/kafka/consumer.rs +++ b/src/connectors/kafka/consumer.rs @@ -235,11 +235,12 @@ impl KafkaStreams { } fn create_consumer(context: KafkaContext) -> Arc { - info!("Creating Kafka consumer from configs {:#?}", context.config); - let kafka_config = &context.config; let consumer_config = kafka_config.to_rdkafka_consumer_config(); - info!("Consumer configs: {:#?}", &consumer_config); + info!( + "Creating Kafka consumer from configs: {:#?}", + &consumer_config + ); let consumer: StreamConsumer = consumer_config .create_with_context(context.clone()) diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs index 48987d98b..dfae9d4df 100644 --- a/src/connectors/kafka/mod.rs +++ b/src/connectors/kafka/mod.rs @@ -203,11 +203,14 @@ impl ConsumerContext for KafkaContext { match rebalance { Rebalance::Revoke(tpl) => { let (pq_waiter_tx, pq_waiter_rx) = std::sync::mpsc::channel(); + + let tpl = TopicPartitionList::from_rdkafka_tpl(tpl); + self.notify(RebalanceEvent::Revoke(tpl, pq_waiter_tx)); + if pq_waiter_rx.recv().is_err() { warn!("Queue termination sender dropped"); } - let tpl = TopicPartitionList::from_rdkafka_tpl(tpl); - self.notify(RebalanceEvent::Revoke(tpl, pq_waiter_tx)); + info!("Rebalance Revoke started"); } Rebalance::Assign(tpl) => { let tpl = TopicPartitionList::from_rdkafka_tpl(tpl); diff --git a/src/connectors/kafka/rebalance_listener.rs b/src/connectors/kafka/rebalance_listener.rs index 40a452b99..7dbbe3985 100644 --- a/src/connectors/kafka/rebalance_listener.rs +++ b/src/connectors/kafka/rebalance_listener.rs @@ -60,6 +60,8 @@ impl RebalanceListener { info!("RebalanceEvent Revoke: {:?}", tpl); let mut stream_state = stream_state.write().await; stream_state.terminate_partition_streams(tpl).await; + drop(stream_state); + if let Err(err) = callback.send(()) { warn!("Error during sending response to context. Cause: {:?}", err); } From 2df47271d52fc592ee2d369cb79b64721ed21ce3 Mon Sep 17 00:00:00 2001 From: hippalus Date: Fri, 27 Dec 2024 23:56:48 +0100 Subject: [PATCH 20/65] add Install dependencies step to coverage.yaml --- .github/workflows/coverage.yaml | 6 +- Cargo.lock | 106 ++++++++++++++++---------------- 2 files changed, 58 insertions(+), 54 deletions(-) diff --git a/.github/workflows/coverage.yaml b/.github/workflows/coverage.yaml index eb5a46da4..495e68902 100644 --- a/.github/workflows/coverage.yaml +++ b/.github/workflows/coverage.yaml @@ -27,6 +27,10 @@ jobs: with: tool: cargo-hack, cargo-llvm-cov, nextest + - name: Install dependencies + run: sudo apt-get install -y libsasl2-dev || echo "Skipping libsasl2-dev installation" + continue-on-error: true + - name: Check with clippy run: cargo hack clippy --verbose --each-feature --no-dev-deps -- -D warnings @@ -39,7 +43,7 @@ jobs: - name: Tests run: cargo hack --each-feature llvm-cov --no-report nextest - - name: Genrate coverage report + - name: Generate coverage report run: cargo llvm-cov report --lcov --output-path coverage.lcov - name: Upload Coverage Report diff --git a/Cargo.lock b/Cargo.lock index ce8b077a0..6960a94dd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -81,7 +81,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e01ed3140b2f8d422c68afa1ed2e85d996ea619c988ac834d255db32138655cb" dependencies = [ "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -218,7 +218,7 @@ dependencies = [ "actix-router", "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -707,7 +707,7 @@ checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -718,7 +718,7 @@ checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -1026,9 +1026,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.5" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c31a0499c1dc64f458ad13872de75c0eb7e3fdb0e67964610c914b034fc5956e" +checksum = "8d6dbb628b8f8555f86d0323c2eb39e3ec81901f4b83e091db8a6a76d316a333" dependencies = [ "jobserver", "libc", @@ -1132,7 +1132,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -1381,7 +1381,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -1392,7 +1392,7 @@ checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -1821,7 +1821,7 @@ checksum = "30542c1ad912e0e3d22a1935c290e12e8a29d704a420177a31faad4a601a0800" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -1834,7 +1834,7 @@ dependencies = [ "proc-macro2", "quote", "rustc_version", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -1856,7 +1856,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -2035,7 +2035,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -2587,7 +2587,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -3129,7 +3129,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -3170,7 +3170,7 @@ dependencies = [ "percent-encoding", "quick-xml", "rand", - "reqwest 0.12.10", + "reqwest 0.12.11", "ring", "serde", "serde_json", @@ -3198,7 +3198,7 @@ dependencies = [ "chrono", "lazy_static", "mime", - "reqwest 0.12.10", + "reqwest 0.12.11", "serde", "serde_json", "thiserror 1.0.69", @@ -3527,7 +3527,7 @@ checksum = "3c0f5fad0874fc7abcd4d750e76917eaebbecaa2c20bde22e1dbeeba8beb758c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -3570,7 +3570,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "64d1ec885c64d0457d564db4ec299b2dae3f9c02808b8ad9c3a089c591b18033" dependencies = [ "proc-macro2", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -3693,7 +3693,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.91", + "syn 2.0.92", "tempfile", ] @@ -3707,7 +3707,7 @@ dependencies = [ "itertools 0.13.0", "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4016,9 +4016,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.10" +version = "0.12.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d3536321cfc54baa8cf3e273d5e1f63f889067829c4b410fcdbac8ca7b80994" +checksum = "7fe060fe50f524be480214aba758c71f99f90ee8c83c5a36b5e9e1d568eb4eb3" dependencies = [ "base64 0.22.1", "bytes", @@ -4102,7 +4102,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.91", + "syn 2.0.92", "unicode-ident", ] @@ -4243,9 +4243,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" +checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "ryu" @@ -4339,22 +4339,22 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.216" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b9781016e935a97e8beecf0c933758c97a5520d32930e460142b4cd80c6338e" +checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.216" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46f859dbbf73865c6627ed570e78961cd3ac92407a2d117204c49232485da55e" +checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4378,7 +4378,7 @@ checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4530,7 +4530,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4573,7 +4573,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4624,7 +4624,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4645,9 +4645,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.91" +version = "2.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d53cbcb5a243bd33b7858b1d7f4aca2153490815872d86d955d6ea29f743c035" +checksum = "70ae51629bf965c5c098cc9e87908a3df5301051a9e087d6f9bef5c9771ed126" dependencies = [ "proc-macro2", "quote", @@ -4677,14 +4677,14 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] name = "sysinfo" -version = "0.33.0" +version = "0.33.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "948512566b1895f93b1592c7574baeb2de842f224f2aab158799ecadb8ebbb46" +checksum = "4fc858248ea01b66f19d8e8a6d55f41deaf91e9d495246fd01368d99935c6c01" dependencies = [ "core-foundation-sys", "libc", @@ -4754,7 +4754,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4765,7 +4765,7 @@ checksum = "7b50fa271071aae2e6ee85f842e2e28ba8cd2c5fb67f11fcb1fd70b276f9e7d4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -4896,7 +4896,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -5139,7 +5139,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -5354,7 +5354,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -5447,7 +5447,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", "wasm-bindgen-shared", ] @@ -5482,7 +5482,7 @@ checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -5621,7 +5621,7 @@ checksum = "9107ddc059d5b6fbfbffdfa7a7fe3e22a226def0b2608f72e9d552763d3e1ad7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -5632,7 +5632,7 @@ checksum = "29bee4b38ea3cde66011baa44dba677c432a78593e202392d1e9070cf2a7fca7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -5888,7 +5888,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", "synstructure", ] @@ -5910,7 +5910,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] @@ -5930,7 +5930,7 @@ checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", "synstructure", ] @@ -5959,7 +5959,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.91", + "syn 2.0.92", ] [[package]] From 839bef8c1e78769ab426e85e4338875c20949a4a Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 28 Dec 2024 05:14:44 +0100 Subject: [PATCH 21/65] improve logging and err handling --- Cargo.lock | 24 ----- Cargo.toml | 1 - src/connectors/common/mod.rs | 57 ++++++++++-- src/connectors/kafka/consumer.rs | 103 ++++++++------------- src/connectors/kafka/mod.rs | 7 +- src/connectors/kafka/partition_stream.rs | 6 +- src/connectors/kafka/processor.rs | 4 +- src/connectors/kafka/rebalance_listener.rs | 9 +- src/connectors/kafka/state.rs | 4 +- src/handlers/http/modal/mod.rs | 2 +- 10 files changed, 108 insertions(+), 109 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6960a94dd..b8b7dd70a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -789,17 +789,6 @@ dependencies = [ "tower-service", ] -[[package]] -name = "backon" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba5289ec98f68f28dd809fd601059e6aa908bb8f6108620930828283d4ee23d7" -dependencies = [ - "fastrand 2.3.0", - "gloo-timers", - "tokio", -] - [[package]] name = "backtrace" version = "0.3.74" @@ -2109,18 +2098,6 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" -[[package]] -name = "gloo-timers" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbb143cf96099802033e0d4f4963b19fd2e0b728bcf076cd9cf7f6634f092994" -dependencies = [ - "futures-channel", - "futures-core", - "js-sys", - "wasm-bindgen", -] - [[package]] name = "h2" version = "0.3.26" @@ -3342,7 +3319,6 @@ dependencies = [ "arrow-schema", "arrow-select", "async-trait", - "backon", "base64 0.22.1", "byteorder", "bytes", diff --git a/Cargo.toml b/Cargo.toml index c00fd62f5..55545c62e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,7 +32,6 @@ mime = "0.3.17" ### connectors dependencies rdkafka = { version = "0.37", features = ["cmake-build", "tracing", "sasl", "ssl", "libz-static"] } -backon = "1.3" ### other dependencies anyhow = { version = "1.0", features = ["backtrace"] } diff --git a/src/connectors/common/mod.rs b/src/connectors/common/mod.rs index f9bf8b137..3401f7937 100644 --- a/src/connectors/common/mod.rs +++ b/src/connectors/common/mod.rs @@ -15,19 +15,62 @@ * along with this program. If not, see . * */ +use rdkafka::error::{KafkaError, RDKafkaErrorCode}; +use thiserror::Error; pub mod config; pub mod processor; pub mod shutdown; -#[derive(Debug, thiserror::Error)] +#[derive(Debug, Error)] pub enum ConnectorError { #[error("Kafka error: {0}")] - Kafka(#[from] rdkafka::error::KafkaError), - #[error("Configuration error: {0}")] - Config(String), + Kafka(KafkaError), + + #[error("Connection error: {0}")] + Connection(String), + + #[error("Fatal error: {0}")] + Fatal(String), + #[error("Processing error: {0}")] - Processing(String), - #[error("Initialization error: {0}")] - Init(String), + Processing(#[from] anyhow::Error), + + #[error("State error: {0}")] + State(String), + + #[error("Authentication error: {0}")] + Auth(String), +} + +impl From for ConnectorError { + fn from(error: KafkaError) -> Self { + if let Some(code) = error.rdkafka_error_code() { + match code { + RDKafkaErrorCode::BrokerTransportFailure + | RDKafkaErrorCode::NetworkException + | RDKafkaErrorCode::AllBrokersDown => ConnectorError::Connection(error.to_string()), + + RDKafkaErrorCode::Fatal | RDKafkaErrorCode::CriticalSystemResource => { + ConnectorError::Fatal(error.to_string()) + } + + RDKafkaErrorCode::Authentication | RDKafkaErrorCode::SaslAuthenticationFailed => { + ConnectorError::Auth(error.to_string()) + } + + _ => ConnectorError::Kafka(error), + } + } else { + ConnectorError::Kafka(error) + } + } +} +impl ConnectorError { + pub fn is_fatal(&self) -> bool { + matches!( + self, + ConnectorError::Fatal(_) | ConnectorError::Auth(_) | ConnectorError::State(_) + ) + } } diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs index d191ee520..d4783e4a8 100644 --- a/src/connectors/kafka/consumer.rs +++ b/src/connectors/kafka/consumer.rs @@ -17,21 +17,21 @@ */ use crate::connectors::common::shutdown::Shutdown; +use crate::connectors::common::ConnectorError; use crate::connectors::kafka::partition_stream::{PartitionStreamReceiver, PartitionStreamSender}; use crate::connectors::kafka::state::StreamState; use crate::connectors::kafka::{ partition_stream, ConsumerRecord, KafkaContext, StreamConsumer, TopicPartition, }; -use backon::{ExponentialBuilder, Retryable}; use futures_util::FutureExt; use rdkafka::consumer::Consumer; -use rdkafka::error::KafkaError; use rdkafka::message::BorrowedMessage; use rdkafka::Statistics; use std::sync::Arc; +use std::time::Duration; use tokio::sync::{mpsc, RwLock}; use tokio_stream::wrappers::ReceiverStream; -use tracing::{error, info}; +use tracing::{error, info, warn}; pub struct KafkaStreams { consumer: Arc, @@ -108,76 +108,49 @@ impl KafkaStreams { std::thread::spawn(move || { tokio_handle.block_on(async move { - let retry_policy = ExponentialBuilder::default().with_max_times(5000); - loop { - let result = KafkaStreams::process_consumer_messages( - &consumer, - &stream_state, - &stream_tx, - &shutdown_handle, - &retry_policy, - ) - .await; - - if let Err(e) = result { - error!( - "Partitioned processing encountered a critical error: {:?}", - e - ); - break; + let result: Result<(), ConnectorError> = tokio::select! { + result = consumer.recv() => { + match result { + Ok(msg) => KafkaStreams::handle_message(msg, &stream_state, &stream_tx).await.map_err(Into::into), + Err(e) => Err(e.into()) + } + } + _ = shutdown_handle.recv() => { + KafkaStreams::shutdown(&consumer, &stream_state).await; + break; + } + }; + + match result { + Ok(_) => continue, + Err(error) => match &error { + ConnectorError::Connection(msg) => { + error!("Connection error: {}", msg); + tokio::time::sleep(Duration::from_secs(1)).await; + } + ConnectorError::Fatal(msg) => { + error!("Fatal error: {}", msg); + break; + } + ConnectorError::Auth(msg) => { + error!("Authentication error: {}", msg); + break; + } + error => { + warn!("Non-fatal error: {}", error); + } + }, } } + + info!("Kafka stream processing terminated"); }); }); ReceiverStream::new(stream_rx) } - async fn process_consumer_messages( - consumer: &Arc, - stream_state: &RwLock, - stream_tx: &mpsc::Sender, - shutdown_handle: &Shutdown, - retry_policy: &ExponentialBuilder, - ) -> anyhow::Result<()> { - tokio::select! { - result = KafkaStreams::receive_with_retry(consumer, retry_policy) => match result { - Ok(msg) => KafkaStreams::handle_message(msg, stream_state, stream_tx).await, - Err(err) => { - anyhow::bail!("Unrecoverable error occurred while receiving Kafka message: {:?}", err); - }, - }, - _ = shutdown_handle.recv() => { - KafkaStreams::handle_shutdown(consumer, stream_state).await; - Ok(()) - }, - else => { - error!("KafkaStreams terminated unexpectedly!"); - Ok(()) - } - } - } - - async fn receive_with_retry<'a>( - consumer: &'a Arc, - retry_policy: &'a ExponentialBuilder, - ) -> Result, KafkaError> { - let recv_fn = || consumer.recv(); - - recv_fn - .retry(*retry_policy) - .sleep(tokio::time::sleep) - .notify(|err, dur| { - tracing::warn!( - "Retrying message reception due to error: {:?}. Waiting for {:?}...", - err, - dur - ); - }) - .await - } - /// Handle individual Kafka message and route it to the proper partition stream async fn handle_message( msg: BorrowedMessage<'_>, @@ -227,7 +200,7 @@ impl KafkaStreams { ps_tx } - async fn handle_shutdown(consumer: &Arc, stream_state: &RwLock) { + async fn shutdown(consumer: &Arc, stream_state: &RwLock) { info!("Gracefully stopping kafka partition streams!"); let mut state = stream_state.write().await; state.clear(); diff --git a/src/connectors/kafka/mod.rs b/src/connectors/kafka/mod.rs index dfae9d4df..211371b44 100644 --- a/src/connectors/kafka/mod.rs +++ b/src/connectors/kafka/mod.rs @@ -72,10 +72,11 @@ impl KafkaContext { let rebalance_sender = self.rebalance_tx.clone(); std::thread::spawn(move || { info!("Sending RebalanceEvent to listener..."); - if rebalance_sender.blocking_send(rebalance_event).is_err() { - warn!("Rebalance event receiver is closed!"); + if let Err(e) = rebalance_sender.blocking_send(rebalance_event) { + warn!("Rebalance event receiver is closed! {:?}", e); + } else { + info!("RebalanceEvent sent successfully!"); } - info!("Sent RebalanceEvent to lister."); }); } diff --git a/src/connectors/kafka/partition_stream.rs b/src/connectors/kafka/partition_stream.rs index e99cd649e..f1a6ca8bc 100644 --- a/src/connectors/kafka/partition_stream.rs +++ b/src/connectors/kafka/partition_stream.rs @@ -20,7 +20,7 @@ use crate::connectors::kafka::{ConsumerRecord, TopicPartition}; use std::sync::Arc; use tokio::sync::{mpsc, Notify}; use tokio_stream::wrappers::ReceiverStream; -use tracing::info; +use tracing::{error, info}; #[derive(Clone)] pub struct PartitionStreamSender { @@ -38,7 +38,9 @@ impl PartitionStreamSender { } pub async fn send(&self, consumer_record: ConsumerRecord) { - self.inner.send(consumer_record).await.unwrap(); + if let Err(e) = self.inner.send(consumer_record).await { + error!("Failed to send message to partition stream: {:?}", e); + } } pub fn sender(&self) -> mpsc::Sender { diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index c18ca1dc9..d4ed999c0 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -156,7 +156,9 @@ where //@see https://github.com/confluentinc/librdkafka/issues/4534 //@see https://github.com/confluentinc/librdkafka/issues/4059 if let Err(e) = self.consumer.commit(&tpl, CommitMode::Sync) { - error!("Failed to commit offsets for {:?}: {:?}", tp, e); + error!(error = %e, "Failed to commit offsets for {:?}", tpl); + } else { + debug!("Committed offsets for {:?}", tpl); } } }) diff --git a/src/connectors/kafka/rebalance_listener.rs b/src/connectors/kafka/rebalance_listener.rs index 7dbbe3985..1b1c40890 100644 --- a/src/connectors/kafka/rebalance_listener.rs +++ b/src/connectors/kafka/rebalance_listener.rs @@ -58,9 +58,12 @@ impl RebalanceListener { Some(RebalanceEvent::Assign(tpl)) => info!("RebalanceEvent Assign: {:?}", tpl), Some(RebalanceEvent::Revoke(tpl, callback)) => { info!("RebalanceEvent Revoke: {:?}", tpl); - let mut stream_state = stream_state.write().await; - stream_state.terminate_partition_streams(tpl).await; - drop(stream_state); + if let Ok(mut stream_state) = stream_state.try_write() { + stream_state.terminate_partition_streams(tpl).await; + drop(stream_state); + } else { + warn!("Stream state lock is busy, skipping rebalance revoke for {:?}", tpl); + } if let Err(err) = callback.send(()) { warn!("Error during sending response to context. Cause: {:?}", err); diff --git a/src/connectors/kafka/state.rs b/src/connectors/kafka/state.rs index ca0904d4d..7b20ae762 100644 --- a/src/connectors/kafka/state.rs +++ b/src/connectors/kafka/state.rs @@ -50,9 +50,9 @@ impl StreamState { for tp in tpl.tpl { if let Some(sender) = self.partition_senders.remove(&tp) { info!("Terminating stream for {:?}", tp); - drop(sender.sender()); sender.terminate(); - info!("Waiting for stream to finish for {:?}", tp); + drop(sender); + info!("Stream terminated for {:?}", tp); } else { info!("Stream already completed for {:?}", tp); } diff --git a/src/handlers/http/modal/mod.rs b/src/handlers/http/modal/mod.rs index e831be829..2be6ca2f1 100644 --- a/src/handlers/http/modal/mod.rs +++ b/src/handlers/http/modal/mod.rs @@ -116,7 +116,7 @@ pub trait ParseableServer { // Spawn the signal handler task let signal_task = tokio::spawn(async move { health_check::handle_signals(shutdown_signal).await; - println!("Received shutdown signal, notifying server to shut down..."); + info!("Received shutdown signal, notifying server to shut down..."); }); // Create the HTTP server From 935fc4081904f7778edd73cec362d620bca5f127 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 28 Dec 2024 05:32:40 +0100 Subject: [PATCH 22/65] change log rate --- docker-compose-distributed-test.yaml | 2 +- docker-compose-test.yaml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index e628a86fc..d3b2ba42b 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -191,7 +191,7 @@ services: environment: - KAFKA_BROKERS=kafka-0:9092 - KAFKA_TOPIC=dist-test-logs-stream - - LOG_RATE=100 + - LOG_RATE=500 - TOTAL_LOGS=20000 depends_on: - kafka-0 diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index 203f14b7d..943cf1a41 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -146,8 +146,8 @@ services: environment: - KAFKA_BROKERS=kafka-0:9092 - KAFKA_TOPIC=test-logs-stream - - LOG_RATE=100 - - TOTAL_LOGS=20000 + - LOG_RATE=1000 + - TOTAL_LOGS=10000 depends_on: - kafka-0 networks: From a44582dd05ca079342ecd14aec81e64530e6218a Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 28 Dec 2024 05:50:29 +0100 Subject: [PATCH 23/65] comment out kafka-ui in docker-compose --- docker-compose-distributed-test.yaml | 40 ++++++++++++++-------------- docker-compose-test.yaml | 40 ++++++++++++++-------------- 2 files changed, 40 insertions(+), 40 deletions(-) diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index d3b2ba42b..4f2334050 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -163,26 +163,26 @@ services: timeout: 5s retries: 5 - kafka-ui: - platform: linux/amd64 - image: provectuslabs/kafka-ui:latest - environment: - KAFKA_CLUSTERS_0_NAME: dist-test - KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 - KAFKA_CLUSTERS_0_METRICS_PORT: 9101 - DYNAMIC_CONFIG_ENABLED: "true" - JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC - networks: - - parseable-internal - depends_on: - - kafka-0 - ports: - - "8080:8080" - deploy: - restart_policy: - condition: on-failure - delay: 20s - max_attempts: 3 + #kafka-ui: + # platform: linux/amd64 + # image: provectuslabs/kafka-ui:latest + # environment: + # KAFKA_CLUSTERS_0_NAME: dist-test + # KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 + # KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + # DYNAMIC_CONFIG_ENABLED: "true" + # JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC + # networks: + # - parseable-internal + # depends_on: + # - kafka-0 + # ports: + # - "8080:8080" + # deploy: + # restart_policy: + # condition: on-failure + # delay: 20s + # max_attempts: 3 kafka-log-generator: build: diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index 943cf1a41..e44fe352a 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -118,26 +118,26 @@ services: timeout: 5s retries: 5 - kafka-ui: - platform: linux/amd64 - image: provectuslabs/kafka-ui:latest - ports: - - "8080:8080" - depends_on: - - kafka-0 - environment: - KAFKA_CLUSTERS_0_NAME: test - KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 - KAFKA_CLUSTERS_0_METRICS_PORT: 9101 - DYNAMIC_CONFIG_ENABLED: "true" - JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC - networks: - - parseable-internal - deploy: - restart_policy: - condition: on-failure - delay: 20s - max_attempts: 3 + #kafka-ui: + # platform: linux/amd64 + # image: provectuslabs/kafka-ui:latest + # ports: + # - "8080:8080" + # depends_on: + # - kafka-0 + # environment: + # KAFKA_CLUSTERS_0_NAME: test + # KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 + # KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + # DYNAMIC_CONFIG_ENABLED: "true" + # JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC + # networks: + # - parseable-internal + # deploy: + # restart_policy: + # condition: on-failure + # delay: 20s + # max_attempts: 3 kafka-log-generator: build: From 8937c4d413577985116c1328df503a390cb9925c Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 28 Dec 2024 07:26:31 +0100 Subject: [PATCH 24/65] refactor py script --- docker-compose-distributed-test.yaml | 131 ++++++++++++++++++++------ docker-compose-test.yaml | 50 +++++----- scripts/kafka_log_stream_generator.py | 26 +++-- 3 files changed, 147 insertions(+), 60 deletions(-) diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index 4f2334050..1b4a84699 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -82,7 +82,7 @@ services: - P_MODE=ingest - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092 + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 # additional settings like security, tuning, etc. networks: - parseable-internal @@ -95,6 +95,8 @@ services: - parseable-query - minio - kafka-0 + - kafka-1 + - kafka-2 deploy: restart_policy: condition: on-failure @@ -141,7 +143,7 @@ services: # KRaft settings - KAFKA_CFG_NODE_ID=0 - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093 + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv # Listeners - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 @@ -150,8 +152,8 @@ services: - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=1 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=1 + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 volumes: - kafka_0_data:/bitnami/kafka @@ -163,43 +165,116 @@ services: timeout: 5s retries: 5 - #kafka-ui: - # platform: linux/amd64 - # image: provectuslabs/kafka-ui:latest - # environment: - # KAFKA_CLUSTERS_0_NAME: dist-test - # KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 - # KAFKA_CLUSTERS_0_METRICS_PORT: 9101 - # DYNAMIC_CONFIG_ENABLED: "true" - # JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC - # networks: - # - parseable-internal - # depends_on: - # - kafka-0 - # ports: - # - "8080:8080" - # deploy: - # restart_policy: - # condition: on-failure - # delay: 20s - # max_attempts: 3 + kafka-1: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=1 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_1_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-2: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=2 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_2_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-ui: + platform: linux/amd64 + image: provectuslabs/kafka-ui:latest + environment: + KAFKA_CLUSTERS_0_NAME: dist-test + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 + KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + DYNAMIC_CONFIG_ENABLED: "true" + JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC + networks: + - parseable-internal + depends_on: + - kafka-0 + - kafka-1 + - kafka-2 + ports: + - "8080:8080" + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 kafka-log-generator: build: context: ./scripts dockerfile: Dockerfile environment: - - KAFKA_BROKERS=kafka-0:9092 + - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - KAFKA_TOPIC=dist-test-logs-stream - - LOG_RATE=500 - - TOTAL_LOGS=20000 + - LOG_RATE=1000 + - TOTAL_LOGS=100000 + - REPLICATION_FACTO=3 depends_on: - kafka-0 + - kafka-1 + - kafka-2 networks: - parseable-internal - restart: "no" + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 volumes: kafka_0_data: driver: local + kafka_1_data: + driver: local + kafka_2_data: + driver: local diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index e44fe352a..7ea445fba 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -118,26 +118,26 @@ services: timeout: 5s retries: 5 - #kafka-ui: - # platform: linux/amd64 - # image: provectuslabs/kafka-ui:latest - # ports: - # - "8080:8080" - # depends_on: - # - kafka-0 - # environment: - # KAFKA_CLUSTERS_0_NAME: test - # KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 - # KAFKA_CLUSTERS_0_METRICS_PORT: 9101 - # DYNAMIC_CONFIG_ENABLED: "true" - # JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC - # networks: - # - parseable-internal - # deploy: - # restart_policy: - # condition: on-failure - # delay: 20s - # max_attempts: 3 + kafka-ui: + platform: linux/amd64 + image: provectuslabs/kafka-ui:latest + ports: + - "8080:8080" + depends_on: + - kafka-0 + environment: + KAFKA_CLUSTERS_0_NAME: test + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 + KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + DYNAMIC_CONFIG_ENABLED: "true" + JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 kafka-log-generator: build: @@ -146,13 +146,17 @@ services: environment: - KAFKA_BROKERS=kafka-0:9092 - KAFKA_TOPIC=test-logs-stream - - LOG_RATE=1000 - - TOTAL_LOGS=10000 + - LOG_RATE=500 + - TOTAL_LOGS=50000 depends_on: - kafka-0 networks: - parseable-internal - restart: "no" + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 volumes: kafka_0_data: diff --git a/scripts/kafka_log_stream_generator.py b/scripts/kafka_log_stream_generator.py index bc7ec86e3..b4f5214cf 100644 --- a/scripts/kafka_log_stream_generator.py +++ b/scripts/kafka_log_stream_generator.py @@ -150,14 +150,16 @@ def generate_log(): def main(): - logger.info("Starting rate-limited log producer...") + logger.info("Starting continuous log producer...") create_topic(KAFKA_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR) - logger.info(f"Broker: {KAFKA_BROKERS}, Topic: {KAFKA_TOPIC}, Rate: {LOG_RATE} logs/sec, Total Logs: {TOTAL_LOGS}") + logger.info(f"Broker: {KAFKA_BROKERS}, Topic: {KAFKA_TOPIC}, Rate: {LOG_RATE} logs/sec") + message_count = 0 start_time = time.time() + batch_start_time = time.time() try: - for i in range(TOTAL_LOGS): + while True: log_data = generate_log() log_str = json.dumps(log_data) @@ -168,9 +170,19 @@ def main(): callback=delivery_report ) - if (i + 1) % REPORT_EVERY == 0: - logger.info(f"{i + 1} messages produced. Flushing producer...") + message_count += 1 + + if message_count % REPORT_EVERY == 0: + current_time = time.time() + batch_elapsed = current_time - batch_start_time + total_elapsed = current_time - start_time + + logger.info(f"Batch of {REPORT_EVERY} messages produced in {batch_elapsed:.2f}s") + logger.info(f"Total messages: {message_count}, Running time: {total_elapsed:.2f}s") + logger.info(f"Current rate: ~{REPORT_EVERY / batch_elapsed:,.0f} logs/sec") + producer.flush() + batch_start_time = current_time # Sleep to maintain the logs/second rate time.sleep(1 / LOG_RATE) @@ -186,10 +198,6 @@ def main(): logger.info("Flushing producer...") producer.flush() - elapsed = time.time() - start_time - logger.info(f"DONE! Produced {TOTAL_LOGS} log messages in {elapsed:.2f} seconds.") - logger.info(f"Effective rate: ~{TOTAL_LOGS / elapsed:,.0f} logs/sec") - if __name__ == "__main__": main() From 00647271e41c94fa21fc8fe6096403401b7f1e6a Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 28 Dec 2024 08:07:14 +0100 Subject: [PATCH 25/65] refactor py script --- scripts/kafka_log_stream_generator.py | 62 +++++++++++++++++---------- 1 file changed, 39 insertions(+), 23 deletions(-) diff --git a/scripts/kafka_log_stream_generator.py b/scripts/kafka_log_stream_generator.py index b4f5214cf..55bb442c5 100644 --- a/scripts/kafka_log_stream_generator.py +++ b/scripts/kafka_log_stream_generator.py @@ -40,9 +40,9 @@ KAFKA_TOPIC = os.getenv("KAFKA_TOPIC", "local-logs-stream") NUM_PARTITIONS = int(os.getenv("NUM_PARTITIONS", "6")) # Default partitions REPLICATION_FACTOR = int(os.getenv("REPLICATION_FACTOR", "1")) # Default RF -TOTAL_LOGS = int(os.getenv("TOTAL_LOGS", "100000")) # Total logs to produce -LOG_RATE = int(os.getenv("LOG_RATE", "500")) # Logs per second -REPORT_EVERY = 10_000 # Progress report frequency +TOTAL_LOGS = int(os.getenv("TOTAL_LOGS", "100")) # Total logs to produce +LOG_RATE = int(os.getenv("LOG_RATE", "50")) # Logs per second +REPORT_EVERY = 5_000 # Progress report frequency producer_conf = { "bootstrap.servers": KAFKA_BROKERS, @@ -157,35 +157,51 @@ def main(): message_count = 0 start_time = time.time() batch_start_time = time.time() + limit_reached = False try: while True: - log_data = generate_log() - log_str = json.dumps(log_data) + current_time = time.time() - # Send to Kafka - producer.produce( - topic=KAFKA_TOPIC, - value=log_str, - callback=delivery_report - ) + if not limit_reached: + if message_count < TOTAL_LOGS: + log_data = generate_log() + log_str = json.dumps(log_data) - message_count += 1 + # Send to Kafka + producer.produce( + topic=KAFKA_TOPIC, + value=log_str, + callback=delivery_report + ) - if message_count % REPORT_EVERY == 0: - current_time = time.time() - batch_elapsed = current_time - batch_start_time - total_elapsed = current_time - start_time + message_count += 1 + + if message_count % REPORT_EVERY == 0: + batch_elapsed = current_time - batch_start_time + total_elapsed = current_time - start_time + + logger.info(f"Batch of {REPORT_EVERY} messages produced in {batch_elapsed:.2f}s") + logger.info(f"Total messages: {message_count}, Running time: {total_elapsed:.2f}s") + logger.info(f"Current rate: ~{REPORT_EVERY / batch_elapsed:,.0f} logs/sec") - logger.info(f"Batch of {REPORT_EVERY} messages produced in {batch_elapsed:.2f}s") - logger.info(f"Total messages: {message_count}, Running time: {total_elapsed:.2f}s") - logger.info(f"Current rate: ~{REPORT_EVERY / batch_elapsed:,.0f} logs/sec") + producer.flush() + batch_start_time = current_time - producer.flush() - batch_start_time = current_time + elif not limit_reached: + logger.info( + f"Reached TOTAL_LOGS limit of {TOTAL_LOGS}. Continuing to run without producing messages...") + producer.flush() + limit_reached = True - # Sleep to maintain the logs/second rate - time.sleep(1 / LOG_RATE) + if limit_reached: + time.sleep(5) + total_elapsed = current_time - start_time + if total_elapsed % 60 < 5: + logger.info(f"Total messages sent: {message_count}") + else: + # Sleep to maintain the logs/second rate + time.sleep(1 / LOG_RATE) except KeyboardInterrupt: logger.warning("Interrupted by user! Flushing remaining messages...") From 535563411439ba0086215b8a05e266e73638ccbf Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 28 Dec 2024 08:45:38 +0100 Subject: [PATCH 26/65] update dist-test with LOG_RATE=500 TOTAL_LOGS=50000 --- docker-compose-distributed-test.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index 1b4a84699..6531dc5ea 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -255,8 +255,8 @@ services: environment: - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - KAFKA_TOPIC=dist-test-logs-stream - - LOG_RATE=1000 - - TOTAL_LOGS=100000 + - LOG_RATE=500 + - TOTAL_LOGS=50000 - REPLICATION_FACTO=3 depends_on: - kafka-0 From 32c17bd789ba0f85c9aba1ff67b1248b4b636e4c Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 28 Dec 2024 16:57:00 +0100 Subject: [PATCH 27/65] update dist-test with topic REPLICATION_FACTOR=3 --- docker-compose-distributed-test.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index 6531dc5ea..beafb566d 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -257,7 +257,7 @@ services: - KAFKA_TOPIC=dist-test-logs-stream - LOG_RATE=500 - TOTAL_LOGS=50000 - - REPLICATION_FACTO=3 + - REPLICATION_FACTOR=3 depends_on: - kafka-0 - kafka-1 From cc236d0df523029724c9f529dd217a7b8c13f851 Mon Sep 17 00:00:00 2001 From: hippalus Date: Mon, 30 Dec 2024 15:40:34 +0100 Subject: [PATCH 28/65] Separate kafka and standard dockerfiles. Add conditional compilation for Kafka features. --- .github/workflows/build-push-edge-kafka.yaml | 45 +++ .github/workflows/integration-test.yaml | 24 ++ Cargo.lock | 194 ++++-------- Cargo.toml | 15 +- Dockerfile | 23 +- Dockerfile.debug | 3 +- Dockerfile.dev | 4 +- Dockerfile.kafka | 58 ++++ ...r-compose-distributed-test-with-kafka.yaml | 280 ++++++++++++++++++ docker-compose-distributed-test.yaml | 152 +--------- docker-compose-test-with-kafka.yaml | 164 ++++++++++ docker-compose-test.yaml | 80 +---- src/cli.rs | 22 +- src/connectors/mod.rs | 2 + src/lib.rs | 5 + src/main.rs | 29 +- src/metadata.rs | 1 + src/option.rs | 25 ++ 18 files changed, 717 insertions(+), 409 deletions(-) create mode 100644 .github/workflows/build-push-edge-kafka.yaml create mode 100644 Dockerfile.kafka create mode 100644 docker-compose-distributed-test-with-kafka.yaml create mode 100644 docker-compose-test-with-kafka.yaml diff --git a/.github/workflows/build-push-edge-kafka.yaml b/.github/workflows/build-push-edge-kafka.yaml new file mode 100644 index 000000000..ecf20bdee --- /dev/null +++ b/.github/workflows/build-push-edge-kafka.yaml @@ -0,0 +1,45 @@ +name: Build and push edge kafka tag + +on: + push: + branches: + - 'main' + paths-ignore: + - 'docs/**' + - 'helm/**' + - 'assets/**' + - '**.md' + +jobs: + docker: + runs-on: ubuntu-latest + steps: + - name: Checkout + uses: actions/checkout@v3 + + - name: Set up QEMU + uses: docker/setup-qemu-action@v3 + + - name: Set up Docker Buildx + uses: docker/setup-buildx-action@v3 + + - name: Login to Docker Hub + uses: docker/login-action@f054a8b539a109f9f41c372932f1ae047eff08c9 + with: + username: ${{ secrets.DOCKERHUB_USERNAME }} + password: ${{ secrets.DOCKERHUB_TOKEN }} + + - name: Extract metadata (tags, labels) for Docker + id: meta + uses: docker/metadata-action@98669ae865ea3cffbcbaa878cf57c20bbf1c6c38 + with: + images: parseable/parseable + + - name: Build and push + uses: docker/build-push-action@ad44023a93711e3deb337508980b4b5e9bcdc5dc + with: + context: . + file: ./Dockerfile.kafka + push: true + tags: parseable/parseable:edge-kafka + platforms: linux/amd64,linux/arm64 diff --git a/.github/workflows/integration-test.yaml b/.github/workflows/integration-test.yaml index fac178932..52f8b9020 100644 --- a/.github/workflows/integration-test.yaml +++ b/.github/workflows/integration-test.yaml @@ -33,3 +33,27 @@ jobs: - name: Stop compose if: always() run: docker compose -f docker-compose-distributed-test.yaml down + + docker-compose-test-with-kafka: + name: Quest Smoke and Load Tests for Standalone deployments with Kafka + runs-on: ubuntu-latest + steps: + - name: Checkout + uses: actions/checkout@v3 + - name: Start compose + run: docker compose -f docker-compose-test-with-kafka.yaml up --build --exit-code-from quest + - name: Stop compose + if: always() + run: docker compose -f docker-compose-test-with-kafka.yaml down + + docker-compose-distributed-test-with-kafka: + name: Quest Smoke and Load Tests for Distributed deployments with Kafka + runs-on: ubuntu-latest + steps: + - name: Checkout + uses: actions/checkout@v3 + - name: Start compose + run: docker compose -f docker-compose-distributed-test-with-kafka.yaml up --build --exit-code-from quest + - name: Stop compose + if: always() + run: docker compose -f docker-compose-distributed-test-with-kafka.yaml down diff --git a/Cargo.lock b/Cargo.lock index b8b7dd70a..a0d9dbd8d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -81,7 +81,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e01ed3140b2f8d422c68afa1ed2e85d996ea619c988ac834d255db32138655cb" dependencies = [ "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -218,7 +218,7 @@ dependencies = [ "actix-router", "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -676,7 +676,7 @@ version = "0.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "df895a515f70646414f4b45c0b79082783b80552b373a68283012928df56f522" dependencies = [ - "bzip2 0.4.4", + "bzip2", "flate2", "futures-core", "futures-io", @@ -707,7 +707,7 @@ checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -718,7 +718,7 @@ checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -950,16 +950,6 @@ dependencies = [ "libc", ] -[[package]] -name = "bzip2" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bafdbf26611df8c14810e268ddceda071c297570a5fb360ceddf617fe417ef58" -dependencies = [ - "bzip2-sys", - "libc", -] - [[package]] name = "bzip2-sys" version = "0.1.11+1.0.8" @@ -1121,7 +1111,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -1370,7 +1360,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -1381,7 +1371,7 @@ checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -1418,7 +1408,7 @@ dependencies = [ "async-compression", "async-trait", "bytes", - "bzip2 0.4.4", + "bzip2", "chrono", "dashmap", "datafusion-catalog", @@ -1810,7 +1800,7 @@ checksum = "30542c1ad912e0e3d22a1935c290e12e8a29d704a420177a31faad4a601a0800" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -1823,7 +1813,7 @@ dependencies = [ "proc-macro2", "quote", "rustc_version", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -1845,7 +1835,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -2024,7 +2014,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -2094,9 +2084,9 @@ checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" +checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "h2" @@ -2153,15 +2143,6 @@ version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" -[[package]] -name = "hashbrown" -version = "0.13.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" -dependencies = [ - "ahash", -] - [[package]] name = "hashbrown" version = "0.14.5" @@ -2178,16 +2159,6 @@ version = "0.15.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" -[[package]] -name = "hashlru" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2c782df656366ccb42fbbac7c7c180535a43d906a138ffb491ec3ed56a239ab" -dependencies = [ - "hashbrown 0.13.2", - "serde", -] - [[package]] name = "heck" version = "0.5.0" @@ -2564,7 +2535,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -2883,12 +2854,6 @@ dependencies = [ "libc", ] -[[package]] -name = "maplit" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e2e65a1a2e43cfcb47a895c4c8b10d1f4a61097f9f254f183aee60cad9c651d" - [[package]] name = "matchers" version = "0.1.0" @@ -2963,12 +2928,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "multimap" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "defc4c55412d89136f966bbb339008b474350e5e6e78d2714439c386b3137a03" - [[package]] name = "nom" version = "7.1.3" @@ -3106,7 +3065,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -3189,6 +3148,15 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" +[[package]] +name = "openssl-src" +version = "300.4.1+3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "faa4eac4138c62414b5622d1b31c5c304f34b406b013c079c2bbc652fdd6678c" +dependencies = [ + "cc", +] + [[package]] name = "openssl-sys" version = "0.9.104" @@ -3197,6 +3165,7 @@ checksum = "45abf306cbf99debc8195b66b7346498d7b10c210de50418b5ccd7ceba08c741" dependencies = [ "cc", "libc", + "openssl-src", "pkg-config", "vcpkg", ] @@ -3322,7 +3291,6 @@ dependencies = [ "base64 0.22.1", "byteorder", "bytes", - "bzip2 0.5.0", "cargo_toml", "chrono", "chrono-humanize", @@ -3335,8 +3303,6 @@ dependencies = [ "fs_extra", "futures", "futures-util", - "hashlru", - "hex", "hostname", "http 0.2.12", "http-auth-basic", @@ -3345,8 +3311,6 @@ dependencies = [ "humantime-serde", "itertools 0.13.0", "lazy_static", - "maplit", - "mime", "nom", "num_cpus", "object_store", @@ -3356,8 +3320,6 @@ dependencies = [ "path-clean", "prometheus", "prometheus-parse", - "prost", - "prost-build", "rand", "rdkafka", "regex", @@ -3375,7 +3337,6 @@ dependencies = [ "static-files", "sysinfo", "thiserror 2.0.9", - "thread-priority", "tokio", "tokio-stream", "tokio-util", @@ -3390,7 +3351,6 @@ dependencies = [ "url", "vergen", "xxhash-rust", - "xz2", "zip", ] @@ -3503,7 +3463,7 @@ checksum = "3c0f5fad0874fc7abcd4d750e76917eaebbecaa2c20bde22e1dbeeba8beb758c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -3539,16 +3499,6 @@ dependencies = [ "zerocopy", ] -[[package]] -name = "prettyplease" -version = "0.2.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64d1ec885c64d0457d564db4ec299b2dae3f9c02808b8ad9c3a089c591b18033" -dependencies = [ - "proc-macro2", - "syn 2.0.92", -] - [[package]] name = "proc-macro-crate" version = "3.2.0" @@ -3653,26 +3603,6 @@ dependencies = [ "prost-derive", ] -[[package]] -name = "prost-build" -version = "0.13.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0f3e5beed80eb580c68e2c600937ac2c4eedabdfd5ef1e5b7ea4f3fba84497b" -dependencies = [ - "heck", - "itertools 0.13.0", - "log", - "multimap", - "once_cell", - "petgraph", - "prettyplease", - "prost", - "prost-types", - "regex", - "syn 2.0.92", - "tempfile", -] - [[package]] name = "prost-derive" version = "0.13.4" @@ -3683,7 +3613,7 @@ dependencies = [ "itertools 0.13.0", "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -3719,9 +3649,9 @@ dependencies = [ [[package]] name = "quick-xml" -version = "0.37.1" +version = "0.37.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f22f29bdff3987b4d8632ef95fd6424ec7e4e0a57e2f4fc63e489e75357f6a03" +checksum = "165859e9e55f79d67b96c5d96f4e88b6f2695a1972849c15a6a3f5c59fc2c003" dependencies = [ "memchr", "serde", @@ -4078,7 +4008,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.92", + "syn 2.0.93", "unicode-ident", ] @@ -4330,7 +4260,7 @@ checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -4354,7 +4284,7 @@ checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -4506,7 +4436,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -4549,7 +4479,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -4600,7 +4530,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -4621,9 +4551,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.92" +version = "2.0.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70ae51629bf965c5c098cc9e87908a3df5301051a9e087d6f9bef5c9771ed126" +checksum = "9c786062daee0d6db1132800e623df74274a0a87322d8e183338e01b3d98d058" dependencies = [ "proc-macro2", "quote", @@ -4653,7 +4583,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -4730,7 +4660,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -4741,21 +4671,7 @@ checksum = "7b50fa271071aae2e6ee85f842e2e28ba8cd2c5fb67f11fcb1fd70b276f9e7d4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", -] - -[[package]] -name = "thread-priority" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cfe075d7053dae61ac5413a34ea7d4913b6e6207844fd726bdd858b37ff72bf5" -dependencies = [ - "bitflags 2.6.0", - "cfg-if", - "libc", - "log", - "rustversion", - "winapi", + "syn 2.0.93", ] [[package]] @@ -4872,7 +4788,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -5115,7 +5031,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -5330,7 +5246,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -5423,7 +5339,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", "wasm-bindgen-shared", ] @@ -5458,7 +5374,7 @@ checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -5597,7 +5513,7 @@ checksum = "9107ddc059d5b6fbfbffdfa7a7fe3e22a226def0b2608f72e9d552763d3e1ad7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -5608,7 +5524,7 @@ checksum = "29bee4b38ea3cde66011baa44dba677c432a78593e202392d1e9070cf2a7fca7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -5831,9 +5747,9 @@ checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" [[package]] name = "xxhash-rust" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7d48f1b18be023c95e7b75f481cac649d74be7c507ff4a407c55cfb957f7934" +checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" [[package]] name = "xz2" @@ -5864,7 +5780,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", "synstructure", ] @@ -5886,7 +5802,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] @@ -5906,7 +5822,7 @@ checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", "synstructure", ] @@ -5935,7 +5851,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.92", + "syn 2.0.93", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index 55545c62e..3c902e616 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -28,10 +28,9 @@ actix-web = { version = "4.9.0", features = ["rustls-0_22"] } actix-cors = "0.7.0" actix-web-prometheus = { version = "0.1" } actix-web-static-files = "4.0" -mime = "0.3.17" ### connectors dependencies -rdkafka = { version = "0.37", features = ["cmake-build", "tracing", "sasl", "ssl", "libz-static"] } +rdkafka = { version = "0.37", optional = true, features = ["cmake-build", "tracing", "libz-static"] } ### other dependencies anyhow = { version = "1.0", features = ["backtrace"] } @@ -41,7 +40,6 @@ base64 = "0.22.0" lazy_static = "1.4" bytes = "1.4" byteorder = "1.4.3" -bzip2 = { version = "*", features = ["static"] } cookie = "0.18.1" chrono = "0.4" chrono-humanize = "0.2" @@ -60,7 +58,6 @@ derive_more = "0.99.18" fs_extra = "1.3" futures = "0.3" futures-util = "0.3.28" -hex = "0.4" hostname = "0.4.0" http = "0.2.7" humantime-serde = "1.1" @@ -83,7 +80,6 @@ serde_json = "1.0" static-files = "0.2" sysinfo = "0.33.0" thiserror = "2.0.9" -thread-priority = "1.0.0" tokio = { version = "1.42", default-features = false, features = [ "sync", "macros", @@ -96,7 +92,6 @@ tracing-subscriber = { version = "0.3.18", features = ["env-filter", "time"] } ulid = { version = "1.0", features = ["serde"] } uptime_lib = "0.3.0" xxhash-rust = { version = "0.8", features = ["xxh3"] } -xz2 = { version = "*", features = ["static"] } nom = "7.1.3" humantime = "2.1.0" human-size = "0.4" @@ -104,9 +99,7 @@ openid = { version = "0.15.0", default-features = false, features = ["rustls"] } url = "2.4.0" http-auth-basic = "0.3.3" serde_repr = "0.1.17" -hashlru = { version = "0.11.0", features = ["serde"] } path-clean = "1.0.1" -prost = "0.13.3" prometheus-parse = "0.2.5" sha2 = "0.10.8" tracing = "0.1.41" @@ -119,10 +112,8 @@ ureq = "2.6" vergen = { version = "8.1", features = ["build", "git", "cargo", "gitcl"] } zip = { version = "2.2.0", default-features = false, features = ["deflate"] } url = "2.4.0" -prost-build = "0.13.3" [dev-dependencies] -maplit = "1.0" rstest = "0.23.0" arrow = "53.0.0" @@ -133,6 +124,10 @@ assets-sha1 = "9d5a45f204d709a2dd96f6a5e0b21b3834ee0e36" [features] debug = [] +rdkafka-ssl-vendored = ["rdkafka", "rdkafka/ssl-vendored"] +rdkafka-ssl = ["rdkafka", "rdkafka/ssl"] +rdkafka-sasl = ["rdkafka", "rdkafka/sasl"] + [profile.release-lto] inherits = "release" lto = "fat" diff --git a/Dockerfile b/Dockerfile index 361facc1e..6961f8f15 100644 --- a/Dockerfile +++ b/Dockerfile @@ -14,26 +14,16 @@ # along with this program. If not, see . # build stage -FROM rust:1.83.0-bookworm as builder +FROM rust:1.83.0-bookworm AS builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " LABEL org.opencontainers.image.vendor="Parseable Inc" LABEL org.opencontainers.image.licenses="AGPL-3.0" -RUN apt-get update && \ - apt-get install --no-install-recommends -y \ - cmake \ - clang \ - librdkafka-dev \ - ca-certificates \ - build-essential \ - libsasl2-dev \ - libssl-dev && \ - rm -rf /var/lib/apt/lists/* - WORKDIR /parseable COPY . . + RUN cargo build --release # final stage @@ -44,13 +34,4 @@ WORKDIR /parseable # Copy the Parseable binary from builder COPY --from=builder /parseable/target/release/parseable /usr/bin/parseable -# Copy only the libraries that binary needs since kafka is statically linked -COPY --from=builder /usr/lib/x86_64-linux-gnu/libsasl2.so.2 /usr/lib/x86_64-linux-gnu/ -COPY --from=builder /usr/lib/x86_64-linux-gnu/libssl.so.3 /usr/lib/x86_64-linux-gnu/ -COPY --from=builder /usr/lib/x86_64-linux-gnu/libcrypto.so.3 /usr/lib/x86_64-linux-gnu/ - -# Copy CA certificates -COPY --from=builder /etc/ssl/certs/ca-certificates.crt /etc/ssl/certs/ - - CMD ["/usr/bin/parseable"] diff --git a/Dockerfile.debug b/Dockerfile.debug index acbc44698..f76f21e30 100644 --- a/Dockerfile.debug +++ b/Dockerfile.debug @@ -14,7 +14,7 @@ # along with this program. If not, see . # build stage -FROM rust:1.77.1-bookworm as builder +FROM rust:1.83.0-bookworm AS builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " @@ -30,6 +30,7 @@ FROM docker.io/debian:bookworm-slim WORKDIR /parseable +# Copy the Parseable binary from builder COPY --from=builder /parseable/target/debug/parseable /usr/bin/parseable CMD ["/usr/bin/parseable"] diff --git a/Dockerfile.dev b/Dockerfile.dev index 5568dca12..a0bb5e674 100644 --- a/Dockerfile.dev +++ b/Dockerfile.dev @@ -14,7 +14,7 @@ # along with this program. If not, see . # build stage -FROM rust:1.77.1-bookworm AS builder +FROM rust:1.83.0-bookworm AS builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " @@ -34,7 +34,7 @@ FROM gcr.io/distroless/cc-debian12:latest WORKDIR /parseable -# Copy the static shell into base image. +# Copy the Parseable binary from builder COPY --from=builder /parseable/target/release/parseable /usr/bin/parseable CMD ["/usr/bin/parseable"] diff --git a/Dockerfile.kafka b/Dockerfile.kafka new file mode 100644 index 000000000..f7f0d7ce0 --- /dev/null +++ b/Dockerfile.kafka @@ -0,0 +1,58 @@ +# Parseable Server (C) 2022 - 2024 Parseable, Inc. +# +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as +# published by the Free Software Foundation, either version 3 of the +# License, or (at your option) any later version. +# +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. +# +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +# build stage +FROM rust:1.83.0-bookworm AS builder + +LABEL org.opencontainers.image.title="Parseable" +LABEL maintainer="Parseable Team " +LABEL org.opencontainers.image.vendor="Parseable Inc" +LABEL org.opencontainers.image.licenses="AGPL-3.0" + +RUN apt-get update && \ + apt-get install --no-install-recommends -y \ + cmake \ + clang \ + librdkafka-dev \ + ca-certificates \ + build-essential \ + libsasl2-dev \ + libssl-dev && \ + rm -rf /var/lib/apt/lists/* + +WORKDIR /parseable +COPY . . + +# Build with kafka feature +RUN cargo build --release --features "rdkafka-ssl,rdkafka-sasl" + +# final stage +FROM gcr.io/distroless/cc-debian12:latest + +WORKDIR /parseable + +# Copy the Parseable binary from builder +COPY --from=builder /parseable/target/release/parseable /usr/bin/parseable + +# Copy only the libraries that binary needs since kafka is statically linked +COPY --from=builder /usr/lib/x86_64-linux-gnu/libsasl2.so.2 /usr/lib/x86_64-linux-gnu/ +COPY --from=builder /usr/lib/x86_64-linux-gnu/libssl.so.3 /usr/lib/x86_64-linux-gnu/ +COPY --from=builder /usr/lib/x86_64-linux-gnu/libcrypto.so.3 /usr/lib/x86_64-linux-gnu/ + +# Copy CA certificates +COPY --from=builder /etc/ssl/certs/ca-certificates.crt /etc/ssl/certs/ + + +CMD ["/usr/bin/parseable"] diff --git a/docker-compose-distributed-test-with-kafka.yaml b/docker-compose-distributed-test-with-kafka.yaml new file mode 100644 index 000000000..062cde1d3 --- /dev/null +++ b/docker-compose-distributed-test-with-kafka.yaml @@ -0,0 +1,280 @@ +networks: + parseable-internal: + +services: + # minio + minio: + image: minio/minio:RELEASE.2023-02-10T18-48-39Z + entrypoint: + - sh + - -euc + - | + mkdir -p /tmp/minio/parseable && \ + minio server /tmp/minio + environment: + - MINIO_ROOT_USER=parseable + - MINIO_ROOT_PASSWORD=supersecret + - MINIO_UPDATE=off + ports: + - "9000:9000" + healthcheck: + test: [ "CMD", "curl", "-f", "http://localhost:9000/minio/health/live" ] + interval: 15s + timeout: 20s + retries: 5 + networks: + - parseable-internal + # query server + parseable-query: + build: + context: . + dockerfile: Dockerfile + platform: linux/amd64 + command: [ "parseable", "s3-store" ] + ports: + - "8000:8000" + environment: + - P_S3_URL=http://minio:9000 + - P_S3_ACCESS_KEY=parseable + - P_S3_SECRET_KEY=supersecret + - P_S3_REGION=us-east-1 + - P_S3_BUCKET=parseable + - P_STAGING_DIR=/tmp/data + - P_USERNAME=parseableadmin + - P_PASSWORD=parseableadmin + - P_CHECK_UPDATE=false + - P_PARQUET_COMPRESSION_ALGO=snappy + - P_MODE=query + networks: + - parseable-internal + healthcheck: + test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] + interval: 15s + timeout: 20s + retries: 5 + depends_on: + - minio + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + # ingest server one + parseable-ingest-one: + build: + context: . + dockerfile: Dockerfile.kafka + platform: linux/amd64 + command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + ports: + - "8000" + environment: + - P_S3_URL=http://minio:9000 + - P_S3_ACCESS_KEY=parseable + - P_S3_SECRET_KEY=supersecret + - P_S3_REGION=us-east-1 + - P_S3_BUCKET=parseable + - P_STAGING_DIR=/tmp/data + - P_USERNAME=parseableadmin + - P_PASSWORD=parseableadmin + - P_CHECK_UPDATE=false + - P_PARQUET_COMPRESSION_ALGO=snappy + - P_MODE=ingest + - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 + - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + # additional settings like security, tuning, etc. + networks: + - parseable-internal + healthcheck: + test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] + interval: 15s + timeout: 20s + retries: 5 + depends_on: + - parseable-query + - minio + - kafka-0 + - kafka-1 + - kafka-2 + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + quest: + platform: linux/amd64 + image: ghcr.io/parseablehq/quest:main + command: + [ + "load", + "http://parseable-query:8000", + "parseableadmin", + "parseableadmin", + "20", + "10", + "5m", + "minio:9000", + "parseable", + "supersecret", + "parseable", + "http://parseable-ingest-one:8000", + "parseableadmin", + "parseableadmin", + ] + networks: + - parseable-internal + depends_on: + - parseable-query + - parseable-ingest-one + - minio + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + kafka-0: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=0 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_0_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-1: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=1 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_1_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-2: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=2 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_2_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-ui: + platform: linux/amd64 + image: provectuslabs/kafka-ui:latest + environment: + KAFKA_CLUSTERS_0_NAME: dist-test + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 + KAFKA_CLUSTERS_0_METRICS_PORT: 9101 + DYNAMIC_CONFIG_ENABLED: "true" + JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC + networks: + - parseable-internal + depends_on: + - kafka-0 + - kafka-1 + - kafka-2 + ports: + - "8080:8080" + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + kafka-log-generator: + build: + context: ./scripts + dockerfile: Dockerfile + environment: + - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - KAFKA_TOPIC=dist-test-logs-stream + - LOG_RATE=1000 + - TOTAL_LOGS=1_000_000 + - REPLICATION_FACTOR=3 + depends_on: + - kafka-0 + - kafka-1 + - kafka-2 + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + +volumes: + kafka_0_data: + driver: local + kafka_1_data: + driver: local + kafka_2_data: + driver: local + diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index beafb566d..ca79941f3 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -65,7 +65,7 @@ services: context: . dockerfile: Dockerfile platform: linux/amd64 - command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + command: [ "parseable", "s3-store", ] ports: - "8000" environment: @@ -81,9 +81,6 @@ services: - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=ingest - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 - - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - # additional settings like security, tuning, etc. networks: - parseable-internal healthcheck: @@ -94,9 +91,6 @@ services: depends_on: - parseable-query - minio - - kafka-0 - - kafka-1 - - kafka-2 deploy: restart_policy: condition: on-failure @@ -134,147 +128,3 @@ services: condition: on-failure delay: 20s max_attempts: 3 - - kafka-0: - image: docker.io/bitnami/kafka:3.9 - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=0 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_0_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - - kafka-1: - image: docker.io/bitnami/kafka:3.9 - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=1 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_1_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - - kafka-2: - image: docker.io/bitnami/kafka:3.9 - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=2 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093,1@kafka-1:9093,2@kafka-2:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=3 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_2_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - - kafka-ui: - platform: linux/amd64 - image: provectuslabs/kafka-ui:latest - environment: - KAFKA_CLUSTERS_0_NAME: dist-test - KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092,kafka-1:9092,kafka-2:9092 - KAFKA_CLUSTERS_0_METRICS_PORT: 9101 - DYNAMIC_CONFIG_ENABLED: "true" - JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC - networks: - - parseable-internal - depends_on: - - kafka-0 - - kafka-1 - - kafka-2 - ports: - - "8080:8080" - deploy: - restart_policy: - condition: on-failure - delay: 20s - max_attempts: 3 - - kafka-log-generator: - build: - context: ./scripts - dockerfile: Dockerfile - environment: - - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - - KAFKA_TOPIC=dist-test-logs-stream - - LOG_RATE=500 - - TOTAL_LOGS=50000 - - REPLICATION_FACTOR=3 - depends_on: - - kafka-0 - - kafka-1 - - kafka-2 - networks: - - parseable-internal - deploy: - restart_policy: - condition: on-failure - delay: 20s - max_attempts: 3 - -volumes: - kafka_0_data: - driver: local - kafka_1_data: - driver: local - kafka_2_data: - driver: local - diff --git a/docker-compose-test-with-kafka.yaml b/docker-compose-test-with-kafka.yaml new file mode 100644 index 000000000..7a8ce9bbb --- /dev/null +++ b/docker-compose-test-with-kafka.yaml @@ -0,0 +1,164 @@ +networks: + parseable-internal: + +services: + minio: + image: minio/minio:RELEASE.2023-02-10T18-48-39Z + entrypoint: + - sh + - -euc + - | + mkdir -p /tmp/minio/parseable && \ + minio server /tmp/minio + environment: + - MINIO_ROOT_USER=parseable + - MINIO_ROOT_PASSWORD=supersecret + - MINIO_UPDATE=off + ports: + - "9000:9000" + healthcheck: + test: [ "CMD", "curl", "-f", "http://localhost:9000/minio/health/live" ] + interval: 15s + timeout: 20s + retries: 5 + networks: + - parseable-internal + + parseable: + build: + context: . + dockerfile: Dockerfile.kafka + platform: linux/amd64 + command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + ports: + - "8000:8000" + environment: + - P_S3_URL=http://minio:9000 + - P_S3_ACCESS_KEY=parseable + - P_S3_SECRET_KEY=supersecret + - P_S3_REGION=us-east-1 + - P_S3_BUCKET=parseable + - P_STAGING_DIR=/tmp/data + - P_USERNAME=parseableadmin + - P_PASSWORD=parseableadmin + - P_CHECK_UPDATE=false + - P_PARQUET_COMPRESSION_ALGO=snappy + - P_KAFKA_CONSUMER_TOPICS=test-logs-stream + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092 + # additional settings like security, tuning, etc. + depends_on: + - minio + - kafka-0 + healthcheck: + test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] + interval: 15s + timeout: 20s + retries: 5 + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + quest: + image: ghcr.io/parseablehq/quest:main + platform: linux/amd64 + command: [ + "load", + "http://parseable:8000", + "parseableadmin", + "parseableadmin", + "20", + "10", + "5m", + "minio:9000", + "parseable", + "supersecret", + "parseable" + ] + depends_on: + - parseable + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + kafka-0: + image: docker.io/bitnami/kafka:3.9 + ports: + - "9092:9092" + - "11001:11001" + environment: + # KRaft settings + - KAFKA_CFG_NODE_ID=0 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093 + - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv + # Listeners + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER + - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT + # Clustering + - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=1 + - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=1 + - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 + volumes: + - kafka_0_data:/bitnami/kafka + networks: + - parseable-internal + healthcheck: + test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] + interval: 10s + timeout: 5s + retries: 5 + + kafka-ui: + platform: linux/amd64 + image: provectuslabs/kafka-ui:latest + ports: + - "8080:8080" + depends_on: + - kafka-0 + environment: + KAFKA_CLUSTERS_0_NAME: test + KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 + KAFKA_CLUSTERS_0_METRICS_PORT: 11001 + DYNAMIC_CONFIG_ENABLED: "true" + JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + + kafka-log-generator: + build: + context: ./scripts + dockerfile: Dockerfile + environment: + - KAFKA_BROKERS=kafka-0:9092 + - KAFKA_TOPIC=test-logs-stream + - LOG_RATE=1000 + - TOTAL_LOGS=500_000 + depends_on: + - kafka-0 + networks: + - parseable-internal + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + +volumes: + kafka_0_data: + driver: local diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index 7ea445fba..0a81d3c8c 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -29,7 +29,7 @@ services: context: . dockerfile: Dockerfile platform: linux/amd64 - command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + command: [ "parseable", "s3-store", ] ports: - "8000:8000" environment: @@ -43,12 +43,8 @@ services: - P_PASSWORD=parseableadmin - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - - P_KAFKA_CONSUMER_TOPICS=test-logs-stream - - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092 - # additional settings like security, tuning, etc. depends_on: - minio - - kafka-0 healthcheck: test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] interval: 15s @@ -87,77 +83,3 @@ services: condition: on-failure delay: 20s max_attempts: 3 - - kafka-0: - image: docker.io/bitnami/kafka:3.9 - ports: - - "9092" - environment: - # KRaft settings - - KAFKA_CFG_NODE_ID=0 - - KAFKA_CFG_PROCESS_ROLES=controller,broker - - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka-0:9093 - - KAFKA_KRAFT_CLUSTER_ID=abcdefghijklmnopqrstuv - # Listeners - - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 - - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://:9092 - - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT - - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER - - KAFKA_CFG_INTER_BROKER_LISTENER_NAME=PLAINTEXT - # Clustering - - KAFKA_CFG_OFFSETS_TOPIC_REPLICATION_FACTOR=1 - - KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=1 - - KAFKA_CFG_TRANSACTION_STATE_LOG_MIN_ISR=2 - volumes: - - kafka_0_data:/bitnami/kafka - networks: - - parseable-internal - healthcheck: - test: [ "CMD-SHELL", "kafka-topics.sh --bootstrap-server localhost:9092 --list || exit 1" ] - interval: 10s - timeout: 5s - retries: 5 - - kafka-ui: - platform: linux/amd64 - image: provectuslabs/kafka-ui:latest - ports: - - "8080:8080" - depends_on: - - kafka-0 - environment: - KAFKA_CLUSTERS_0_NAME: test - KAFKA_CLUSTERS_0_BOOTSTRAPSERVERS: kafka-0:9092 - KAFKA_CLUSTERS_0_METRICS_PORT: 9101 - DYNAMIC_CONFIG_ENABLED: "true" - JAVA_OPTS: -Xms256m -Xmx512m -XX:+UseG1GC - networks: - - parseable-internal - deploy: - restart_policy: - condition: on-failure - delay: 20s - max_attempts: 3 - - kafka-log-generator: - build: - context: ./scripts - dockerfile: Dockerfile - environment: - - KAFKA_BROKERS=kafka-0:9092 - - KAFKA_TOPIC=test-logs-stream - - LOG_RATE=500 - - TOTAL_LOGS=50000 - depends_on: - - kafka-0 - networks: - - parseable-internal - deploy: - restart_policy: - condition: on-failure - delay: 20s - max_attempts: 3 - -volumes: - kafka_0_data: - driver: local diff --git a/src/cli.rs b/src/cli.rs index 1f1e9f4b6..1dec9136e 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -16,11 +16,17 @@ * */ +#[allow(unused_imports)] use clap::{value_parser, Arg, ArgGroup, Command, CommandFactory, FromArgMatches}; use std::path::PathBuf; use url::Url; +#[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" +))] use crate::connectors::common::config::ConnectorConfig; use crate::{ oidc::{self, OpenidConfig}, @@ -161,7 +167,7 @@ impl Cli { } pub fn create_cli_command_with_clap(name: &'static str) -> Command { - let mut command = Command::new(name) + let command = Command::new(name) .next_line_help(false) .arg( Arg::new(Self::TRINO_ENDPOINT) @@ -452,8 +458,20 @@ impl Cli { .multiple(true) ); - command = command.subcommand(ConnectorConfig::command()); + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + { + command.subcommand(ConnectorConfig::command()) + } + #[cfg(not(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + )))] command } } diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 5d8038cd5..924b7fead 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -76,6 +76,8 @@ async fn run_kafka2parseable

( where P: Processor, ()> + Send + Sync + 'static, { + info!("Initializing KafkaSink connector..."); + let kafka_config = Arc::new(config.clone()); let (kafka_context, rebalance_rx) = KafkaContext::new(kafka_config); diff --git a/src/lib.rs b/src/lib.rs index bff6e2114..2cc764c14 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -22,6 +22,11 @@ pub mod analytics; pub mod banner; mod catalog; mod cli; +#[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" +))] pub mod connectors; mod event; pub mod handlers; diff --git a/src/main.rs b/src/main.rs index f747ddc1c..28eeff45b 100644 --- a/src/main.rs +++ b/src/main.rs @@ -16,8 +16,14 @@ * */ +#[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" +))] +use parseable::connectors; use parseable::{ - banner, connectors, metrics, + banner, metrics, option::{Mode, CONFIG}, rbac, storage, IngestServer, ParseableServer, QueryServer, Server, }; @@ -47,11 +53,26 @@ async fn main() -> anyhow::Result<()> { metadata.set_global(); let prometheus = metrics::build_metrics_handler(); - let parseable_server = server.init(&prometheus); - let connectors_task = connectors::init(&prometheus); - tokio::try_join!(parseable_server, connectors_task)?; + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + { + let connectors_task = connectors::init(&prometheus); + tokio::try_join!(parseable_server, connectors_task)?; + } + + #[cfg(not(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + )))] + { + parseable_server.await?; + } Ok(()) } diff --git a/src/metadata.rs b/src/metadata.rs index 129defed8..71c126da3 100644 --- a/src/metadata.rs +++ b/src/metadata.rs @@ -165,6 +165,7 @@ impl StreamInfo { Ok(Arc::new(schema)) } + #[allow(dead_code)] pub fn schema_raw( &self, stream_name: &str, diff --git a/src/option.rs b/src/option.rs index 57e8db65d..399a5f1f5 100644 --- a/src/option.rs +++ b/src/option.rs @@ -17,6 +17,11 @@ */ use crate::cli::Cli; +#[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" +))] use crate::connectors::common::config::ConnectorConfig; use crate::storage::object_storage::parseable_json_path; use crate::storage::{ @@ -43,6 +48,11 @@ pub struct Config { pub parseable: Cli, storage: Arc, pub storage_name: &'static str, + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] pub connector_config: Option, } @@ -101,6 +111,11 @@ parseable [command] --help parseable: cli, storage: Arc::new(storage), storage_name: "drive", + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] connector_config: ConnectorConfig::from(m), } } @@ -118,6 +133,11 @@ parseable [command] --help parseable: cli, storage: Arc::new(storage), storage_name: "s3", + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] connector_config: ConnectorConfig::from(m), } } @@ -135,6 +155,11 @@ parseable [command] --help parseable: cli, storage: Arc::new(storage), storage_name: "blob_store", + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] connector_config: ConnectorConfig::from(m), } } From aade3a8ce35410817f4638411ec3a9fd439fcf40 Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 31 Dec 2024 19:50:06 +0100 Subject: [PATCH 29/65] fix rust fmt --- src/connectors/kafka/rebalance_listener.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/connectors/kafka/rebalance_listener.rs b/src/connectors/kafka/rebalance_listener.rs index 1b1c40890..fe7d40278 100644 --- a/src/connectors/kafka/rebalance_listener.rs +++ b/src/connectors/kafka/rebalance_listener.rs @@ -64,7 +64,6 @@ impl RebalanceListener { } else { warn!("Stream state lock is busy, skipping rebalance revoke for {:?}", tpl); } - if let Err(err) = callback.send(()) { warn!("Error during sending response to context. Cause: {:?}", err); } From 9ba40b53144b9e482551a9f9d7626c91d724b05f Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 2 Jan 2025 08:58:49 +0100 Subject: [PATCH 30/65] Use dedicated runtime for KafkaSinkConnector to ensure true parallelism. Fix thread parallelism issue where tasks were running on main thread due to actix-tokio-runtime being single-threaded. --- src/connectors/common/mod.rs | 12 ++++++++++ src/connectors/kafka/config.rs | 12 ++++++++++ src/connectors/kafka/sink.rs | 42 ++++++++++++++++++++++------------ 3 files changed, 51 insertions(+), 15 deletions(-) diff --git a/src/connectors/common/mod.rs b/src/connectors/common/mod.rs index 3401f7937..f44e2a6a7 100644 --- a/src/connectors/common/mod.rs +++ b/src/connectors/common/mod.rs @@ -17,6 +17,8 @@ */ use rdkafka::error::{KafkaError, RDKafkaErrorCode}; use thiserror::Error; +use tokio::runtime; +use tokio::runtime::Builder; pub mod config; pub mod processor; @@ -74,3 +76,13 @@ impl ConnectorError { ) } } + +pub fn build_runtime(worker_threads: usize, thread_name: &str) -> anyhow::Result { + Builder::new_multi_thread() + .enable_all() + .thread_name(thread_name) + .worker_threads(worker_threads) + .max_blocking_threads(worker_threads) + .build() + .map_err(|e| anyhow::anyhow!(e)) +} diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index c95777208..c48d61ebb 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -44,6 +44,16 @@ pub struct KafkaConfig { )] pub client_id: String, + #[arg( + long = "partition-listener-concurrency", + env = "P_KAFKA_PARTITION_LISTENER_CONCURRENCY", + value_name = "concurrency", + required = false, + default_value_t = 1, + help = "Number of parallel threads for Kafka partition listeners. Each partition gets processed on a dedicated thread." + )] + pub partition_listener_concurrency: usize, + #[command(flatten)] #[group(id = "consumer", required = false)] pub consumer: Option, @@ -846,6 +856,8 @@ impl Default for KafkaConfig { // Common configuration with standard broker port bootstrap_servers: "localhost:9092".to_string(), client_id: "parseable-connect".to_string(), + // Single threaded listener for all assigned partitions + partition_listener_concurrency: 1, // Component-specific configurations with production-ready defaults consumer: Some(ConsumerConfig::default()), producer: Some(ProducerConfig::default()), diff --git a/src/connectors/kafka/sink.rs b/src/connectors/kafka/sink.rs index d1c02b71c..447955686 100644 --- a/src/connectors/kafka/sink.rs +++ b/src/connectors/kafka/sink.rs @@ -15,22 +15,25 @@ * along with this program. If not, see . * */ - +use crate::connectors::common::build_runtime; use crate::connectors::common::processor::Processor; use crate::connectors::kafka::consumer::KafkaStreams; use crate::connectors::kafka::processor::StreamWorker; use crate::connectors::kafka::ConsumerRecord; use anyhow::Result; use futures_util::StreamExt; +use rdkafka::consumer::Consumer; use std::sync::Arc; +use tokio::runtime::Runtime; use tracing::{error, info}; pub struct KafkaSinkConnector

where P: Processor, ()>, { - kafka_streams: KafkaStreams, - worker: Arc>, + streams: KafkaStreams, + stream_processor: Arc>, + runtime: Runtime, } impl

KafkaSinkConnector

@@ -38,30 +41,39 @@ where P: Processor, ()> + Send + Sync + 'static, { pub fn new(kafka_streams: KafkaStreams, processor: P) -> Self { - let worker = Arc::new(StreamWorker::new( + let consumer = kafka_streams.consumer(); + let stream_processor = Arc::new(StreamWorker::new( Arc::new(processor), - kafka_streams.consumer(), + Arc::clone(&consumer), )); + let runtime = build_runtime( + consumer.context().config.partition_listener_concurrency, + "kafka-sink-worker", + ) + .expect("Failed to build runtime"); + let _ = runtime.enter(); + Self { - kafka_streams, - worker, + streams: kafka_streams, + stream_processor, + runtime, } } pub async fn run(self) -> Result<()> { - self.kafka_streams + self.streams .partitioned() - .map(|partition_queue| { - let worker = Arc::clone(&self.worker); - let tp = partition_queue.topic_partition().clone(); - tokio::spawn(async move { - partition_queue - .run_drain(|record_stream| async { + .map(|partition_stream| { + let worker = Arc::clone(&self.stream_processor); + let tp = partition_stream.topic_partition().clone(); + self.runtime.spawn(async move { + partition_stream + .run_drain(|partition_records| async { info!("Starting task for partition: {:?}", tp); worker - .process_partition(tp.clone(), record_stream) + .process_partition(tp.clone(), partition_records) .await .unwrap(); }) From 5c67134d2b079675f11ca59a02b30d7586a0c706 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 4 Jan 2025 01:36:09 +0100 Subject: [PATCH 31/65] add schema version when deserialize ParseableEvent --- Cargo.lock | 99 ++++++++++++++++--------------- src/connectors/kafka/processor.rs | 5 +- 2 files changed, 53 insertions(+), 51 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a0d9dbd8d..bd4a66389 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -81,7 +81,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e01ed3140b2f8d422c68afa1ed2e85d996ea619c988ac834d255db32138655cb" dependencies = [ "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -218,7 +218,7 @@ dependencies = [ "actix-router", "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -707,18 +707,18 @@ checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] name = "async-trait" -version = "0.1.83" +version = "0.1.84" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" +checksum = "1b1244b10dcd56c92219da4e14caa97e312079e185f04ba3eea25061561dc0a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -1005,9 +1005,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.6" +version = "1.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d6dbb628b8f8555f86d0323c2eb39e3ec81901f4b83e091db8a6a76d316a333" +checksum = "a012a0df96dd6d06ba9a1b29d6402d1a5d77c6befd2566afdc26e10603dc93d7" dependencies = [ "jobserver", "libc", @@ -1111,7 +1111,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -1360,7 +1360,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -1371,7 +1371,7 @@ checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -1800,7 +1800,7 @@ checksum = "30542c1ad912e0e3d22a1935c290e12e8a29d704a420177a31faad4a601a0800" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -1813,7 +1813,7 @@ dependencies = [ "proc-macro2", "quote", "rustc_version", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -1835,7 +1835,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -2014,7 +2014,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -2535,7 +2535,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -3065,7 +3065,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -3106,7 +3106,7 @@ dependencies = [ "percent-encoding", "quick-xml", "rand", - "reqwest 0.12.11", + "reqwest 0.12.12", "ring", "serde", "serde_json", @@ -3134,7 +3134,7 @@ dependencies = [ "chrono", "lazy_static", "mime", - "reqwest 0.12.11", + "reqwest 0.12.12", "serde", "serde_json", "thiserror 1.0.69", @@ -3463,7 +3463,7 @@ checksum = "3c0f5fad0874fc7abcd4d750e76917eaebbecaa2c20bde22e1dbeeba8beb758c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -3613,7 +3613,7 @@ dependencies = [ "itertools 0.13.0", "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -3922,9 +3922,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.11" +version = "0.12.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fe060fe50f524be480214aba758c71f99f90ee8c83c5a36b5e9e1d568eb4eb3" +checksum = "43e734407157c3c2034e0258f5e4473ddb361b1e85f95a66690d67264d7cd1da" dependencies = [ "base64 0.22.1", "bytes", @@ -4008,7 +4008,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.93", + "syn 2.0.94", "unicode-ident", ] @@ -4260,7 +4260,7 @@ checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4284,7 +4284,7 @@ checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4436,7 +4436,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4479,7 +4479,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4530,7 +4530,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4551,9 +4551,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.93" +version = "2.0.94" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c786062daee0d6db1132800e623df74274a0a87322d8e183338e01b3d98d058" +checksum = "987bc0be1cdea8b10216bd06e2ca407d40b9543468fafd3ddfb02f36e77f71f3" dependencies = [ "proc-macro2", "quote", @@ -4583,7 +4583,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4623,12 +4623,13 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.14.0" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28cce251fcbc87fac86a866eeb0d6c2d536fc16d06f184bb61aeae11aa4cee0c" +checksum = "9a8a559c81686f576e8cd0290cd2a24a2a9ad80c98b3478856500fcbd7acd704" dependencies = [ "cfg-if", "fastrand 2.3.0", + "getrandom", "once_cell", "rustix", "windows-sys 0.59.0", @@ -4660,7 +4661,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4671,7 +4672,7 @@ checksum = "7b50fa271071aae2e6ee85f842e2e28ba8cd2c5fb67f11fcb1fd70b276f9e7d4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -4788,7 +4789,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -5031,7 +5032,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -5246,7 +5247,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -5339,7 +5340,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", "wasm-bindgen-shared", ] @@ -5374,7 +5375,7 @@ checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -5513,7 +5514,7 @@ checksum = "9107ddc059d5b6fbfbffdfa7a7fe3e22a226def0b2608f72e9d552763d3e1ad7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -5524,7 +5525,7 @@ checksum = "29bee4b38ea3cde66011baa44dba677c432a78593e202392d1e9070cf2a7fca7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -5716,9 +5717,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" -version = "0.6.20" +version = "0.6.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36c1fec1a2bb5866f07c25f68c26e565c4c200aebb96d7e55710c19d3e8ac49b" +checksum = "39281189af81c07ec09db316b302a3e67bf9bd7cbf6c820b50e35fee9c2fa980" dependencies = [ "memchr", ] @@ -5780,7 +5781,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", "synstructure", ] @@ -5802,7 +5803,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] @@ -5822,7 +5823,7 @@ checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", "synstructure", ] @@ -5851,7 +5852,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.93", + "syn 2.0.94", ] [[package]] diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index d4ed999c0..15b0210b3 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -22,7 +22,7 @@ use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; use crate::event::format; use crate::event::format::EventFormat; use crate::handlers::http::ingest::create_stream_if_not_exists; -use crate::metadata::STREAM_INFO; +use crate::metadata::{SchemaVersion, STREAM_INFO}; use crate::storage::StreamType; use async_trait::async_trait; use chrono::Utc; @@ -66,7 +66,8 @@ impl ParseableSinkProcessor { }; // TODO: Implement a buffer (e.g., a wrapper around [Box]) to optimize the creation of ParseableEvent by compacting the internal RecordBatch. - let (record_batch, is_first) = event.into_recordbatch(&schema, None, None)?; + let (record_batch, is_first) = + event.into_recordbatch(&schema, None, None, SchemaVersion::V1)?; let p_event = crate::event::Event { rb: record_batch, From ce2fca0476316187ee5d10d5f5d2dc07387f7e78 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 4 Jan 2025 01:39:56 +0100 Subject: [PATCH 32/65] rename Event as ParseableEvent --- src/connectors/kafka/processor.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index 15b0210b3..975a1b4c4 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -21,6 +21,7 @@ use crate::connectors::kafka::config::BufferConfig; use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; use crate::event::format; use crate::event::format::EventFormat; +use crate::event::Event as ParseableEvent; use crate::handlers::http::ingest::create_stream_if_not_exists; use crate::metadata::{SchemaVersion, STREAM_INFO}; use crate::storage::StreamType; @@ -41,7 +42,7 @@ impl ParseableSinkProcessor { async fn deserialize( &self, consumer_record: &ConsumerRecord, - ) -> anyhow::Result> { + ) -> anyhow::Result> { let stream_name = consumer_record.topic.as_str(); create_stream_if_not_exists(stream_name, &StreamType::UserDefined.to_string()).await?; @@ -69,7 +70,7 @@ impl ParseableSinkProcessor { let (record_batch, is_first) = event.into_recordbatch(&schema, None, None, SchemaVersion::V1)?; - let p_event = crate::event::Event { + let p_event = ParseableEvent { rb: record_batch, stream_name: stream_name.to_string(), origin_format: "json", From 835e9b9fd6a9104b1e0df338120f328e37d7a20f Mon Sep 17 00:00:00 2001 From: hippalus Date: Sat, 4 Jan 2025 01:51:38 +0100 Subject: [PATCH 33/65] -v flag to clean up volumes when stopping containers. Remove the error suppression in favor of proper error handling. --- .github/workflows/coverage.yaml | 14 ++++++++++---- .github/workflows/integration-test.yaml | 16 ++++++++-------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/.github/workflows/coverage.yaml b/.github/workflows/coverage.yaml index 495e68902..988b83a27 100644 --- a/.github/workflows/coverage.yaml +++ b/.github/workflows/coverage.yaml @@ -14,7 +14,7 @@ jobs: coverage: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - uses: dtolnay/rust-toolchain@stable with: components: clippy @@ -27,9 +27,15 @@ jobs: with: tool: cargo-hack, cargo-llvm-cov, nextest - - name: Install dependencies - run: sudo apt-get install -y libsasl2-dev || echo "Skipping libsasl2-dev installation" - continue-on-error: true + - name: Install System Dependencies + run: | + sudo apt-get update + sudo apt-get install -y \ + libsasl2-dev \ + libssl-dev \ + pkg-config \ + build-essential + if: runner.os == 'Linux' - name: Check with clippy run: cargo hack clippy --verbose --each-feature --no-dev-deps -- -D warnings diff --git a/.github/workflows/integration-test.yaml b/.github/workflows/integration-test.yaml index a68b0e6e6..cb8df920e 100644 --- a/.github/workflows/integration-test.yaml +++ b/.github/workflows/integration-test.yaml @@ -14,45 +14,45 @@ jobs: runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Start compose run: docker compose -f docker-compose-test.yaml up --build --exit-code-from quest - name: Stop compose if: always() - run: docker compose -f docker-compose-test.yaml down + run: docker compose -f docker-compose-test.yaml down -v docker-compose-distributed-test: name: Quest Smoke and Load Tests for Distributed deployments runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Start compose run: docker compose -f docker-compose-distributed-test.yaml up --build --exit-code-from quest - name: Stop compose if: always() - run: docker compose -f docker-compose-distributed-test.yaml down + run: docker compose -f docker-compose-distributed-test.yaml down -v docker-compose-test-with-kafka: name: Quest Smoke and Load Tests for Standalone deployments with Kafka runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Start compose run: docker compose -f docker-compose-test-with-kafka.yaml up --build --exit-code-from quest - name: Stop compose if: always() - run: docker compose -f docker-compose-test-with-kafka.yaml down + run: docker compose -f docker-compose-test-with-kafka.yaml down -v docker-compose-distributed-test-with-kafka: name: Quest Smoke and Load Tests for Distributed deployments with Kafka runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Start compose run: docker compose -f docker-compose-distributed-test-with-kafka.yaml up --build --exit-code-from quest - name: Stop compose if: always() - run: docker compose -f docker-compose-distributed-test-with-kafka.yaml down + run: docker compose -f docker-compose-distributed-test-with-kafka.yaml down -v From bb12d043e892724ade95318452c2d0c308e30f7a Mon Sep 17 00:00:00 2001 From: hippalus Date: Mon, 6 Jan 2025 11:43:53 +0100 Subject: [PATCH 34/65] fix ParseableSinkProcessor.deserialize --- Cargo.lock | 1 + src/connectors/kafka/processor.rs | 25 ++++++++++++++----------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 122da09b6..6c1982012 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3334,6 +3334,7 @@ dependencies = [ "base64 0.22.1", "byteorder", "bytes", + "bzip2", "cargo_toml", "chrono", "chrono-humanize", diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index 975a1b4c4..644b15833 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -20,10 +20,10 @@ use crate::connectors::common::processor::Processor; use crate::connectors::kafka::config::BufferConfig; use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; use crate::event::format; -use crate::event::format::EventFormat; +use crate::event::format::{EventFormat, LogSource}; use crate::event::Event as ParseableEvent; use crate::handlers::http::ingest::create_stream_if_not_exists; -use crate::metadata::{SchemaVersion, STREAM_INFO}; +use crate::metadata::STREAM_INFO; use crate::storage::StreamType; use async_trait::async_trait; use chrono::Utc; @@ -59,19 +59,22 @@ impl ParseableSinkProcessor { } Some(payload) => { let data: Value = serde_json::from_slice(payload.as_ref())?; + let json_event = format::json::Event { data }; - let event = format::json::Event { - data, - tags: String::default(), - metadata: String::default(), - }; + let time_partition = STREAM_INFO.get_time_partition(stream_name)?; + let static_schema_flag = STREAM_INFO.get_static_schema_flag(stream_name)?; + let schema_version = STREAM_INFO.get_schema_version(stream_name)?; - // TODO: Implement a buffer (e.g., a wrapper around [Box]) to optimize the creation of ParseableEvent by compacting the internal RecordBatch. - let (record_batch, is_first) = - event.into_recordbatch(&schema, None, None, SchemaVersion::V1)?; + let (rb, is_first) = json_event.into_recordbatch( + &schema, + static_schema_flag.as_ref(), + time_partition.as_ref(), + schema_version, + &LogSource::Json, + )?; let p_event = ParseableEvent { - rb: record_batch, + rb, stream_name: stream_name.to_string(), origin_format: "json", origin_size: payload.len() as u64, From 736af65c029614556393c9c82f1e64393f32d3bc Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 7 Jan 2025 10:06:35 +0100 Subject: [PATCH 35/65] fix ParseableSinkProcessor.deserialize --- src/connectors/kafka/processor.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index 644b15833..4f4474439 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -20,7 +20,7 @@ use crate::connectors::common::processor::Processor; use crate::connectors::kafka::config::BufferConfig; use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; use crate::event::format; -use crate::event::format::{EventFormat, LogSource}; +use crate::event::format::EventFormat; use crate::event::Event as ParseableEvent; use crate::handlers::http::ingest::create_stream_if_not_exists; use crate::metadata::STREAM_INFO; @@ -70,7 +70,6 @@ impl ParseableSinkProcessor { static_schema_flag.as_ref(), time_partition.as_ref(), schema_version, - &LogSource::Json, )?; let p_event = ParseableEvent { From f1d26069230ebccfa602efd616a0ee0c9eac9f4c Mon Sep 17 00:00:00 2001 From: hippalus Date: Sun, 12 Jan 2025 02:01:45 +0100 Subject: [PATCH 36/65] update build.rs --- build.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.rs b/build.rs index 0cd638eea..e73bf7082 100644 --- a/build.rs +++ b/build.rs @@ -28,7 +28,7 @@ pub fn main() -> Result<()> { Emitter::default() .add_instructions(&BuildBuilder::all_build()?)? .add_instructions(&CargoBuilder::all_cargo()?)? - .add_instructions(&GitclBuilder::all_git()?)? + .add_instructions(&GitclBuilder::default().all().sha(true).build()?)? .add_instructions(&RustcBuilder::all_rustc()?)? .add_instructions(&SysinfoBuilder::all_sysinfo()?)? .emit()?; From e399f0b6a02a53f3b996ee6c643365dae30bd4df Mon Sep 17 00:00:00 2001 From: hippalus Date: Sun, 12 Jan 2025 04:51:29 +0100 Subject: [PATCH 37/65] fix comment --- Cargo.toml | 2 +- src/connectors/mod.rs | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 1605398a9..66f22322c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -35,7 +35,7 @@ url = "2.4.0" # Connectors dependencies rdkafka = { version = "0.37", optional = true, features = ["cmake-build", "tracing", "libz-static"] } -### other dependencies +# Authentication and Security argon2 = "0.5.0" base64 = "0.22.0" cookie = "0.18.1" diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 924b7fead..12b5aa6b5 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -40,6 +40,9 @@ pub mod kafka; pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { if matches!(CONFIG.parseable.mode, Mode::Ingest | Mode::All) { match CONFIG.connector_config.clone() { + None => { + warn!("Kafka connector configuration is missing. Skipping Kafka pipeline."); + } Some(connectors) => { let shutdown_handle = Shutdown::default(); let registry = prometheus.registry.clone(); @@ -52,15 +55,13 @@ pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { info!("Connector received shutdown signal!"); } }); + match connectors.connectors { Connectors::KafkaSink(config) => { run_kafka2parseable(config, registry, processor, shutdown_handle).await? } } } - None => { - warn!("Kafka connector configuration is missing. Skipping Kafka pipeline."); - } } } From d5256fdfce134429cde075601ed5e63a6d60b045 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sun, 12 Jan 2025 04:52:29 +0100 Subject: [PATCH 38/65] fmt --- src/connectors/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 12b5aa6b5..146ad6d75 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -55,7 +55,7 @@ pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { info!("Connector received shutdown signal!"); } }); - + match connectors.connectors { Connectors::KafkaSink(config) => { run_kafka2parseable(config, registry, processor, shutdown_handle).await? From 7e878e2632f7dfd5511f6b55b109181254f4c7c1 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sun, 12 Jan 2025 06:11:31 +0100 Subject: [PATCH 39/65] kafka-streams-thread name --- src/connectors/kafka/consumer.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connectors/kafka/consumer.rs b/src/connectors/kafka/consumer.rs index d4783e4a8..3bfc01702 100644 --- a/src/connectors/kafka/consumer.rs +++ b/src/connectors/kafka/consumer.rs @@ -106,7 +106,7 @@ impl KafkaStreams { let tokio_handle = tokio::runtime::Handle::current(); let shutdown_handle = self.shutdown_handle.clone(); - std::thread::spawn(move || { + std::thread::Builder::new().name("kafka-streams-thread".to_string()).spawn(move || { tokio_handle.block_on(async move { loop { let result: Result<(), ConnectorError> = tokio::select! { @@ -146,7 +146,7 @@ impl KafkaStreams { info!("Kafka stream processing terminated"); }); - }); + }).expect("Failed to spawn Kafka partitioned stream thread"); ReceiverStream::new(stream_rx) } From 595baaa773942ad5d98c6c78cb398077b3ae37eb Mon Sep 17 00:00:00 2001 From: hippalus Date: Sun, 12 Jan 2025 09:16:24 +0100 Subject: [PATCH 40/65] Introduced haproxy to support Kafka consumer rebalance testing and distributed ingestion with multiple servers. --- ...r-compose-distributed-test-with-kafka.yaml | 96 ++++++++++++++++++- parseable-ingest-haproxy.cfg | 45 +++++++++ 2 files changed, 136 insertions(+), 5 deletions(-) create mode 100644 parseable-ingest-haproxy.cfg diff --git a/docker-compose-distributed-test-with-kafka.yaml b/docker-compose-distributed-test-with-kafka.yaml index 062cde1d3..627d950f2 100644 --- a/docker-compose-distributed-test-with-kafka.yaml +++ b/docker-compose-distributed-test-with-kafka.yaml @@ -2,6 +2,30 @@ networks: parseable-internal: services: + # HAProxy Load Balancer + parseable-ingest-haproxy: + image: haproxy:3.0.7-alpine3.21 + ports: + - "9001:9001" # HAProxy stats + - "8001:8001" # Load balanced ingestion endpoint + volumes: + - ./parseable-ingest-haproxy.cfg:/usr/local/etc/haproxy/haproxy.cfg:ro + depends_on: + - parseable-ingest-one + - parseable-ingest-two + networks: + - parseable-internal + healthcheck: + test: [ "CMD", "haproxy", "-c", "-f", "/usr/local/etc/haproxy/haproxy.cfg" ] + interval: 15s + timeout: 10s + retries: 3 + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + # minio minio: image: minio/minio:RELEASE.2023-02-10T18-48-39Z @@ -24,6 +48,9 @@ services: retries: 5 networks: - parseable-internal + volumes: + - minio_data:/tmp/minio + # query server parseable-query: build: @@ -59,6 +86,9 @@ services: condition: on-failure delay: 20s max_attempts: 3 + volumes: + - parseable_query_data:/tmp/data + # ingest server one parseable-ingest-one: build: @@ -66,7 +96,7 @@ services: dockerfile: Dockerfile.kafka platform: linux/amd64 command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] - ports: + expose: - "8000" environment: - P_S3_URL=http://minio:9000 @@ -83,6 +113,53 @@ services: - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - P_KAFKA_PARTITION_LISTENER_CONCURRENCY=3 + # additional settings like security, tuning, etc. + networks: + - parseable-internal + healthcheck: + test: [ "CMD", "curl", "-f", "http://localhost:8000/api/v1/liveness" ] + interval: 15s + timeout: 20s + retries: 5 + depends_on: + - parseable-query + - minio + - kafka-0 + - kafka-1 + - kafka-2 + deploy: + restart_policy: + condition: on-failure + delay: 20s + max_attempts: 3 + volumes: + - parseable_ingest_one_data:/tmp/data + + parseable-ingest-two: + build: + context: . + dockerfile: Dockerfile.kafka + platform: linux/amd64 + command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + expose: + - "8000" + environment: + - P_S3_URL=http://minio:9000 + - P_S3_ACCESS_KEY=parseable + - P_S3_SECRET_KEY=supersecret + - P_S3_REGION=us-east-1 + - P_S3_BUCKET=parseable + - P_STAGING_DIR=/tmp/data + - P_USERNAME=parseableadmin + - P_PASSWORD=parseableadmin + - P_CHECK_UPDATE=false + - P_PARQUET_COMPRESSION_ALGO=snappy + - P_MODE=ingest + - P_INGESTOR_ENDPOINT=parseable-ingest-two:8000 + - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream + - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 + - P_KAFKA_PARTITION_LISTENER_CONCURRENCY=3 # additional settings like security, tuning, etc. networks: - parseable-internal @@ -102,7 +179,9 @@ services: condition: on-failure delay: 20s max_attempts: 3 - + volumes: + - parseable_ingest_two_data:/tmp/data + quest: platform: linux/amd64 image: ghcr.io/parseablehq/quest:main @@ -119,7 +198,7 @@ services: "parseable", "supersecret", "parseable", - "http://parseable-ingest-one:8000", + "http://parseable-ingest-haproxy:8001", "parseableadmin", "parseableadmin", ] @@ -127,7 +206,7 @@ services: - parseable-internal depends_on: - parseable-query - - parseable-ingest-one + - parseable-ingest-haproxy - minio deploy: restart_policy: @@ -271,10 +350,17 @@ services: max_attempts: 3 volumes: + minio_data: + driver: local + parseable_query_data: + driver: local + parseable_ingest_one_data: + driver: local + parseable_ingest_two_data: + driver: local kafka_0_data: driver: local kafka_1_data: driver: local kafka_2_data: driver: local - diff --git a/parseable-ingest-haproxy.cfg b/parseable-ingest-haproxy.cfg new file mode 100644 index 000000000..073150996 --- /dev/null +++ b/parseable-ingest-haproxy.cfg @@ -0,0 +1,45 @@ +global + log stdout format raw local0 + maxconn 60000 + daemon + +defaults + log global + mode http + option httplog + option dontlognull + timeout connect 5000 + timeout client 50000 + timeout server 50000 + +frontend stats + bind *:9001 + stats enable + stats uri / + stats refresh 30s + stats admin if TRUE + +frontend ingestion_frontend + bind *:8001 + mode http + default_backend ingestion_backend + +backend ingestion_backend + mode http + balance roundrobin + option forwardfor + + # Health check configuration + option httpchk GET /api/v1/liveness + http-check expect status 200 + + # Backend servers + server ingest1 parseable-ingest-one:8000 check inter 5s rise 2 fall 3 + server ingest2 parseable-ingest-two:8000 check inter 5s rise 2 fall 3 + + # Session persistence + hash-type consistent + + # Retry configuration + retries 3 + option redispatch From 7eddf03dd62c50f5d2aef97c4f618a8593fbdcc8 Mon Sep 17 00:00:00 2001 From: hippalus Date: Sun, 12 Jan 2025 23:02:09 +0100 Subject: [PATCH 41/65] Update log generator to send kube logs to kafka --- scripts/Dockerfile | 1 + scripts/kafka_log_stream_generator.py | 162 ++++++++++++++------------ src/kafka.rs | 0 3 files changed, 91 insertions(+), 72 deletions(-) delete mode 100644 src/kafka.rs diff --git a/scripts/Dockerfile b/scripts/Dockerfile index 0ad2d62a1..8a0a9d27d 100644 --- a/scripts/Dockerfile +++ b/scripts/Dockerfile @@ -21,6 +21,7 @@ RUN apt-get update && apt-get install -y --no-install-recommends \ && rm -rf /var/lib/apt/lists/* RUN pip install confluent-kafka +RUN pip install faker WORKDIR /app COPY kafka_log_stream_generator.py /app/ diff --git a/scripts/kafka_log_stream_generator.py b/scripts/kafka_log_stream_generator.py index 55bb442c5..ab6cf89ae 100644 --- a/scripts/kafka_log_stream_generator.py +++ b/scripts/kafka_log_stream_generator.py @@ -13,29 +13,31 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . +import json +import logging import os +import random import sys import time -import json -import logging +import uuid from datetime import datetime, timezone -from random import choice, randint -from uuid import uuid4 +from typing import Dict, Any from confluent_kafka import Producer from confluent_kafka.admin import AdminClient from confluent_kafka.cimpl import NewTopic +from faker import Faker logging.basicConfig( level=logging.INFO, format="%(asctime)s [%(levelname)s] %(message)s", - handlers=[ - logging.StreamHandler(sys.stdout) # Log to stdout - ] + handlers=[logging.StreamHandler(sys.stdout)] ) logger = logging.getLogger(__name__) +fake = Faker() +# Kafka Configuration KAFKA_BROKERS = os.getenv("KAFKA_BROKERS", "localhost:9092") KAFKA_TOPIC = os.getenv("KAFKA_TOPIC", "local-logs-stream") NUM_PARTITIONS = int(os.getenv("NUM_PARTITIONS", "6")) # Default partitions @@ -44,6 +46,17 @@ LOG_RATE = int(os.getenv("LOG_RATE", "50")) # Logs per second REPORT_EVERY = 5_000 # Progress report frequency +# Kubernetes Configuration +K8S_NAMESPACES = ["default", "kube-system", "monitoring", "logging", "app"] +CONTAINER_IMAGES = [ + "parseable/parseable:v1.8.1", + "parseable/query-service:v1.8.1", + "parseable/ingester:v1.8.1", + "parseable/frontend:v1.8.1" +] +NODE_TYPES = ["compute", "storage", "ingestion"] +COMPONENTS = ["query", "storage", "ingestion", "frontend"] + producer_conf = { "bootstrap.servers": KAFKA_BROKERS, "queue.buffering.max.messages": 200_000, @@ -59,19 +72,76 @@ admin_client = AdminClient({"bootstrap.servers": KAFKA_BROKERS}) producer = Producer(producer_conf) -LOG_TEMPLATE = { - "timestamp": "", - "correlation_id": "", - "level": "INFO", - "message": "", - "pod": {"name": "", "namespace": "", "node": ""}, - "request": {"method": "", "path": "", "remote_address": ""}, - "response": {"status_code": 200, "latency_ms": 0}, - "metadata": {"container_id": "", "image": "", "environment": ""}, -} +def generate_kubernetes_metadata() -> Dict[str, str]: + namespace = random.choice(K8S_NAMESPACES) + sts_name = f"parseable-{random.choice(COMPONENTS)}" + pod_index = str(random.randint(0, 5)) + pod_name = f"{sts_name}-{pod_index}" + + return { + "kubernetes_namespace_name": namespace, + "kubernetes_pod_name": pod_name, + "kubernetes_pod_id": str(uuid.uuid4()), + "kubernetes_pod_ip": f"10.{random.randint(0, 255)}.{random.randint(0, 255)}.{random.randint(0, 255)}", + "kubernetes_host": f"ip-10-0-{random.randint(0, 255)}-{random.randint(0, 255)}.ec2.internal", + "kubernetes_container_name": random.choice(COMPONENTS), + "kubernetes_container_image": random.choice(CONTAINER_IMAGES), + "kubernetes_container_hash": fake.sha256(), + "kubernetes_docker_id": fake.sha256()[:12], + "kubernetes_labels_app": "parseable", + "kubernetes_labels_component": random.choice(COMPONENTS), + "kubernetes_labels_pbc_nodetype": random.choice(NODE_TYPES), + "kubernetes_labels_spot": random.choice(["true", "false"]), + "kubernetes_labels_sts_name": sts_name, + "kubernetes_labels_statefulset.kubernetes.io/pod-name": pod_name, + "kubernetes_labels_apps.kubernetes.io/pod-index": pod_index, + "kubernetes_labels_controller-revision-hash": fake.sha256()[:10], + "kubernetes_labels_original_sts_name": sts_name, + "kubernetes_labels_parseable_cr": "parseable-cluster" + } + + +def generate_log_entry() -> Dict[str, Any]: + now = datetime.now(timezone.utc) + + # Generate request-related data + status_code = random.choice([200, 200, 200, 201, 400, 401, 403, 404, 500]) + response_time = random.randint(10, 2000) + + # Basic log structure + log_entry = { + "app_meta": json.dumps({"version": "v0.8.0", "component": random.choice(COMPONENTS)}), + "device_id": random.randint(1000, 9999), + "host": f"ip-{random.randint(0, 255)}-{random.randint(0, 255)}-{random.randint(0, 255)}-{random.randint(0, 255)}", + "level": random.choice(["INFO", "INFO", "INFO", "WARN", "ERROR"]), + "location": fake.city(), + "message": fake.sentence(), + "os": random.choice(["linux/amd64", "linux/arm64"]), + "process_id": random.randint(1, 65535), + "request_body": json.dumps({"query": "SELECT * FROM logs LIMIT 100"}), + "response_time": response_time, + "runtime": "python3.9", + "session_id": str(uuid.uuid4()), + "source": "application", + "source_time": now.isoformat(), + "status_code": status_code, + "stream": "stdout", + "time": int(now.timestamp() * 1000), + "timezone": "UTC", + "user_agent": fake.user_agent(), + "user_id": random.randint(1000, 9999), + "uuid": str(uuid.uuid4()), + "version": "v0.8.0" + } + + # Add Kubernetes metadata + log_entry.update(generate_kubernetes_metadata()) + + return log_entry -def create_topic(topic_name, num_partitions, replication_factor): + +def create_topic(topic_name: str, num_partitions: int, replication_factor: int) -> None: new_topic = NewTopic( topic=topic_name, num_partitions=num_partitions, @@ -99,56 +169,6 @@ def delivery_report(err, msg): logger.debug(f"Message delivered to {msg.topic()} [{msg.partition()}]") -def generate_log(): - log = LOG_TEMPLATE.copy() - - # Timestamp & correlation - log["timestamp"] = datetime.now(timezone.utc).isoformat() - log["correlation_id"] = str(uuid4()) - - # Random level/message - levels = ["INFO", "WARNING", "ERROR", "DEBUG"] - messages = [ - "Received incoming HTTP request", - "Processed request successfully", - "Failed to process request", - "Request timeout encountered", - "Service unavailable", - ] - log["level"] = choice(levels) - log["message"] = choice(messages) - - # Populate request fields - methods = ["GET", "POST", "PUT", "DELETE"] - paths = ["/api/resource", "/api/login", "/api/logout", "/api/data"] - log["request"] = { - "method": choice(methods), - "path": choice(paths), - "remote_address": f"192.168.1.{randint(1, 255)}", - } - - # Populate response fields - log["response"] = { - "status_code": choice([200, 201, 400, 401, 403, 404, 500]), - "latency_ms": randint(10, 1000), - } - - # Populate pod and metadata fields - log["pod"] = { - "name": f"pod-{randint(1, 100)}", - "namespace": choice(["default", "kube-system", "production", "staging"]), - "node": f"node-{randint(1, 10)}", - } - - log["metadata"] = { - "container_id": f"container-{randint(1000, 9999)}", - "image": f"example/image:{randint(1, 5)}.0", - "environment": choice(["dev", "staging", "prod"]), - } - - return log - - def main(): logger.info("Starting continuous log producer...") create_topic(KAFKA_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR) @@ -165,7 +185,7 @@ def main(): if not limit_reached: if message_count < TOTAL_LOGS: - log_data = generate_log() + log_data = generate_log_entry() log_str = json.dumps(log_data) # Send to Kafka @@ -196,9 +216,6 @@ def main(): if limit_reached: time.sleep(5) - total_elapsed = current_time - start_time - if total_elapsed % 60 < 5: - logger.info(f"Total messages sent: {message_count}") else: # Sleep to maintain the logs/second rate time.sleep(1 / LOG_RATE) @@ -213,6 +230,7 @@ def main(): finally: logger.info("Flushing producer...") producer.flush() + logger.info("Generator stopped.") if __name__ == "__main__": diff --git a/src/kafka.rs b/src/kafka.rs deleted file mode 100644 index e69de29bb..000000000 From 1226fb22a01798b564a51b64d9aa2b8e93af9fcf Mon Sep 17 00:00:00 2001 From: Devdutt Shenoi Date: Sat, 18 Jan 2025 22:29:22 +0530 Subject: [PATCH 42/65] fix: missing subcommand name --- src/cli.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/cli.rs b/src/cli.rs index 14013c4c8..d54be20eb 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -53,7 +53,7 @@ pub const DEFAULT_PASSWORD: &str = "admin"; long_about = r#" Cloud Native, log analytics platform for modern applications. -Usage: +Usage: parseable [command] [options..] @@ -81,6 +81,7 @@ pub struct Cli { #[derive(Parser)] pub enum StorageOptions { + #[command(name = "local-store")] Local(LocalStoreArgs), #[command(name = "s3-store")] @@ -126,7 +127,7 @@ pub struct Options { // Server configuration #[arg( long, - env = "P_ADDR", + env = "P_ADDR", default_value = "0.0.0.0:8000", value_parser = validation::socket_addr, help = "Address and port for Parseable HTTP(s) server" From 5b5e7b03776af0a9baf1818e66c70069384b6fb0 Mon Sep 17 00:00:00 2001 From: Devdutt Shenoi Date: Sat, 18 Jan 2025 22:33:04 +0530 Subject: [PATCH 43/65] remove unused code --- src/cli.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/cli.rs b/src/cli.rs index d54be20eb..25bfc8a3e 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -34,12 +34,6 @@ use crate::{ storage::{AzureBlobConfig, FSConfig, S3Config}, }; -#[cfg(not(any( - all(target_os = "linux", target_arch = "x86_64"), - all(target_os = "macos", target_arch = "aarch64") -)))] -use std::string::String as KafkaSslProtocol; - /// Default username and password for Parseable server, used by default for local mode. /// NOTE: obviously not recommended for production pub const DEFAULT_USERNAME: &str = "admin"; From 13c23e95d6732ac78c7084313240aba9579b3c4f Mon Sep 17 00:00:00 2001 From: hippalus Date: Mon, 27 Jan 2025 02:56:45 +0100 Subject: [PATCH 44/65] Refactor connector cli options and processor.rs to handle batch events. --- ...r-compose-distributed-test-with-kafka.yaml | 4 +- docker-compose-test-with-kafka.yaml | 2 +- src/cli.rs | 31 ++++-- src/connectors/common/config.rs | 103 ------------------ src/connectors/common/mod.rs | 25 ++++- src/connectors/kafka/config.rs | 85 +++++++++------ src/connectors/kafka/processor.rs | 103 +++++++++--------- src/connectors/kafka/rebalance_listener.rs | 4 +- src/connectors/mod.rs | 16 +-- src/main.rs | 13 +-- src/option.rs | 30 +++++ 11 files changed, 197 insertions(+), 219 deletions(-) delete mode 100644 src/connectors/common/config.rs diff --git a/docker-compose-distributed-test-with-kafka.yaml b/docker-compose-distributed-test-with-kafka.yaml index 627d950f2..b493c6f0f 100644 --- a/docker-compose-distributed-test-with-kafka.yaml +++ b/docker-compose-distributed-test-with-kafka.yaml @@ -95,7 +95,7 @@ services: context: . dockerfile: Dockerfile.kafka platform: linux/amd64 - command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + command: [ "parseable", "s3-store", ] expose: - "8000" environment: @@ -141,7 +141,7 @@ services: context: . dockerfile: Dockerfile.kafka platform: linux/amd64 - command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + command: [ "parseable", "s3-store", ] expose: - "8000" environment: diff --git a/docker-compose-test-with-kafka.yaml b/docker-compose-test-with-kafka.yaml index 7a8ce9bbb..16cec82a2 100644 --- a/docker-compose-test-with-kafka.yaml +++ b/docker-compose-test-with-kafka.yaml @@ -29,7 +29,7 @@ services: context: . dockerfile: Dockerfile.kafka platform: linux/amd64 - command: [ "parseable", "s3-store", "connectors", "kafka-sink" ] + command: [ "parseable", "s3-store", ] ports: - "8000:8000" environment: diff --git a/src/cli.rs b/src/cli.rs index dc34123dc..64f37ee23 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -26,7 +26,7 @@ use url::Url; feature = "rdkafka-ssl-vendored", feature = "rdkafka-sasl" ))] -use crate::connectors::common::config::ConnectorConfig; +use crate::connectors::kafka::config::KafkaConfig; use crate::{ oidc::{self, OpenidConfig}, @@ -91,6 +91,13 @@ pub struct LocalStoreArgs { pub options: Options, #[command(flatten)] pub storage: FSConfig, + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + #[command(flatten)] + pub kafka: KafkaConfig, } #[derive(Parser)] @@ -99,6 +106,13 @@ pub struct S3StoreArgs { pub options: Options, #[command(flatten)] pub storage: S3Config, + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + #[command(flatten)] + pub kafka: KafkaConfig, } #[derive(Parser)] @@ -107,6 +121,13 @@ pub struct BlobStoreArgs { pub options: Options, #[command(flatten)] pub storage: AzureBlobConfig, + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + #[command(flatten)] + pub kafka: KafkaConfig, } #[derive(Parser, Debug)] @@ -309,14 +330,6 @@ pub struct Options { #[arg(long, env = "P_MS_CLARITY_TAG", help = "Tag for MS Clarity")] pub ms_clarity_tag: Option, - - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] - #[command(flatten)] - pub connector: Option, } #[derive(Parser, Debug)] diff --git a/src/connectors/common/config.rs b/src/connectors/common/config.rs deleted file mode 100644 index ec05f7708..000000000 --- a/src/connectors/common/config.rs +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Parseable Server (C) 2022 - 2024 Parseable, Inc. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -use crate::connectors::kafka::config::KafkaConfig; -use clap::{ArgMatches, FromArgMatches, Parser, Subcommand, ValueEnum}; -use std::fmt; -use std::str::FromStr; - -#[derive(Debug, Clone, Parser)] -#[command(name = "connectors", about = "Configure Parseable connectors")] -#[group(id = "connector-config")] -pub struct ConnectorConfig { - #[command(subcommand)] - pub connectors: Connectors, - - #[arg( - value_enum, - long = "bad-data-policy", - required = false, - default_value_t = BadData::Fail, - env = "P_CONNECTOR_BAD_DATA_POLICY", - help = "Policy for handling bad data" - )] - pub bad_data: BadData, -} - -#[derive(Debug, Clone, Subcommand)] -pub enum Connectors { - #[command( - name = "kafka-sink", - about = "Configure Kafka Sink", - next_help_heading = "KAFKA OPTIONS" - )] - KafkaSink(#[command(flatten)] KafkaConfig), - // KinesisSink, - // PulsarSink, etc. -} - -impl fmt::Display for Connectors { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Connectors::KafkaSink(_) => write!(f, "KafkaSink"), - } - } -} - -#[derive(ValueEnum, Default, Clone, Debug, PartialEq, Eq, Hash)] -pub enum BadData { - #[default] - Fail, - Drop, - Dlt, //TODO: Implement Dead Letter Topic support when needed -} - -impl FromStr for BadData { - type Err = String; - - fn from_str(s: &str) -> Result { - match s.to_lowercase().as_str() { - "drop" => Ok(BadData::Drop), - "fail" => Ok(BadData::Fail), - "dlt" => Ok(BadData::Dlt), - _ => Err(format!("Invalid bad data policy: {}", s)), - } - } -} - -impl Default for ConnectorConfig { - fn default() -> Self { - ConnectorConfig { - bad_data: BadData::Drop, - connectors: Connectors::KafkaSink(KafkaConfig::default()), - } - } -} - -impl ConnectorConfig { - pub fn from(matches: &ArgMatches) -> Option { - matches - .subcommand_matches("connectors") - .and_then(|connector_matches| { - match ConnectorConfig::from_arg_matches(connector_matches) { - Ok(config) => Some(config), - Err(err) => err.exit(), - } - }) - } -} diff --git a/src/connectors/common/mod.rs b/src/connectors/common/mod.rs index f44e2a6a7..cb77d983c 100644 --- a/src/connectors/common/mod.rs +++ b/src/connectors/common/mod.rs @@ -15,12 +15,14 @@ * along with this program. If not, see . * */ + +use clap::ValueEnum; use rdkafka::error::{KafkaError, RDKafkaErrorCode}; +use std::str::FromStr; use thiserror::Error; use tokio::runtime; use tokio::runtime::Builder; -pub mod config; pub mod processor; pub mod shutdown; @@ -77,6 +79,27 @@ impl ConnectorError { } } +#[derive(ValueEnum, Default, Clone, Debug, PartialEq, Eq, Hash)] +pub enum BadData { + #[default] + Fail, + Drop, + Dlt, //TODO: Implement Dead Letter Topic support when needed +} + +impl FromStr for BadData { + type Err = String; + + fn from_str(s: &str) -> Result { + match s.to_lowercase().as_str() { + "drop" => Ok(BadData::Drop), + "fail" => Ok(BadData::Fail), + "dlt" => Ok(BadData::Dlt), + _ => Err(format!("Invalid bad data policy: {}", s)), + } + } +} + pub fn build_runtime(worker_threads: usize, thread_name: &str) -> anyhow::Result { Builder::new_multi_thread() .enable_all() diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index c48d61ebb..85c705ce5 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -1,38 +1,20 @@ -use clap::{Args, ValueEnum}; +use crate::connectors::common::BadData; +use clap::{Args, Parser, ValueEnum}; use rdkafka::{ClientConfig, Offset}; use serde::{Deserialize, Serialize}; use std::path::PathBuf; use std::time::Duration; -#[derive(ValueEnum, Debug, Clone)] -pub enum SourceOffset { - Earliest, - Latest, - Group, -} - -impl SourceOffset { - pub fn get_offset(&self) -> Offset { - match self { - SourceOffset::Earliest => Offset::Beginning, - SourceOffset::Latest => Offset::End, - SourceOffset::Group => Offset::Stored, - } - } -} - -#[derive(Debug, Clone, Args)] -#[command(name = "kafka-config", about = "Configure Kafka connector settings")] -#[group(id = "kafka")] +#[derive(Debug, Clone, Parser)] pub struct KafkaConfig { #[arg( long = "bootstrap-servers", env = "P_KAFKA_BOOTSTRAP_SERVERS", value_name = "bootstrap-servers", - required = true, + required = false, help = "Comma-separated list of Kafka bootstrap servers" )] - pub bootstrap_servers: String, + pub bootstrap_servers: Option, #[arg( long = "client-id", @@ -49,22 +31,29 @@ pub struct KafkaConfig { env = "P_KAFKA_PARTITION_LISTENER_CONCURRENCY", value_name = "concurrency", required = false, - default_value_t = 1, + default_value_t = 2, help = "Number of parallel threads for Kafka partition listeners. Each partition gets processed on a dedicated thread." )] pub partition_listener_concurrency: usize, #[command(flatten)] - #[group(id = "consumer", required = false)] pub consumer: Option, #[command(flatten)] - #[group(id = "producer", required = false)] pub producer: Option, #[command(flatten)] - #[group(id = "security", required = false)] pub security: Option, + + #[arg( + value_enum, + long = "bad-data-policy", + required = false, + default_value_t = BadData::Fail, + env = "P_CONNECTOR_BAD_DATA_POLICY", + help = "Policy for handling bad data" + )] + pub bad_data: BadData, } #[derive(Debug, Clone, Args)] @@ -73,7 +62,7 @@ pub struct ConsumerConfig { long = "consumer-topics", env = "P_KAFKA_CONSUMER_TOPICS", value_name = "consumer-topics", - required = true, + required = false, value_delimiter = ',', help = "Comma-separated list of topics" )] @@ -537,7 +526,12 @@ impl KafkaConfig { // Basic configuration config - .set("bootstrap.servers", &self.bootstrap_servers) + .set( + "bootstrap.servers", + self.bootstrap_servers + .as_ref() + .expect("Bootstrap servers must not be empty"), + ) .set("client.id", &self.client_id); // Consumer configuration @@ -560,7 +554,12 @@ impl KafkaConfig { // Basic configuration config - .set("bootstrap.servers", &self.bootstrap_servers) + .set( + "bootstrap.servers", + self.bootstrap_servers + .as_ref() + .expect("Bootstrap servers must not be empty"), + ) .set("client.id", &self.client_id); // Producer configuration @@ -591,7 +590,7 @@ impl KafkaConfig { } pub fn validate(&self) -> anyhow::Result<()> { - if self.bootstrap_servers.is_empty() { + if self.bootstrap_servers.is_none() { anyhow::bail!("Bootstrap servers must not be empty"); } @@ -854,15 +853,16 @@ impl Default for KafkaConfig { fn default() -> Self { Self { // Common configuration with standard broker port - bootstrap_servers: "localhost:9092".to_string(), + bootstrap_servers: Some("localhost:9092".to_string()), client_id: "parseable-connect".to_string(), - // Single threaded listener for all assigned partitions - partition_listener_concurrency: 1, + // Listener for all assigned partitions + partition_listener_concurrency: 2, // Component-specific configurations with production-ready defaults consumer: Some(ConsumerConfig::default()), producer: Some(ProducerConfig::default()), // Security configuration with plaintext protocol security: Some(SecurityConfig::default()), + bad_data: BadData::default(), } } } @@ -986,3 +986,20 @@ impl std::str::FromStr for Acks { } } } + +#[derive(ValueEnum, Debug, Clone)] +pub enum SourceOffset { + Earliest, + Latest, + Group, +} + +impl SourceOffset { + pub fn get_offset(&self) -> Offset { + match self { + SourceOffset::Earliest => Offset::Beginning, + SourceOffset::Latest => Offset::End, + SourceOffset::Group => Offset::Stored, + } + } +} diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index 6b7655a74..e06835121 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -33,61 +33,67 @@ use serde_json::Value; use std::collections::HashMap; use std::sync::Arc; use tokio_stream::wrappers::ReceiverStream; -use tracing::{debug, error, warn}; +use tracing::{debug, error}; #[derive(Default, Debug, Clone)] pub struct ParseableSinkProcessor; impl ParseableSinkProcessor { - async fn deserialize( + async fn build_event_from_chunk( &self, - consumer_record: &ConsumerRecord, - ) -> anyhow::Result> { - let stream_name = consumer_record.topic.as_str(); + records: &[ConsumerRecord], + ) -> anyhow::Result { + let stream_name = records + .first() + .map(|r| r.topic.as_str()) + .unwrap_or_default(); + + create_stream_if_not_exists(stream_name, StreamType::UserDefined, LogSource::Json).await?; - create_stream_if_not_exists(stream_name, StreamType::UserDefined, LogSource::default()) - .await?; let schema = STREAM_INFO.schema_raw(stream_name)?; + let time_partition = STREAM_INFO.get_time_partition(stream_name)?; + let static_schema_flag = STREAM_INFO.get_static_schema_flag(stream_name)?; + let schema_version = STREAM_INFO.get_schema_version(stream_name)?; + + let (json_vec, total_payload_size) = Self::json_vec(records); + let batch_json_event = format::json::Event { + data: Value::Array(json_vec.to_vec()), + }; + + let (rb, is_first) = batch_json_event.into_recordbatch( + &schema, + static_schema_flag, + time_partition.as_ref(), + schema_version, + )?; + + let p_event = ParseableEvent { + rb, + stream_name: stream_name.to_string(), + origin_format: "json", + origin_size: total_payload_size, + is_first_event: is_first, + parsed_timestamp: Utc::now().naive_utc(), + time_partition: None, + custom_partition_values: HashMap::new(), + stream_type: StreamType::UserDefined, + }; + + Ok(p_event) + } - match &consumer_record.payload { - None => { - warn!( - "Skipping tombstone or empty payload in partition {} key {}", - consumer_record.partition, - consumer_record.key_str() - ); - Ok(None) - } - Some(payload) => { - let data: Value = serde_json::from_slice(payload.as_ref())?; - let json_event = format::json::Event { data }; - - let time_partition = STREAM_INFO.get_time_partition(stream_name)?; - let static_schema_flag = STREAM_INFO.get_static_schema_flag(stream_name)?; - let schema_version = STREAM_INFO.get_schema_version(stream_name)?; - - let (rb, is_first) = json_event.into_recordbatch( - &schema, - static_schema_flag, - time_partition.as_ref(), - schema_version, - )?; - - let p_event = ParseableEvent { - rb, - stream_name: stream_name.to_string(), - origin_format: "json", - origin_size: payload.len() as u64, - is_first_event: is_first, - parsed_timestamp: Utc::now().naive_utc(), - time_partition: None, - custom_partition_values: HashMap::new(), - stream_type: StreamType::UserDefined, - }; - - Ok(Some(p_event)) + fn json_vec(records: &[ConsumerRecord]) -> (Vec, u64) { + let mut json_vec = Vec::with_capacity(records.len()); + let mut total_payload_size = 0u64; + + for record in records.iter().filter_map(|r| r.payload.as_ref()) { + total_payload_size += record.len() as u64; + if let Ok(value) = serde_json::from_slice::(record) { + json_vec.push(value); } } + + (json_vec, total_payload_size) } } @@ -97,11 +103,10 @@ impl Processor, ()> for ParseableSinkProcessor { let len = records.len(); debug!("Processing {} records", len); - for cr in records { - if let Some(event) = self.deserialize(&cr).await? { - event.process().await?; - } - } + self.build_event_from_chunk(&records) + .await? + .process() + .await?; debug!("Processed {} records", len); Ok(()) diff --git a/src/connectors/kafka/rebalance_listener.rs b/src/connectors/kafka/rebalance_listener.rs index fe7d40278..8372e1b5b 100644 --- a/src/connectors/kafka/rebalance_listener.rs +++ b/src/connectors/kafka/rebalance_listener.rs @@ -49,7 +49,7 @@ impl RebalanceListener { let shutdown_handle = self.shutdown_handle.clone(); let tokio_runtime_handle = Handle::current(); - std::thread::spawn(move || { + std::thread::Builder::new().name("rebalance-listener-thread".to_string()).spawn(move || { tokio_runtime_handle.block_on(async move { loop { tokio::select! { @@ -82,6 +82,6 @@ impl RebalanceListener { } } }) - }); + }).expect("Failed to start rebalance listener thread"); } } diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 37393f1de..757c49a97 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -16,7 +16,6 @@ * */ -use crate::connectors::common::config::Connectors; use crate::connectors::common::processor::Processor; use crate::connectors::common::shutdown::Shutdown; use crate::connectors::kafka::config::KafkaConfig; @@ -39,11 +38,12 @@ pub mod kafka; pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { if matches!(CONFIG.options.mode, Mode::Ingest | Mode::All) { - match CONFIG.options.connector.clone() { - None => { - warn!("Kafka connector configuration is missing. Skipping Kafka pipeline."); + match CONFIG.kafka_config.validate() { + Err(e) => { + warn!("Kafka connector configuration invalid. {}", e); } - Some(connectors) => { + Ok(_) => { + let config = CONFIG.kafka_config.clone(); let shutdown_handle = Shutdown::default(); let registry = prometheus.registry.clone(); let processor = ParseableSinkProcessor; @@ -56,11 +56,7 @@ pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { } }); - match connectors.connectors { - Connectors::KafkaSink(config) => { - run_kafka2parseable(config, registry, processor, shutdown_handle).await? - } - } + run_kafka2parseable(config, registry, processor, shutdown_handle).await?; } } } diff --git a/src/main.rs b/src/main.rs index 9574cc048..7c1d7e0d4 100644 --- a/src/main.rs +++ b/src/main.rs @@ -66,7 +66,6 @@ async fn main() -> anyhow::Result<()> { }); let prometheus = metrics::build_metrics_handler(); - let parseable_server = server.init(&prometheus, shutdown_rx); #[cfg(any( feature = "rdkafka-ssl", @@ -74,13 +73,10 @@ async fn main() -> anyhow::Result<()> { feature = "rdkafka-sasl" ))] { - // load kafka server - if CONFIG.options.mode != Mode::Query { - let connectors_task = connectors::init(&prometheus); - tokio::try_join!(parseable_server, connectors_task)?; - } else { - parseable_server.await?; - } + let parseable_server = server.init(&prometheus, shutdown_rx); + let connectors = connectors::init(&prometheus); + + tokio::try_join!(parseable_server, connectors)?; } #[cfg(not(any( @@ -89,6 +85,7 @@ async fn main() -> anyhow::Result<()> { feature = "rdkafka-sasl" )))] { + let parseable_server = server.init(&prometheus, shutdown_rx); parseable_server.await?; } diff --git a/src/option.rs b/src/option.rs index f8bf4dccc..806c5ee20 100644 --- a/src/option.rs +++ b/src/option.rs @@ -17,6 +17,12 @@ */ use crate::cli::{Cli, Options, StorageOptions, DEFAULT_PASSWORD, DEFAULT_USERNAME}; +#[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" +))] +use crate::connectors::kafka::config::KafkaConfig; use crate::storage::object_storage::parseable_json_path; use crate::storage::{ObjectStorageError, ObjectStorageProvider}; use bytes::Bytes; @@ -37,6 +43,12 @@ pub struct Config { pub options: Options, storage: Arc, pub storage_name: &'static str, + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + pub kafka_config: KafkaConfig, } impl Config { @@ -63,17 +75,35 @@ impl Config { options: args.options, storage: Arc::new(args.storage), storage_name: "drive", + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + kafka_config: args.kafka, } } StorageOptions::S3(args) => Config { options: args.options, storage: Arc::new(args.storage), storage_name: "s3", + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + kafka_config: args.kafka, }, StorageOptions::Blob(args) => Config { options: args.options, storage: Arc::new(args.storage), storage_name: "blob_store", + #[cfg(any( + feature = "rdkafka-ssl", + feature = "rdkafka-ssl-vendored", + feature = "rdkafka-sasl" + ))] + kafka_config: args.kafka, }, } } From 1e23d3742714a65dba66a4c5ea266871cffc2c9c Mon Sep 17 00:00:00 2001 From: hippalus Date: Mon, 27 Jan 2025 03:59:17 +0100 Subject: [PATCH 45/65] up step rust to 1.84 --- Dockerfile | 2 +- Dockerfile.debug | 2 +- Dockerfile.dev | 2 +- Dockerfile.kafka | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Dockerfile b/Dockerfile index 6961f8f15..d424dc14f 100644 --- a/Dockerfile +++ b/Dockerfile @@ -14,7 +14,7 @@ # along with this program. If not, see . # build stage -FROM rust:1.83.0-bookworm AS builder +FROM rust:1.84.0-bookworm AS builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " diff --git a/Dockerfile.debug b/Dockerfile.debug index 51facae9e..a1d95ee99 100644 --- a/Dockerfile.debug +++ b/Dockerfile.debug @@ -14,7 +14,7 @@ # along with this program. If not, see . # build stage -FROM rust:1.83.0-bookworm AS builder +FROM rust:1.84.0-bookworm AS builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " diff --git a/Dockerfile.dev b/Dockerfile.dev index a0bb5e674..227ade1ce 100644 --- a/Dockerfile.dev +++ b/Dockerfile.dev @@ -14,7 +14,7 @@ # along with this program. If not, see . # build stage -FROM rust:1.83.0-bookworm AS builder +FROM rust:1.84.0-bookworm AS builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " diff --git a/Dockerfile.kafka b/Dockerfile.kafka index f7f0d7ce0..de06d40a2 100644 --- a/Dockerfile.kafka +++ b/Dockerfile.kafka @@ -14,7 +14,7 @@ # along with this program. If not, see . # build stage -FROM rust:1.83.0-bookworm AS builder +FROM rust:1.84.0-bookworm AS builder LABEL org.opencontainers.image.title="Parseable" LABEL maintainer="Parseable Team " From ddea62eff9dd3cd3a6758460f14cbb1a501ea965 Mon Sep 17 00:00:00 2001 From: hippalus Date: Mon, 27 Jan 2025 22:54:33 +0100 Subject: [PATCH 46/65] Add RUST_LOG env and add feature flags to build.yaml workflow. --- .github/workflows/build.yaml | 6 +++--- docker-compose-distributed-test-with-kafka.yaml | 5 ++++- docker-compose-distributed-test.yaml | 2 ++ docker-compose-test-with-kafka.yaml | 3 ++- docker-compose-test.yaml | 1 + 5 files changed, 12 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 5802a1a1c..ebf3055f2 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -35,7 +35,7 @@ jobs: with: use-cross: true command: build - args: --target ${{matrix.target}} + args: --target ${{matrix.target}} --features "rdkafka-ssl,rdkafka-sasl" build-windows: name: Build for windows @@ -52,7 +52,7 @@ jobs: override: true - name: Build on windows - run: cargo build --target x86_64-pc-windows-msvc + run: cargo build --target x86_64-pc-windows-msvc --features "rdkafka-ssl,rdkafka-sasl" build-macos: name: Build for ${{matrix.target}} @@ -73,4 +73,4 @@ jobs: override: true - name: Build on ${{ matrix.target }} - run: cargo build --target ${{ matrix.target }} + run: cargo build --target ${{ matrix.target }} --features "rdkafka-ssl,rdkafka-sasl" diff --git a/docker-compose-distributed-test-with-kafka.yaml b/docker-compose-distributed-test-with-kafka.yaml index b493c6f0f..e2f20a96b 100644 --- a/docker-compose-distributed-test-with-kafka.yaml +++ b/docker-compose-distributed-test-with-kafka.yaml @@ -72,6 +72,7 @@ services: - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=query + - RUST_LOG=info networks: - parseable-internal healthcheck: @@ -114,6 +115,7 @@ services: - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - P_KAFKA_PARTITION_LISTENER_CONCURRENCY=3 + - RUST_LOG=info # additional settings like security, tuning, etc. networks: - parseable-internal @@ -160,6 +162,7 @@ services: - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - P_KAFKA_PARTITION_LISTENER_CONCURRENCY=3 + - RUST_LOG=info # additional settings like security, tuning, etc. networks: - parseable-internal @@ -334,7 +337,7 @@ services: environment: - KAFKA_BROKERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - KAFKA_TOPIC=dist-test-logs-stream - - LOG_RATE=1000 + - LOG_RATE=5000 - TOTAL_LOGS=1_000_000 - REPLICATION_FACTOR=3 depends_on: diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index ca79941f3..bd01dfb72 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -45,6 +45,7 @@ services: - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=query + - RUST_LOG=info networks: - parseable-internal healthcheck: @@ -81,6 +82,7 @@ services: - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=ingest - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 + - RUST_LOG=info networks: - parseable-internal healthcheck: diff --git a/docker-compose-test-with-kafka.yaml b/docker-compose-test-with-kafka.yaml index 16cec82a2..c532bd470 100644 --- a/docker-compose-test-with-kafka.yaml +++ b/docker-compose-test-with-kafka.yaml @@ -45,6 +45,7 @@ services: - P_PARQUET_COMPRESSION_ALGO=snappy - P_KAFKA_CONSUMER_TOPICS=test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092 + - RUST_LOG=info # additional settings like security, tuning, etc. depends_on: - minio @@ -147,7 +148,7 @@ services: environment: - KAFKA_BROKERS=kafka-0:9092 - KAFKA_TOPIC=test-logs-stream - - LOG_RATE=1000 + - LOG_RATE=5000 - TOTAL_LOGS=500_000 depends_on: - kafka-0 diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index 0a81d3c8c..dce83dd60 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -43,6 +43,7 @@ services: - P_PASSWORD=parseableadmin - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy + - RUST_LOG=info depends_on: - minio healthcheck: From a0116afd9b7243559acd320d931b539e5b8b67fb Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 28 Jan 2025 00:01:18 +0100 Subject: [PATCH 47/65] add OpenSSL setup to build.yaml workflow. --- .github/workflows/build.yaml | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index ebf3055f2..059ab57b8 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -20,14 +20,25 @@ jobs: target: - aarch64-unknown-linux-gnu # linux(arm) - x86_64-unknown-linux-gnu # linux(64 bit) + include: + - target: aarch64-unknown-linux-gnu + openssl_package: libssl-dev:arm64 + - target: x86_64-unknown-linux-gnu + openssl_package: libssl-dev steps: - uses: actions/checkout@v2 + - name: Install OpenSSL development packages + run: | + sudo dpkg --add-architecture arm64 + sudo apt-get update + sudo apt-get install -y ${{ matrix.openssl_package }} + - uses: actions-rs/toolchain@v1 with: toolchain: stable - profile: minimal # minimal component installation (ie, no documentation) + profile: minimal target: ${{ matrix.target }} override: true @@ -44,6 +55,13 @@ jobs: steps: - uses: actions/checkout@v2 + - name: Install OpenSSL + run: | + vcpkg install openssl:x64-windows-static + echo "OPENSSL_DIR=C:/vcpkg/installed/x64-windows-static" >> $GITHUB_ENV + echo "OPENSSL_STATIC=1" >> $GITHUB_ENV + shell: bash + - uses: actions-rs/toolchain@v1 with: toolchain: stable @@ -65,6 +83,14 @@ jobs: steps: - uses: actions/checkout@v2 + + - name: Install OpenSSL + run: brew install openssl@3 + + - name: Set OpenSSL Environment Variables + run: | + echo "OPENSSL_DIR=$(brew --prefix openssl@3)" >> $GITHUB_ENV + - uses: actions-rs/toolchain@v1 with: toolchain: stable From b02b2a129404560cc3b76c36d0985185de7942a4 Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 28 Jan 2025 00:13:56 +0100 Subject: [PATCH 48/65] update build.yaml --- .github/workflows/build.yaml | 71 ++++++++++++++++++------------------ 1 file changed, 35 insertions(+), 36 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 059ab57b8..732c47958 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -20,54 +20,54 @@ jobs: target: - aarch64-unknown-linux-gnu # linux(arm) - x86_64-unknown-linux-gnu # linux(64 bit) - include: - - target: aarch64-unknown-linux-gnu - openssl_package: libssl-dev:arm64 - - target: x86_64-unknown-linux-gnu - openssl_package: libssl-dev steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - - name: Install OpenSSL development packages - run: | - sudo dpkg --add-architecture arm64 - sudo apt-get update - sudo apt-get install -y ${{ matrix.openssl_package }} + - name: Install cross + run: cargo install cross --git https://github.com/cross-rs/cross - - uses: actions-rs/toolchain@v1 + - name: Configure Cross.toml + run: | + cat > Cross.toml << 'EOF' + [target.aarch64-unknown-linux-gnu] + pre-build = [ + "apt-get update && apt-get install -y --no-install-recommends libssl-dev" + ] + + [target.x86_64-unknown-linux-gnu] + pre-build = [ + "apt-get update && apt-get install -y --no-install-recommends libssl-dev" + ] + EOF + + - uses: dtolnay/rust-toolchain@stable with: - toolchain: stable - profile: minimal - target: ${{ matrix.target }} - override: true + targets: ${{ matrix.target }} - - uses: actions-rs/cargo@v1 - with: - use-cross: true - command: build - args: --target ${{matrix.target}} --features "rdkafka-ssl,rdkafka-sasl" + - name: Build with cross + run: cross build --target ${{ matrix.target }} --features "rdkafka-ssl,rdkafka-sasl" build-windows: name: Build for windows runs-on: windows-latest steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 + + - name: Setup vcpkg + uses: lukka/run-vcpkg@v11 - name: Install OpenSSL run: | vcpkg install openssl:x64-windows-static - echo "OPENSSL_DIR=C:/vcpkg/installed/x64-windows-static" >> $GITHUB_ENV - echo "OPENSSL_STATIC=1" >> $GITHUB_ENV - shell: bash + echo "OPENSSL_DIR=$env:VCPKG_ROOT/installed/x64-windows-static" >> $env:GITHUB_ENV + echo "OPENSSL_STATIC=1" >> $env:GITHUB_ENV + shell: pwsh - - uses: actions-rs/toolchain@v1 + - uses: dtolnay/rust-toolchain@stable with: - toolchain: stable - profile: minimal # minimal component installation (ie, no documentation) - default: true - override: true + targets: x86_64-pc-windows-msvc - name: Build on windows run: cargo build --target x86_64-pc-windows-msvc --features "rdkafka-ssl,rdkafka-sasl" @@ -82,7 +82,7 @@ jobs: - x86_64-apple-darwin # macos(intel 64 bit) steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Install OpenSSL run: brew install openssl@3 @@ -90,13 +90,12 @@ jobs: - name: Set OpenSSL Environment Variables run: | echo "OPENSSL_DIR=$(brew --prefix openssl@3)" >> $GITHUB_ENV + echo "OPENSSL_INCLUDE_DIR=$(brew --prefix openssl@3)/include" >> $GITHUB_ENV + echo "OPENSSL_LIB_DIR=$(brew --prefix openssl@3)/lib" >> $GITHUB_ENV - - uses: actions-rs/toolchain@v1 + - uses: dtolnay/rust-toolchain@stable with: - toolchain: stable - profile: minimal - target: ${{ matrix.target }} - override: true + targets: ${{ matrix.target }} - name: Build on ${{ matrix.target }} run: cargo build --target ${{ matrix.target }} --features "rdkafka-ssl,rdkafka-sasl" From 689318f8c644056abc6334de230676e6bced544f Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 28 Jan 2025 00:30:44 +0100 Subject: [PATCH 49/65] update build.yaml --- .github/workflows/build.yaml | 46 ++++++++++++++++++++++++++++++------ 1 file changed, 39 insertions(+), 7 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 732c47958..dd761a834 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -30,14 +30,40 @@ jobs: - name: Configure Cross.toml run: | cat > Cross.toml << 'EOF' + [build.env] + passthrough = [ + "RUST_BACKTRACE", + "RUST_LOG", + "OPENSSL_DIR", + "OPENSSL_LIB_DIR", + "OPENSSL_INCLUDE_DIR", + "SASL2_DIR", + "PKG_CONFIG_PATH", + "PKG_CONFIG_ALLOW_CROSS" + ] + [target.aarch64-unknown-linux-gnu] pre-build = [ - "apt-get update && apt-get install -y --no-install-recommends libssl-dev" + "dpkg --add-architecture arm64", + "apt-get update", + "apt-get install -y --no-install-recommends pkg-config libssl-dev:arm64 gcc-aarch64-linux-gnu libc6-dev-arm64-cross libsasl2-dev:arm64", + "export OPENSSL_DIR=/usr", + "export OPENSSL_LIB_DIR=/usr/lib/aarch64-linux-gnu", + "export OPENSSL_INCLUDE_DIR=/usr/include", + "export SASL2_DIR=/usr", + "export PKG_CONFIG_PATH=/usr/lib/aarch64-linux-gnu/pkgconfig", + "export PKG_CONFIG_ALLOW_CROSS=1" ] [target.x86_64-unknown-linux-gnu] pre-build = [ - "apt-get update && apt-get install -y --no-install-recommends libssl-dev" + "apt-get update", + "apt-get install -y --no-install-recommends pkg-config libssl-dev libsasl2-dev", + "export OPENSSL_DIR=/usr", + "export OPENSSL_LIB_DIR=/usr/lib/x86_64-linux-gnu", + "export OPENSSL_INCLUDE_DIR=/usr/include", + "export SASL2_DIR=/usr", + "export PKG_CONFIG_PATH=/usr/lib/x86_64-linux-gnu/pkgconfig" ] EOF @@ -57,12 +83,16 @@ jobs: - name: Setup vcpkg uses: lukka/run-vcpkg@v11 + with: + vcpkgGitCommitId: '5e0cab206a5ea620130caf672fce3e4a6b5666a1' - - name: Install OpenSSL + - name: Install dependencies run: | - vcpkg install openssl:x64-windows-static + vcpkg install openssl:x64-windows-static cyrus-sasl:x64-windows-static echo "OPENSSL_DIR=$env:VCPKG_ROOT/installed/x64-windows-static" >> $env:GITHUB_ENV echo "OPENSSL_STATIC=1" >> $env:GITHUB_ENV + echo "SASL2_DIR=$env:VCPKG_ROOT/installed/x64-windows-static" >> $env:GITHUB_ENV + echo "SASL2_STATIC=1" >> $env:GITHUB_ENV shell: pwsh - uses: dtolnay/rust-toolchain@stable @@ -84,14 +114,16 @@ jobs: steps: - uses: actions/checkout@v4 - - name: Install OpenSSL - run: brew install openssl@3 + - name: Install dependencies + run: | + brew install openssl@3 cyrus-sasl - - name: Set OpenSSL Environment Variables + - name: Set environment variables run: | echo "OPENSSL_DIR=$(brew --prefix openssl@3)" >> $GITHUB_ENV echo "OPENSSL_INCLUDE_DIR=$(brew --prefix openssl@3)/include" >> $GITHUB_ENV echo "OPENSSL_LIB_DIR=$(brew --prefix openssl@3)/lib" >> $GITHUB_ENV + echo "SASL2_DIR=$(brew --prefix cyrus-sasl)" >> $GITHUB_ENV - uses: dtolnay/rust-toolchain@stable with: From b4715285226f041d517e6eecd7d462fe609e3626 Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 28 Jan 2025 00:41:59 +0100 Subject: [PATCH 50/65] update build.yaml --- .github/workflows/build.yaml | 43 ++++++++++++++++++++++++++++++------ 1 file changed, 36 insertions(+), 7 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index dd761a834..c252f3c3c 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -67,6 +67,7 @@ jobs: ] EOF + - uses: dtolnay/rust-toolchain@stable with: targets: ${{ matrix.target }} @@ -77,23 +78,28 @@ jobs: build-windows: name: Build for windows runs-on: windows-latest + env: + VCPKG_ROOT: ${{ github.workspace }}/vcpkg steps: - uses: actions/checkout@v4 - name: Setup vcpkg - uses: lukka/run-vcpkg@v11 - with: - vcpkgGitCommitId: '5e0cab206a5ea620130caf672fce3e4a6b5666a1' + shell: bash + run: | + git clone https://github.com/microsoft/vcpkg.git + cd vcpkg + ./bootstrap-vcpkg.bat + ./vcpkg integrate install - name: Install dependencies + shell: pwsh run: | - vcpkg install openssl:x64-windows-static cyrus-sasl:x64-windows-static + $env:VCPKG_ROOT/vcpkg install openssl:x64-windows-static cyrus-sasl:x64-windows-static echo "OPENSSL_DIR=$env:VCPKG_ROOT/installed/x64-windows-static" >> $env:GITHUB_ENV echo "OPENSSL_STATIC=1" >> $env:GITHUB_ENV echo "SASL2_DIR=$env:VCPKG_ROOT/installed/x64-windows-static" >> $env:GITHUB_ENV echo "SASL2_STATIC=1" >> $env:GITHUB_ENV - shell: pwsh - uses: dtolnay/rust-toolchain@stable with: @@ -105,6 +111,8 @@ jobs: build-macos: name: Build for ${{matrix.target}} runs-on: macos-latest + env: + MACOSX_DEPLOYMENT_TARGET: "11.0" # Updated minimum target strategy: matrix: target: @@ -116,18 +124,39 @@ jobs: - name: Install dependencies run: | - brew install openssl@3 cyrus-sasl + brew install openssl@3 cyrus-sasl pkg-config - name: Set environment variables run: | + # OpenSSL configuration echo "OPENSSL_DIR=$(brew --prefix openssl@3)" >> $GITHUB_ENV echo "OPENSSL_INCLUDE_DIR=$(brew --prefix openssl@3)/include" >> $GITHUB_ENV echo "OPENSSL_LIB_DIR=$(brew --prefix openssl@3)/lib" >> $GITHUB_ENV + echo "PKG_CONFIG_PATH=$(brew --prefix openssl@3)/lib/pkgconfig:$(brew --prefix cyrus-sasl)/lib/pkgconfig" >> $GITHUB_ENV + + # SASL configuration echo "SASL2_DIR=$(brew --prefix cyrus-sasl)" >> $GITHUB_ENV + echo "LDFLAGS=-L$(brew --prefix openssl@3)/lib -L$(brew --prefix cyrus-sasl)/lib" >> $GITHUB_ENV + echo "CPPFLAGS=-I$(brew --prefix openssl@3)/include -I$(brew --prefix cyrus-sasl)/include" >> $GITHUB_ENV - uses: dtolnay/rust-toolchain@stable with: targets: ${{ matrix.target }} + - name: Create .cargo/config.toml + run: | + mkdir -p .cargo + cat > .cargo/config.toml << 'EOF' + [target.'cfg(target_os = "macos")'] + rustflags = [ + "-C", "link-arg=-L/usr/local/lib", + "-C", "link-arg=-L$(brew --prefix openssl@3)/lib", + "-C", "link-arg=-L$(brew --prefix cyrus-sasl)/lib" + ] + EOF + - name: Build on ${{ matrix.target }} - run: cargo build --target ${{ matrix.target }} --features "rdkafka-ssl,rdkafka-sasl" + run: | + # Ensure all dynamic libraries are findable + export DYLD_LIBRARY_PATH="$(brew --prefix openssl@3)/lib:$(brew --prefix cyrus-sasl)/lib:$DYLD_LIBRARY_PATH" + cargo build --target ${{ matrix.target }} --features "rdkafka-ssl,rdkafka-sasl" From 549ac426a2ae13ae43b625252166241b098a5ad6 Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 28 Jan 2025 23:28:04 +0100 Subject: [PATCH 51/65] configure kafka future and build workflow --- .github/workflows/build-push-edge-kafka.yaml | 2 +- .github/workflows/build.yaml | 132 +++++-------------- Cargo.lock | 97 +++++++++----- Cargo.toml | 6 +- Dockerfile.kafka | 2 +- src/cli.rs | 24 +--- src/lib.rs | 6 +- src/main.rs | 26 +--- src/option.rs | 30 +---- 9 files changed, 114 insertions(+), 211 deletions(-) diff --git a/.github/workflows/build-push-edge-kafka.yaml b/.github/workflows/build-push-edge-kafka.yaml index ecf20bdee..1131d2fa4 100644 --- a/.github/workflows/build-push-edge-kafka.yaml +++ b/.github/workflows/build-push-edge-kafka.yaml @@ -15,7 +15,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Set up QEMU uses: docker/setup-qemu-action@v3 diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index c252f3c3c..c7de61a50 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -1,3 +1,5 @@ +name: Ensure parseable builds on all release targets with Kafka feature + on: pull_request: paths-ignore: @@ -9,7 +11,6 @@ on: branches: - main -name: Ensure parseable builds on all release targets jobs: build-linux: name: Build for ${{matrix.target}} @@ -24,95 +25,49 @@ jobs: steps: - uses: actions/checkout@v4 - - name: Install cross - run: cargo install cross --git https://github.com/cross-rs/cross - - - name: Configure Cross.toml + - name: Install dependencies run: | - cat > Cross.toml << 'EOF' - [build.env] - passthrough = [ - "RUST_BACKTRACE", - "RUST_LOG", - "OPENSSL_DIR", - "OPENSSL_LIB_DIR", - "OPENSSL_INCLUDE_DIR", - "SASL2_DIR", - "PKG_CONFIG_PATH", - "PKG_CONFIG_ALLOW_CROSS" - ] - - [target.aarch64-unknown-linux-gnu] - pre-build = [ - "dpkg --add-architecture arm64", - "apt-get update", - "apt-get install -y --no-install-recommends pkg-config libssl-dev:arm64 gcc-aarch64-linux-gnu libc6-dev-arm64-cross libsasl2-dev:arm64", - "export OPENSSL_DIR=/usr", - "export OPENSSL_LIB_DIR=/usr/lib/aarch64-linux-gnu", - "export OPENSSL_INCLUDE_DIR=/usr/include", - "export SASL2_DIR=/usr", - "export PKG_CONFIG_PATH=/usr/lib/aarch64-linux-gnu/pkgconfig", - "export PKG_CONFIG_ALLOW_CROSS=1" - ] - - [target.x86_64-unknown-linux-gnu] - pre-build = [ - "apt-get update", - "apt-get install -y --no-install-recommends pkg-config libssl-dev libsasl2-dev", - "export OPENSSL_DIR=/usr", - "export OPENSSL_LIB_DIR=/usr/lib/x86_64-linux-gnu", - "export OPENSSL_INCLUDE_DIR=/usr/include", - "export SASL2_DIR=/usr", - "export PKG_CONFIG_PATH=/usr/lib/x86_64-linux-gnu/pkgconfig" - ] - EOF - - - - uses: dtolnay/rust-toolchain@stable + sudo apt-get update + sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libssl-dev + + - uses: actions-rs/toolchain@v1 with: - targets: ${{ matrix.target }} + toolchain: stable + profile: minimal # minimal component installation (ie, no documentation) + target: ${{ matrix.target }} + override: true - - name: Build with cross - run: cross build --target ${{ matrix.target }} --features "rdkafka-ssl,rdkafka-sasl" + - uses: actions-rs/cargo@v1 + with: + use-cross: true + command: build + args: --target ${{matrix.target}} --features kafka build-windows: name: Build for windows runs-on: windows-latest - env: - VCPKG_ROOT: ${{ github.workspace }}/vcpkg steps: - uses: actions/checkout@v4 - - name: Setup vcpkg - shell: bash - run: | - git clone https://github.com/microsoft/vcpkg.git - cd vcpkg - ./bootstrap-vcpkg.bat - ./vcpkg integrate install - - name: Install dependencies - shell: pwsh run: | - $env:VCPKG_ROOT/vcpkg install openssl:x64-windows-static cyrus-sasl:x64-windows-static - echo "OPENSSL_DIR=$env:VCPKG_ROOT/installed/x64-windows-static" >> $env:GITHUB_ENV - echo "OPENSSL_STATIC=1" >> $env:GITHUB_ENV - echo "SASL2_DIR=$env:VCPKG_ROOT/installed/x64-windows-static" >> $env:GITHUB_ENV - echo "SASL2_STATIC=1" >> $env:GITHUB_ENV + choco install cmake -y + choco install llvm -y - - uses: dtolnay/rust-toolchain@stable + - uses: actions-rs/toolchain@v1 with: - targets: x86_64-pc-windows-msvc + toolchain: stable + profile: minimal # minimal component installation (ie, no documentation) + default: true + override: true - name: Build on windows - run: cargo build --target x86_64-pc-windows-msvc --features "rdkafka-ssl,rdkafka-sasl" + run: cargo build --target x86_64-pc-windows-msvc --features kafka build-macos: name: Build for ${{matrix.target}} runs-on: macos-latest - env: - MACOSX_DEPLOYMENT_TARGET: "11.0" # Updated minimum target strategy: matrix: target: @@ -124,39 +79,14 @@ jobs: - name: Install dependencies run: | - brew install openssl@3 cyrus-sasl pkg-config + brew install cmake llvm librdkafka openssl - - name: Set environment variables - run: | - # OpenSSL configuration - echo "OPENSSL_DIR=$(brew --prefix openssl@3)" >> $GITHUB_ENV - echo "OPENSSL_INCLUDE_DIR=$(brew --prefix openssl@3)/include" >> $GITHUB_ENV - echo "OPENSSL_LIB_DIR=$(brew --prefix openssl@3)/lib" >> $GITHUB_ENV - echo "PKG_CONFIG_PATH=$(brew --prefix openssl@3)/lib/pkgconfig:$(brew --prefix cyrus-sasl)/lib/pkgconfig" >> $GITHUB_ENV - - # SASL configuration - echo "SASL2_DIR=$(brew --prefix cyrus-sasl)" >> $GITHUB_ENV - echo "LDFLAGS=-L$(brew --prefix openssl@3)/lib -L$(brew --prefix cyrus-sasl)/lib" >> $GITHUB_ENV - echo "CPPFLAGS=-I$(brew --prefix openssl@3)/include -I$(brew --prefix cyrus-sasl)/include" >> $GITHUB_ENV - - - uses: dtolnay/rust-toolchain@stable + - uses: actions-rs/toolchain@v1 with: - targets: ${{ matrix.target }} - - - name: Create .cargo/config.toml - run: | - mkdir -p .cargo - cat > .cargo/config.toml << 'EOF' - [target.'cfg(target_os = "macos")'] - rustflags = [ - "-C", "link-arg=-L/usr/local/lib", - "-C", "link-arg=-L$(brew --prefix openssl@3)/lib", - "-C", "link-arg=-L$(brew --prefix cyrus-sasl)/lib" - ] - EOF + toolchain: stable + profile: minimal + target: ${{ matrix.target }} + override: true - name: Build on ${{ matrix.target }} - run: | - # Ensure all dynamic libraries are findable - export DYLD_LIBRARY_PATH="$(brew --prefix openssl@3)/lib:$(brew --prefix cyrus-sasl)/lib:$DYLD_LIBRARY_PATH" - cargo build --target ${{ matrix.target }} --features "rdkafka-ssl,rdkafka-sasl" + run: cargo build --target ${{ matrix.target }} --features kafka \ No newline at end of file diff --git a/Cargo.lock b/Cargo.lock index eb03e073f..69da2cefb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -286,7 +286,7 @@ checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" dependencies = [ "cfg-if", "const-random", - "getrandom", + "getrandom 0.2.15", "once_cell", "version_check", "zerocopy", @@ -929,9 +929,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.16.0" +version = "3.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" +checksum = "1628fb46dfa0b37568d12e5edd512553eccf6a22a78e8bde00bb4aed84d5bdbf" [[package]] name = "byteorder" @@ -1155,9 +1155,9 @@ dependencies = [ [[package]] name = "cmake" -version = "0.1.52" +version = "0.1.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c682c223677e0e5b6b7f63a64b9351844c3f1b1678a68b7ee617e30fb082620e" +checksum = "e24a03c8b52922d68a1589ad61032f2c1aa5a8158d2aa0d93c6e9534944bbad6" dependencies = [ "cc", ] @@ -1194,7 +1194,7 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ - "getrandom", + "getrandom 0.2.15", "once_cell", "tiny-keccak", ] @@ -2198,6 +2198,18 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "getrandom" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43a49c392881ce6d5c3b8cb70f98717b7c07aabbdff06687b9030dbfbe2725f8" +dependencies = [ + "cfg-if", + "libc", + "wasi 0.13.3+wasi-0.2.2", + "windows-targets 0.52.6", +] + [[package]] name = "gimli" version = "0.31.1" @@ -2377,9 +2389,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.9.5" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d71d3574edd2771538b901e6549113b4006ece66150fb69c0fb6d9a2adae946" +checksum = "f2d708df4e7140240a16cd6ab0ab65c972d7433ab77819ea693fde9c43811e2a" [[package]] name = "httpdate" @@ -2435,9 +2447,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.5.2" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "256fb8d4bd6413123cc9d91832d78325c48ff41677595be797d90f42969beae0" +checksum = "cc2b571658e38e0c01b1fdca3bbbe93c00d3d71693ff2770043f8c29bc7d6f80" dependencies = [ "bytes", "futures-channel", @@ -2476,7 +2488,7 @@ checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2" dependencies = [ "futures-util", "http 1.2.0", - "hyper 1.5.2", + "hyper 1.6.0", "hyper-util", "rustls 0.23.21", "rustls-native-certs", @@ -2493,7 +2505,7 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2b90d566bffbce6a75bd8b09a05aa8c2cb1fabb6cb348f8840c9e4c90a0d83b0" dependencies = [ - "hyper 1.5.2", + "hyper 1.6.0", "hyper-util", "pin-project-lite", "tokio", @@ -2511,7 +2523,7 @@ dependencies = [ "futures-util", "http 1.2.0", "http-body 1.0.1", - "hyper 1.5.2", + "hyper 1.6.0", "pin-project-lite", "socket2", "tokio", @@ -3227,7 +3239,7 @@ dependencies = [ "futures", "httparse", "humantime", - "hyper 1.5.2", + "hyper 1.6.0", "itertools 0.13.0", "md-5", "parking_lot", @@ -3483,7 +3495,6 @@ dependencies = [ "humantime-serde", "itertools 0.14.0", "lazy_static", - "mime", "nom", "num_cpus", "object_store", @@ -3866,7 +3877,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2fe5ef3495d7d2e377ff17b1a8ce2ee2ec2a18cde8b6ad6619d65d0701c135d" dependencies = [ "bytes", - "getrandom", + "getrandom 0.2.15", "rand", "ring", "rustc-hash", @@ -3929,7 +3940,7 @@ version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" dependencies = [ - "getrandom", + "getrandom 0.2.15", ] [[package]] @@ -4140,7 +4151,7 @@ dependencies = [ "http 1.2.0", "http-body 1.0.1", "http-body-util", - "hyper 1.5.2", + "hyper 1.6.0", "hyper-rustls 0.27.5", "hyper-util", "ipnet", @@ -4181,7 +4192,7 @@ checksum = "c17fa4cb658e3583423e915b9f3acc01cceaee1860e33d59ebae66adc3a2dc0d" dependencies = [ "cc", "cfg-if", - "getrandom", + "getrandom 0.2.15", "libc", "spin", "untrusted", @@ -4325,9 +4336,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.10.1" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2bf47e6ff922db3825eb750c4e2ff784c6ff8fb9e13046ef6a1d1c5401b0b37" +checksum = "917ce264624a4b4db1c364dcc35bfca9ded014d0a958cd47ad3e960e988ea51c" dependencies = [ "web-time", ] @@ -4361,9 +4372,9 @@ checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "ryu" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" +checksum = "6ea1a2d0a644769cc99faa24c3ad26b379b786fe7c36fd3c546254801650e6dd" [[package]] name = "same-file" @@ -4471,9 +4482,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.137" +version = "1.0.138" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "930cfb6e6abf99298aaad7d29abbef7a9999a9a8806a40088f55f0dcec03146b" +checksum = "d434192e7da787e94a6ea7e9670b26a036d0ca41e0b7efb2676dd32bae872949" dependencies = [ "indexmap 2.7.1", "itoa", @@ -4839,13 +4850,13 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.15.0" +version = "3.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a8a559c81686f576e8cd0290cd2a24a2a9ad80c98b3478856500fcbd7acd704" +checksum = "38c246215d7d24f48ae091a2902398798e05d978b24315d6efbc00ede9a8bb91" dependencies = [ "cfg-if", "fastrand 2.3.0", - "getrandom", + "getrandom 0.3.1", "once_cell", "rustix", "windows-sys 0.59.0", @@ -5113,7 +5124,7 @@ dependencies = [ "http 1.2.0", "http-body 1.0.1", "http-body-util", - "hyper 1.5.2", + "hyper 1.6.0", "hyper-timeout", "hyper-util", "percent-encoding", @@ -5338,9 +5349,9 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.15" +version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11cd88e12b17c6494200a9c1b683a04fcac9573ed74cd1b62aeb2727c5592243" +checksum = "a210d160f08b701c8721ba1c726c11662f877ea6b7094007e1ca9a1041945034" [[package]] name = "unicode-normalization" @@ -5438,7 +5449,7 @@ version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b3758f5e68192bb96cc8f9b7e2c2cfdabb435499a28499a42f8f984092adad4b" dependencies = [ - "getrandom", + "getrandom 0.2.15", ] [[package]] @@ -5568,6 +5579,15 @@ version = "0.11.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" +[[package]] +name = "wasi" +version = "0.13.3+wasi-0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26816d2e1a4a36a2940b96c5296ce403917633dff8f3440e9b236ed6f6bacad2" +dependencies = [ + "wit-bindgen-rt", +] + [[package]] name = "wasm-bindgen" version = "0.2.100" @@ -5970,9 +5990,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" -version = "0.6.24" +version = "0.6.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8d71a593cc5c42ad7876e2c1fda56f314f3754c084128833e64f1345ff8a03a" +checksum = "ad699df48212c6cc6eb4435f35500ac6fd3b9913324f938aea302022ce19d310" dependencies = [ "memchr", ] @@ -5987,6 +6007,15 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "wit-bindgen-rt" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3268f3d866458b787f390cf61f4bbb563b922d091359f9608842999eaee3943c" +dependencies = [ + "bitflags 2.8.0", +] + [[package]] name = "write16" version = "1.0.0" diff --git a/Cargo.toml b/Cargo.toml index b0bdcc65f..adca0074a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -27,7 +27,6 @@ actix-web-prometheus = { version = "0.1" } actix-web-static-files = "4.0" http = "0.2.7" http-auth-basic = "0.3.3" -mime = "0.3.17" tonic = { version = "0.12.3", features = ["tls", "transport", "gzip", "zstd"] } tonic-web = "0.12.3" tower-http = { version = "0.6.1", features = ["cors"] } @@ -144,10 +143,7 @@ assets-sha1 = "4516db38c8e556707b29b33569f9b1e53d5165f2" [features] debug = [] - -rdkafka-ssl-vendored = ["rdkafka", "rdkafka/ssl-vendored"] -rdkafka-ssl = ["rdkafka", "rdkafka/ssl"] -rdkafka-sasl = ["rdkafka", "rdkafka/sasl"] +kafka = ["rdkafka", "rdkafka/ssl-vendored", "rdkafka/ssl", "rdkafka/sasl"] [profile.release-lto] inherits = "release" diff --git a/Dockerfile.kafka b/Dockerfile.kafka index de06d40a2..be9c7c07e 100644 --- a/Dockerfile.kafka +++ b/Dockerfile.kafka @@ -36,7 +36,7 @@ WORKDIR /parseable COPY . . # Build with kafka feature -RUN cargo build --release --features "rdkafka-ssl,rdkafka-sasl" +RUN cargo build --release --features kafka # final stage FROM gcr.io/distroless/cc-debian12:latest diff --git a/src/cli.rs b/src/cli.rs index 64f37ee23..d19eadd9b 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -21,11 +21,7 @@ use std::path::PathBuf; use url::Url; -#[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" -))] +#[cfg(feature = "kafka")] use crate::connectors::kafka::config::KafkaConfig; use crate::{ @@ -91,11 +87,7 @@ pub struct LocalStoreArgs { pub options: Options, #[command(flatten)] pub storage: FSConfig, - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + #[cfg(feature = "kafka")] #[command(flatten)] pub kafka: KafkaConfig, } @@ -106,11 +98,7 @@ pub struct S3StoreArgs { pub options: Options, #[command(flatten)] pub storage: S3Config, - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + #[cfg(feature = "kafka")] #[command(flatten)] pub kafka: KafkaConfig, } @@ -121,11 +109,7 @@ pub struct BlobStoreArgs { pub options: Options, #[command(flatten)] pub storage: AzureBlobConfig, - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + #[cfg(feature = "kafka")] #[command(flatten)] pub kafka: KafkaConfig, } diff --git a/src/lib.rs b/src/lib.rs index 56e4d1b64..d7247176f 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -23,11 +23,7 @@ pub mod audit; pub mod banner; mod catalog; mod cli; -#[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" -))] +#[cfg(feature = "kafka")] pub mod connectors; pub mod correlation; mod event; diff --git a/src/main.rs b/src/main.rs index 7c1d7e0d4..82776ed43 100644 --- a/src/main.rs +++ b/src/main.rs @@ -15,12 +15,7 @@ * along with this program. If not, see . * */ - -#[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" -))] +#[cfg(feature = "kafka")] use parseable::connectors; use parseable::{ banner, metrics, @@ -29,8 +24,8 @@ use parseable::{ }; use tokio::signal::ctrl_c; use tokio::sync::oneshot; -use tracing::info; use tracing::level_filters::LevelFilter; +use tracing::{info, warn}; use tracing_subscriber::layer::SubscriberExt; use tracing_subscriber::util::SubscriberInitExt; use tracing_subscriber::{fmt, EnvFilter, Registry}; @@ -61,17 +56,13 @@ async fn main() -> anyhow::Result<()> { block_until_shutdown_signal().await; // Trigger graceful shutdown - println!("Received shutdown signal, notifying server to shut down..."); + warn!("Received shutdown signal, notifying server to shut down..."); shutdown_trigger.send(()).unwrap(); }); let prometheus = metrics::build_metrics_handler(); - - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + // Start servers + #[cfg(feature = "kafka")] { let parseable_server = server.init(&prometheus, shutdown_rx); let connectors = connectors::init(&prometheus); @@ -79,11 +70,7 @@ async fn main() -> anyhow::Result<()> { tokio::try_join!(parseable_server, connectors)?; } - #[cfg(not(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - )))] + #[cfg(not(feature = "kafka"))] { let parseable_server = server.init(&prometheus, shutdown_rx); parseable_server.await?; @@ -101,6 +88,7 @@ pub fn init_logger(default_level: LevelFilter) { .with_thread_ids(true) .with_line_number(true) .with_timer(tracing_subscriber::fmt::time::UtcTime::rfc_3339()) + .with_target(true) .compact(); Registry::default() diff --git a/src/option.rs b/src/option.rs index 9c524a2a5..7050c8ea1 100644 --- a/src/option.rs +++ b/src/option.rs @@ -17,11 +17,7 @@ */ use crate::cli::{Cli, Options, StorageOptions, DEFAULT_PASSWORD, DEFAULT_USERNAME}; -#[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" -))] +#[cfg(feature = "kafka")] use crate::connectors::kafka::config::KafkaConfig; use crate::storage::object_storage::parseable_json_path; use crate::storage::{ObjectStorageError, ObjectStorageProvider}; @@ -43,11 +39,7 @@ pub struct Config { pub options: Options, storage: Arc, pub storage_name: &'static str, - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + #[cfg(feature = "kafka")] pub kafka_config: KafkaConfig, } @@ -75,11 +67,7 @@ impl Config { options: args.options, storage: Arc::new(args.storage), storage_name: "drive", - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + #[cfg(feature = "kafka")] kafka_config: args.kafka, } } @@ -87,22 +75,14 @@ impl Config { options: args.options, storage: Arc::new(args.storage), storage_name: "s3", - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + #[cfg(feature = "kafka")] kafka_config: args.kafka, }, StorageOptions::Blob(args) => Config { options: args.options, storage: Arc::new(args.storage), storage_name: "blob_store", - #[cfg(any( - feature = "rdkafka-ssl", - feature = "rdkafka-ssl-vendored", - feature = "rdkafka-sasl" - ))] + #[cfg(feature = "kafka")] kafka_config: args.kafka, }, } From 4508698e18c51e54b76587c7a5e7205f0c963c62 Mon Sep 17 00:00:00 2001 From: hippalus Date: Tue, 28 Jan 2025 23:59:25 +0100 Subject: [PATCH 52/65] install sasl --- .github/workflows/build.yaml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index c7de61a50..8a7339445 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -28,7 +28,7 @@ jobs: - name: Install dependencies run: | sudo apt-get update - sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libssl-dev + sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libssl-dev pkg-config - uses: actions-rs/toolchain@v1 with: @@ -37,7 +37,8 @@ jobs: target: ${{ matrix.target }} override: true - - uses: actions-rs/cargo@v1 + - name: Build with Kafka feature + uses: actions-rs/cargo@v1 with: use-cross: true command: build @@ -79,7 +80,7 @@ jobs: - name: Install dependencies run: | - brew install cmake llvm librdkafka openssl + brew install cmake llvm librdkafka openssl cyrus-sasl pkg-config - uses: actions-rs/toolchain@v1 with: From af476e4e84fecb9b8764fce4df3e2232804c3119 Mon Sep 17 00:00:00 2001 From: hippalus Date: Wed, 29 Jan 2025 00:17:40 +0100 Subject: [PATCH 53/65] install ca-cert --- .github/workflows/build.yaml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 8a7339445..894e3f39e 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -28,7 +28,7 @@ jobs: - name: Install dependencies run: | sudo apt-get update - sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libssl-dev pkg-config + sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2 libssl-dev pkg-config ca-certificates build-essential - uses: actions-rs/toolchain@v1 with: @@ -51,11 +51,6 @@ jobs: steps: - uses: actions/checkout@v4 - - name: Install dependencies - run: | - choco install cmake -y - choco install llvm -y - - uses: actions-rs/toolchain@v1 with: toolchain: stable @@ -64,7 +59,7 @@ jobs: override: true - name: Build on windows - run: cargo build --target x86_64-pc-windows-msvc --features kafka + run: cargo build --target x86_64-pc-windows-msvc build-macos: name: Build for ${{matrix.target}} @@ -80,7 +75,7 @@ jobs: - name: Install dependencies run: | - brew install cmake llvm librdkafka openssl cyrus-sasl pkg-config + brew install cmake llvm librdkafka openssl cyrus-sasl pkg-config ca-certificates build-essential - uses: actions-rs/toolchain@v1 with: From 751ab01b10e5af7512569eef461ecd6e377656c8 Mon Sep 17 00:00:00 2001 From: hippalus Date: Wed, 29 Jan 2025 00:22:55 +0100 Subject: [PATCH 54/65] install sasl2 --- .github/workflows/build.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 894e3f39e..6e154d08d 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -28,7 +28,7 @@ jobs: - name: Install dependencies run: | sudo apt-get update - sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2 libssl-dev pkg-config ca-certificates build-essential + sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2-2 libssl-dev pkg-config ca-certificates build-essential - uses: actions-rs/toolchain@v1 with: From 4ba221e301b23a9bc131483b02cbc6346fbd7b0b Mon Sep 17 00:00:00 2001 From: hippalus Date: Wed, 29 Jan 2025 00:28:54 +0100 Subject: [PATCH 55/65] install sasl2 --- .github/workflows/build.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 6e154d08d..22896b332 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -28,7 +28,7 @@ jobs: - name: Install dependencies run: | sudo apt-get update - sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2-2 libssl-dev pkg-config ca-certificates build-essential + sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2-2 libssl-dev pkg-config ca-certificates - uses: actions-rs/toolchain@v1 with: From b184ed584585da7d9ca58043fdb4b1ba9127ab72 Mon Sep 17 00:00:00 2001 From: hippalus Date: Wed, 29 Jan 2025 00:51:33 +0100 Subject: [PATCH 56/65] build essential --- .github/workflows/build.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 22896b332..b1bceb50c 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -28,7 +28,7 @@ jobs: - name: Install dependencies run: | sudo apt-get update - sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2-2 libssl-dev pkg-config ca-certificates + sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2-2 libssl-dev pkg-config ca-certificates build-essential - uses: actions-rs/toolchain@v1 with: @@ -75,7 +75,7 @@ jobs: - name: Install dependencies run: | - brew install cmake llvm librdkafka openssl cyrus-sasl pkg-config ca-certificates build-essential + brew install cmake llvm librdkafka openssl cyrus-sasl pkg-config ca-certificates - uses: actions-rs/toolchain@v1 with: From 5f2bf057e7bc10ab0fc5e4663fd25a8d4f79b82c Mon Sep 17 00:00:00 2001 From: hippalus Date: Wed, 29 Jan 2025 01:12:42 +0100 Subject: [PATCH 57/65] Use separate jobs for kafka and default build. --- .github/workflows/build.yaml | 174 +++++++++++++++++++++++++---------- 1 file changed, 124 insertions(+), 50 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index b1bceb50c..7bb7e5523 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -1,4 +1,4 @@ -name: Ensure parseable builds on all release targets with Kafka feature +name: Ensure parseable builds on all release targets on: pull_request: @@ -12,77 +12,151 @@ on: - main jobs: - build-linux: - name: Build for ${{matrix.target}} - runs-on: ubuntu-latest + # Default build without Kafka + build-default: + name: Build Default ${{matrix.target}} + runs-on: ${{ matrix.os }} strategy: fail-fast: false matrix: - target: - - aarch64-unknown-linux-gnu # linux(arm) - - x86_64-unknown-linux-gnu # linux(64 bit) + include: + # Linux builds + - os: ubuntu-latest + target: x86_64-unknown-linux-gnu + - os: ubuntu-latest + target: aarch64-unknown-linux-gnu + # macOS builds + - os: macos-latest + target: x86_64-apple-darwin + - os: macos-latest + target: aarch64-apple-darwin + # Windows build + - os: windows-latest + target: x86_64-pc-windows-msvc steps: - uses: actions/checkout@v4 - - name: Install dependencies - run: | - sudo apt-get update - sudo apt-get install -y cmake clang librdkafka-dev libsasl2-dev libsasl2-2 libssl-dev pkg-config ca-certificates build-essential + - name: Setup Rust toolchain + uses: dtolnay/rust-toolchain@stable + with: + targets: ${{ matrix.target }} - - uses: actions-rs/toolchain@v1 + - name: Cache dependencies + uses: actions/cache@v4 with: - toolchain: stable - profile: minimal # minimal component installation (ie, no documentation) - target: ${{ matrix.target }} - override: true + path: | + ~/.cargo/registry + ~/.cargo/git + target + key: ${{ runner.os }}-cargo-${{ matrix.target }}-default-${{ hashFiles('**/Cargo.lock') }} - - name: Build with Kafka feature + - name: Build uses: actions-rs/cargo@v1 with: - use-cross: true + use-cross: ${{ runner.os == 'Linux' }} command: build - args: --target ${{matrix.target}} --features kafka + args: --target ${{ matrix.target }} --release - build-windows: - name: Build for windows - runs-on: windows-latest + # Kafka build for supported platforms + build-kafka: + name: Build Kafka ${{matrix.target}} + runs-on: ${{ matrix.os }} + strategy: + fail-fast: false + matrix: + include: + # Linux builds + - os: ubuntu-latest + target: x86_64-unknown-linux-gnu + - os: ubuntu-latest + target: aarch64-unknown-linux-gnu + # macOS builds + - os: macos-latest + target: x86_64-apple-darwin + - os: macos-latest + target: aarch64-apple-darwin steps: - uses: actions/checkout@v4 - - uses: actions-rs/toolchain@v1 - with: - toolchain: stable - profile: minimal # minimal component installation (ie, no documentation) - default: true - override: true - - - name: Build on windows - run: cargo build --target x86_64-pc-windows-msvc + # Linux-specific dependencies + - name: Install Linux dependencies + if: runner.os == 'Linux' + run: | + sudo apt-get update + sudo apt-get install -y \ + build-essential \ + pkg-config \ + cmake \ + clang \ + zlib1g-dev \ + libzstd-dev \ + liblz4-dev \ + libssl-dev \ + libsasl2-dev \ + python3 \ + gcc-aarch64-linux-gnu \ + g++-aarch64-linux-gnu - build-macos: - name: Build for ${{matrix.target}} - runs-on: macos-latest - strategy: - matrix: - target: - - aarch64-apple-darwin # macos(arm) - - x86_64-apple-darwin # macos(intel 64 bit) + # macOS-specific dependencies + - name: Install macOS dependencies + if: runner.os == 'macOS' + run: | + brew install \ + cmake \ + llvm \ + pkg-config \ + zstd \ + lz4 \ + openssl@3.4.0 \ + cyrus-sasl \ + python3 - steps: - - uses: actions/checkout@v4 + # OpenSSL setup for Linux + - name: Setup OpenSSL on Linux + if: runner.os == 'Linux' + run: | + curl -O https://www.openssl.org/source/openssl-3.4.0.tar.gz + tar xf openssl-3.4.0.tar.gz + cd openssl-3.4.0 + ./config --prefix=/usr/local/ssl --openssldir=/usr/local/ssl shared zlib + make -j$(nproc) + sudo make install + echo "OPENSSL_ROOT_DIR=/usr/local/ssl" >> $GITHUB_ENV + echo "OPENSSL_INCLUDE_DIR=/usr/local/ssl/include" >> $GITHUB_ENV + echo "OPENSSL_LIB_DIR=/usr/local/ssl/lib" >> $GITHUB_ENV + echo "LD_LIBRARY_PATH=/usr/local/ssl/lib:$LD_LIBRARY_PATH" >> $GITHUB_ENV + echo "PKG_CONFIG_PATH=/usr/local/ssl/lib/pkgconfig:$PKG_CONFIG_PATH" >> $GITHUB_ENV - - name: Install dependencies + # OpenSSL setup for macOS + - name: Setup OpenSSL on macOS + if: runner.os == 'macOS' run: | - brew install cmake llvm librdkafka openssl cyrus-sasl pkg-config ca-certificates + echo "OPENSSL_ROOT_DIR=$(brew --prefix openssl@3.4.0)" >> $GITHUB_ENV + echo "OPENSSL_INCLUDE_DIR=$(brew --prefix openssl@3.4.0)/include" >> $GITHUB_ENV + echo "OPENSSL_LIB_DIR=$(brew --prefix openssl@3.4.0)/lib" >> $GITHUB_ENV + echo "PKG_CONFIG_PATH=$(brew --prefix openssl@3.4.0)/lib/pkgconfig:$PKG_CONFIG_PATH" >> $GITHUB_ENV - - uses: actions-rs/toolchain@v1 + - name: Setup Rust toolchain + uses: dtolnay/rust-toolchain@stable with: - toolchain: stable - profile: minimal - target: ${{ matrix.target }} - override: true + targets: ${{ matrix.target }} - - name: Build on ${{ matrix.target }} - run: cargo build --target ${{ matrix.target }} --features kafka \ No newline at end of file + - name: Cache dependencies + uses: actions/cache@v4 + with: + path: | + ~/.cargo/registry + ~/.cargo/git + target + key: ${{ runner.os }}-cargo-${{ matrix.target }}-kafka-${{ hashFiles('**/Cargo.lock') }} + + - name: Build with Kafka + uses: actions-rs/cargo@v1 + with: + use-cross: ${{ runner.os == 'Linux' }} + command: build + args: --target ${{ matrix.target }} --features kafka --release + env: + LIBRDKAFKA_SSL_VENDORED: 1 \ No newline at end of file From a1a7e8b2e8a9d50ba5ef522b6f7c985766a79ed5 Mon Sep 17 00:00:00 2001 From: hippalus Date: Wed, 29 Jan 2025 01:18:34 +0100 Subject: [PATCH 58/65] Use separate jobs for kafka and default build. --- .github/workflows/build.yaml | 27 +-------------------------- 1 file changed, 1 insertion(+), 26 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 7bb7e5523..d50457fd4 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -109,35 +109,10 @@ jobs: pkg-config \ zstd \ lz4 \ - openssl@3.4.0 \ + openssl@3.0 \ cyrus-sasl \ python3 - # OpenSSL setup for Linux - - name: Setup OpenSSL on Linux - if: runner.os == 'Linux' - run: | - curl -O https://www.openssl.org/source/openssl-3.4.0.tar.gz - tar xf openssl-3.4.0.tar.gz - cd openssl-3.4.0 - ./config --prefix=/usr/local/ssl --openssldir=/usr/local/ssl shared zlib - make -j$(nproc) - sudo make install - echo "OPENSSL_ROOT_DIR=/usr/local/ssl" >> $GITHUB_ENV - echo "OPENSSL_INCLUDE_DIR=/usr/local/ssl/include" >> $GITHUB_ENV - echo "OPENSSL_LIB_DIR=/usr/local/ssl/lib" >> $GITHUB_ENV - echo "LD_LIBRARY_PATH=/usr/local/ssl/lib:$LD_LIBRARY_PATH" >> $GITHUB_ENV - echo "PKG_CONFIG_PATH=/usr/local/ssl/lib/pkgconfig:$PKG_CONFIG_PATH" >> $GITHUB_ENV - - # OpenSSL setup for macOS - - name: Setup OpenSSL on macOS - if: runner.os == 'macOS' - run: | - echo "OPENSSL_ROOT_DIR=$(brew --prefix openssl@3.4.0)" >> $GITHUB_ENV - echo "OPENSSL_INCLUDE_DIR=$(brew --prefix openssl@3.4.0)/include" >> $GITHUB_ENV - echo "OPENSSL_LIB_DIR=$(brew --prefix openssl@3.4.0)/lib" >> $GITHUB_ENV - echo "PKG_CONFIG_PATH=$(brew --prefix openssl@3.4.0)/lib/pkgconfig:$PKG_CONFIG_PATH" >> $GITHUB_ENV - - name: Setup Rust toolchain uses: dtolnay/rust-toolchain@stable with: From 40a91f88f82e79c62c3660dd5cd84bc044befef1 Mon Sep 17 00:00:00 2001 From: hippalus Date: Wed, 29 Jan 2025 01:33:29 +0100 Subject: [PATCH 59/65] update kafka build dep --- .github/workflows/build.yaml | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index d50457fd4..924b60336 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -98,6 +98,18 @@ jobs: python3 \ gcc-aarch64-linux-gnu \ g++-aarch64-linux-gnu + + # Install cross-compilation specific packages + if [ "${{ matrix.target }}" = "aarch64-unknown-linux-gnu" ]; then + sudo apt-get install -y \ + gcc-aarch64-linux-gnu \ + g++-aarch64-linux-gnu \ + libc6-dev-arm64-cross \ + libsasl2-dev:arm64 \ + libssl-dev:arm64 \ + pkg-config-aarch64-linux-gnu + fi + # macOS-specific dependencies - name: Install macOS dependencies @@ -134,4 +146,11 @@ jobs: command: build args: --target ${{ matrix.target }} --features kafka --release env: - LIBRDKAFKA_SSL_VENDORED: 1 \ No newline at end of file + LIBRDKAFKA_SSL_VENDORED: 1 + PKG_CONFIG_ALLOW_CROSS: "1" + PKG_CONFIG_PATH: "/usr/lib/aarch64-linux-gnu/pkgconfig" + SASL2_DIR: "/usr/lib/aarch64-linux-gnu" + OPENSSL_DIR: "/usr/lib/aarch64-linux-gnu" + OPENSSL_ROOT_DIR: "/usr/lib/aarch64-linux-gnu" + OPENSSL_STATIC: "1" + SASL2_STATIC: "0" From 14a9d4b1eef6774d7a85afb76973d079ab9d8653 Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 30 Jan 2025 22:29:40 +0100 Subject: [PATCH 60/65] remove aarch64-unknown-linux-gnu for kafka builds --- .github/workflows/build.yaml | 2 -- Dockerfile | 2 +- Dockerfile.debug | 2 +- Dockerfile.kafka | 12 ++++++++++-- 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index e716ef790..29dda8485 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -66,8 +66,6 @@ jobs: # Linux builds - os: ubuntu-latest target: x86_64-unknown-linux-gnu - - os: ubuntu-latest - target: aarch64-unknown-linux-gnu # macOS builds - os: macos-latest target: x86_64-apple-darwin diff --git a/Dockerfile b/Dockerfile index 40883f50d..a9980e8d9 100644 --- a/Dockerfile +++ b/Dockerfile @@ -25,7 +25,7 @@ LABEL org.opencontainers.image.licenses="AGPL-3.0" WORKDIR /parseable # Cache dependencies -COPY Cargo.toml Cargo.lock build.rs .git ./ +COPY Cargo.toml Cargo.lock build.rs ./ RUN mkdir src && echo "fn main() {}" > src/main.rs && cargo build --release && rm -rf src # Build the actual binary diff --git a/Dockerfile.debug b/Dockerfile.debug index de3f2e2b2..1d21291f0 100644 --- a/Dockerfile.debug +++ b/Dockerfile.debug @@ -24,7 +24,7 @@ LABEL org.opencontainers.image.licenses="AGPL-3.0" WORKDIR /parseable # Cache dependencies -COPY Cargo.toml Cargo.lock build.rs .git ./ +COPY Cargo.toml Cargo.lock build.rs ./ RUN mkdir src && echo "fn main() {}" > src/main.rs && cargo build && rm -rf src # Build the actual binary diff --git a/Dockerfile.kafka b/Dockerfile.kafka index be9c7c07e..0c45fe7d6 100644 --- a/Dockerfile.kafka +++ b/Dockerfile.kafka @@ -33,9 +33,17 @@ RUN apt-get update && \ rm -rf /var/lib/apt/lists/* WORKDIR /parseable -COPY . . +COPY Cargo.toml Cargo.lock build.rs ./ -# Build with kafka feature +# Create a dummy main.rs to pre-cache dependencies +RUN mkdir src && echo "fn main() {}" > src/main.rs && \ + cargo build --release --features kafka && \ + rm -rf src + +# Copy the actual source code +COPY src ./src + +# Build the actual binary with kafka feature RUN cargo build --release --features kafka # final stage From f17dc5fd5ebf439b01124b709adc121fd1c3442a Mon Sep 17 00:00:00 2001 From: hippalus Date: Thu, 30 Jan 2025 23:55:19 +0100 Subject: [PATCH 61/65] add topic validation and sasl-sys --- Cargo.lock | 25 +++++++++++++------------ Cargo.toml | 5 +++-- src/connectors/kafka/config.rs | 3 +++ 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 69da2cefb..6ac609a06 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2490,7 +2490,7 @@ dependencies = [ "http 1.2.0", "hyper 1.6.0", "hyper-util", - "rustls 0.23.21", + "rustls 0.23.22", "rustls-native-certs", "rustls-pki-types", "tokio", @@ -3513,6 +3513,7 @@ dependencies = [ "rstest", "rustls 0.22.4", "rustls-pemfile 2.2.0", + "sasl2-sys", "semver", "serde", "serde_json", @@ -3863,7 +3864,7 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.21", + "rustls 0.23.22", "socket2", "thiserror 2.0.11", "tokio", @@ -3881,7 +3882,7 @@ dependencies = [ "rand", "ring", "rustc-hash", - "rustls 0.23.21", + "rustls 0.23.22", "rustls-pki-types", "slab", "thiserror 2.0.11", @@ -4162,7 +4163,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.21", + "rustls 0.23.22", "rustls-native-certs", "rustls-pemfile 2.2.0", "rustls-pki-types", @@ -4291,9 +4292,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.21" +version = "0.23.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f287924602bf649d949c63dc8ac8b235fa5387d394020705b80c4eb597ce5b8" +checksum = "9fb9263ab4eb695e42321db096e3b8fbd715a59b154d5c88d82db2175b681ba7" dependencies = [ "log", "once_cell", @@ -5046,7 +5047,7 @@ version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" dependencies = [ - "rustls 0.23.21", + "rustls 0.23.22", "tokio", ] @@ -5097,9 +5098,9 @@ dependencies = [ [[package]] name = "toml_edit" -version = "0.22.22" +version = "0.22.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" +checksum = "02a8b472d1a3d7c18e2d61a489aee3453fd9031c33e4f55bd533f4a7adca1bee" dependencies = [ "indexmap 2.7.1", "serde", @@ -5407,7 +5408,7 @@ dependencies = [ "flate2", "log", "once_cell", - "rustls 0.23.21", + "rustls 0.23.22", "rustls-pki-types", "url", "webpki-roots 0.26.7", @@ -5990,9 +5991,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" -version = "0.6.25" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad699df48212c6cc6eb4435f35500ac6fd3b9913324f938aea302022ce19d310" +checksum = "7e49d2d35d3fad69b39b94139037ecfb4f359f08958b9c11e7315ce770462419" dependencies = [ "memchr", ] diff --git a/Cargo.toml b/Cargo.toml index adca0074a..662b8d02f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,6 +34,7 @@ url = "2.4.0" # Connectors dependencies rdkafka = { version = "0.37", optional = true, features = ["cmake-build", "tracing", "libz-static"] } +sasl2-sys = { version = "0.1.22", optional = true, features = ["vendored"] } # Authentication and Security argon2 = "0.5.0" @@ -107,7 +108,7 @@ derive_more = { version = "1", features = ["full"] } itertools = "0.14" lazy_static = "1.4" nom = "7.1.3" -once_cell = "1.17.1" +once_cell = "1.20" rand = "0.8.5" regex = "1.7.3" reqwest = { version = "0.11.27", default-features = false, features = [ @@ -143,7 +144,7 @@ assets-sha1 = "4516db38c8e556707b29b33569f9b1e53d5165f2" [features] debug = [] -kafka = ["rdkafka", "rdkafka/ssl-vendored", "rdkafka/ssl", "rdkafka/sasl"] +kafka = ["rdkafka", "sasl2-sys", "sasl2-sys/vendored", "rdkafka/ssl-vendored", "rdkafka/ssl", "rdkafka/sasl"] [profile.release-lto] inherits = "release" diff --git a/src/connectors/kafka/config.rs b/src/connectors/kafka/config.rs index 85c705ce5..ee7e77446 100644 --- a/src/connectors/kafka/config.rs +++ b/src/connectors/kafka/config.rs @@ -615,6 +615,9 @@ impl ConsumerConfig { if self.group_id.is_empty() { anyhow::bail!("Consumer group ID must not be empty"); } + if self.topics.is_empty() { + anyhow::bail!("At least one topic must be specified"); + } Ok(()) } From 60704180abe7f7cd2c2d57e3f774b70a54e09ef7 Mon Sep 17 00:00:00 2001 From: hippalus Date: Fri, 31 Jan 2025 00:30:15 +0100 Subject: [PATCH 62/65] disable x86_64-apple-darwin in kafka build --- .github/workflows/build.yaml | 3 --- 1 file changed, 3 deletions(-) diff --git a/.github/workflows/build.yaml b/.github/workflows/build.yaml index 29dda8485..080a9ba68 100644 --- a/.github/workflows/build.yaml +++ b/.github/workflows/build.yaml @@ -66,9 +66,6 @@ jobs: # Linux builds - os: ubuntu-latest target: x86_64-unknown-linux-gnu - # macOS builds - - os: macos-latest - target: x86_64-apple-darwin - os: macos-latest target: aarch64-apple-darwin From 5e4a10238a196a625895a14eace0d34b0e9316f6 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha <131262146+nikhilsinhaparseable@users.noreply.github.com> Date: Mon, 3 Feb 2025 17:11:14 +0530 Subject: [PATCH 63/65] remove RUST_LOG=info Co-authored-by: Devdutt Shenoi Signed-off-by: Nikhil Sinha <131262146+nikhilsinhaparseable@users.noreply.github.com> --- docker-compose-distributed-test-with-kafka.yaml | 1 - docker-compose-distributed-test.yaml | 1 - docker-compose-test-with-kafka.yaml | 1 - docker-compose-test.yaml | 1 - 4 files changed, 4 deletions(-) diff --git a/docker-compose-distributed-test-with-kafka.yaml b/docker-compose-distributed-test-with-kafka.yaml index e2f20a96b..b77a3100c 100644 --- a/docker-compose-distributed-test-with-kafka.yaml +++ b/docker-compose-distributed-test-with-kafka.yaml @@ -72,7 +72,6 @@ services: - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=query - - RUST_LOG=info networks: - parseable-internal healthcheck: diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index bd01dfb72..b06d0ad31 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -45,7 +45,6 @@ services: - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=query - - RUST_LOG=info networks: - parseable-internal healthcheck: diff --git a/docker-compose-test-with-kafka.yaml b/docker-compose-test-with-kafka.yaml index c532bd470..0e747c097 100644 --- a/docker-compose-test-with-kafka.yaml +++ b/docker-compose-test-with-kafka.yaml @@ -45,7 +45,6 @@ services: - P_PARQUET_COMPRESSION_ALGO=snappy - P_KAFKA_CONSUMER_TOPICS=test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092 - - RUST_LOG=info # additional settings like security, tuning, etc. depends_on: - minio diff --git a/docker-compose-test.yaml b/docker-compose-test.yaml index dce83dd60..0a81d3c8c 100644 --- a/docker-compose-test.yaml +++ b/docker-compose-test.yaml @@ -43,7 +43,6 @@ services: - P_PASSWORD=parseableadmin - P_CHECK_UPDATE=false - P_PARQUET_COMPRESSION_ALGO=snappy - - RUST_LOG=info depends_on: - minio healthcheck: From 9b306876dc1830197b84a0baf9fb9ab44e8e8e65 Mon Sep 17 00:00:00 2001 From: Nitish Tiwari Date: Tue, 4 Feb 2025 11:30:30 +0530 Subject: [PATCH 64/65] Apply suggestions from code review Co-authored-by: Devdutt Shenoi Signed-off-by: Nitish Tiwari --- docker-compose-distributed-test-with-kafka.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker-compose-distributed-test-with-kafka.yaml b/docker-compose-distributed-test-with-kafka.yaml index b77a3100c..d65f7390b 100644 --- a/docker-compose-distributed-test-with-kafka.yaml +++ b/docker-compose-distributed-test-with-kafka.yaml @@ -114,7 +114,6 @@ services: - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - P_KAFKA_PARTITION_LISTENER_CONCURRENCY=3 - - RUST_LOG=info # additional settings like security, tuning, etc. networks: - parseable-internal @@ -161,7 +160,6 @@ services: - P_KAFKA_CONSUMER_TOPICS=dist-test-logs-stream - P_KAFKA_BOOTSTRAP_SERVERS=kafka-0:9092,kafka-1:9092,kafka-2:9092 - P_KAFKA_PARTITION_LISTENER_CONCURRENCY=3 - - RUST_LOG=info # additional settings like security, tuning, etc. networks: - parseable-internal From 4eaf7bf63c8f38181ff2757366ee18b0904d7e65 Mon Sep 17 00:00:00 2001 From: Nitish Tiwari Date: Tue, 4 Feb 2025 11:31:11 +0530 Subject: [PATCH 65/65] Update docker-compose-distributed-test.yaml Co-authored-by: Devdutt Shenoi Signed-off-by: Nitish Tiwari --- docker-compose-distributed-test.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/docker-compose-distributed-test.yaml b/docker-compose-distributed-test.yaml index b06d0ad31..ca79941f3 100644 --- a/docker-compose-distributed-test.yaml +++ b/docker-compose-distributed-test.yaml @@ -81,7 +81,6 @@ services: - P_PARQUET_COMPRESSION_ALGO=snappy - P_MODE=ingest - P_INGESTOR_ENDPOINT=parseable-ingest-one:8000 - - RUST_LOG=info networks: - parseable-internal healthcheck: