From 08688d9182790ed514142bdfba808bfff4ff87c9 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Wed, 7 Sep 2022 19:42:20 -0600 Subject: [PATCH 01/48] enhancement(pulsar sink): Refactor to use StreamSink This commit heavily refactors the Pulsar Sink to use the StreamSink interface and is modeled after the Kafka Sink. It also adds additional features that bring it in line with Kafka Sink feature set. This includes: * Refactoring to use StreamSink instead of Sink interace. See https://github.com/vectordotdev/vector/issues/9261 * Supports dynamic topics using a topic template * Refactor configurations in advance of adding Pulsar source * Rework message parsing to support logs and metrics, with support for dynamic keys and properties This work is heavily modeled after Kafka sink. This means there has been some duplication of some utility code. However, it has not been refactored to remove the duplication as there wasn't a clear pattern of where such shared code should be put. Additionally, this refactor seems to be much simpler by using StreamSink but does require some workarounds limitations in the Pulsar client library by wrapping certain resources in Arc that *may* have performance implications. I am not famaliar enough to know if there might be some efficiencies by structuring this differently. Remaining work: * Add a few more tests --- Cargo.toml | 2 +- src/internal_events/pulsar.rs | 22 ++ src/sinks/mod.rs | 2 +- src/sinks/pulsar.rs | 453 ---------------------------- src/sinks/pulsar/config.rs | 237 +++++++++++++++ src/sinks/pulsar/mod.rs | 5 + src/sinks/pulsar/request_builder.rs | 127 ++++++++ src/sinks/pulsar/service.rs | 194 ++++++++++++ src/sinks/pulsar/sink.rs | 93 ++++++ src/sinks/pulsar/tests.rs | 103 +++++++ 10 files changed, 783 insertions(+), 455 deletions(-) delete mode 100644 src/sinks/pulsar.rs create mode 100644 src/sinks/pulsar/config.rs create mode 100644 src/sinks/pulsar/mod.rs create mode 100644 src/sinks/pulsar/request_builder.rs create mode 100644 src/sinks/pulsar/service.rs create mode 100644 src/sinks/pulsar/sink.rs create mode 100644 src/sinks/pulsar/tests.rs diff --git a/Cargo.toml b/Cargo.toml index cee5a9783d742..1afdd5a494b6c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -664,7 +664,7 @@ sinks-new_relic_logs = ["sinks-http"] sinks-new_relic = [] sinks-papertrail = ["dep:syslog"] sinks-prometheus = ["dep:prometheus-parser", "dep:snap", "dep:serde_with"] -sinks-pulsar = ["dep:avro-rs", "dep:pulsar"] +sinks-pulsar = ["dep:avro-rs", "dep:pulsar", "dep:lru"] sinks-redis = ["dep:redis"] sinks-sematext = ["sinks-elasticsearch", "sinks-influxdb"] sinks-socket = ["sinks-utils-udp"] diff --git a/src/internal_events/pulsar.rs b/src/internal_events/pulsar.rs index e79e1e8d2c674..340fbe29f41fc 100644 --- a/src/internal_events/pulsar.rs +++ b/src/internal_events/pulsar.rs @@ -33,3 +33,25 @@ impl InternalEvent for PulsarSendingError { }); } } + +pub struct PulsarPropertyExtractionError<'a> { + pub property_field: &'a str, +} + +impl InternalEvent for PulsarPropertyExtractionError<'_> { + fn emit(self) { + error!( + message = "Failed to extract properties. Value should be a map of String -> Bytes.", + error_code = "extracing_property", + error_type = error_type::PARSER_FAILED, + stage = error_stage::RECEIVING, + property_field = self.property_field, + ); + counter!( + "component_errors_total", 1, + "error_code" => "extracing_property", + "error_type" => error_type::PARSER_FAILED, + "stage" => error_stage::RECEIVING, + ); + } +} diff --git a/src/sinks/mod.rs b/src/sinks/mod.rs index 0927cde670690..75b24aff3e42f 100644 --- a/src/sinks/mod.rs +++ b/src/sinks/mod.rs @@ -287,7 +287,7 @@ pub enum Sinks { /// Apache Pulsar. #[cfg(feature = "sinks-pulsar")] - Pulsar(#[configurable(derived)] pulsar::PulsarSinkConfig), + Pulsar(#[configurable(derived)] pulsar::config::PulsarSinkConfig), /// Redis. #[cfg(feature = "sinks-redis")] diff --git a/src/sinks/pulsar.rs b/src/sinks/pulsar.rs deleted file mode 100644 index c3e0547a7e92e..0000000000000 --- a/src/sinks/pulsar.rs +++ /dev/null @@ -1,453 +0,0 @@ -use std::{ - num::NonZeroUsize, - pin::Pin, - task::{Context, Poll}, -}; - -use bytes::BytesMut; -use codecs::{encoding::SerializerConfig, TextSerializerConfig}; -use futures::{future::BoxFuture, ready, stream::FuturesUnordered, FutureExt, Sink, Stream}; -use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; -use pulsar::error::AuthenticationError; -use pulsar::{ - message::proto, producer::SendFuture, proto::CommandSendReceipt, Authentication, - Error as PulsarError, Producer, Pulsar, TokioExecutor, -}; -use snafu::{ResultExt, Snafu}; -use tokio_util::codec::Encoder as _; -use vector_common::internal_event::{ - ByteSize, BytesSent, EventsSent, InternalEventHandle as _, Protocol, Registered, -}; -use vector_config::configurable_component; -use vector_core::config::log_schema; - -use crate::{ - codecs::{Encoder, EncodingConfig, Transformer}, - config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, - event::{Event, EventFinalizers, EventStatus, Finalizable}, - internal_events::PulsarSendingError, - sinks::util::metadata::RequestMetadata, -}; - -#[derive(Debug, Snafu)] -enum BuildError { - #[snafu(display("creating pulsar producer failed: {}", source))] - CreatePulsarSink { source: PulsarError }, -} - -/// Configuration for the `pulsar` sink. -#[configurable_component(sink("pulsar"))] -#[derive(Clone, Debug)] -pub struct PulsarSinkConfig { - /// The endpoint to which the Pulsar client should connect to. - #[serde(alias = "address")] - endpoint: String, - - /// The Pulsar topic name to write events to. - topic: String, - - #[configurable(derived)] - pub encoding: EncodingConfig, - - #[configurable(derived)] - auth: Option, - - #[configurable(derived)] - #[serde( - default, - deserialize_with = "crate::serde::bool_or_struct", - skip_serializing_if = "crate::serde::skip_serializing_if_default" - )] - pub acknowledgements: AcknowledgementsConfig, -} - -/// Authentication configuration. -#[configurable_component] -#[derive(Clone, Debug)] -struct AuthConfig { - /// Basic authentication name/username. - /// - /// This can be used either for basic authentication (username/password) or JWT authentication. - /// When used for JWT, the value should be `token`. - name: Option, - - /// Basic authentication password/token. - /// - /// This can be used either for basic authentication (username/password) or JWT authentication. - /// When used for JWT, the value should be the signed JWT, in the compact representation. - token: Option, - - #[configurable(derived)] - oauth2: Option, -} - -/// OAuth2-specific authenticatgion configuration. -#[configurable_component] -#[derive(Clone, Debug)] -pub struct OAuth2Config { - /// The issuer URL. - issuer_url: String, - - /// The credentials URL. - /// - /// A data URL is also supported. - credentials_url: String, - - /// The OAuth2 audience. - audience: Option, - - /// The OAuth2 scope. - scope: Option, -} - -type PulsarProducer = Producer; -type BoxedPulsarProducer = Box; - -enum PulsarSinkState { - None, - Ready(BoxedPulsarProducer), - Sending( - BoxFuture< - 'static, - ( - BoxedPulsarProducer, - Result, - RequestMetadata, - EventFinalizers, - ), - >, - ), -} - -struct PulsarSink { - transformer: Transformer, - encoder: Encoder<()>, - state: PulsarSinkState, - in_flight: FuturesUnordered< - BoxFuture< - 'static, - ( - Result, - RequestMetadata, - EventFinalizers, - ), - >, - >, - bytes_sent: Registered, -} - -impl GenerateConfig for PulsarSinkConfig { - fn generate_config() -> toml::Value { - toml::Value::try_from(Self { - endpoint: "pulsar://127.0.0.1:6650".to_string(), - topic: "topic-1234".to_string(), - encoding: TextSerializerConfig::new().into(), - auth: None, - acknowledgements: Default::default(), - }) - .unwrap() - } -} - -#[async_trait::async_trait] -impl SinkConfig for PulsarSinkConfig { - async fn build( - &self, - _cx: SinkContext, - ) -> crate::Result<(super::VectorSink, super::Healthcheck)> { - let producer = self - .create_pulsar_producer() - .await - .context(CreatePulsarSinkSnafu)?; - - let transformer = self.encoding.transformer(); - let serializer = self.encoding.build()?; - let encoder = Encoder::<()>::new(serializer); - - let sink = PulsarSink::new(producer, transformer, encoder)?; - - let producer = self - .create_pulsar_producer() - .await - .context(CreatePulsarSinkSnafu)?; - let healthcheck = healthcheck(producer).boxed(); - - Ok((super::VectorSink::from_event_sink(sink), healthcheck)) - } - - fn input(&self) -> Input { - Input::log() - } - - fn acknowledgements(&self) -> &AcknowledgementsConfig { - &self.acknowledgements - } -} - -impl PulsarSinkConfig { - async fn create_pulsar_producer(&self) -> Result { - let mut builder = Pulsar::builder(&self.endpoint, TokioExecutor); - if let Some(auth) = &self.auth { - builder = match ( - auth.name.as_ref(), - auth.token.as_ref(), - auth.oauth2.as_ref(), - ) { - (Some(name), Some(token), None) => builder.with_auth(Authentication { - name: name.clone(), - data: token.as_bytes().to_vec(), - }), - (None, None, Some(oauth2)) => builder.with_auth_provider( - OAuth2Authentication::client_credentials(OAuth2Params { - issuer_url: oauth2.issuer_url.clone(), - credentials_url: oauth2.credentials_url.clone(), - audience: oauth2.audience.clone(), - scope: oauth2.scope.clone(), - }), - ), - _ => return Err(PulsarError::Authentication(AuthenticationError::Custom( - "Invalid auth config: can only specify name and token or oauth2 configuration" - .to_string(), - ))), - }; - } - - let pulsar = builder.build().await?; - if let SerializerConfig::Avro { avro } = self.encoding.config() { - pulsar - .producer() - .with_options(pulsar::producer::ProducerOptions { - schema: Some(proto::Schema { - schema_data: avro.schema.as_bytes().into(), - r#type: proto::schema::Type::Avro as i32, - ..Default::default() - }), - ..Default::default() - }) - .with_topic(&self.topic) - .build() - .await - } else { - pulsar.producer().with_topic(&self.topic).build().await - } - } -} - -async fn healthcheck(producer: PulsarProducer) -> crate::Result<()> { - producer.check_connection().await.map_err(Into::into) -} - -impl PulsarSink { - fn new( - producer: PulsarProducer, - transformer: Transformer, - encoder: Encoder<()>, - ) -> crate::Result { - Ok(Self { - transformer, - encoder, - state: PulsarSinkState::Ready(Box::new(producer)), - in_flight: FuturesUnordered::new(), - bytes_sent: register!(BytesSent::from(Protocol::TCP)), - }) - } - - fn poll_in_flight_prepare(&mut self, cx: &mut Context<'_>) -> Poll<()> { - if let PulsarSinkState::Sending(fut) = &mut self.state { - let (producer, result, metadata, finalizers) = ready!(fut.as_mut().poll(cx)); - - self.state = PulsarSinkState::Ready(producer); - self.in_flight.push(Box::pin(async move { - let result = match result { - Ok(fut) => fut.await, - Err(error) => Err(error), - }; - (result, metadata, finalizers) - })); - } - - Poll::Ready(()) - } -} - -impl Sink for PulsarSink { - type Error = (); - - fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - ready!(self.poll_in_flight_prepare(cx)); - Poll::Ready(Ok(())) - } - - fn start_send(mut self: Pin<&mut Self>, mut event: Event) -> Result<(), Self::Error> { - assert!( - matches!(self.state, PulsarSinkState::Ready(_)), - "Expected `poll_ready` to be called first." - ); - - let event_time = event.maybe_as_log().and_then(|log| { - log.get(log_schema().timestamp_key()) - .and_then(|v| v.as_timestamp().map(|dt| dt.timestamp_millis())) - }); - - let metadata_builder = RequestMetadata::builder(&event); - self.transformer.transform(&mut event); - - let finalizers = event.take_finalizers(); - let mut bytes = BytesMut::new(); - self.encoder.encode(event, &mut bytes).map_err(|_| { - finalizers.update_status(EventStatus::Errored); - // Error is handled by `Encoder`. - })?; - - let bytes_len = - NonZeroUsize::new(bytes.len()).expect("payload should never be zero length"); - let metadata = metadata_builder.with_request_size(bytes_len); - - let mut producer = match std::mem::replace(&mut self.state, PulsarSinkState::None) { - PulsarSinkState::Ready(producer) => producer, - _ => unreachable!(), - }; - - let _ = std::mem::replace( - &mut self.state, - PulsarSinkState::Sending(Box::pin(async move { - let mut builder = producer.create_message().with_content(bytes.as_ref()); - if let Some(et) = event_time { - builder = builder.event_time(et as u64); - } - let result = builder.send().await; - (producer, result, metadata, finalizers) - })), - ); - - Ok(()) - } - - fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - ready!(self.poll_in_flight_prepare(cx)); - - let this = Pin::into_inner(self); - while !this.in_flight.is_empty() { - match ready!(Pin::new(&mut this.in_flight).poll_next(cx)) { - Some((Ok(result), metadata, finalizers)) => { - trace!( - message = "Pulsar sink produced message.", - message_id = ?result.message_id, - producer_id = %result.producer_id, - sequence_id = %result.sequence_id, - ); - - finalizers.update_status(EventStatus::Delivered); - - emit!(EventsSent { - count: metadata.event_count(), - byte_size: metadata.events_byte_size(), - output: None, - }); - - this.bytes_sent - .emit(ByteSize(metadata.request_encoded_size())); - } - Some((Err(error), metadata, finalizers)) => { - finalizers.update_status(EventStatus::Errored); - emit!(PulsarSendingError { - error: Box::new(error), - count: metadata.event_count() as u64, - }); - return Poll::Ready(Err(())); - } - None => break, - } - } - - Poll::Ready(Ok(())) - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_flush(cx) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn generate_config() { - crate::test_util::test_generate_config::(); - } -} - -#[cfg(feature = "pulsar-integration-tests")] -#[cfg(test)] -mod integration_tests { - use futures::StreamExt; - use pulsar::SubType; - - use super::*; - use crate::sinks::VectorSink; - use crate::test_util::{ - components::{assert_sink_compliance, SINK_TAGS}, - random_lines_with_stream, random_string, trace_init, - }; - - fn pulsar_address() -> String { - std::env::var("PULSAR_ADDRESS").unwrap_or_else(|_| "pulsar://127.0.0.1:6650".into()) - } - - #[tokio::test] - async fn pulsar_happy() { - trace_init(); - - let num_events = 1_000; - let (input, events) = random_lines_with_stream(100, num_events, None); - - let topic = format!("test-{}", random_string(10)); - let cnf = PulsarSinkConfig { - endpoint: pulsar_address(), - topic: topic.clone(), - encoding: TextSerializerConfig::new().into(), - auth: None, - acknowledgements: Default::default(), - }; - - let pulsar = Pulsar::::builder(&cnf.endpoint, TokioExecutor) - .build() - .await - .unwrap(); - let mut consumer = pulsar - .consumer() - .with_topic(&topic) - .with_consumer_name("VectorTestConsumer") - .with_subscription_type(SubType::Shared) - .with_subscription("VectorTestSub") - .with_options(pulsar::consumer::ConsumerOptions { - read_compacted: Some(false), - ..Default::default() - }) - .build::() - .await - .unwrap(); - - let producer = cnf.create_pulsar_producer().await.unwrap(); - let transformer = cnf.encoding.transformer(); - let serializer = cnf.encoding.build().unwrap(); - let encoder = Encoder::<()>::new(serializer); - - assert_sink_compliance(&SINK_TAGS, async move { - let sink = PulsarSink::new(producer, transformer, encoder).unwrap(); - VectorSink::from_event_sink(sink).run(events).await - }) - .await - .expect("Running sink failed"); - - for line in input { - let msg = match consumer.next().await.unwrap() { - Ok(msg) => msg, - Err(error) => panic!("{:?}", error), - }; - consumer.ack(&msg).await.unwrap(); - assert_eq!(String::from_utf8_lossy(&msg.payload.data), line); - } - } -} diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs new file mode 100644 index 0000000000000..28fa7a70c4722 --- /dev/null +++ b/src/sinks/pulsar/config.rs @@ -0,0 +1,237 @@ +use crate::{ + codecs::EncodingConfig, + config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, + sinks::{ + pulsar::sink::{healthcheck, PulsarSink}, + Healthcheck, VectorSink, + }, +}; +use codecs::{encoding::SerializerConfig, TextSerializerConfig}; +use futures_util::FutureExt; +use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; +use pulsar::error::AuthenticationError; +use pulsar::{ + message::proto, Authentication, Error as PulsarError, ProducerOptions, Pulsar, TokioExecutor, +}; +use snafu::ResultExt; +use vector_config::configurable_component; +use vector_core::config::DataType; + +pub(crate) const QUEUED_MIN_MESSAGES: u64 = 100000; + +/// Configuration for the `pulsar` sink. +#[configurable_component(sink("pulsar"))] +#[derive(Clone, Debug)] +pub struct PulsarSinkConfig { + /// The endpoint to which the Pulsar client should connect to. + #[serde(alias = "address")] + pub endpoint: String, + + /// The Pulsar topic name to write events to. + #[configurable(metadata(templateable))] + pub topic: String, + + /// The log field name or tags key to use for the topic key. + /// + /// If the field does not exist in the log or in tags, a blank value will be used. If unspecified, the key is not sent. + /// + /// Pulsar uses a hash of the key to choose the topic-partition or uses round-robin if the record has no key. + pub key_field: Option, + + /// The log field name to use for the Pulsar properties. + /// + /// If omitted, no properties will be written. + pub properties_key: Option, + + /// Determines the default batch size, if not set, defaults to 1000 + #[configurable(derived)] + #[serde(default)] + pub batch_size: Option, + + /// Enables compression for messages. No compression by default. + /// Valid values are + #[configurable(derived)] + #[serde(default)] + pub compression: Option, + + #[configurable(derived)] + pub encoding: EncodingConfig, + + #[configurable(derived)] + pub auth: Option, + + #[configurable(derived)] + #[serde( + default, + deserialize_with = "crate::serde::bool_or_struct", + skip_serializing_if = "crate::serde::skip_serializing_if_default" + )] + pub acknowledgements: AcknowledgementsConfig, +} + +/// Identifies the compression options that are available within Pulsar +#[configurable_component] +#[derive(Clone, Debug)] +pub enum PulsarCompression { + /// No compression + None, + /// LZ4 compression + Lz4, + /// Zlib compression + Zlib, + /// Zstd compression + Zstd, + /// Snappy compression + Snappy, +} + +/// Authentication configuration. +#[configurable_component] +#[derive(Clone, Debug)] +pub struct AuthConfig { + /// Basic authentication name/username. + /// + /// This can be used either for basic authentication (username/password) or JWT authentication. + /// When used for JWT, the value should be `token`. + name: Option, + + /// Basic authentication password/token. + /// + /// This can be used either for basic authentication (username/password) or JWT authentication. + /// When used for JWT, the value should be the signed JWT, in the compact representation. + token: Option, + + #[configurable(derived)] + oauth2: Option, +} + +/// OAuth2-specific authenticatgion configuration. +#[configurable_component] +#[derive(Clone, Debug)] +pub struct OAuth2Config { + /// The issuer URL. + issuer_url: String, + + /// The credentials URL. + /// + /// A data URL is also supported. + credentials_url: String, + + /// The OAuth2 audience. + audience: Option, + + /// The OAuth2 scope. + scope: Option, +} + +impl PulsarSinkConfig { + pub(crate) async fn create_pulsar_client(&self) -> Result, PulsarError> { + let mut builder = Pulsar::builder(&self.endpoint, TokioExecutor); + if let Some(auth) = &self.auth { + builder = match ( + auth.name.as_ref(), + auth.token.as_ref(), + auth.oauth2.as_ref(), + ) { + (Some(name), Some(token), None) => builder.with_auth(Authentication { + name: name.clone(), + data: token.as_bytes().to_vec(), + }), + (None, None, Some(oauth2)) => builder.with_auth_provider( + OAuth2Authentication::client_credentials(OAuth2Params { + issuer_url: oauth2.issuer_url.clone(), + credentials_url: oauth2.credentials_url.clone(), + audience: oauth2.audience.clone(), + scope: oauth2.scope.clone(), + }), + ), + _ => return Err(PulsarError::Authentication(AuthenticationError::Custom( + "Invalid auth config: can only specify name and token or oauth2 configuration" + .to_string(), + ))), + }; + } + + builder.build().await + } + pub(crate) fn build_producer_options(&self) -> ProducerOptions { + let mut opts = ProducerOptions { + encrypted: None, + metadata: Default::default(), + schema: None, + batch_size: None, + compression: None, + }; + if let Some(compression) = &self.compression { + match compression { + PulsarCompression::None => opts.compression = Some(proto::CompressionType::None), + PulsarCompression::Lz4 => opts.compression = Some(proto::CompressionType::Lz4), + PulsarCompression::Zlib => opts.compression = Some(proto::CompressionType::Zlib), + PulsarCompression::Zstd => opts.compression = Some(proto::CompressionType::Zstd), + PulsarCompression::Snappy => { + opts.compression = Some(proto::CompressionType::Snappy) + } + } + } + opts.batch_size = self.batch_size.to_owned(); + if let SerializerConfig::Avro { avro } = self.encoding.config() { + opts.schema = Some(proto::Schema { + schema_data: avro.schema.as_bytes().into(), + r#type: proto::schema::Type::Avro as i32, + ..Default::default() + }); + } + opts + } +} + +impl GenerateConfig for PulsarSinkConfig { + fn generate_config() -> toml::Value { + toml::Value::try_from(Self { + endpoint: "pulsar://127.0.0.1:6650".to_string(), + topic: "topic-1234".to_string(), + key_field: None, + properties_key: None, + batch_size: None, + compression: None, + encoding: TextSerializerConfig::new().into(), + auth: None, + acknowledgements: Default::default(), + }) + .unwrap() + } +} + +#[async_trait::async_trait] +impl SinkConfig for PulsarSinkConfig { + async fn build(&self, _cx: SinkContext) -> vector_core::Result<(VectorSink, Healthcheck)> { + let client = self + .create_pulsar_client() + .await + .context(super::sink::CreatePulsarSinkSnafu)?; + + let sink = PulsarSink::new(client, self.clone())?; + + let hc = healthcheck(self.clone()).boxed(); + + Ok((VectorSink::from_event_streamsink(sink), hc)) + } + + fn input(&self) -> Input { + Input::new(self.encoding.config().input_type() & (DataType::Log | DataType::Metric)) + } + + fn acknowledgements(&self) -> &AcknowledgementsConfig { + &self.acknowledgements + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn generate_config() { + PulsarSinkConfig::generate_config(); + } +} diff --git a/src/sinks/pulsar/mod.rs b/src/sinks/pulsar/mod.rs new file mode 100644 index 0000000000000..2faa678ef4072 --- /dev/null +++ b/src/sinks/pulsar/mod.rs @@ -0,0 +1,5 @@ +pub(crate) mod config; +pub(crate) mod request_builder; +pub(crate) mod service; +pub(crate) mod sink; +pub(crate) mod tests; diff --git a/src/sinks/pulsar/request_builder.rs b/src/sinks/pulsar/request_builder.rs new file mode 100644 index 0000000000000..9e8b59677fa50 --- /dev/null +++ b/src/sinks/pulsar/request_builder.rs @@ -0,0 +1,127 @@ +use bytes::{Bytes, BytesMut}; +use std::collections::HashMap; +use tokio_util::codec::Encoder as _; +use vector_core::{config::LogSchema, ByteSizeOf}; + +use crate::{ + codecs::{Encoder, Transformer}, + event::{Event, Finalizable, Value}, + internal_events::PulsarPropertyExtractionError, + sinks::pulsar::service::{PulsarRequest, PulsarRequestMetadata}, + template::Template, +}; + +pub struct PulsarRequestBuilder { + pub key_field: Option, + pub properties_key: Option, + pub topic_template: Template, + pub transformer: Transformer, + pub encoder: Encoder<()>, + pub log_schema: &'static LogSchema, +} + +impl PulsarRequestBuilder { + pub fn build_request(&mut self, mut event: Event) -> Option { + let topic = self.topic_template.render_string(&event).ok()?; + let metadata = PulsarRequestMetadata { + finalizers: event.take_finalizers(), + key: get_key(&event, &self.key_field), + timestamp_millis: get_timestamp_millis(&event, self.log_schema), + properties: get_properties(&event, &self.properties_key), + topic, + }; + let event_byte_size = event.size_of(); + self.transformer.transform(&mut event); + let mut body = BytesMut::new(); + self.encoder.encode(event, &mut body).ok()?; + let body = body.freeze(); + Some(PulsarRequest { + body, + metadata, + event_byte_size, + }) + } +} + +fn get_key(event: &Event, key_field: &Option) -> Option { + key_field.as_ref().and_then(|key_field| match event { + Event::Log(log) => log + .get(key_field.as_str()) + .map(|value| value.coerce_to_bytes()), + Event::Metric(metric) => metric + .tags() + .and_then(|tags| tags.get(key_field)) + .map(|value| value.clone().into()), + _ => None, + }) +} + +fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option { + match &event { + Event::Log(log) => log + .get(log_schema.timestamp_key()) + .and_then(|v| v.as_timestamp()) + .copied(), + Event::Metric(metric) => metric.timestamp(), + _ => None, + } + .map(|ts| ts.timestamp_millis()) +} + +fn get_properties( + event: &Event, + properties_key: &Option, +) -> Option> { + properties_key.as_ref().and_then(|properties_key| { + if let Event::Log(log) = event { + if let Some(properties) = log.get(properties_key.as_str()) { + match properties { + Value::Object(headers_map) => { + let mut property_map = HashMap::new(); + for (key, value) in headers_map { + if let Value::Bytes(value_bytes) = value { + property_map.insert(key.clone(), value_bytes.clone()); + } else { + emit!(PulsarPropertyExtractionError { + property_field: properties_key + }); + } + } + return Some(property_map); + } + _ => { + emit!(PulsarPropertyExtractionError { + property_field: properties_key + }); + } + } + } + } + None + }) +} + +#[cfg(test)] +mod tests { + use std::collections::BTreeMap; + + use bytes::Bytes; + + use super::*; + use crate::event::LogEvent; + + #[test] + fn pulsar_get_headers() { + let properties_key = "properties"; + let mut property_values = BTreeMap::new(); + property_values.insert("a-key".to_string(), Value::Bytes(Bytes::from("a-value"))); + property_values.insert("b-key".to_string(), Value::Bytes(Bytes::from("b-value"))); + + let mut event = Event::Log(LogEvent::from("hello")); + event.as_mut_log().insert(properties_key, property_values); + + let properties = get_properties(&event, &Some(properties_key.to_string())).unwrap(); + assert_eq!(properties.get("a-key").unwrap(), "a-value".as_bytes()); + assert_eq!(properties.get("b-key").unwrap(), "b-value".as_bytes()); + } +} diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs new file mode 100644 index 0000000000000..f2eb899b3c4c3 --- /dev/null +++ b/src/sinks/pulsar/service.rs @@ -0,0 +1,194 @@ +use std::collections::HashMap; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use bytes::Bytes; +use futures::future::BoxFuture; +use lru::LruCache; +use pulsar::{Error as PulsarError, Executor, Producer, ProducerOptions, Pulsar}; +use tokio::sync::Mutex; +use tower::Service; +use vector_core::{ + internal_event::{ + ByteSize, BytesSent, EventsSent, InternalEventHandle as _, Protocol, Registered, + }, + stream::DriverResponse, +}; + +use crate::event::{EventFinalizers, EventStatus, Finalizable}; +use crate::internal_events::PulsarSendingError; + +pub struct PulsarRequest { + pub body: Bytes, + pub metadata: PulsarRequestMetadata, + pub event_byte_size: usize, +} + +pub struct PulsarRequestMetadata { + pub finalizers: EventFinalizers, + pub key: Option, + pub properties: Option>, + pub timestamp_millis: Option, + pub topic: String, +} + +pub struct PulsarResponse { + event_byte_size: usize, +} + +impl DriverResponse for PulsarResponse { + fn event_status(&self) -> EventStatus { + EventStatus::Delivered + } + + fn events_sent(&self) -> EventsSent { + EventsSent { + count: 1, + byte_size: self.event_byte_size, + output: None, + } + } +} + +impl Finalizable for PulsarRequest { + fn take_finalizers(&mut self) -> EventFinalizers { + std::mem::take(&mut self.metadata.finalizers) + } +} + +type SafeLru = Arc>>, PulsarError>>>>; +pub struct PulsarService { + pulsar_client: Pulsar, + producer_cache: SafeLru, + producer_options: ProducerOptions, + bytes_sent: Registered, +} + +impl PulsarService { + pub(crate) fn new( + pulsar_client: Pulsar, + producer_options: ProducerOptions, + producer_cache_size: Option, + ) -> PulsarService { + // Use a LRUCache to store a limited set of producers + // Producers in Pulsar use a send buffer, so we want to limit the number of these + let producer_cache = Arc::new(Mutex::new(LruCache::new( + producer_cache_size.unwrap_or(100), + ))); + PulsarService { + pulsar_client, + producer_cache, + producer_options, + bytes_sent: register!(BytesSent::from(Protocol("pulsar".into()))), + } + } + + /// Build a producer that is wrapped in an Arc to allow for the producer + /// to control access. + /// + /// NOTE: Pulsar client library should likely be improved to simplify this + async fn build_producer( + client: Pulsar, + producer_options: ProducerOptions, + topic: &String, + ) -> Result>>, PulsarError> { + let prod = client + .producer() + .with_topic(topic) + .with_options(producer_options); + match prod.build().await { + Ok(p) => Ok(Arc::new(Mutex::new(p))), + Err(e) => Err(e), + } + } + + /// Pulsar requires a producer object be created per topic + /// This method will build a producer if it hasn't been created or caches it otherwise + async fn get_or_build_producer( + producer_cache: SafeLru, + client: Pulsar, + producer_options: ProducerOptions, + topic: String, + ) -> Arc>> { + let mut pc = producer_cache.lock().await; + match pc.contains(&topic) { + false => { + pc.put( + topic.clone(), + PulsarService::build_producer(client, producer_options, &topic).await, + ); + let f = pc.get(&topic).unwrap().as_ref().unwrap(); + Arc::clone(f) + } + true => { + let f = pc.get(&topic).unwrap().as_ref().unwrap(); + Arc::clone(f) + } + } + } +} + +impl Service for PulsarService { + type Response = PulsarResponse; + type Error = PulsarError; + type Future = BoxFuture<'static, Result>; + + fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll> { + Poll::Ready(Ok(())) + } + + fn call(&mut self, request: PulsarRequest) -> Self::Future { + let prod_future = PulsarService::get_or_build_producer( + Arc::clone(&self.producer_cache), + self.pulsar_client.clone(), + self.producer_options.clone(), + request.metadata.topic.clone(), + ); + let bytes_sent = self.bytes_sent.clone(); + let ts = request.metadata.timestamp_millis.to_owned(); + Box::pin(async move { + let p = prod_future.await; + let mut lp = p.lock().await; + let body = request.body.clone(); + let mut msg_builder = lp.create_message().with_content(body.as_ref()); + if let Some(key) = request.metadata.key { + msg_builder = msg_builder.with_key(String::from_utf8_lossy(&*key)); + } + if let Some(timestamp) = ts { + msg_builder = msg_builder.event_time(timestamp as u64); + } + if let Some(properties) = request.metadata.properties { + for (key, value) in properties { + msg_builder = + msg_builder.with_property(key, String::from_utf8_lossy(&*value.clone())); + } + } + + match msg_builder.send().await { + Ok(resp) => match resp.await { + Ok(_) => { + //TODO: not totally accurate, doesn't include metadata or key + bytes_sent.emit(ByteSize(request.body.len())); + Ok(PulsarResponse { + event_byte_size: request.event_byte_size, + }) + } + Err(e) => { + emit!(PulsarSendingError { + error: Box::new(PulsarError::Custom("failed to send".to_string())), + count: 1 + }); + Err(e) + } + }, + Err(e) => { + emit!(PulsarSendingError { + error: Box::new(PulsarError::Custom("failed to send".to_string())), + count: 1, + }); + Err(e) + } + } + }) + } +} diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs new file mode 100644 index 0000000000000..817d9282afab8 --- /dev/null +++ b/src/sinks/pulsar/sink.rs @@ -0,0 +1,93 @@ +use async_trait::async_trait; + +use futures::{future, stream::BoxStream, StreamExt}; +use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; +use snafu::{ResultExt, Snafu}; +use tower::limit::ConcurrencyLimit; +use vector_core::config::log_schema; +use vector_core::event::LogEvent; +use vector_core::sink::StreamSink; + +use crate::sinks::pulsar::config::{PulsarSinkConfig, QUEUED_MIN_MESSAGES}; +use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; +use crate::sinks::pulsar::service::PulsarService; +use crate::sinks::util::SinkBuilderExt; +use crate::template::{Template, TemplateParseError}; +use crate::{ + codecs::{Encoder, Transformer}, + event::Event, +}; + +#[derive(Debug, Snafu)] +#[snafu(visibility(pub(crate)))] +pub(crate) enum BuildError { + #[snafu(display("creating pulsar producer failed: {}", source))] + CreatePulsarSink { source: PulsarError }, + #[snafu(display("invalid topic template: {}", source))] + TopicTemplate { source: TemplateParseError }, +} + +pub(crate) struct PulsarSink { + transformer: Transformer, + encoder: Encoder<()>, + service: PulsarService, + topic: Template, + key_field: Option, + properties_key: Option, +} + +pub(crate) async fn healthcheck(config: PulsarSinkConfig) -> crate::Result<()> { + trace!("Healthcheck started."); + let client = config.create_pulsar_client().await?; + let topic = Template::try_from(config.topic) + .context(TopicTemplateSnafu)? + .render_string(&LogEvent::from_str_legacy(""))?; + client.lookup_topic(topic).await?; + trace!("Healthcheck completed."); + Ok(()) +} + +impl PulsarSink { + pub(crate) fn new( + client: Pulsar, + config: PulsarSinkConfig, + ) -> crate::Result { + let producer_opts = config.build_producer_options(); + let transformer = config.encoding.transformer(); + let serializer = config.encoding.build()?; + let encoder = Encoder::<()>::new(serializer); + let service = PulsarService::new(client, producer_opts, None); + + Ok(PulsarSink { + properties_key: config.properties_key, + key_field: config.key_field, + transformer, + encoder, + service, + topic: Template::try_from(config.topic).context(TopicTemplateSnafu)?, + }) + } + + async fn run_inner(self: Box, input: BoxStream<'_, Event>) -> Result<(), ()> { + let service = ConcurrencyLimit::new(self.service, QUEUED_MIN_MESSAGES as usize); + let mut request_builder = PulsarRequestBuilder { + key_field: self.key_field, + properties_key: self.properties_key, + topic_template: self.topic, + transformer: self.transformer, + encoder: self.encoder, + log_schema: log_schema(), + }; + let sink = input + .filter_map(|event| future::ready(request_builder.build_request(event))) + .into_driver(service); + sink.run().await + } +} + +#[async_trait] +impl StreamSink for PulsarSink { + async fn run(self: Box, input: BoxStream<'_, Event>) -> Result<(), ()> { + self.run_inner(input).await + } +} diff --git a/src/sinks/pulsar/tests.rs b/src/sinks/pulsar/tests.rs new file mode 100644 index 0000000000000..74a053487d486 --- /dev/null +++ b/src/sinks/pulsar/tests.rs @@ -0,0 +1,103 @@ +#[cfg(feature = "pulsar-integration-tests")] +#[cfg(test)] +mod integration_tests { + use crate::sinks::pulsar::config::PulsarSinkConfig; + use crate::sinks::pulsar::sink::PulsarSink; + use codecs::TextSerializerConfig; + use futures::StreamExt; + use pulsar::SubType; + use std::collections::BTreeMap; + + use crate::event::Value; + use crate::sinks::VectorSink; + use crate::test_util::components::assert_sink_compliance; + use crate::test_util::{ + components::SINK_TAGS, random_lines_with_stream, random_string, trace_init, + }; + use bytes::Bytes; + + fn pulsar_address() -> String { + std::env::var("PULSAR_ADDRESS").unwrap_or_else(|_| "pulsar://127.0.0.1:6650".into()) + } + + async fn pulsar_happy_reuse(mut cnf: PulsarSinkConfig) { + trace_init(); + + let prop_1_key = "prop-1-key"; + let prop_1_value = "prop-1-value"; + let num_events = 1_000; + let (input, events) = random_lines_with_stream(100, num_events, None); + let prop_key_opt = cnf.properties_key.clone(); + let input_events = events.map(move |mut events| { + // if a property_key is defined, add some properties! + if let Some(prop_key) = &prop_key_opt { + let properties_key = prop_key; + let mut property_values = BTreeMap::new(); + property_values.insert( + prop_1_key.to_owned(), + Value::Bytes(Bytes::from(prop_1_value)), + ); + events.iter_logs_mut().for_each(move |log| { + log.insert(properties_key.as_str(), property_values.clone()); + }); + events + } else { + events + } + }); + + let topic = format!("test-{}", random_string(10)); + + cnf.topic = topic.clone(); + + let pulsar = cnf.create_pulsar_client().await.unwrap(); + let mut consumer = pulsar + .consumer() + .with_topic(&topic) + .with_consumer_name("VectorTestConsumer") + .with_subscription_type(SubType::Shared) + .with_subscription("VectorTestSub") + .with_options(pulsar::consumer::ConsumerOptions { + read_compacted: Some(false), + ..Default::default() + }) + .build::() + .await + .unwrap(); + + assert_sink_compliance(&SINK_TAGS, async move { + let sink = PulsarSink::new(pulsar, cnf).unwrap(); + let sink = VectorSink::from_event_streamsink(sink); + sink.run(input_events).await + }) + .await + .expect("Running sink failed"); + + for line in input { + let msg = match consumer.next().await.unwrap() { + Ok(msg) => msg, + Err(error) => panic!("{:?}", error), + }; + consumer.ack(&msg).await.unwrap(); + assert_eq!(String::from_utf8_lossy(&msg.payload.data), line); + } + } + + #[tokio::test] + async fn pulsar_happy() { + let cnf = PulsarSinkConfig { + endpoint: pulsar_address(), + // overriden by test + topic: "".to_string(), + key_field: None, + properties_key: None, + batch_size: None, + compression: None, + encoding: TextSerializerConfig::new().into(), + auth: None, + acknowledgements: Default::default(), + }; + + pulsar_happy_reuse(cnf).await + } +} From 4839a046d2cfdec002aca3b09250b517cdedd5cf Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Mon, 12 Sep 2022 10:45:40 -0600 Subject: [PATCH 02/48] Update src/internal_events/pulsar.rs Co-authored-by: Spencer Gilbert --- src/internal_events/pulsar.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/internal_events/pulsar.rs b/src/internal_events/pulsar.rs index 340fbe29f41fc..5799476ebd7f3 100644 --- a/src/internal_events/pulsar.rs +++ b/src/internal_events/pulsar.rs @@ -46,6 +46,7 @@ impl InternalEvent for PulsarPropertyExtractionError<'_> { error_type = error_type::PARSER_FAILED, stage = error_stage::RECEIVING, property_field = self.property_field, + internal_log_rate_secs = 10, ); counter!( "component_errors_total", 1, From 44d156fc15e2572ccc8a7385485d7e6c7d34bf43 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Mon, 12 Sep 2022 10:46:04 -0600 Subject: [PATCH 03/48] Update src/sinks/pulsar/config.rs Co-authored-by: Spencer Gilbert --- src/sinks/pulsar/config.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 28fa7a70c4722..9338e5e6cf4cf 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -48,8 +48,6 @@ pub struct PulsarSinkConfig { #[serde(default)] pub batch_size: Option, - /// Enables compression for messages. No compression by default. - /// Valid values are #[configurable(derived)] #[serde(default)] pub compression: Option, From 548fdcecef351397b8fafcc6f2afcd58aeb6e055 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Mon, 12 Sep 2022 10:46:16 -0600 Subject: [PATCH 04/48] Update src/sinks/pulsar/config.rs Co-authored-by: Spencer Gilbert --- src/sinks/pulsar/config.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 9338e5e6cf4cf..a6414d6e689b4 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -43,7 +43,9 @@ pub struct PulsarSinkConfig { /// If omitted, no properties will be written. pub properties_key: Option, - /// Determines the default batch size, if not set, defaults to 1000 + /// Determines the batch size. + /// + /// Defaults to 1000. #[configurable(derived)] #[serde(default)] pub batch_size: Option, From f26685de22ee3fc472bd57692b3211818fde5d89 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Mon, 12 Sep 2022 10:46:23 -0600 Subject: [PATCH 05/48] Update src/sinks/pulsar/config.rs Co-authored-by: Spencer Gilbert --- src/sinks/pulsar/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index a6414d6e689b4..a10cd94271e13 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -69,7 +69,7 @@ pub struct PulsarSinkConfig { pub acknowledgements: AcknowledgementsConfig, } -/// Identifies the compression options that are available within Pulsar +/// Identifies the compression options that are available within Pulsar. #[configurable_component] #[derive(Clone, Debug)] pub enum PulsarCompression { From dfcac00cc8fe7fcbc20010a3e2aba9c483b4ad00 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Mon, 12 Sep 2022 10:46:41 -0600 Subject: [PATCH 06/48] Update src/sinks/pulsar/config.rs Co-authored-by: Spencer Gilbert --- src/sinks/pulsar/config.rs | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index a10cd94271e13..1dc8b93802df2 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -73,15 +73,27 @@ pub struct PulsarSinkConfig { #[configurable_component] #[derive(Clone, Debug)] pub enum PulsarCompression { - /// No compression + /// No compression. None, - /// LZ4 compression + + /// [LZ4][lz4] compression. + /// + /// [lz4]: https://lz4.github.io/lz4/ Lz4, - /// Zlib compression + + /// [Zlib][zlib] compression. + /// + /// [zlib]: https://www.zlib.net Zlib, - /// Zstd compression + + /// [Zstd][zstd] compression. + /// + /// [zstd]: https://zstd.net Zstd, - /// Snappy compression + + /// [Snappy][snappy] compression. + /// + /// [snappy]: https://google.github.io/snappy/ Snappy, } From 63a7965ca8317a6444d002d17cca21429880be45 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Mon, 12 Sep 2022 10:47:14 -0600 Subject: [PATCH 07/48] Update src/sinks/pulsar/config.rs Co-authored-by: Spencer Gilbert --- src/sinks/pulsar/config.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 1dc8b93802df2..79724d78ee13a 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -166,6 +166,7 @@ impl PulsarSinkConfig { builder.build().await } + pub(crate) fn build_producer_options(&self) -> ProducerOptions { let mut opts = ProducerOptions { encrypted: None, From fb244fa42fd2a97bda863b625ebf2d14da0be03a Mon Sep 17 00:00:00 2001 From: Nathan Fox Date: Tue, 13 Sep 2022 13:15:01 -0400 Subject: [PATCH 08/48] cargo fmt --- src/sinks/pulsar/config.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 79724d78ee13a..b1ae758a4864c 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -75,22 +75,22 @@ pub struct PulsarSinkConfig { pub enum PulsarCompression { /// No compression. None, - + /// [LZ4][lz4] compression. /// /// [lz4]: https://lz4.github.io/lz4/ Lz4, - + /// [Zlib][zlib] compression. /// /// [zlib]: https://www.zlib.net Zlib, - + /// [Zstd][zstd] compression. /// /// [zstd]: https://zstd.net Zstd, - + /// [Snappy][snappy] compression. /// /// [snappy]: https://google.github.io/snappy/ @@ -166,7 +166,7 @@ impl PulsarSinkConfig { builder.build().await } - + pub(crate) fn build_producer_options(&self) -> ProducerOptions { let mut opts = ProducerOptions { encrypted: None, From 573b9eebeb707f3a43e51fb7e3a40696581f713c Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Thu, 17 Nov 2022 18:07:42 -0700 Subject: [PATCH 09/48] Update src/sinks/pulsar/config.rs Co-authored-by: neuronull --- src/sinks/pulsar/config.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index b1ae758a4864c..14eecd137d095 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -60,6 +60,9 @@ pub struct PulsarSinkConfig { #[configurable(derived)] pub auth: Option, + #[configurable(derived)] + #[serde(default)] + pub request: TowerRequestConfig, #[configurable(derived)] #[serde( default, From 0514d8f84b60aa08f56b3b9258d925a2225e85a8 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Fri, 18 Nov 2022 11:54:53 -0700 Subject: [PATCH 10/48] WIP: Refactor to use new abstractions, not compiling This changes the Pulsar sink to use the new helpers like: * Request Builder * Tower Request / Retry Settings * Improved size_of However, it is not currently complining and I need some help --- Cargo.toml | 2 +- src/sinks/pulsar/config.rs | 16 +-- src/sinks/pulsar/encoder.rs | 30 +++++ src/sinks/pulsar/mod.rs | 2 + src/sinks/pulsar/request_builder.rs | 164 +++++++++------------------- src/sinks/pulsar/service.rs | 58 ++++++---- src/sinks/pulsar/sink.rs | 99 ++++++++++++++--- src/sinks/pulsar/tests.rs | 10 ++ src/sinks/pulsar/util.rs | 89 +++++++++++++++ 9 files changed, 308 insertions(+), 162 deletions(-) create mode 100644 src/sinks/pulsar/encoder.rs create mode 100644 src/sinks/pulsar/util.rs diff --git a/Cargo.toml b/Cargo.toml index 82a907f263824..4a4462eef2fa0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -285,7 +285,7 @@ ordered-float = { version = "3.4.0", default-features = false } percent-encoding = { version = "2.2.0", default-features = false } pin-project = { version = "1.0.12", default-features = false } postgres-openssl = { version = "0.5.0", default-features = false, features = ["runtime"], optional = true } -pulsar = { version = "4.1.3", default-features = false, features = ["tokio-runtime", "auth-oauth2"], optional = true } +pulsar = { version = "4.2.0", default-features = false, features = ["tokio-runtime", "auth-oauth2"], optional = true } rand = { version = "0.8.5", default-features = false, features = ["small_rng"] } rand_distr = { version = "0.4.3", default-features = false } rdkafka = { version = "0.29.0", default-features = false, features = ["tokio", "libz", "ssl", "zstd"], optional = true } diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 14eecd137d095..be10876bf6374 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -16,6 +16,7 @@ use pulsar::{ use snafu::ResultExt; use vector_config::configurable_component; use vector_core::config::DataType; +use crate::sinks::util::TowerRequestConfig; pub(crate) const QUEUED_MIN_MESSAGES: u64 = 100000; @@ -63,6 +64,7 @@ pub struct PulsarSinkConfig { #[configurable(derived)] #[serde(default)] pub request: TowerRequestConfig, + #[configurable(derived)] #[serde( default, @@ -173,6 +175,7 @@ impl PulsarSinkConfig { pub(crate) fn build_producer_options(&self) -> ProducerOptions { let mut opts = ProducerOptions { encrypted: None, + access_mode: Some(0), metadata: Default::default(), schema: None, batch_size: None, @@ -205,6 +208,7 @@ impl GenerateConfig for PulsarSinkConfig { fn generate_config() -> toml::Value { toml::Value::try_from(Self { endpoint: "pulsar://127.0.0.1:6650".to_string(), + request: TowerRequestConfig::default(), topic: "topic-1234".to_string(), key_field: None, properties_key: None, @@ -220,7 +224,7 @@ impl GenerateConfig for PulsarSinkConfig { #[async_trait::async_trait] impl SinkConfig for PulsarSinkConfig { - async fn build(&self, _cx: SinkContext) -> vector_core::Result<(VectorSink, Healthcheck)> { + async fn build(&self, _cx: SinkContext) -> crate::Result<(VectorSink, Healthcheck)> { let client = self .create_pulsar_client() .await @@ -241,13 +245,3 @@ impl SinkConfig for PulsarSinkConfig { &self.acknowledgements } } - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn generate_config() { - PulsarSinkConfig::generate_config(); - } -} diff --git a/src/sinks/pulsar/encoder.rs b/src/sinks/pulsar/encoder.rs new file mode 100644 index 0000000000000..63d25f2e05711 --- /dev/null +++ b/src/sinks/pulsar/encoder.rs @@ -0,0 +1,30 @@ +//! Encoding for the `Pulsar` sink. +use crate::{ + event::Event, + sinks::util::encoding::{write_all, Encoder}, +}; +use bytes::BytesMut; +use std::io; +use tokio_util::codec::Encoder as _; + +#[derive(Clone, Debug)] +pub(super) struct PulsarEncoder { + pub(super) encoder: crate::codecs::Encoder<()>, + pub(super) transformer: crate::codecs::Transformer, +} + +impl Encoder for PulsarEncoder { + fn encode_input(&self, mut input: Event, writer: &mut dyn io::Write) -> io::Result { + let mut body = BytesMut::new(); + self.transformer.transform(&mut input); + let mut encoder = self.encoder.clone(); + encoder + .encode(input, &mut body) + .map_err(|_| io::Error::new(io::ErrorKind::Other, "unable to encode"))?; + + let body = body.freeze(); + write_all(writer, 1, body.as_ref())?; + + Ok(body.len()) + } +} diff --git a/src/sinks/pulsar/mod.rs b/src/sinks/pulsar/mod.rs index 2faa678ef4072..85c7c325b38f7 100644 --- a/src/sinks/pulsar/mod.rs +++ b/src/sinks/pulsar/mod.rs @@ -3,3 +3,5 @@ pub(crate) mod request_builder; pub(crate) mod service; pub(crate) mod sink; pub(crate) mod tests; +pub(crate) mod encoder; +mod util; diff --git a/src/sinks/pulsar/request_builder.rs b/src/sinks/pulsar/request_builder.rs index 9e8b59677fa50..6fa27ac73f5ed 100644 --- a/src/sinks/pulsar/request_builder.rs +++ b/src/sinks/pulsar/request_builder.rs @@ -1,127 +1,69 @@ -use bytes::{Bytes, BytesMut}; +use bytes::{Bytes}; use std::collections::HashMap; -use tokio_util::codec::Encoder as _; -use vector_core::{config::LogSchema, ByteSizeOf}; +use std::io; +use vector_common::finalization::EventFinalizers; +use vector_common::request_metadata::RequestMetadata; use crate::{ - codecs::{Encoder, Transformer}, - event::{Event, Finalizable, Value}, - internal_events::PulsarPropertyExtractionError, - sinks::pulsar::service::{PulsarRequest, PulsarRequestMetadata}, - template::Template, + event::{Event, Finalizable}, + sinks::pulsar::service::{PulsarRequest}, }; +use crate::sinks::pulsar::encoder::PulsarEncoder; +use crate::sinks::pulsar::sink::PulsarEvent; +use crate::sinks::util::metadata::RequestMetadataBuilder; +use crate::sinks::util::{Compression, RequestBuilder}; +use crate::sinks::util::request_builder::EncodeResult; -pub struct PulsarRequestBuilder { - pub key_field: Option, - pub properties_key: Option, - pub topic_template: Template, - pub transformer: Transformer, - pub encoder: Encoder<()>, - pub log_schema: &'static LogSchema, +#[derive(Clone)] +pub(super) struct PulsarMetadata { + pub finalizers: EventFinalizers, + pub key: Option, + pub properties: Option>, + pub timestamp_millis: Option, + pub topic: String, } -impl PulsarRequestBuilder { - pub fn build_request(&mut self, mut event: Event) -> Option { - let topic = self.topic_template.render_string(&event).ok()?; - let metadata = PulsarRequestMetadata { - finalizers: event.take_finalizers(), - key: get_key(&event, &self.key_field), - timestamp_millis: get_timestamp_millis(&event, self.log_schema), - properties: get_properties(&event, &self.properties_key), - topic, - }; - let event_byte_size = event.size_of(); - self.transformer.transform(&mut event); - let mut body = BytesMut::new(); - self.encoder.encode(event, &mut body).ok()?; - let body = body.freeze(); - Some(PulsarRequest { - body, - metadata, - event_byte_size, - }) - } +pub(super) struct PulsarRequestBuilder { + pub(super) encoder: PulsarEncoder, } -fn get_key(event: &Event, key_field: &Option) -> Option { - key_field.as_ref().and_then(|key_field| match event { - Event::Log(log) => log - .get(key_field.as_str()) - .map(|value| value.coerce_to_bytes()), - Event::Metric(metric) => metric - .tags() - .and_then(|tags| tags.get(key_field)) - .map(|value| value.clone().into()), - _ => None, - }) -} +impl RequestBuilder for PulsarRequestBuilder { + type Metadata = PulsarMetadata; + type Events = Event; + type Encoder = PulsarEncoder; + type Payload = Bytes; + type Request = PulsarRequest; + type Error = io::Error; -fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option { - match &event { - Event::Log(log) => log - .get(log_schema.timestamp_key()) - .and_then(|v| v.as_timestamp()) - .copied(), - Event::Metric(metric) => metric.timestamp(), - _ => None, + fn compression(&self) -> Compression { + // we don't use this compression from vector as Pulsar supports more + // compression formats + Compression::None } - .map(|ts| ts.timestamp_millis()) -} - -fn get_properties( - event: &Event, - properties_key: &Option, -) -> Option> { - properties_key.as_ref().and_then(|properties_key| { - if let Event::Log(log) = event { - if let Some(properties) = log.get(properties_key.as_str()) { - match properties { - Value::Object(headers_map) => { - let mut property_map = HashMap::new(); - for (key, value) in headers_map { - if let Value::Bytes(value_bytes) = value { - property_map.insert(key.clone(), value_bytes.clone()); - } else { - emit!(PulsarPropertyExtractionError { - property_field: properties_key - }); - } - } - return Some(property_map); - } - _ => { - emit!(PulsarPropertyExtractionError { - property_field: properties_key - }); - } - } - } - } - None - }) -} - -#[cfg(test)] -mod tests { - use std::collections::BTreeMap; - use bytes::Bytes; - - use super::*; - use crate::event::LogEvent; - - #[test] - fn pulsar_get_headers() { - let properties_key = "properties"; - let mut property_values = BTreeMap::new(); - property_values.insert("a-key".to_string(), Value::Bytes(Bytes::from("a-value"))); - property_values.insert("b-key".to_string(), Value::Bytes(Bytes::from("b-value"))); + fn encoder(&self) -> &Self::Encoder { + &self.encoder + } - let mut event = Event::Log(LogEvent::from("hello")); - event.as_mut_log().insert(properties_key, property_values); + fn split_input(&self, mut input: PulsarEvent) -> (Self::Metadata, RequestMetadataBuilder, Self::Events) { + let builder = RequestMetadataBuilder::from_events(&input); + let metadata = PulsarMetadata { + finalizers: input.event.take_finalizers(), + key: input.key, + timestamp_millis: input.timestamp_millis, + properties: input.properties, + topic: input.topic + }; + (metadata, builder, input.event) + } - let properties = get_properties(&event, &Some(properties_key.to_string())).unwrap(); - assert_eq!(properties.get("a-key").unwrap(), "a-value".as_bytes()); - assert_eq!(properties.get("b-key").unwrap(), "b-value".as_bytes()); + fn build_request(&self, metadata: Self::Metadata, request_metadata: RequestMetadata, payload: EncodeResult) -> Self::Request { + let body = payload.into_payload(); + PulsarRequest { + body, + metadata, + request_metadata, + } } } + diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index f2eb899b3c4c3..657a753b0c91a 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -1,4 +1,3 @@ -use std::collections::HashMap; use std::sync::Arc; use std::task::{Context, Poll}; @@ -8,28 +7,26 @@ use lru::LruCache; use pulsar::{Error as PulsarError, Executor, Producer, ProducerOptions, Pulsar}; use tokio::sync::Mutex; use tower::Service; +use vector_common::internal_event::{CountByteSize}; use vector_core::{ internal_event::{ - ByteSize, BytesSent, EventsSent, InternalEventHandle as _, Protocol, Registered, + BytesSent, Protocol, Registered, }, stream::DriverResponse, }; use crate::event::{EventFinalizers, EventStatus, Finalizable}; use crate::internal_events::PulsarSendingError; +use std::num::NonZeroUsize; +use vector_common::request_metadata::{MetaDescriptive, RequestMetadata}; +use crate::sinks::pulsar::request_builder::PulsarMetadata; +use crate::sinks::util::retries::RetryLogic; +#[derive(Clone)] pub struct PulsarRequest { pub body: Bytes, - pub metadata: PulsarRequestMetadata, - pub event_byte_size: usize, -} - -pub struct PulsarRequestMetadata { - pub finalizers: EventFinalizers, - pub key: Option, - pub properties: Option>, - pub timestamp_millis: Option, - pub topic: String, + pub metadata: PulsarMetadata, + pub request_metadata: RequestMetadata } pub struct PulsarResponse { @@ -41,12 +38,8 @@ impl DriverResponse for PulsarResponse { EventStatus::Delivered } - fn events_sent(&self) -> EventsSent { - EventsSent { - count: 1, - byte_size: self.event_byte_size, - output: None, - } + fn events_sent(&self) -> CountByteSize { + CountByteSize(1, self.event_byte_size) } } @@ -56,6 +49,26 @@ impl Finalizable for PulsarRequest { } } +impl MetaDescriptive for PulsarRequest { + fn get_metadata(&self) -> RequestMetadata { + self.request_metadata + } +} + +/// Pulsar retry logic. +#[derive(Debug, Default, Clone)] +pub struct PulsarRetryLogic; + +impl RetryLogic for PulsarRetryLogic { + type Error = PulsarError; + type Response = PulsarResponse; + + fn is_retriable_error(&self, error: &Self::Error) -> bool { + // TODO improve retry logic + true + } +} + type SafeLru = Arc>>, PulsarError>>>>; pub struct PulsarService { pulsar_client: Pulsar, @@ -68,12 +81,12 @@ impl PulsarService { pub(crate) fn new( pulsar_client: Pulsar, producer_options: ProducerOptions, - producer_cache_size: Option, + producer_cache_size: Option, ) -> PulsarService { // Use a LRUCache to store a limited set of producers // Producers in Pulsar use a send buffer, so we want to limit the number of these let producer_cache = Arc::new(Mutex::new(LruCache::new( - producer_cache_size.unwrap_or(100), + producer_cache_size.unwrap_or(NonZeroUsize::new(100).unwrap()), ))); PulsarService { pulsar_client, @@ -144,7 +157,6 @@ impl Service for PulsarService { self.producer_options.clone(), request.metadata.topic.clone(), ); - let bytes_sent = self.bytes_sent.clone(); let ts = request.metadata.timestamp_millis.to_owned(); Box::pin(async move { let p = prod_future.await; @@ -167,10 +179,8 @@ impl Service for PulsarService { match msg_builder.send().await { Ok(resp) => match resp.await { Ok(_) => { - //TODO: not totally accurate, doesn't include metadata or key - bytes_sent.emit(ByteSize(request.body.len())); Ok(PulsarResponse { - event_byte_size: request.event_byte_size, + event_byte_size: request.request_metadata.events_byte_size(), }) } Err(e) => { diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index 817d9282afab8..32f20b2db49b4 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -1,22 +1,29 @@ +use std::collections::HashMap; use async_trait::async_trait; +use bytes::Bytes; -use futures::{future, stream::BoxStream, StreamExt}; +use futures::{stream::BoxStream, StreamExt}; use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; use snafu::{ResultExt, Snafu}; -use tower::limit::ConcurrencyLimit; +use serde::Serialize; +use tower::ServiceBuilder; +use vector_buffers::EventCount; +use vector_common::byte_size_of::ByteSizeOf; use vector_core::config::log_schema; use vector_core::event::LogEvent; use vector_core::sink::StreamSink; -use crate::sinks::pulsar::config::{PulsarSinkConfig, QUEUED_MIN_MESSAGES}; +use crate::sinks::pulsar::config::PulsarSinkConfig; use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; -use crate::sinks::pulsar::service::PulsarService; -use crate::sinks::util::SinkBuilderExt; +use crate::sinks::pulsar::service::{PulsarRetryLogic, PulsarService}; +use crate::sinks::util::{ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig, TowerRequestSettings}; use crate::template::{Template, TemplateParseError}; use crate::{ codecs::{Encoder, Transformer}, event::Event, }; +use crate::sinks::pulsar::encoder::PulsarEncoder; +use crate::sinks::pulsar::util; #[derive(Debug, Snafu)] #[snafu(visibility(pub(crate)))] @@ -31,11 +38,44 @@ pub(crate) struct PulsarSink { transformer: Transformer, encoder: Encoder<()>, service: PulsarService, + request_settings: TowerRequestSettings, topic: Template, key_field: Option, properties_key: Option, } +/// Stores the event together with the extracted keys, topics, etc +/// This is passed into the `RequestBuilder` which then splits it out into the event +/// and metadata containing the keys, and metadata +/// This event needs to be created prior to building the request so we can filter out +/// any events that error whilst rendering the templates. +#[derive(Serialize)] +pub(super) struct PulsarEvent { + pub(super) event: Event, + pub(super) topic: String, + pub(super) key: Option, + pub(super) properties: Option>, + pub(super) timestamp_millis: Option, +} + +impl EventCount for PulsarEvent { + fn event_count(&self) -> usize { + // A PulsarEvent represents one event. + 1 + } +} + +impl ByteSizeOf for PulsarEvent { + fn allocated_bytes(&self) -> usize { + self.event.size_of() + + self.topic.size_of() + + self.key.map_or(0, |bytes| bytes.size_of()) + + self.properties.map_or(0, + |props| + props.iter().map(|(key, val)| key.capacity() + val.size_of()).sum()) + } +} + pub(crate) async fn healthcheck(config: PulsarSinkConfig) -> crate::Result<()> { trace!("Healthcheck started."); let client = config.create_pulsar_client().await?; @@ -51,10 +91,12 @@ impl PulsarSink { pub(crate) fn new( client: Pulsar, config: PulsarSinkConfig, - ) -> crate::Result { + ) -> crate::Result + { let producer_opts = config.build_producer_options(); let transformer = config.encoding.transformer(); let serializer = config.encoding.build()?; + let request_settings = config.request.unwrap_with(&TowerRequestConfig::default()); let encoder = Encoder::<()>::new(serializer); let service = PulsarService::new(client, producer_opts, None); @@ -62,24 +104,51 @@ impl PulsarSink { properties_key: config.properties_key, key_field: config.key_field, transformer, + request_settings, encoder, service, topic: Template::try_from(config.topic).context(TopicTemplateSnafu)?, }) } + /// Transforms an event into a Pulsar event by rendering the required template fields. + /// Returns None if there is an error whilst rendering. + fn make_pulsar_event(&self, event: Event) -> Option { + let topic = self.topic.render_string(&event).ok()?; + let key = util::get_key(&event, &self.key_field); + let timestamp_millis = util::get_timestamp_millis(&event, log_schema()); + let properties = util::get_properties(&event, &self.properties_key); + Some(PulsarEvent{ + event, + topic, + key, + timestamp_millis, + properties + }) + } + async fn run_inner(self: Box, input: BoxStream<'_, Event>) -> Result<(), ()> { - let service = ConcurrencyLimit::new(self.service, QUEUED_MIN_MESSAGES as usize); - let mut request_builder = PulsarRequestBuilder { - key_field: self.key_field, - properties_key: self.properties_key, - topic_template: self.topic, - transformer: self.transformer, - encoder: self.encoder, - log_schema: log_schema(), + let service = ServiceBuilder::new() + .settings(self.request_settings, PulsarRetryLogic) + .service(self.service); + let request_builder = PulsarRequestBuilder { + encoder: PulsarEncoder{ + transformer: self.transformer.clone(), + encoder: self.encoder.clone(), + } }; let sink = input - .filter_map(|event| future::ready(request_builder.build_request(event))) + .filter_map(|event| std::future::ready(self.make_pulsar_event(event))) + .request_builder(None, request_builder) + .filter_map(|request| async move { + match request { + Err(e) => { + error!("Failed to build Pulsar request: {:?}.", e); + None + } + Ok(req) => Some(req), + } + }) .into_driver(service); sink.run().await } diff --git a/src/sinks/pulsar/tests.rs b/src/sinks/pulsar/tests.rs index 74a053487d486..a53f9aa5ed0a0 100644 --- a/src/sinks/pulsar/tests.rs +++ b/src/sinks/pulsar/tests.rs @@ -1,3 +1,13 @@ +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn generate_config() { + PulsarSinkConfig::generate_config(); + } +} + #[cfg(feature = "pulsar-integration-tests")] #[cfg(test)] mod integration_tests { diff --git a/src/sinks/pulsar/util.rs b/src/sinks/pulsar/util.rs new file mode 100644 index 0000000000000..1fce9f085db8c --- /dev/null +++ b/src/sinks/pulsar/util.rs @@ -0,0 +1,89 @@ +use bytes::Bytes; +use std::collections::HashMap; +use value::Value; +use vector_core::config::LogSchema; +use vector_core::event::Event; +use crate::internal_events::PulsarPropertyExtractionError; + +pub fn get_key(event: &Event, key_field: &Option) -> Option { + key_field.as_ref().and_then(|key_field| match event { + Event::Log(log) => log + .get(key_field.as_str()) + .map(|value| value.coerce_to_bytes()), + Event::Metric(metric) => metric + .tags() + .and_then(|tags| tags.get(key_field)) + .map(|value| value.clone().into()), + _ => None, + }) +} + +pub fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option { + match &event { + Event::Log(log) => log + .get(log_schema.timestamp_key()) + .and_then(|v| v.as_timestamp()) + .copied(), + Event::Metric(metric) => metric.timestamp(), + _ => None, + } + .map(|ts| ts.timestamp_millis()) +} + +pub fn get_properties( + event: &Event, + properties_key: &Option, +) -> Option> { + properties_key.as_ref().and_then(|properties_key| { + if let Event::Log(log) = event { + if let Some(properties) = log.get(properties_key.as_str()) { + match properties { + Value::Object(headers_map) => { + let mut property_map = HashMap::new(); + for (key, value) in headers_map { + if let Value::Bytes(value_bytes) = value { + property_map.insert(key.clone(), value_bytes.clone()); + } else { + emit!(PulsarPropertyExtractionError { + property_field: properties_key + }); + } + } + return Some(property_map); + } + _ => { + emit!(PulsarPropertyExtractionError { + property_field: properties_key + }); + } + } + } + } + None + }) +} + +#[cfg(test)] +mod tests { + use std::collections::BTreeMap; + + use bytes::Bytes; + + use super::*; + use crate::event::LogEvent; + + #[test] + fn pulsar_get_headers() { + let properties_key = "properties"; + let mut property_values = BTreeMap::new(); + property_values.insert("a-key".to_string(), Value::Bytes(Bytes::from("a-value"))); + property_values.insert("b-key".to_string(), Value::Bytes(Bytes::from("b-value"))); + + let mut event = Event::Log(LogEvent::from("hello")); + event.as_mut_log().insert(properties_key, property_values); + + let properties = get_properties(&event, &Some(properties_key.to_string())).unwrap(); + assert_eq!(properties.get("a-key").unwrap(), "a-value".as_bytes()); + assert_eq!(properties.get("b-key").unwrap(), "b-value".as_bytes()); + } +} From e20a60a96970f61bae57032c937c4fdff85ad524 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Sun, 20 Nov 2022 13:55:56 -0700 Subject: [PATCH 11/48] fix all errors and pass tests --- src/sinks/pulsar/config.rs | 4 +- src/sinks/pulsar/mod.rs | 2 +- src/sinks/pulsar/request_builder.rs | 27 ++++++----- src/sinks/pulsar/service.rs | 35 +++++++-------- src/sinks/pulsar/sink.rs | 70 +++++++++++++---------------- src/sinks/pulsar/tests.rs | 7 ++- src/sinks/pulsar/util.rs | 36 ++++++++++++--- 7 files changed, 101 insertions(+), 80 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index be10876bf6374..62ae8c82f1b20 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -1,3 +1,4 @@ +use crate::sinks::util::TowerRequestConfig; use crate::{ codecs::EncodingConfig, config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, @@ -16,9 +17,6 @@ use pulsar::{ use snafu::ResultExt; use vector_config::configurable_component; use vector_core::config::DataType; -use crate::sinks::util::TowerRequestConfig; - -pub(crate) const QUEUED_MIN_MESSAGES: u64 = 100000; /// Configuration for the `pulsar` sink. #[configurable_component(sink("pulsar"))] diff --git a/src/sinks/pulsar/mod.rs b/src/sinks/pulsar/mod.rs index 85c7c325b38f7..49dfd20acbd9e 100644 --- a/src/sinks/pulsar/mod.rs +++ b/src/sinks/pulsar/mod.rs @@ -1,7 +1,7 @@ pub(crate) mod config; +pub(crate) mod encoder; pub(crate) mod request_builder; pub(crate) mod service; pub(crate) mod sink; pub(crate) mod tests; -pub(crate) mod encoder; mod util; diff --git a/src/sinks/pulsar/request_builder.rs b/src/sinks/pulsar/request_builder.rs index 6fa27ac73f5ed..2d32c03b2719b 100644 --- a/src/sinks/pulsar/request_builder.rs +++ b/src/sinks/pulsar/request_builder.rs @@ -1,18 +1,18 @@ -use bytes::{Bytes}; +use bytes::Bytes; use std::collections::HashMap; use std::io; use vector_common::finalization::EventFinalizers; use vector_common::request_metadata::RequestMetadata; -use crate::{ - event::{Event, Finalizable}, - sinks::pulsar::service::{PulsarRequest}, -}; use crate::sinks::pulsar::encoder::PulsarEncoder; use crate::sinks::pulsar::sink::PulsarEvent; use crate::sinks::util::metadata::RequestMetadataBuilder; -use crate::sinks::util::{Compression, RequestBuilder}; use crate::sinks::util::request_builder::EncodeResult; +use crate::sinks::util::{Compression, RequestBuilder}; +use crate::{ + event::{Event, Finalizable}, + sinks::pulsar::service::PulsarRequest, +}; #[derive(Clone)] pub(super) struct PulsarMetadata { @@ -45,19 +45,27 @@ impl RequestBuilder for PulsarRequestBuilder { &self.encoder } - fn split_input(&self, mut input: PulsarEvent) -> (Self::Metadata, RequestMetadataBuilder, Self::Events) { + fn split_input( + &self, + mut input: PulsarEvent, + ) -> (Self::Metadata, RequestMetadataBuilder, Self::Events) { let builder = RequestMetadataBuilder::from_events(&input); let metadata = PulsarMetadata { finalizers: input.event.take_finalizers(), key: input.key, timestamp_millis: input.timestamp_millis, properties: input.properties, - topic: input.topic + topic: input.topic, }; (metadata, builder, input.event) } - fn build_request(&self, metadata: Self::Metadata, request_metadata: RequestMetadata, payload: EncodeResult) -> Self::Request { + fn build_request( + &self, + metadata: Self::Metadata, + request_metadata: RequestMetadata, + payload: EncodeResult, + ) -> Self::Request { let body = payload.into_payload(); PulsarRequest { body, @@ -66,4 +74,3 @@ impl RequestBuilder for PulsarRequestBuilder { } } } - diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 657a753b0c91a..9093bcf49db48 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -7,26 +7,21 @@ use lru::LruCache; use pulsar::{Error as PulsarError, Executor, Producer, ProducerOptions, Pulsar}; use tokio::sync::Mutex; use tower::Service; -use vector_common::internal_event::{CountByteSize}; -use vector_core::{ - internal_event::{ - BytesSent, Protocol, Registered, - }, - stream::DriverResponse, -}; +use vector_common::internal_event::CountByteSize; +use vector_core::stream::DriverResponse; use crate::event::{EventFinalizers, EventStatus, Finalizable}; use crate::internal_events::PulsarSendingError; -use std::num::NonZeroUsize; -use vector_common::request_metadata::{MetaDescriptive, RequestMetadata}; use crate::sinks::pulsar::request_builder::PulsarMetadata; use crate::sinks::util::retries::RetryLogic; +use std::num::NonZeroUsize; +use vector_common::request_metadata::{MetaDescriptive, RequestMetadata}; #[derive(Clone)] -pub struct PulsarRequest { +pub(super) struct PulsarRequest { pub body: Bytes, pub metadata: PulsarMetadata, - pub request_metadata: RequestMetadata + pub request_metadata: RequestMetadata, } pub struct PulsarResponse { @@ -41,6 +36,10 @@ impl DriverResponse for PulsarResponse { fn events_sent(&self) -> CountByteSize { CountByteSize(1, self.event_byte_size) } + + fn bytes_sent(&self) -> Option<(usize, &str)> { + Some((self.event_byte_size, "pulsar")) + } } impl Finalizable for PulsarRequest { @@ -63,18 +62,19 @@ impl RetryLogic for PulsarRetryLogic { type Error = PulsarError; type Response = PulsarResponse; - fn is_retriable_error(&self, error: &Self::Error) -> bool { + fn is_retriable_error(&self, _error: &Self::Error) -> bool { // TODO improve retry logic true } } type SafeLru = Arc>>, PulsarError>>>>; + +#[derive(Clone)] pub struct PulsarService { pulsar_client: Pulsar, producer_cache: SafeLru, producer_options: ProducerOptions, - bytes_sent: Registered, } impl PulsarService { @@ -92,7 +92,6 @@ impl PulsarService { pulsar_client, producer_cache, producer_options, - bytes_sent: register!(BytesSent::from(Protocol("pulsar".into()))), } } @@ -178,11 +177,9 @@ impl Service for PulsarService { match msg_builder.send().await { Ok(resp) => match resp.await { - Ok(_) => { - Ok(PulsarResponse { - event_byte_size: request.request_metadata.events_byte_size(), - }) - } + Ok(_) => Ok(PulsarResponse { + event_byte_size: request.request_metadata.events_byte_size(), + }), Err(e) => { emit!(PulsarSendingError { error: Box::new(PulsarError::Custom("failed to send".to_string())), diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index 32f20b2db49b4..dace283e08d35 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -1,29 +1,30 @@ -use std::collections::HashMap; use async_trait::async_trait; use bytes::Bytes; +use std::collections::HashMap; use futures::{stream::BoxStream, StreamExt}; use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; -use snafu::{ResultExt, Snafu}; use serde::Serialize; +use snafu::{ResultExt, Snafu}; use tower::ServiceBuilder; use vector_buffers::EventCount; use vector_common::byte_size_of::ByteSizeOf; -use vector_core::config::log_schema; use vector_core::event::LogEvent; use vector_core::sink::StreamSink; use crate::sinks::pulsar::config::PulsarSinkConfig; +use crate::sinks::pulsar::encoder::PulsarEncoder; use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; use crate::sinks::pulsar::service::{PulsarRetryLogic, PulsarService}; -use crate::sinks::util::{ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig, TowerRequestSettings}; +use crate::sinks::pulsar::util; +use crate::sinks::util::{ + ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig, TowerRequestSettings, +}; use crate::template::{Template, TemplateParseError}; use crate::{ codecs::{Encoder, Transformer}, event::Event, }; -use crate::sinks::pulsar::encoder::PulsarEncoder; -use crate::sinks::pulsar::util; #[derive(Debug, Snafu)] #[snafu(visibility(pub(crate)))] @@ -39,9 +40,8 @@ pub(crate) struct PulsarSink { encoder: Encoder<()>, service: PulsarService, request_settings: TowerRequestSettings, - topic: Template, - key_field: Option, - properties_key: Option, + config: PulsarSinkConfig, + topic_template: Template, } /// Stores the event together with the extracted keys, topics, etc @@ -67,12 +67,15 @@ impl EventCount for PulsarEvent { impl ByteSizeOf for PulsarEvent { fn allocated_bytes(&self) -> usize { - self.event.size_of() + - self.topic.size_of() + - self.key.map_or(0, |bytes| bytes.size_of()) + - self.properties.map_or(0, - |props| - props.iter().map(|(key, val)| key.capacity() + val.size_of()).sum()) + self.event.size_of() + + self.topic.size_of() + + self.key.as_ref().map_or(0, |bytes| bytes.clone().size_of()) + + self.properties.as_ref().map_or(0, |props| { + props + .iter() + .map(|(key, val)| key.capacity() + val.size_of()) + .sum() + }) } } @@ -91,39 +94,22 @@ impl PulsarSink { pub(crate) fn new( client: Pulsar, config: PulsarSinkConfig, - ) -> crate::Result - { + ) -> crate::Result { let producer_opts = config.build_producer_options(); let transformer = config.encoding.transformer(); let serializer = config.encoding.build()?; let request_settings = config.request.unwrap_with(&TowerRequestConfig::default()); let encoder = Encoder::<()>::new(serializer); let service = PulsarService::new(client, producer_opts, None); + let topic = config.topic.clone(); Ok(PulsarSink { - properties_key: config.properties_key, - key_field: config.key_field, + config, transformer, request_settings, encoder, service, - topic: Template::try_from(config.topic).context(TopicTemplateSnafu)?, - }) - } - - /// Transforms an event into a Pulsar event by rendering the required template fields. - /// Returns None if there is an error whilst rendering. - fn make_pulsar_event(&self, event: Event) -> Option { - let topic = self.topic.render_string(&event).ok()?; - let key = util::get_key(&event, &self.key_field); - let timestamp_millis = util::get_timestamp_millis(&event, log_schema()); - let properties = util::get_properties(&event, &self.properties_key); - Some(PulsarEvent{ - event, - topic, - key, - timestamp_millis, - properties + topic_template: Template::try_from(topic).context(TopicTemplateSnafu)?, }) } @@ -132,13 +118,19 @@ impl PulsarSink { .settings(self.request_settings, PulsarRetryLogic) .service(self.service); let request_builder = PulsarRequestBuilder { - encoder: PulsarEncoder{ + encoder: PulsarEncoder { transformer: self.transformer.clone(), encoder: self.encoder.clone(), - } + }, }; let sink = input - .filter_map(|event| std::future::ready(self.make_pulsar_event(event))) + .filter_map(|event| { + std::future::ready(util::make_pulsar_event( + &self.topic_template, + &self.config, + event, + )) + }) .request_builder(None, request_builder) .filter_map(|request| async move { match request { diff --git a/src/sinks/pulsar/tests.rs b/src/sinks/pulsar/tests.rs index a53f9aa5ed0a0..c0dc7647abd9e 100644 --- a/src/sinks/pulsar/tests.rs +++ b/src/sinks/pulsar/tests.rs @@ -1,6 +1,7 @@ #[cfg(test)] -mod tests { - use super::*; +mod config { + use crate::sinks::pulsar::config::PulsarSinkConfig; + use vector_config::component::GenerateConfig; #[test] fn generate_config() { @@ -13,6 +14,7 @@ mod tests { mod integration_tests { use crate::sinks::pulsar::config::PulsarSinkConfig; use crate::sinks::pulsar::sink::PulsarSink; + use crate::sinks::util::TowerRequestConfig; use codecs::TextSerializerConfig; use futures::StreamExt; use pulsar::SubType; @@ -106,6 +108,7 @@ mod integration_tests { encoding: TextSerializerConfig::new().into(), auth: None, acknowledgements: Default::default(), + request: TowerRequestConfig::default(), }; pulsar_happy_reuse(cnf).await diff --git a/src/sinks/pulsar/util.rs b/src/sinks/pulsar/util.rs index 1fce9f085db8c..cd65fe88b75b7 100644 --- a/src/sinks/pulsar/util.rs +++ b/src/sinks/pulsar/util.rs @@ -1,11 +1,35 @@ +use crate::internal_events::PulsarPropertyExtractionError; +use crate::sinks::pulsar::config::PulsarSinkConfig; +use crate::sinks::pulsar::sink::PulsarEvent; +use crate::template::Template; use bytes::Bytes; use std::collections::HashMap; use value::Value; +use vector_core::config::log_schema; use vector_core::config::LogSchema; use vector_core::event::Event; -use crate::internal_events::PulsarPropertyExtractionError; -pub fn get_key(event: &Event, key_field: &Option) -> Option { +/// Transforms an event into a Pulsar event by rendering the required template fields. +/// Returns None if there is an error whilst rendering. +pub(super) fn make_pulsar_event( + topic: &Template, + config: &PulsarSinkConfig, + event: Event, +) -> Option { + let topic = topic.render_string(&event).ok()?; + let key = get_key(&event, &config.key_field); + let timestamp_millis = get_timestamp_millis(&event, log_schema()); + let properties = get_properties(&event, &config.properties_key); + Some(PulsarEvent { + event, + topic, + key, + timestamp_millis, + properties, + }) +} + +fn get_key(event: &Event, key_field: &Option) -> Option { key_field.as_ref().and_then(|key_field| match event { Event::Log(log) => log .get(key_field.as_str()) @@ -13,12 +37,12 @@ pub fn get_key(event: &Event, key_field: &Option) -> Option { Event::Metric(metric) => metric .tags() .and_then(|tags| tags.get(key_field)) - .map(|value| value.clone().into()), + .map(|value| value.to_owned().into()), _ => None, }) } -pub fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option { +fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option { match &event { Event::Log(log) => log .get(log_schema.timestamp_key()) @@ -27,10 +51,10 @@ pub fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Op Event::Metric(metric) => metric.timestamp(), _ => None, } - .map(|ts| ts.timestamp_millis()) + .map(|ts| ts.timestamp_millis()) } -pub fn get_properties( +fn get_properties( event: &Event, properties_key: &Option, ) -> Option> { From 20d6160184ca2d19761363ffc082716f43761391 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Sun, 20 Nov 2022 13:56:19 -0700 Subject: [PATCH 12/48] update cargo.lock --- Cargo.lock | 1175 +++++++++++++++++++++++++++------------------------- 1 file changed, 606 insertions(+), 569 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index afd7a3340eb47..eb7ef7dfbbece 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -54,9 +54,9 @@ dependencies = [ [[package]] name = "ahash" -version = "0.8.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57e6e951cfbb2db8de1828d49073a113a29fd7117b1596caa781a258c7e38d72" +checksum = "bf6ccdb167abbf410dcb915cabd428929d7f6a04980b54a11f26a39f1c7f7107" dependencies = [ "cfg-if", "getrandom 0.2.8", @@ -66,9 +66,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "0.7.18" +version = "0.7.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e37cfd5e7657ada45f742d6e99ca5788580b5c529dc78faf11ece6dc702656f" +checksum = "b4f55bd91a0978cbfd91c457a164bab8b4001c833b7f323132c0a4e1922dd44e" dependencies = [ "memchr", ] @@ -208,15 +208,6 @@ version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4c527152e37cf757a3f78aae5a06fbeefdb07ccc535c980a3208ee3060dd544" -[[package]] -name = "arrayvec" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd9fd44efafa8690358b7408d253adf110036b88f55672a933f01d616ad9b1b9" -dependencies = [ - "nodrop", -] - [[package]] name = "arrayvec" version = "0.5.2" @@ -246,9 +237,9 @@ dependencies = [ [[package]] name = "assert-json-diff" -version = "2.0.1" +version = "2.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50f1c3703dd33532d7f0ca049168930e9099ecac238e23cf932f3a69c42f06da" +checksum = "47e4f2b81832e72834d7518d8487a0396a28cc408186a2e8854c0f98011faf12" dependencies = [ "serde", "serde_json", @@ -270,11 +261,11 @@ dependencies = [ [[package]] name = "async-channel" -version = "1.6.1" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2114d64672151c0c5eaa5e131ec84a74f06e1e559830dabba01ca30605d66319" +checksum = "e14485364214912d3b19cc3435dde4df66065127f05fa0d75c712f36f12c2f28" dependencies = [ - "concurrent-queue", + "concurrent-queue 1.2.4", "event-listener", "futures-core", ] @@ -296,34 +287,35 @@ dependencies = [ [[package]] name = "async-executor" -version = "1.4.1" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "871f9bb5e0a22eeb7e8cf16641feb87c9dc67032ccf8ff49e772eb9941d3a965" +checksum = "17adb73da160dfb475c183343c8cccd80721ea5a605d3eb57125f0a7b7a92d0b" dependencies = [ + "async-lock", "async-task", - "concurrent-queue", + "concurrent-queue 2.0.0", "fastrand", "futures-lite", - "once_cell", "slab", ] [[package]] name = "async-fs" -version = "1.5.0" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b3ca4f8ff117c37c278a2f7415ce9be55560b846b5bc4412aaa5d29c1c3dae2" +checksum = "279cf904654eeebfa37ac9bb1598880884924aab82e290aa65c9e77a0e142e06" dependencies = [ "async-lock", + "autocfg", "blocking", "futures-lite", ] [[package]] name = "async-global-executor" -version = "2.2.0" +version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5262ed948da60dd8956c6c5aca4d4163593dddb7b32d73267c93dab7b2e98940" +checksum = "f1b6f5d7df27bd294849f8eec66ecfc63d11814df7a4f5d74168a2394467b776" dependencies = [ "async-channel", "async-executor", @@ -331,7 +323,6 @@ dependencies = [ "async-lock", "blocking", "futures-lite", - "num_cpus", "once_cell", ] @@ -386,7 +377,7 @@ checksum = "c121a894495d7d3fc3d4e15e0a9843e422e4d1d9e3c514d8062a1c94b35b005d" dependencies = [ "Inflector", "async-graphql-parser", - "darling 0.14.1", + "darling 0.14.2", "proc-macro-crate", "proc-macro2", "quote", @@ -432,15 +423,16 @@ dependencies = [ [[package]] name = "async-io" -version = "1.6.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a811e6a479f2439f0c04038796b5cfb3d2ad56c230e0f2d3f7b04d68cfee607b" +checksum = "e8121296a9f05be7f34aa4196b1747243b3b62e048bb7906f644f3fbfc490cf7" dependencies = [ - "concurrent-queue", + "async-lock", + "autocfg", + "concurrent-queue 1.2.4", "futures-lite", "libc", "log", - "once_cell", "parking", "polling", "slab", @@ -451,31 +443,34 @@ dependencies = [ [[package]] name = "async-lock" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e97a171d191782fba31bb902b14ad94e24a68145032b7eedf871ab0bc0d077b6" +checksum = "c8101efe8695a6c17e02911402145357e718ac92d3ff88ae8419e84b1707b685" dependencies = [ "event-listener", + "futures-lite", ] [[package]] name = "async-net" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5373304df79b9b4395068fb080369ec7178608827306ce4d081cba51cac551df" +checksum = "4051e67316bc7eff608fe723df5d32ed639946adcd69e07df41fd42a7b411f1f" dependencies = [ "async-io", + "autocfg", "blocking", "futures-lite", ] [[package]] name = "async-process" -version = "1.3.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83137067e3a2a6a06d67168e49e68a0957d215410473a740cea95a2425c0b7c6" +checksum = "02111fd8655a613c25069ea89fc8d9bb89331fa77486eb3bc059ee757cfa481c" dependencies = [ "async-io", + "autocfg", "blocking", "cfg-if", "event-listener", @@ -532,9 +527,9 @@ dependencies = [ [[package]] name = "async-task" -version = "4.2.0" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30696a84d817107fc028e049980e09d5e140e8da8f1caeb17e8e950658a3cea9" +checksum = "7a40729d2133846d9ed0ea60a8b9541bccddab49cd30f0715a1da672fe9a2524" [[package]] name = "async-trait" @@ -1019,7 +1014,7 @@ version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b02e06ea63498c43bc0217ea4d16605d4e58d85c12fc23f6572ff6d0a840c61" dependencies = [ - "itoa 1.0.1", + "itoa 1.0.4", "num-integer", "ryu", "time", @@ -1064,7 +1059,7 @@ dependencies = [ "http", "http-body", "hyper", - "itoa 1.0.1", + "itoa 1.0.4", "matchit", "memchr", "mime", @@ -1221,9 +1216,9 @@ checksum = "e6b4d9b1225d28d360ec6a231d65af1fd99a2a095154c8040689617290569c5c" [[package]] name = "bit-set" -version = "0.5.2" +version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e11e16035ea35e4e5997b393eacbf6f63983188f7a2ad25bfb13465f5ad59de" +checksum = "0700ddab506f33b20a03b13996eccd309a48e5ff77d0d95926aa0210fb4e95f1" dependencies = [ "bit-vec 0.6.3", ] @@ -1278,9 +1273,9 @@ dependencies = [ [[package]] name = "block-buffer" -version = "0.10.2" +version = "0.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bf7fe51849ea569fd452f37822f606a5cabb684dc918707a0193fd4664ff324" +checksum = "69cce20737498f97b993470a6e536b8523f0af7892a4f928cceb1ac5e52ebe7e" dependencies = [ "generic-array", ] @@ -1337,9 +1332,9 @@ dependencies = [ "hyperlocal", "log", "pin-project-lite", - "rustls 0.20.4", + "rustls 0.20.7", "rustls-native-certs 0.6.2", - "rustls-pemfile 1.0.0", + "rustls-pemfile 1.0.1", "serde", "serde_derive", "serde_json", @@ -1366,9 +1361,9 @@ dependencies = [ [[package]] name = "bson" -version = "2.3.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a24ecf39f5a314493ede1bb015984735d41aa6aedb59cafb95492d40cd893330" +checksum = "99d76085681585d39016f4d3841eb019201fc54d2dd0d92ad1e4fab3bfb32754" dependencies = [ "ahash 0.7.6", "base64", @@ -1380,7 +1375,7 @@ dependencies = [ "serde_bytes", "serde_json", "time", - "uuid 0.8.2", + "uuid 1.2.2", ] [[package]] @@ -1409,9 +1404,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.9.1" +version = "3.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a45a46ab1f2412e53d3a0ade76ffad2025804294569aae387231a0cd6e0899" +checksum = "572f695136211188308f16ad2ca5c851a712c464060ae6974944458eb83880ba" [[package]] name = "bytecheck" @@ -1436,9 +1431,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.9.1" +version = "1.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdead85bdec19c194affaeeb670c0e41fe23de31459efd1c174d049269cf02cc" +checksum = "aaa3a8d9a1ca92e282c96a32d6511b695d7d994d1d102ba85d279f9b2756947f" [[package]] name = "byteorder" @@ -1467,9 +1462,9 @@ dependencies = [ [[package]] name = "bytes-utils" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1934a3ef9cac8efde4966a92781e77713e1ba329f1d42e446c7d7eba340d8ef1" +checksum = "e47d3a8076e283f3acd27400535992edb3ba4b5bb72f8891ad8fbe7932a7d4b9" dependencies = [ "bytes 1.2.1", "either", @@ -1510,9 +1505,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.73" +version = "1.0.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fff2a6927b3bb87f9595d67196a70493f627687a71d87a0d692242c33f58c11" +checksum = "76a284da2e6fe2092f2353e51713435363112dfd60030e22add80be333fb928f" dependencies = [ "jobserver", ] @@ -1652,14 +1647,14 @@ dependencies = [ [[package]] name = "clap" -version = "3.2.22" +version = "3.2.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86447ad904c7fb335a790c9d7fe3d0d971dc523b8ccd1561a520de9a85302750" +checksum = "71655c45cb9845d3270c9d6df84ebe72b4dad3c2ba3f7023ad47c144e4e473a5" dependencies = [ "bitflags", - "clap_lex 0.2.3", + "clap_lex 0.2.4", "indexmap", - "textwrap 0.15.1", + "textwrap 0.16.0", ] [[package]] @@ -1675,7 +1670,7 @@ dependencies = [ "once_cell", "strsim 0.10.0", "termcolor", - "terminal_size 0.2.1", + "terminal_size 0.2.2", ] [[package]] @@ -1693,9 +1688,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.2.3" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87eba3c8c7f42ef17f6c659fc7416d0f4758cd3e58861ee63c5fa4a4dde649e4" +checksum = "2850f2f5a82cbf437dd5af4d49848fbdfc27c157c3d010345776f952765261c5" dependencies = [ "os_str_bytes", ] @@ -1711,9 +1706,9 @@ dependencies = [ [[package]] name = "clipboard-win" -version = "4.4.1" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f3e1238132dc01f081e1cbb9dace14e5ef4c3a51ee244bd982275fb514605db" +checksum = "c4ab1b92798304eedc095b53942963240037c0516452cb11aeba709d420b2219" dependencies = [ "error-code", "str-buf", @@ -1722,9 +1717,9 @@ dependencies = [ [[package]] name = "cmake" -version = "0.1.48" +version = "0.1.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8ad8cef104ac57b68b89df3208164d228503abbdce70f6880ffa3d970e7443a" +checksum = "db34956e100b30725f2eb215f90d4871051239535632f84fea3bc92722c66b7c" dependencies = [ "cc", ] @@ -1744,7 +1739,7 @@ dependencies = [ "memchr", "once_cell", "ordered-float 3.4.0", - "prost 0.11.0", + "prost 0.11.2", "regex", "serde", "serde_json", @@ -1799,9 +1794,9 @@ dependencies = [ [[package]] name = "combine" -version = "4.6.4" +version = "4.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a604e93b79d1808327a6fca85a6f2d69de66461e7620f5a4cbf5fb4d1d7c948" +checksum = "35ed6e9d84f0b51a7f52daf1c7d71dd136fd7a3f41a8462b8cdb8c78d920fad4" dependencies = [ "bytes 1.2.1", "futures-core", @@ -1813,13 +1808,22 @@ dependencies = [ [[package]] name = "concurrent-queue" -version = "1.2.2" +version = "1.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30ed07550be01594c6026cff2a1d7fe9c8f683caa798e12b68694ac9e88286a3" +checksum = "af4780a44ab5696ea9e28294517f1fffb421a83a25af521333c838635509db9c" dependencies = [ "cache-padded", ] +[[package]] +name = "concurrent-queue" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd7bef69dc86e3c610e4e7aed41035e2a7ed12e72dd7530f61327a6579a4390b" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "console" version = "0.15.2" @@ -1839,8 +1843,8 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e57ff02e8ad8e06ab9731d5dc72dc23bef9200778eae1a89d555d8c42e5d4a86" dependencies = [ - "prost 0.11.0", - "prost-types 0.11.1", + "prost 0.11.2", + "prost-types 0.11.2", "tonic", "tracing-core 0.1.30", ] @@ -1857,7 +1861,7 @@ dependencies = [ "futures 0.3.25", "hdrhistogram", "humantime", - "prost-types 0.11.1", + "prost-types 0.11.2", "serde", "serde_json", "thread_local", @@ -1911,9 +1915,9 @@ checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" [[package]] name = "cpufeatures" -version = "0.2.2" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59a6001667ab124aebae2a495118e11d30984c3a653e99d86d58971708cf5e4b" +checksum = "28d997bd5e24a5928dd43e46dc529867e207907fe0b239c3477d924f7f2ca320" dependencies = [ "libc", ] @@ -1961,10 +1965,10 @@ dependencies = [ "atty", "cast", "ciborium", - "clap 3.2.22", + "clap 3.2.23", "criterion-plot", "futures 0.3.25", - "itertools 0.10.5", + "itertools", "lazy_static", "num-traits", "oorandom", @@ -1986,14 +1990,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" dependencies = [ "cast", - "itertools 0.10.5", + "itertools", ] [[package]] name = "crossbeam-channel" -version = "0.5.4" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5aaa7bd5fb665c6864b5f963dd9097905c54125909c7aa94c9e18507cdbe6c53" +checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" dependencies = [ "cfg-if", "crossbeam-utils", @@ -2001,9 +2005,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6455c0ca19f0d2fbf751b908d5c55c1f5cbc65e03c4225427254b46890bdde1e" +checksum = "715e8152b692bba2d374b53d4875445368fdf21a94751410af607a5ac677d1fc" dependencies = [ "cfg-if", "crossbeam-epoch", @@ -2012,23 +2016,22 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.8" +version = "0.9.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1145cf131a2c6ba0615079ab6a638f7e1973ac9c2634fcbeaaad6114246efe8c" +checksum = "96bf8df95e795db1a4aca2957ad884a2df35413b24bbeb3114422f3cc21498e8" dependencies = [ "autocfg", "cfg-if", "crossbeam-utils", - "lazy_static", - "memoffset", + "memoffset 0.7.1", "scopeguard", ] [[package]] name = "crossbeam-queue" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd42583b04998a5363558e5f9291ee5a5ff6b49944332103f251e7479a82aa7" +checksum = "ebb3d1683412e9be6a15533314f00ec223c0762c522a3f77f048b265aab4470c" dependencies = [ "cfg-if", "crossbeam-utils", @@ -2036,9 +2039,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.12" +version = "0.8.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edbafec5fa1f196ca66527c1b12c2ec4745ca14b50f1ad8f9f6f720b55d11fac" +checksum = "422f23e724af1240ec469ea1e834d87a4b59ce2efe2c6a96256b0c47e2fd86aa" dependencies = [ "cfg-if", ] @@ -2077,9 +2080,9 @@ checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" [[package]] name = "crypto-common" -version = "0.1.3" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57952ca27b5e3606ff4dd79b0020231aaf9d6aa76dc05fd30137538c50bd3ce8" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" dependencies = [ "generic-array", "typenum", @@ -2109,9 +2112,9 @@ dependencies = [ [[package]] name = "ctor" -version = "0.1.22" +version = "0.1.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f877be4f7c9f246b183111634f75baa039715e3f46ce860677d3b19a69fb229c" +checksum = "6d2301688392eb071b0bf1a37be05c469d3cc4dbbd95df672fe28ab021e6a096" dependencies = [ "quote", "syn", @@ -2145,6 +2148,50 @@ dependencies = [ "zeroize", ] +[[package]] +name = "cxx" +version = "1.0.82" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4a41a86530d0fe7f5d9ea779916b7cadd2d4f9add748b99c2c029cbbdfaf453" +dependencies = [ + "cc", + "cxxbridge-flags", + "cxxbridge-macro", + "link-cplusplus", +] + +[[package]] +name = "cxx-build" +version = "1.0.82" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06416d667ff3e3ad2df1cd8cd8afae5da26cf9cec4d0825040f88b5ca659a2f0" +dependencies = [ + "cc", + "codespan-reporting", + "once_cell", + "proc-macro2", + "quote", + "scratch", + "syn", +] + +[[package]] +name = "cxxbridge-flags" +version = "1.0.82" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "820a9a2af1669deeef27cb271f476ffd196a2c4b6731336011e0ba63e2c7cf71" + +[[package]] +name = "cxxbridge-macro" +version = "1.0.82" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a08a6e2fcc370a089ad3b4aaf54db3b1b4cee38ddabce5896b33eb693275f470" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "darling" version = "0.13.4" @@ -2157,12 +2204,12 @@ dependencies = [ [[package]] name = "darling" -version = "0.14.1" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4529658bdda7fd6769b8614be250cdcfc3aeb0ee72fe66f9e41e5e5eb73eac02" +checksum = "b0dd3cd20dc6b5a876612a6e5accfe7f3dd883db6d07acfbf14c128f61550dfa" dependencies = [ - "darling_core 0.14.1", - "darling_macro 0.14.1", + "darling_core 0.14.2", + "darling_macro 0.14.2", ] [[package]] @@ -2181,9 +2228,9 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.14.1" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "649c91bc01e8b1eac09fb91e8dbc7d517684ca6be8ebc75bb9cafc894f9fdb6f" +checksum = "a784d2ccaf7c98501746bf0be29b2022ba41fd62a2e622af997a03e9f972859f" dependencies = [ "fnv", "ident_case", @@ -2206,24 +2253,26 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.14.1" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddfc69c5bfcbd2fc09a0f38451d2daf0e372e367986a83906d1b0dbc88134fb5" +checksum = "7618812407e9402654622dd402b0a89dff9ba93badd6540781526117b92aab7e" dependencies = [ - "darling_core 0.14.1", + "darling_core 0.14.2", "quote", "syn", ] [[package]] name = "dashmap" -version = "5.2.0" +version = "5.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c8858831f7781322e539ea39e72449c46b059638250c14344fec8d0aa6e539c" +checksum = "907076dfda823b0b36d2a1bb5f90c96660a5bbcd7729e10727f07858f22c4edc" dependencies = [ "cfg-if", - "num_cpus", - "parking_lot", + "hashbrown", + "lock_api", + "once_cell", + "parking_lot_core", ] [[package]] @@ -2234,12 +2283,9 @@ checksum = "3ee2393c4a91429dffb4bedf19f4d6abf27d8a732c8ce4980305d782e5426d57" [[package]] name = "data-url" -version = "0.1.1" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a30bfce702bcfa94e906ef82421f2c0e61c076ad76030c16ee5d2e9a32fe193" -dependencies = [ - "matches", -] +checksum = "8d7439c3735f405729d52c3fbbe4de140eaf938a1fe47d227c27f8254d4302a5" [[package]] name = "datadog-filter" @@ -2270,7 +2316,7 @@ dependencies = [ "serde_json", "thiserror", "tracing 0.1.37", - "tracing-test 0.2.1", + "tracing-test 0.2.3", "value", "vector-common", "vrl-compiler", @@ -2280,7 +2326,7 @@ dependencies = [ name = "datadog-search-syntax" version = "0.1.0" dependencies = [ - "itertools 0.10.5", + "itertools", "once_cell", "pest", "pest_derive", @@ -2295,9 +2341,9 @@ checksum = "b72465f46d518f6015d9cf07f7f3013a95dd6b9c2747c3d65ae0cce43929d14f" [[package]] name = "deadpool" -version = "0.9.4" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a90d58a15f5acfe41afcac9775d8e92f2338d14482220c778c6e42aa77778182" +checksum = "421fe0f90f2ab22016f32a9881be5134fdd71c65298917084b0c7477cbc3856e" dependencies = [ "async-trait", "deadpool-runtime", @@ -2340,9 +2386,9 @@ dependencies = [ [[package]] name = "derive_arbitrary" -version = "1.1.6" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "226ad66541d865d7a7173ad6a9e691c33fdb910ac723f4bc734b3e5294a1f931" +checksum = "4903dff04948f22033ca30232ab8eca2c3fc4c913a8b6a34ee5199699814817f" dependencies = [ "proc-macro2", "quote", @@ -2364,9 +2410,9 @@ dependencies = [ [[package]] name = "diff" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e25ea47919b1560c4e3b7fe0aaab9becf5b84a10325ddf7db0f0ba5e1026499" +checksum = "56254986775e3233ffa9c4d7d3faaf6d36a2c09d30b20687e9f88bc8bafc16c8" [[package]] name = "difflib" @@ -2385,11 +2431,11 @@ dependencies = [ [[package]] name = "digest" -version = "0.10.5" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adfbc57365a37acbd2ebf2b64d7e69bb766e2fea813521ed536f5d0520dcf86c" +checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" dependencies = [ - "block-buffer 0.10.2", + "block-buffer 0.10.3", "crypto-common", "subtle", ] @@ -2474,9 +2520,9 @@ checksum = "4f94fa09c2aeea5b8839e414b7b841bf429fd25b9c522116ac97ee87856d88b2" [[package]] name = "ed25519" -version = "1.4.1" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d5c4b5e5959dc2c2b89918d8e2cc40fcdd623cef026ed09d2f0ee05199dc8e4" +checksum = "1e9c280362032ea4203659fc489832d0204ef09f247a0506f170dafcac08c369" dependencies = [ "signature", ] @@ -2495,9 +2541,9 @@ dependencies = [ [[package]] name = "either" -version = "1.6.1" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e78d4f1cc4ae33bbfc157ed5d5a5ef3bc29227303d595861deb238fcec4e9457" +checksum = "90e5c1c8368803113bf0c9584fc495a58b86dc8a29edbf8fe877d21d9507e797" [[package]] name = "ena" @@ -2635,9 +2681,9 @@ dependencies = [ [[package]] name = "erased-serde" -version = "0.3.20" +version = "0.3.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad132dd8d0d0b546348d7d86cb3191aad14b34e5f979781fc005c80d4ac67ffd" +checksum = "54558e0ba96fbe24280072642eceb9d7d442e32c7ec0ea9e7ecd7b4ea2cf4e11" dependencies = [ "serde", ] @@ -2698,9 +2744,9 @@ dependencies = [ [[package]] name = "event-listener" -version = "2.5.2" +version = "2.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77f3309417938f28bf8228fcff79a4a37103981e3e186d2ccd19c74b38f4eb71" +checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" [[package]] name = "executor-trait" @@ -2745,22 +2791,22 @@ checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" [[package]] name = "fastrand" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3fcf0cee53519c866c09b5de1f6c56ff9d647101f81c1964fa632e148896cdf" +checksum = "a7a407cfaa3385c4ae6b23e84623d48c2798d06e3e6a1878f7f59f17b3f86499" dependencies = [ "instant", ] [[package]] name = "fd-lock" -version = "3.0.5" +version = "3.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46e245f4c8ec30c6415c56cb132c07e69e74f1942f6b4a4061da748b49f486ca" +checksum = "bb21c69b9fea5e15dbc1049e4b77145dd0ba1c84019c488102de0dc4ea4b0a27" dependencies = [ "cfg-if", - "rustix 0.34.4", - "windows-sys 0.30.0", + "rustix 0.36.2", + "windows-sys 0.42.0", ] [[package]] @@ -2797,14 +2843,14 @@ dependencies = [ [[package]] name = "filetime" -version = "0.2.16" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0408e2626025178a6a7f7ffc05a25bc47103229f19c113755de7bf63816290c" +checksum = "4b9663d381d07ae25dc88dbdf27df458faa83a9b25336bcac83d5e452b5fc9d3" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.2.13", - "winapi", + "redox_syscall 0.2.16", + "windows-sys 0.42.0", ] [[package]] @@ -2829,9 +2875,9 @@ dependencies = [ [[package]] name = "fixedbitset" -version = "0.4.1" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "279fb028e20b3c4c320317955b77c5e0c9701f05a1d309905d6fc702cdc5053e" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flate2" @@ -2858,7 +2904,7 @@ dependencies = [ "futures-core", "futures-sink", "pin-project", - "spin 0.9.3", + "spin 0.9.4", ] [[package]] @@ -3036,9 +3082,9 @@ dependencies = [ [[package]] name = "generic-array" -version = "0.14.5" +version = "0.14.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd48d33ec7f05fbfa152300fdad764757cbded343c1aa1cff2fbaf4134851803" +checksum = "bff49e947297f3312447abdca79f45f4738097cc82b06e72054d2223f601f1b9" dependencies = [ "typenum", "version_check", @@ -3070,9 +3116,9 @@ dependencies = [ [[package]] name = "ghost" -version = "0.1.4" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76c813ffb63e8fd3df6f1ac3cc1ea392c7612ac2de4d0b44dcbfe03e5c4bf94a" +checksum = "eb19fe8de3ea0920d282f7b77dd4227aea6b8b999b42cdf0ca41b2472b14443a" dependencies = [ "proc-macro2", "quote", @@ -3440,7 +3486,7 @@ version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e" dependencies = [ - "digest 0.10.5", + "digest 0.10.6", ] [[package]] @@ -3462,7 +3508,7 @@ checksum = "75f43d41e26995c17e71ee126451dd3941010b0514a81a9d11f3b341debc2399" dependencies = [ "bytes 1.2.1", "fnv", - "itoa 1.0.1", + "itoa 1.0.4", ] [[package]] @@ -3536,7 +3582,7 @@ dependencies = [ "http-body", "httparse", "httpdate", - "itoa 1.0.1", + "itoa 1.0.4", "pin-project-lite", "socket2", "tokio", @@ -3582,14 +3628,14 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.0" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d87c48c02e0dc5e3b849a2041db3029fd066650f8f717c07bf8ed78ccb895cac" +checksum = "59df7c4e19c950e6e0e868dcc0a300b09a9b88e9ec55bd879ca819087a77355d" dependencies = [ "http", "hyper", "log", - "rustls 0.20.4", + "rustls 0.20.7", "rustls-native-certs 0.6.2", "tokio", "tokio-rustls", @@ -3635,17 +3681,28 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.50" +version = "0.1.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd911b35d940d2bd0bea0f9100068e5b97b51a1cbe13d13382f132e0365257a0" +checksum = "64c122667b287044802d6ce17ee2ddf13207ed924c712de9a66a5814d5b64765" dependencies = [ "android_system_properties", "core-foundation-sys", + "iana-time-zone-haiku", "js-sys", "wasm-bindgen", "winapi", ] +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" +dependencies = [ + "cxx", + "cxx-build", +] + [[package]] name = "ident_case" version = "1.0.1" @@ -3675,9 +3732,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "1.9.1" +version = "1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10a35a97730320ffe8e2d410b5d3b69279b98d2c14bdb8b70ea89ecf7888d41e" +checksum = "1885e79c1fc4b10f0e172c475f458b7f7b93061064d98c3293e98c5ba0c8b399" dependencies = [ "autocfg", "hashbrown", @@ -3764,15 +3821,19 @@ dependencies = [ [[package]] name = "io-lifetimes" -version = "0.6.1" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9448015e586b611e5d322f6703812bbca2f1e709d5773ecd38ddb4e3bb649504" +checksum = "59ce5ef949d49ee85593fc4d3f3f95ad61657076395cbbce23e2121fc5542074" [[package]] name = "io-lifetimes" -version = "0.7.3" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ea37f355c05dde75b84bba2d767906ad522e97cd9e2eef2be7a4ab7fb442c06" +checksum = "a7d367024b3f3414d8e01f437f704f41a9f64ab36f9067fa73e526ad4c763c87" +dependencies = [ + "libc", + "windows-sys 0.42.0", +] [[package]] name = "iovec" @@ -3785,21 +3846,21 @@ dependencies = [ [[package]] name = "ipconfig" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "723519edce41262b05d4143ceb95050e4c614f483e78e9fd9e39a8275a84ad98" +checksum = "bd302af1b90f2463a98fa5ad469fc212c8e3175a41c3068601bfa2727591c5be" dependencies = [ "socket2", "widestring 0.5.1", "winapi", - "winreg 0.7.0", + "winreg", ] [[package]] name = "ipnet" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879d54834c8c76457ef4293a689b2a8c59b076067ad77b15efafbb05f92a592b" +checksum = "f88c5561171189e69df9d98bcf18fd5f9558300f7ea7b801eb8a0fd748bd8745" [[package]] name = "ipnetwork" @@ -3810,15 +3871,6 @@ dependencies = [ "serde", ] -[[package]] -name = "itertools" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "284f18f85651fe11e8a991b2adb42cb078325c996ed026d994719efcfca1d54b" -dependencies = [ - "either", -] - [[package]] name = "itertools" version = "0.10.5" @@ -3836,9 +3888,9 @@ checksum = "b71991ff56294aa922b450139ee08b3bfc70982c6b2c7562771375cf73542dd4" [[package]] name = "itoa" -version = "1.0.1" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1aab8fc367588b89dcee83ab0fd66b72b50b72fa1904d7095045ace2b0c81c35" +checksum = "4217ad341ebadf8d8e724e264f13e593e0648f5b3e94b3896a5df283be015ecc" [[package]] name = "jni" @@ -3847,7 +3899,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "039022cdf4d7b1cf548d31f60ae783138e5fd42013f6271049d7df7afadef96c" dependencies = [ "cesu8", - "combine 4.6.4", + "combine 4.6.6", "jni-sys", "log", "thiserror", @@ -3862,18 +3914,18 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.24" +version = "0.1.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af25a77299a7f711a01975c35a6a424eb6862092cc2d6c72c4ed6cbc56dfc1fa" +checksum = "068b1ee6743e4d11fb9c6a1e6064b3693a1b600e7f5f5988047d98b3dc9fb90b" dependencies = [ "libc", ] [[package]] name = "js-sys" -version = "0.3.57" +version = "0.3.60" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "671a26f820db17c2a2750743f1dd03bafd15b98c9f30c7c2628c024c05d73397" +checksum = "49409df3e3bf0856b916e2ceaca09ee28e6871cf7d9ce97a692cacfdb2a25a47" dependencies = [ "wasm-bindgen", ] @@ -3935,7 +3987,7 @@ dependencies = [ "http", "percent-encoding", "serde", - "serde-value 0.7.0", + "serde-value", "serde_json", "url", ] @@ -3953,15 +4005,18 @@ dependencies = [ [[package]] name = "keccak" -version = "0.1.0" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67c21572b4949434e4fc1e1978b99c5f77064153c59d998bf13ecd96fb5ecba7" +checksum = "3afef3b6eff9ce9d8ff9b3601125eec7f0c8cbac7abd14f355d053fa56c98768" +dependencies = [ + "cpufeatures", +] [[package]] name = "kqueue" -version = "1.0.6" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d6112e8f37b59803ac47a42d14f1f3a59bbf72fc6857ffc5be455e28a691f8e" +checksum = "2c8fc60ba15bf51257aa9807a48a61013db043fcf3a78cb0d916e8e396dcad98" dependencies = [ "kqueue-sys", "libc", @@ -4057,7 +4112,7 @@ version = "0.75.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7769af142ee2e46bfa44bd393cf7f40b9d8b80d2e11f6317399551ed17760beb" dependencies = [ - "ahash 0.8.0", + "ahash 0.8.2", "backoff", "derivative", "futures 0.3.25", @@ -4086,7 +4141,7 @@ dependencies = [ "bit-set", "diff", "ena", - "itertools 0.10.5", + "itertools", "lalrpop-util", "petgraph", "pico-args", @@ -4185,15 +4240,15 @@ dependencies = [ [[package]] name = "libm" -version = "0.2.2" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33a33a362ce288760ec6a508b94caaec573ae7d3bbbd91b87aa0bad4456839db" +checksum = "348108ab3fba42ec82ff6e9564fc4ca0247bdccdc68dd8af9764bbc79c3c8ffb" [[package]] name = "libz-sys" -version = "1.1.6" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92e7e15d7610cce1d9752e137625f14e61a28cd45929b6e12e47b50fe154ee2e" +checksum = "9702761c3935f8cc2f101793272e202c72b99da8f4224a19ddcf1279a6450bbf" dependencies = [ "cc", "libc", @@ -4201,11 +4256,20 @@ dependencies = [ "vcpkg", ] +[[package]] +name = "link-cplusplus" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9272ab7b96c9046fbc5bc56c06c117cb639fe2d509df0c421cad82d2915cf369" +dependencies = [ + "cc", +] + [[package]] name = "linked-hash-map" -version = "0.5.4" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fb9b38af92608140b86b693604b9ffcc5824240a484d1ecd4795bacb2fe88f3" +checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" [[package]] name = "linked_hash_set" @@ -4218,15 +4282,15 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.0.42" +version = "0.0.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5284f00d480e1c39af34e72f8ad60b94f47007e3481cd3b731c1d67190ddc7b7" +checksum = "d4d2456c373231a208ad294c33dc5bff30051eafd954cd4caae83a712b12854d" [[package]] name = "linux-raw-sys" -version = "0.0.46" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4d2456c373231a208ad294c33dc5bff30051eafd954cd4caae83a712b12854d" +checksum = "bb68f22743a3fb35785f1e7f844ca5a3de2dde5bd0c0ef5b372065814699b121" [[package]] name = "listenfd" @@ -4241,9 +4305,9 @@ dependencies = [ [[package]] name = "lock_api" -version = "0.4.7" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327fa5b6a6940e4699ec49a9beae1ea4845c6bab9314e4f84ac68742139d8c53" +checksum = "435011366fe56583b16cf956f9df0095b405b82d76425bc8981c0e22e60ec4df" dependencies = [ "autocfg", "scopeguard", @@ -4327,9 +4391,9 @@ dependencies = [ [[package]] name = "luajit-src" -version = "210.4.0+resty124ff8d" +version = "210.4.3+resty8384278" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f76fb2e2c0c7192e18719d321c9a148f7625c4dcbe3df5f4c19e685e4c286f6c" +checksum = "19ee5d5afddf1ec76ffa55ca7c3001f2f8a703834beba53c56a38ea6641cef44" dependencies = [ "cc", ] @@ -4421,7 +4485,7 @@ version = "0.10.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6365506850d44bff6e2fbcb5176cf63650e48bd45ef2fe2665ae1570e0f4b9ca" dependencies = [ - "digest 0.10.5", + "digest 0.10.6", ] [[package]] @@ -4454,6 +4518,15 @@ dependencies = [ "autocfg", ] +[[package]] +name = "memoffset" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5de893c32cde5f383baa4c04c5d6dbdd735cfd4a794b0debdb2bb1b421da5ff4" +dependencies = [ + "autocfg", +] + [[package]] name = "metrics" version = "0.20.1" @@ -4482,7 +4555,7 @@ version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6097e2772147f332c9aedba572e9cd334b7946e1762d8ae4d05db0faf962542a" dependencies = [ - "itoa 1.0.1", + "itoa 1.0.4", "lockfree-object-pool", "metrics", "metrics-util", @@ -4537,23 +4610,23 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.5.1" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b29bd4bc3f33391105ebee3589c19197c4271e3e5a9ec9bfe8127eeff8f082" +checksum = "96590ba8f175222643a85693f33d26e9c8a015f599c216509b1a6894af675d34" dependencies = [ "adler", ] [[package]] name = "mio" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57ee1c23c7c63b0c9250c339ffdc69255f110b298b901b9f6c82547b7b87caaf" +checksum = "e5d732bc30207a6423068df043e3d02e0735b155ad7ce1a6f76fe2baa5b158de" dependencies = [ "libc", "log", "wasi 0.11.0+wasi-snapshot-preview1", - "windows-sys 0.36.1", + "windows-sys 0.42.0", ] [[package]] @@ -4602,7 +4675,7 @@ dependencies = [ "percent-encoding", "rand 0.8.5", "rustc_version_runtime", - "rustls 0.20.4", + "rustls 0.20.7", "rustls-pemfile 0.3.0", "serde", "serde_bytes", @@ -4626,9 +4699,9 @@ dependencies = [ [[package]] name = "multer" -version = "2.0.2" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f8f35e687561d5c1667590911e6698a8cb714a134a7505718a182e7bc9d3836" +checksum = "6ed4198ce7a4cbd2a57af78d28c6fbb57d81ac5f1d6ad79ac6c5587419cbdf22" dependencies = [ "bytes 1.2.1", "encoding_rs", @@ -4638,7 +4711,7 @@ dependencies = [ "log", "memchr", "mime", - "spin 0.9.3", + "spin 0.9.4", "version_check", ] @@ -4650,9 +4723,9 @@ checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" [[package]] name = "native-tls" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd7e2f3618557f980e0b17e8856252eee3c97fa12c54dff0ca290fb6266ca4a9" +checksum = "07226173c32f2926027b63cce4bcd8076c3552846cbe7925f3aaffeac0a3b92e" dependencies = [ "lazy_static", "libc", @@ -4677,7 +4750,7 @@ dependencies = [ "blocking", "crossbeam-channel", "fastrand", - "itoa 1.0.1", + "itoa 1.0.4", "json", "lazy_static", "libc", @@ -4722,7 +4795,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af5a8477ac96877b5bd1fd67e0c28736c12943aba24eda92b127e036b0c8f400" dependencies = [ "indexmap", - "itertools 0.10.5", + "itertools", "ndarray", "noisy_float", "num-integer", @@ -4761,7 +4834,7 @@ dependencies = [ "cc", "cfg-if", "libc", - "memoffset", + "memoffset 0.6.5", ] [[package]] @@ -4785,7 +4858,7 @@ dependencies = [ "bitflags", "cfg-if", "libc", - "memoffset", + "memoffset 0.6.5", ] [[package]] @@ -4819,12 +4892,6 @@ version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b93853da6d84c2e3c7d730d6473e8817692dd89be387eb01b94d7f108ecb5b8c" -[[package]] -name = "nodrop" -version = "0.1.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72ef4a56884ca558e5ddb05a1d1e7e1bfd9a68d9ed024c21704cc98872dae1bb" - [[package]] name = "noisy_float" version = "0.2.0" @@ -4920,29 +4987,29 @@ dependencies = [ [[package]] name = "num-complex" -version = "0.4.0" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26873667bbbb7c5182d4a37c1add32cdf09f841af72da53318fdb81543c15085" +checksum = "7ae39348c8bc5fbd7f40c727a9925f03517afd2ab27d46702108b6a7e5414c19" dependencies = [ "num-traits", ] [[package]] name = "num-format" -version = "0.4.0" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bafe4179722c2894288ee77a9f044f02811c86af699344c498b0840c698a2465" +checksum = "54b862ff8df690cf089058c98b183676a7ed0f974cc08b426800093227cbff3b" dependencies = [ - "arrayvec 0.4.12", - "itoa 0.4.8", - "num-bigint 0.2.6", + "arrayvec 0.7.2", + "itoa 1.0.4", + "num-bigint 0.4.3", ] [[package]] name = "num-integer" -version = "0.1.44" +version = "0.1.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2cc698a63b549a70bc047073d2949cce27cd1c7b0a4a862d08a8031bc2801db" +checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" dependencies = [ "autocfg", "num-traits", @@ -4971,9 +5038,9 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.13.1" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19e64526ebdee182341572e50e9ad03965aa510cd94427a4549448f285e957a1" +checksum = "f6058e64324c71e02bc2b150e4f3bc8286db6c83092132ffa3f6b1eab0f9def5" dependencies = [ "hermit-abi", "libc", @@ -5002,9 +5069,9 @@ dependencies = [ [[package]] name = "num_threads" -version = "0.1.5" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aba1801fb138d8e85e11d0fc70baf4fe1cdfffda7c6cd34a854905df588e5ed0" +checksum = "2819ce041d2ee131036f4fc9d6ae7ae125a3a40e97ba64d04fe799ad9dabbb44" dependencies = [ "libc", ] @@ -5017,9 +5084,9 @@ checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" [[package]] name = "oauth2" -version = "4.1.0" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80e47cfc4c0a1a519d9a025ebfbac3a2439d1b5cdf397d72dcb79b11d9920dab" +checksum = "eeaf26a72311c087f8c5ba617c96fac67a5c04f430e716ac8d8ab2de62e23368" dependencies = [ "base64", "chrono", @@ -5030,7 +5097,7 @@ dependencies = [ "serde", "serde_json", "serde_path_to_error", - "sha2 0.9.9", + "sha2 0.10.6", "thiserror", "url", ] @@ -5095,26 +5162,25 @@ checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" [[package]] name = "openidconnect" -version = "2.3.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "691c1ba89b0a112f3062b946ef160711c3aea33e1476b6877a904f2f83856781" +checksum = "87af7097640fedbe64718ac1c9b0549d72da747a3f527cd089215f96c6f691d5" dependencies = [ "base64", "chrono", "http", - "itertools 0.9.0", + "itertools", "log", "num-bigint 0.4.3", "oauth2", "rand 0.8.5", "ring", "serde", - "serde-value 0.6.0", + "serde-value", "serde_derive", "serde_json", "serde_path_to_error", "thiserror", - "untrusted", "url", ] @@ -5160,9 +5226,9 @@ dependencies = [ [[package]] name = "openssl-sys" -version = "0.9.76" +version = "0.9.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5230151e44c0f05157effb743e8d517472843121cf9243e8b81393edb5acd9ce" +checksum = "b03b84c3b2d099b81f0953422b4d4ad58761589d0229b5506356afca05a3670a" dependencies = [ "autocfg", "cc", @@ -5180,23 +5246,14 @@ dependencies = [ "chrono", "hex", "ordered-float 3.4.0", - "prost 0.11.0", - "prost-build 0.11.1", + "prost 0.11.2", + "prost-build 0.11.2", "tonic", "tonic-build", "value", "vector-core", ] -[[package]] -name = "ordered-float" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" -dependencies = [ - "num-traits", -] - [[package]] name = "ordered-float" version = "2.10.0" @@ -5217,9 +5274,9 @@ dependencies = [ [[package]] name = "os_info" -version = "3.2.0" +version = "3.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "023df84d545ef479cf67fd2f4459a613585c9db4852c2fad12ab70587859d340" +checksum = "c4750134fb6a5d49afc80777394ad5d95b04bc12068c6abb92fae8f43817270f" dependencies = [ "log", "winapi", @@ -5237,9 +5294,9 @@ dependencies = [ [[package]] name = "os_str_bytes" -version = "6.0.0" +version = "6.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e22443d1643a904602595ba1cd8f7d896afe56d26712531c5ff73a15b2fbf64" +checksum = "7b5bf27447411e9ee3ff51186bf7a08e16c341efdde93f4d823e8844429bed7e" [[package]] name = "overload" @@ -5265,15 +5322,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.2" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "995f667a6c822200b0433ac218e05582f0e2efa1b922a3fd2fbaadc5f87bab37" +checksum = "4dc9e0dc2adc1c69d09143aff38d3d30c5c3f0df0dad82e6d25547af174ebec0" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.2.13", + "redox_syscall 0.2.16", "smallvec", - "windows-sys 0.34.0", + "windows-sys 0.42.0", ] [[package]] @@ -5297,7 +5354,7 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271779f35b581956db91a3e55737327a03aa051e90b1c47aeb189508533adfd7" dependencies = [ - "digest 0.10.5", + "digest 0.10.6", ] [[package]] @@ -5308,9 +5365,9 @@ checksum = "9e9ed2178b0575fff8e1b83b58ba6f75e727aafac2e1b6c795169ad3b17eb518" [[package]] name = "pem" -version = "1.0.2" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9a3b09a20e374558580a4914d3b7d89bd61b954a5a5e1dcbea98753addb1947" +checksum = "03c64931a1a212348ec4f3b4362585eca7159d0d09cbdf4a7f74f02173596fd4" dependencies = [ "base64", ] @@ -5376,9 +5433,9 @@ dependencies = [ [[package]] name = "petgraph" -version = "0.6.0" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a13a2fa9d0b63e5f22328828741e523766fff0ee9e779316902290dff3f824f" +checksum = "e6d5014253a1331579ce62aa67443b4a658c5e7dd03d4bc6d302b94474888143" dependencies = [ "fixedbitset", "indexmap", @@ -5386,30 +5443,30 @@ dependencies = [ [[package]] name = "phf" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4724fa946c8d1e7cd881bd3dbee63ce32fc1e9e191e35786b3dc1320a3f68131" +checksum = "928c6535de93548188ef63bb7c4036bd415cd8f36ad25af44b9789b2ee72a48c" dependencies = [ - "phf_shared 0.11.0", + "phf_shared 0.11.1", ] [[package]] name = "phf_codegen" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32ba0c43d7a1b6492b2924a62290cfd83987828af037b0743b38e6ab092aee58" +checksum = "a56ac890c5e3ca598bbdeaa99964edb5b0258a583a9eb6ef4e89fc85d9224770" dependencies = [ "phf_generator", - "phf_shared 0.11.0", + "phf_shared 0.11.1", ] [[package]] name = "phf_generator" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b450720b6f75cfbfabc195814bd3765f337a4f9a83186f8537297cac12f6705" +checksum = "b1181c94580fa345f50f19d738aaa39c0ed30a600d95cb2d3e23f94266f14fbf" dependencies = [ - "phf_shared 0.11.0", + "phf_shared 0.11.1", "rand 0.8.5", ] @@ -5424,9 +5481,9 @@ dependencies = [ [[package]] name = "phf_shared" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9dd5609d4b2df87167f908a32e1b146ce309c16cf35df76bc11f440b756048e4" +checksum = "e1fb5f6f826b772a8d4c0394209441e7d37cbbb967ae9c7e0e8134365c9ee676" dependencies = [ "siphasher", ] @@ -5495,9 +5552,9 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.25" +version = "0.3.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1df8c4ec4b0627e53bdf214615ad287367e482558cf84b109250b37464dc03ae" +checksum = "6ac9a59f73473f1b8d852421e59e64809f025994837ef743615c6d0c5b305160" [[package]] name = "platforms" @@ -5507,9 +5564,9 @@ checksum = "989d43012e2ca1c4a02507c67282691a0a3207f9dc67cec596b43fe925b3d325" [[package]] name = "plotters" -version = "0.3.1" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a3fd9ec30b9749ce28cd91f255d569591cdf937fe280c312143e3c4bad6f2a" +checksum = "2538b639e642295546c50fcd545198c9d64ee2a38620a628724a3b266d5fbf97" dependencies = [ "num-traits", "plotters-backend", @@ -5520,25 +5577,26 @@ dependencies = [ [[package]] name = "plotters-backend" -version = "0.3.2" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d88417318da0eaf0fdcdb51a0ee6c3bed624333bff8f946733049380be67ac1c" +checksum = "193228616381fecdc1224c62e96946dfbc73ff4384fba576e052ff8c1bea8142" [[package]] name = "plotters-svg" -version = "0.3.1" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "521fa9638fa597e1dc53e9412a4f9cefb01187ee1f7413076f9e6749e2885ba9" +checksum = "f9a81d2759aae1dae668f783c308bc5c8ebd191ff4184aaa1b37f65a6ae5a56f" dependencies = [ "plotters-backend", ] [[package]] name = "polling" -version = "2.2.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "685404d509889fade3e86fe3a5803bca2ec09b0c0778d5ada6ec8bf7a8de5259" +checksum = "ab4609a838d88b73d8238967b60dd115cc08d38e2bbaf51ee1e4b695f89122e2" dependencies = [ + "autocfg", "cfg-if", "libc", "log", @@ -5548,9 +5606,9 @@ dependencies = [ [[package]] name = "portable-atomic" -version = "0.3.4" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "763095e04dcbeb889b2ab35296ecb18a20fe16b4e9877ce64aab73d8fd05a8c3" +checksum = "15eb2c6e362923af47e13c23ca5afb859e83d54452c55b0b9ac763b8f7c1ac16" [[package]] name = "portpicker" @@ -5604,9 +5662,9 @@ dependencies = [ [[package]] name = "ppv-lite86" -version = "0.2.16" +version = "0.2.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb9f9e6e233e5c4a35559a617bf40a4ec447db2e84c20b55a6f83167b7e57872" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "precomputed-hash" @@ -5616,26 +5674,26 @@ checksum = "925383efa346730478fb4838dbe9137d2a47675ad789c546d150a6e1dd4ab31c" [[package]] name = "predicates" -version = "2.1.1" +version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5aab5be6e4732b473071984b3164dbbfb7a3674d30ea5ff44410b6bcd960c3c" +checksum = "ed6bd09a7f7e68f3f0bf710fb7ab9c4615a488b58b5f653382a687701e458c92" dependencies = [ "difflib", - "itertools 0.10.5", + "itertools", "predicates-core", ] [[package]] name = "predicates-core" -version = "1.0.3" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da1c2388b1513e1b605fcec39a95e0a9e8ef088f71443ef37099fa9ae6673fcb" +checksum = "72f883590242d3c6fc5bf50299011695fa6590c2c70eac95ee1bdb9a733ad1a2" [[package]] name = "predicates-tree" -version = "1.0.5" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d86de6de25020a36c6d3643a86d9a6a9f552107c0559c60ea03551b5e16c032" +checksum = "54ff541861505aabf6ea722d2131ee980b8276e10a1297b94e896dd8b621850d" dependencies = [ "predicates-core", "termtree", @@ -5654,9 +5712,9 @@ dependencies = [ [[package]] name = "prettyplease" -version = "0.1.10" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9e07e3a46d0771a8a06b5f4441527802830b43e679ba12f44960f48dd4c6803" +checksum = "c142c0e46b57171fe0c528bee8c5b7569e80f0c17e377cd0e30ea57dbc11bb51" dependencies = [ "proc-macro2", "syn", @@ -5691,10 +5749,11 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "1.1.3" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e17d47ce914bf4de440332250b0edd23ce48c005f59fab39d3335866b114f11a" +checksum = "eda0fc3b0fb7c975631757e14d9049da17374063edb6ebbcbc54d880d4fe94e9" dependencies = [ + "once_cell", "thiserror", "toml", ] @@ -5745,9 +5804,9 @@ dependencies = [ "indexmap", "nom", "num_enum", - "prost 0.11.0", - "prost-build 0.11.1", - "prost-types 0.11.1", + "prost 0.11.2", + "prost-build 0.11.2", + "prost-types 0.11.2", "snafu", "value", "vector-common", @@ -5797,12 +5856,12 @@ dependencies = [ [[package]] name = "prost" -version = "0.11.0" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c3c31cdec40583bb68f0b18403400d01ec4289c383aa047560439952c4dd7" +checksum = "a0841812012b2d4a6145fae9a6af1534873c32aa67fff26bd09f8fa42c83f95a" dependencies = [ "bytes 1.2.1", - "prost-derive 0.11.0", + "prost-derive 0.11.2", ] [[package]] @@ -5815,7 +5874,7 @@ dependencies = [ "cfg-if", "cmake", "heck 0.4.0", - "itertools 0.10.5", + "itertools", "lazy_static", "log", "multimap", @@ -5829,20 +5888,22 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f835c582e6bd972ba8347313300219fed5bfa52caf175298d860b61ff6069bb" +checksum = "1d8b442418ea0822409d9e7d047cbf1e7e9e1760b172bf9982cf29d517c93511" dependencies = [ "bytes 1.2.1", "heck 0.4.0", - "itertools 0.10.5", + "itertools", "lazy_static", "log", "multimap", "petgraph", - "prost 0.11.0", - "prost-types 0.11.1", + "prettyplease", + "prost 0.11.2", + "prost-types 0.11.2", "regex", + "syn", "tempfile", "which", ] @@ -5854,7 +5915,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b670f45da57fb8542ebdbb6105a925fe571b67f9e7ed9f47a06a84e72b4e7cc" dependencies = [ "anyhow", - "itertools 0.10.5", + "itertools", "proc-macro2", "quote", "syn", @@ -5862,12 +5923,12 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.11.0" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7345d5f0e08c0536d7ac7229952590239e77abf0a0100a1b1d890add6ea96364" +checksum = "164ae68b6587001ca506d3bf7f1000bfa248d0e1217b618108fba4ec1d0cc306" dependencies = [ "anyhow", - "itertools 0.10.5", + "itertools", "proc-macro2", "quote", "syn", @@ -5885,12 +5946,12 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dfaa718ad76a44b3415e6c4d53b17c8f99160dcb3a99b10470fce8ad43f6e3e" +checksum = "747761bc3dc48f9a34553bf65605cf6cb6288ba219f3450b4275dbd81539551a" dependencies = [ "bytes 1.2.1", - "prost 0.11.0", + "prost 0.11.2", ] [[package]] @@ -5915,9 +5976,9 @@ dependencies = [ [[package]] name = "pulsar" -version = "4.1.3" +version = "4.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b55e5d7c8ffe92d9d788624ade45200d19ffd371bbc6dde99b57f8843aaaf92" +checksum = "93e9a7c4f8fb18df618103e9a33f93da4c672928d15500fc521f145c18af4a48" dependencies = [ "async-trait", "bit-vec 0.6.3", @@ -5934,9 +5995,9 @@ dependencies = [ "oauth2", "openidconnect", "pem", - "prost 0.10.4", - "prost-build 0.10.4", - "prost-derive 0.10.1", + "prost 0.11.2", + "prost-build 0.11.2", + "prost-derive 0.11.2", "rand 0.8.5", "regex", "serde", @@ -6044,7 +6105,7 @@ checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" dependencies = [ "libc", "rand_chacha 0.3.1", - "rand_core 0.6.3", + "rand_core 0.6.4", ] [[package]] @@ -6064,7 +6125,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" dependencies = [ "ppv-lite86", - "rand_core 0.6.3", + "rand_core 0.6.4", ] [[package]] @@ -6078,9 +6139,9 @@ dependencies = [ [[package]] name = "rand_core" -version = "0.6.3" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d34f1408f55294453790c48b2f1ebbb1c5b4b7563eb1f418bcfcfdbb06ebb4e7" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" dependencies = [ "getrandom 0.2.8", ] @@ -6110,14 +6171,14 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d25bf25ec5ae4a3f1b92f929810509a2f53d7dca2f50b794ff57e3face536c8f" dependencies = [ - "rand_core 0.6.3", + "rand_core 0.6.4", ] [[package]] name = "raw-cpuid" -version = "10.3.0" +version = "10.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "738bc47119e3eeccc7e94c4a506901aea5e7b4944ecd0829cbebf4af04ceda12" +checksum = "a6823ea29436221176fe662da99998ad3b4db2c7f31e7b6f5fe43adccd6320bb" dependencies = [ "bitflags", ] @@ -6139,11 +6200,10 @@ checksum = "60a357793950651c4ed0f3f52338f53b2f809f32d83a07f72909fa13e4c6c1e3" [[package]] name = "rayon" -version = "1.5.2" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd249e82c21598a9a426a4e00dd7adc1d640b22445ec8545feef801d1a74c221" +checksum = "1e060280438193c554f654141c9ea9417886713b7acd75974c85b18a69a88e0b" dependencies = [ - "autocfg", "crossbeam-deque", "either", "rayon-core", @@ -6151,9 +6211,9 @@ dependencies = [ [[package]] name = "rayon-core" -version = "1.9.2" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f51245e1e62e1f1629cbfec37b5793bbabcaeb90f30e94d2ba03564687353e4" +checksum = "cac410af5d00ab6884528b4ab69d1e8e146e8d471201800fa1b4524126de6ad3" dependencies = [ "crossbeam-channel", "crossbeam-deque", @@ -6215,10 +6275,10 @@ dependencies = [ "arc-swap", "async-trait", "bytes 1.2.1", - "combine 4.6.4", + "combine 4.6.6", "futures 0.3.25", "futures-util", - "itoa 1.0.1", + "itoa 1.0.4", "native-tls", "percent-encoding", "pin-project-lite", @@ -6237,9 +6297,9 @@ checksum = "41cc0f7e4d5d4544e8861606a285bb08d3e70712ccc7d2b84d7c0ccfaf4b05ce" [[package]] name = "redox_syscall" -version = "0.2.13" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62f25bc4c7e55e0b0b7a1d43fb893f4fa1361d0abe38b9ce4f323c2adfe6ef42" +checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" dependencies = [ "bitflags", ] @@ -6262,7 +6322,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b033d837a7cf162d7993aded9304e30a83213c648b6e389db233191f891e5c2b" dependencies = [ "getrandom 0.2.8", - "redox_syscall 0.2.13", + "redox_syscall 0.2.16", "thiserror", ] @@ -6288,9 +6348,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.6.27" +version = "0.6.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3f87b73ce11b1619a3c6332f45341e0047173771e8b8b73f87bfeefb7b56244" +checksum = "456c603be3e8d448b072f410900c09faf164fbce2d480456f50eea6e25f9c848" [[package]] name = "remove_dir_all" @@ -6335,8 +6395,8 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.20.4", - "rustls-pemfile 1.0.0", + "rustls 0.20.7", + "rustls-pemfile 1.0.1", "serde", "serde_json", "serde_urlencoded", @@ -6350,7 +6410,7 @@ dependencies = [ "wasm-bindgen-futures", "web-sys", "webpki-roots", - "winreg 0.10.1", + "winreg", ] [[package]] @@ -6528,30 +6588,30 @@ dependencies = [ [[package]] name = "rustix" -version = "0.34.4" +version = "0.35.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f5d1c6ed6d1c6915aa64749b809fc1bafff49d160f5d927463658093d7d62ab" +checksum = "727a1a6d65f786ec22df8a81ca3121107f235970dc1705ed681d3e6e8b9cd5f9" dependencies = [ "bitflags", "errno", - "io-lifetimes 0.6.1", + "io-lifetimes 0.7.5", "libc", - "linux-raw-sys 0.0.42", - "winapi", + "linux-raw-sys 0.0.46", + "windows-sys 0.42.0", ] [[package]] name = "rustix" -version = "0.35.11" +version = "0.36.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbb2fda4666def1433b1b05431ab402e42a1084285477222b72d6c564c417cef" +checksum = "203974af07ea769452490ee8de3e5947971efc3a090dca8a779dd432d3fa46a7" dependencies = [ "bitflags", "errno", - "io-lifetimes 0.7.3", + "io-lifetimes 1.0.1", "libc", - "linux-raw-sys 0.0.46", - "windows-sys 0.36.1", + "linux-raw-sys 0.1.2", + "windows-sys 0.42.0", ] [[package]] @@ -6569,9 +6629,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.4" +version = "0.20.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fbfeb8d0ddb84706bc597a5574ab8912817c52a397f819e5b614e2265206921" +checksum = "539a2bfe908f471bfa933876bd1eb6a19cf2176d375f82ef7f99530a40e48c2c" dependencies = [ "log", "ring", @@ -6598,7 +6658,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0167bac7a9f490495f3c33013e7722b53cb087ecbe082fb0c6387c96f634ea50" dependencies = [ "openssl-probe", - "rustls-pemfile 1.0.0", + "rustls-pemfile 1.0.1", "schannel", "security-framework", ] @@ -6623,18 +6683,18 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7522c9de787ff061458fe9a829dc790a3f5b22dc571694fc5883f448b94d9a9" +checksum = "0864aeff53f8c05aa08d86e5ef839d3dfcf07aeba2db32f12db0ef716e87bd55" dependencies = [ "base64", ] [[package]] name = "rustversion" -version = "1.0.6" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2cc38e8fa666e2de3c4aba7edeb5ffc5246c1c2ed0e3d17e560aeeba736b23f" +checksum = "97477e48b4cf8603ad5f7aaf897467cf42ab4218a38ef76fb14c2d6773a6d6a8" [[package]] name = "rusty-fork" @@ -6686,9 +6746,9 @@ dependencies = [ [[package]] name = "sasl2-sys" -version = "0.1.19+2.1.27" +version = "0.1.20+2.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21d3579e03127aee0792cc0e2d739fe05b1652f396ee92127d15b2748be9adf7" +checksum = "9e645bd98535fc8fd251c43ba7c7c1f9be1e0369c99b6a5ea719052a773e655c" dependencies = [ "cc", "duct", @@ -6743,9 +6803,9 @@ dependencies = [ [[package]] name = "scoped-tls" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea6a9290e3c9cf0f18145ef7ffa62d68ee0bf5fcd651017e586dc7fd5da448c2" +checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" [[package]] name = "scopeguard" @@ -6753,6 +6813,12 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" +[[package]] +name = "scratch" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8132065adcfd6e02db789d9285a0deb2f3fcb04002865ab67d5fb103533898" + [[package]] name = "sct" version = "0.6.1" @@ -6854,16 +6920,6 @@ dependencies = [ "toml", ] -[[package]] -name = "serde-value" -version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a65a7291a8a568adcae4c10a677ebcedbc6c9cec91c054dee2ce40b0e3290eb" -dependencies = [ - "ordered-float 1.1.1", - "serde", -] - [[package]] name = "serde-value" version = "0.7.0" @@ -6930,12 +6986,12 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.87" +version = "1.0.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ce777b7b150d76b9cf60d28b55f5847135a003f7d7350c6be7a773508ce7d45" +checksum = "8e8b3801309262e8184d9687fb697586833e939767aea0dda89f5a8e650e8bd7" dependencies = [ "indexmap", - "itoa 1.0.1", + "itoa 1.0.4", "ryu", "serde", ] @@ -6951,9 +7007,9 @@ dependencies = [ [[package]] name = "serde_path_to_error" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7868ad3b8196a8a0aea99a8220b124278ee5320a55e4fde97794b6f85b1a377" +checksum = "184c643044780f7ceb59104cef98a5a6f12cb2288a7bc701ab93a362b49fd47d" dependencies = [ "serde", ] @@ -6971,9 +7027,9 @@ dependencies = [ [[package]] name = "serde_repr" -version = "0.1.7" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98d0516900518c29efa217c298fa1f4e6c6ffc85ae29fd7f4ee48f176e1a9ed5" +checksum = "1fe39d9fbb0ebf5eb2c7cb7e2a47e4f462fad1379f1166b8ae49ad9eae89a7ca" dependencies = [ "proc-macro2", "quote", @@ -6987,7 +7043,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3491c14715ca2294c4d6a88f15e84739788c1d030eed8c110436aafdaa2f3fd" dependencies = [ "form_urlencoded", - "itoa 1.0.1", + "itoa 1.0.4", "ryu", "serde", ] @@ -7036,7 +7092,7 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3452b4c0f6c1e357f73fdb87cd1efabaa12acf328c7a528e252893baeb3f4aa" dependencies = [ - "darling 0.14.1", + "darling 0.14.2", "proc-macro2", "quote", "syn", @@ -7061,7 +7117,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6d232d893b10de3eb7258ff01974d6ee20663d8e833263c99409d4b13a0209da" dependencies = [ "indexmap", - "itoa 1.0.1", + "itoa 1.0.4", "ryu", "serde", "unsafe-libyaml", @@ -7075,18 +7131,18 @@ checksum = "028f48d513f9678cda28f6e4064755b3fbb2af6acd672f2c209b62323f7aea0f" dependencies = [ "cfg-if", "cpufeatures", - "digest 0.10.5", + "digest 0.10.6", ] [[package]] name = "sha1" -version = "0.10.1" +version = "0.10.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c77f4e7f65455545c2153c1253d25056825e77ee2533f0e41deb65a93a34852f" +checksum = "f04293dc80c3993519f2d7f6f511707ee7094fe0c6d3406feb330cdb3540eba3" dependencies = [ "cfg-if", "cpufeatures", - "digest 0.10.5", + "digest 0.10.6", ] [[package]] @@ -7110,16 +7166,16 @@ checksum = "82e6b795fe2e3b1e845bafcb27aa35405c4d47cdfc92af5fc8d3002f76cebdc0" dependencies = [ "cfg-if", "cpufeatures", - "digest 0.10.5", + "digest 0.10.6", ] [[package]] name = "sha3" -version = "0.10.1" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "881bf8156c87b6301fc5ca6b27f11eeb2761224c7081e69b409d5a1951a70c86" +checksum = "bdf0c33fae925bdc080598b84bc15c55e7b9a4a43b3c704da051f977469691c9" dependencies = [ - "digest 0.10.5", + "digest 0.10.6", "keccak", ] @@ -7144,9 +7200,9 @@ dependencies = [ [[package]] name = "signal-hook" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "647c97df271007dcea485bb74ffdb57f2e683f1306c854f468a0c244badabf2d" +checksum = "a253b5e89e2698464fc26b545c9edceb338e18a89effeeecfea192c3025be29d" dependencies = [ "libc", "signal-hook-registry", @@ -7179,22 +7235,22 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5dfecc059e81632eef1dd9b79e22fc28b8fe69b30d3357512a77a0ad8ee3c782" dependencies = [ "pkcs8", - "rand_core 0.6.3", + "rand_core 0.6.4", "signature", "zeroize", ] [[package]] name = "signature" -version = "1.5.0" +version = "1.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f054c6c1a6e95179d6f23ed974060dcefb2d9388bb7256900badad682c499de4" +checksum = "74233d3b3b2f6d4b006dc19dee745e73e2a6bfb6f93607cd3b02bd5b00797d7c" [[package]] name = "similar" -version = "2.2.0" +version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62ac7f900db32bf3fd12e0117dd3dc4da74bc52ebaac97f39668446d89694803" +checksum = "420acb44afdae038210c99e69aae24109f32f15500aa708e81d46c9f29d55fcf" dependencies = [ "bstr 0.2.17", "unicode-segmentation", @@ -7230,9 +7286,12 @@ checksum = "ceb945e54128e09c43d8e4f1277851bd5044c6fc540bbaa2ad888f60b3da9ae7" [[package]] name = "slab" -version = "0.4.6" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb703cfe953bccee95685111adeedb76fabe4e97549a58d16f03ea7b9367bb32" +checksum = "4614a76b2a8be0058caa9dbbaf66d988527d86d003c11a94fbd335d7661edcef" +dependencies = [ + "autocfg", +] [[package]] name = "smallvec" @@ -7300,9 +7359,9 @@ dependencies = [ [[package]] name = "snap" -version = "1.0.5" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45456094d1983e2ee2a18fdfebce3189fa451699d0502cb8e3b49dba5ba41451" +checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" [[package]] name = "socket2" @@ -7322,9 +7381,9 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "spin" -version = "0.9.3" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c530c2b0d0bf8b69304b39fe2001993e267461948b890cd037d8ad4293fa1a0d" +checksum = "7f6002a767bff9e83f8eeecf883ecb8011875a21ae8da43bffb817a57e78cc09" dependencies = [ "lock_api", ] @@ -7346,9 +7405,9 @@ checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" [[package]] name = "str-buf" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d44a3643b4ff9caf57abcee9c2c621d6c03d9135e0d8b589bd9afb5992cb176a" +checksum = "9e08d8363704e6c71fc928674353e6b7c23dcea9d82d7012c8faf2a3a025f8d0" [[package]] name = "stream-cancel" @@ -7535,7 +7594,7 @@ dependencies = [ "cfg-if", "fastrand", "libc", - "redox_syscall 0.2.13", + "redox_syscall 0.2.16", "remove_dir_all", "winapi", ] @@ -7583,19 +7642,19 @@ dependencies = [ [[package]] name = "terminal_size" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8440c860cf79def6164e4a0a983bcc2305d82419177a0e0c71930d049e3ac5a1" +checksum = "40ca90c434fd12083d1a6bdcbe9f92a14f96c8a1ba600ba451734ac334521f7a" dependencies = [ - "rustix 0.35.11", - "windows-sys 0.36.1", + "rustix 0.35.13", + "windows-sys 0.42.0", ] [[package]] name = "termtree" -version = "0.2.4" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "507e9898683b6c43a9aa55b64259b721b52ba226e0f3779137e50ad114a4c90b" +checksum = "95059e91184749cb66be6dc994f67f182b6d897cb3df74a5bf66b5e709295fd8" [[package]] name = "test-case" @@ -7608,9 +7667,9 @@ dependencies = [ [[package]] name = "test-case-macros" -version = "2.2.1" +version = "2.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c95968eedc6fc4f5c21920e0f4264f78ec5e4c56bb394f319becc1a5830b3e54" +checksum = "e45b7bf6e19353ddd832745c8fcf77a17a93171df7151187f26623f2b75b5b26" dependencies = [ "cfg-if", "proc-macro-error", @@ -7630,9 +7689,9 @@ dependencies = [ [[package]] name = "textwrap" -version = "0.15.1" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "949517c0cf1bf4ee812e2e07e08ab448e3ae0d23472aee8a06c985f0c8815b16" +checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" @@ -7665,9 +7724,9 @@ dependencies = [ [[package]] name = "tikv-jemalloc-sys" -version = "0.5.0+5.3.0" +version = "0.5.2+5.3.0-patched" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aeab4310214fe0226df8bfeb893a291a58b19682e8a07e1e1d4483ad4200d315" +checksum = "ec45c14da997d0925c7835883e4d5c181f196fa142f8c19d7643d1e9af2592c3" dependencies = [ "cc", "fs_extra", @@ -7686,22 +7745,32 @@ dependencies = [ [[package]] name = "time" -version = "0.3.14" +version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c3f9a28b618c3a6b9251b6908e9c99e04b9e5c02e6581ccbb67d59c34ef7f9b" +checksum = "a561bf4617eebd33bca6434b988f39ed798e527f51a1e797d0ee4f61c0a38376" dependencies = [ - "itoa 1.0.1", + "itoa 1.0.4", "libc", "num_threads", "serde", + "time-core", "time-macros", ] +[[package]] +name = "time-core" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" + [[package]] name = "time-macros" -version = "0.2.4" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42657b1a6f4d817cda8e7a0ace261fe0cc946cf3a80314390b22cc61ae080792" +checksum = "d967f99f534ca7e495c575c62638eebc2898a8c84c119b89e250477bc4ba16b2" +dependencies = [ + "time-core", +] [[package]] name = "tiny-keccak" @@ -7739,9 +7808,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.21.2" +version = "1.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9e03c497dc955702ba729190dc4aac6f2a0ce97f913e5b1b5912fc5039d9099" +checksum = "d76ce4a75fb488c605c54bf610f221cea8b0dafb53333c1a67e8ee199dcd2ae3" dependencies = [ "autocfg", "bytes 1.2.1", @@ -7781,9 +7850,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b557f72f448c511a979e2564e55d74e6c4432fc96ff4f6241bc6bded342643b7" +checksum = "9724f9a975fb987ef7a3cd9be0350edcbe130698af5b8f7a631e23d42d052484" dependencies = [ "proc-macro2", "quote", @@ -7838,11 +7907,11 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.23.3" +version = "0.23.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4151fda0cf2798550ad0b34bcfc9b9dcc2a9d2471c895c68f3a8818e54f2389e" +checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" dependencies = [ - "rustls 0.20.4", + "rustls 0.20.7", "tokio", "webpki 0.22.0", ] @@ -7880,15 +7949,16 @@ checksum = "f714dd15bead90401d77e04243611caec13726c2408afd5b31901dfcdcb3b181" dependencies = [ "futures-util", "log", - "rustls 0.20.4", + "rustls 0.20.7", "tokio", "tungstenite", ] [[package]] name = "tokio-util" -version = "0.7.1" -source = "git+https://github.com/vectordotdev/tokio?rev=3aa231cf6f33f74ca29077163879f0de9a207ad8#3aa231cf6f33f74ca29077163879f0de9a207ad8" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bb2e075f03b3d66d8d8785356224ba688d2906a371015e225beeb65ca92c740" dependencies = [ "bytes 1.2.1", "futures-core", @@ -7929,10 +7999,10 @@ dependencies = [ "hyper-timeout", "percent-encoding", "pin-project", - "prost 0.11.0", - "prost-derive 0.11.0", + "prost 0.11.2", + "prost-derive 0.11.2", "rustls-native-certs 0.6.2", - "rustls-pemfile 1.0.0", + "rustls-pemfile 1.0.1", "tokio", "tokio-rustls", "tokio-stream", @@ -7952,7 +8022,7 @@ checksum = "48c6fd7c2581e36d63388a9e04c350c21beb7a8b059580b2e93993c526899ddc" dependencies = [ "prettyplease", "proc-macro2", - "prost-build 0.11.1", + "prost-build 0.11.2", "quote", "syn", ] @@ -7979,9 +8049,9 @@ dependencies = [ [[package]] name = "tower-http" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d342c6d58709c0a6d48d48dabbb62d4ef955cf5f0f3bbfd845838e7ae88dbae" +checksum = "3c530c8675c1dbf98facee631536fa116b5fb6382d7dd6dc1b118d970eafe3ba" dependencies = [ "base64", "bitflags", @@ -8000,15 +8070,15 @@ dependencies = [ [[package]] name = "tower-layer" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "343bc9466d3fe6b0f960ef45960509f84480bf4fd96f92901afe7ff3df9d3a62" +checksum = "c20c8dbed6283a09604c3e69b4b7eeb54e298b8a600d4d5ecb5ad39de609f1d0" [[package]] name = "tower-service" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" +checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tower-test" @@ -8198,14 +8268,14 @@ dependencies = [ [[package]] name = "tracing-test" -version = "0.2.1" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3eb7bda2e93bbc9c5b247034acc6a4b3d04f033a3d4b8fc1cb87d4d1c7c7ebd7" +checksum = "9e3d272c44878d2bbc9f4a20ad463724f03e19dbc667c6e84ac433ab7ffcc70b" dependencies = [ "lazy_static", "tracing-core 0.1.30", "tracing-subscriber 0.3.16", - "tracing-test-macro 0.2.1", + "tracing-test-macro 0.2.3", ] [[package]] @@ -8221,9 +8291,9 @@ dependencies = [ [[package]] name = "tracing-test-macro" -version = "0.2.1" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4801dca35e4e2cee957c469bd4a1c370fadb7894c0d50721a40eba3523e6e91c" +checksum = "744324b12d69a9fc1edea4b38b7b1311295b662d161ad5deac17bb1358224a08" dependencies = [ "lazy_static", "quote", @@ -8457,36 +8527,36 @@ checksum = "099b7128301d285f79ddd55b9a83d5e6b9e97c92e0ea0daebee7263e932de992" [[package]] name = "unicode-ident" -version = "1.0.0" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d22af068fba1eb5edcb4aea19d382b2a3deb4c8f9d475c589b6ada9e0fd493ee" +checksum = "6ceab39d59e4c9499d4e5a8ee0e2735b891bb7308ac83dfb4e80cad195c9f6f3" [[package]] name = "unicode-normalization" -version = "0.1.19" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d54590932941a9e9266f0832deed84ebe1bf2e4c9e4a3554d393d18f5e854bf9" +checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" dependencies = [ "tinyvec", ] [[package]] name = "unicode-segmentation" -version = "1.9.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8820f5d777f6224dc4be3632222971ac30164d4a258d595640799554ebfd99" +checksum = "0fdbf052a0783de01e944a6ce7a8cb939e295b1e7be835a1112c3b9a7f047a5a" [[package]] name = "unicode-width" -version = "0.1.9" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ed742d4ea2bd1176e236172c8429aaf54486e7ac098db29ffe6529e0ce50973" +checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" [[package]] name = "unicode-xid" -version = "0.2.2" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ccb82d61f80a663efe1f787a51b16b5a51e3314d6ac365b08639f52387b33f3" +checksum = "f962df74c8c05a667b5ee8bcf162993134c104e96440b663c8daa176dc772d8c" [[package]] name = "unreachable" @@ -8534,9 +8604,9 @@ dependencies = [ [[package]] name = "urlencoding" -version = "2.1.0" +version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68b90931029ab9b034b300b797048cf23723400aa757e8a2bfb9d748102f9821" +checksum = "e8db7427f936968176eaa7cdf81b7f98b980b18495ec28f1b5791ac3bfe3eea9" [[package]] name = "utf-8" @@ -8699,7 +8769,7 @@ dependencies = [ "indoc", "infer 0.11.0", "inventory", - "itertools 0.10.5", + "itertools", "k8s-openapi", "kube", "lapin", @@ -8734,9 +8804,9 @@ dependencies = [ "postgres-openssl", "prometheus-parser", "proptest", - "prost 0.11.0", - "prost-build 0.11.1", - "prost-types 0.11.1", + "prost 0.11.2", + "prost-build 0.11.2", + "prost-types 0.11.2", "pulsar", "quickcheck", "rand 0.8.5", @@ -8996,9 +9066,9 @@ dependencies = [ "parking_lot", "pin-project", "proptest", - "prost 0.11.0", - "prost-build 0.11.1", - "prost-types 0.11.1", + "prost 0.11.2", + "prost-build 0.11.2", + "prost-types 0.11.2", "quanta", "quickcheck", "quickcheck_macros", @@ -9387,9 +9457,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.30" +version = "0.4.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f741de44b75e14c35df886aff5f1eb73aa114fa5d4d00dcd37b5e01259bf3b2" +checksum = "23639446165ca5a5de86ae1d8896b737ae80319560fbaa4c2887b7da6e7ebd7d" dependencies = [ "cfg-if", "js-sys", @@ -9428,9 +9498,9 @@ checksum = "1c38c045535d93ec4f0b4defec448e4291638ee608530863b1e2ba115d4fff7f" [[package]] name = "web-sys" -version = "0.3.57" +version = "0.3.60" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b17e741662c70c8bd24ac5c5b18de314a2c26c32bf8346ee1e6f53de919c283" +checksum = "bcda906d8be16e728fd5adc5b729afad4e444e106ab28cd1c7256e54fa61510f" dependencies = [ "js-sys", "wasm-bindgen", @@ -9474,9 +9544,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.22.3" +version = "0.22.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d8de8415c823c8abd270ad483c6feeac771fad964890779f9a8cb24fbbc1bf" +checksum = "368bfe657969fb01238bb756d351dcade285e0f6fcbd36dcb23359a5169975be" dependencies = [ "webpki 0.22.0", ] @@ -9492,13 +9562,13 @@ dependencies = [ [[package]] name = "which" -version = "4.2.5" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c4fb54e6113b6a8772ee41c3404fb0301ac79604489467e0a9ce1f3e97c24ae" +checksum = "1c831fbbee9e129a8cf93e7747a82da9d95ba8e16621cae60ec2cdc849bacb7b" dependencies = [ "either", - "lazy_static", "libc", + "once_cell", ] [[package]] @@ -9562,32 +9632,6 @@ dependencies = [ "windows-sys 0.36.1", ] -[[package]] -name = "windows-sys" -version = "0.30.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "030b7ff91626e57a05ca64a07c481973cbb2db774e4852c9c7ca342408c6a99a" -dependencies = [ - "windows_aarch64_msvc 0.30.0", - "windows_i686_gnu 0.30.0", - "windows_i686_msvc 0.30.0", - "windows_x86_64_gnu 0.30.0", - "windows_x86_64_msvc 0.30.0", -] - -[[package]] -name = "windows-sys" -version = "0.34.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5acdd78cb4ba54c0045ac14f62d8f94a03d10047904ae2a40afa1e99d8f70825" -dependencies = [ - "windows_aarch64_msvc 0.34.0", - "windows_i686_gnu 0.34.0", - "windows_i686_msvc 0.34.0", - "windows_x86_64_gnu 0.34.0", - "windows_x86_64_msvc 0.34.0", -] - [[package]] name = "windows-sys" version = "0.36.1" @@ -9602,16 +9646,25 @@ dependencies = [ ] [[package]] -name = "windows_aarch64_msvc" -version = "0.30.0" +name = "windows-sys" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29277a4435d642f775f63c7d1faeb927adba532886ce0287bd985bffb16b6bca" +checksum = "5a3e1820f08b8513f676f7ab6c1f99ff312fb97b553d30ff4dd86f9f15728aa7" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc 0.42.0", + "windows_i686_gnu 0.42.0", + "windows_i686_msvc 0.42.0", + "windows_x86_64_gnu 0.42.0", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc 0.42.0", +] [[package]] -name = "windows_aarch64_msvc" -version = "0.34.0" +name = "windows_aarch64_gnullvm" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17cffbe740121affb56fad0fc0e421804adf0ae00891205213b5cecd30db881d" +checksum = "41d2aa71f6f0cbe00ae5167d90ef3cfe66527d6f613ca78ac8024c3ccab9a19e" [[package]] name = "windows_aarch64_msvc" @@ -9620,16 +9673,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9bb8c3fd39ade2d67e9874ac4f3db21f0d710bee00fe7cab16949ec184eeaa47" [[package]] -name = "windows_i686_gnu" -version = "0.30.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1145e1989da93956c68d1864f32fb97c8f561a8f89a5125f6a2b7ea75524e4b8" - -[[package]] -name = "windows_i686_gnu" -version = "0.34.0" +name = "windows_aarch64_msvc" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2564fde759adb79129d9b4f54be42b32c89970c18ebf93124ca8870a498688ed" +checksum = "dd0f252f5a35cac83d6311b2e795981f5ee6e67eb1f9a7f64eb4500fbc4dcdb4" [[package]] name = "windows_i686_gnu" @@ -9638,16 +9685,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "180e6ccf01daf4c426b846dfc66db1fc518f074baa793aa7d9b9aaeffad6a3b6" [[package]] -name = "windows_i686_msvc" -version = "0.30.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4a09e3a0d4753b73019db171c1339cd4362c8c44baf1bcea336235e955954a6" - -[[package]] -name = "windows_i686_msvc" -version = "0.34.0" +name = "windows_i686_gnu" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cd9d32ba70453522332c14d38814bceeb747d80b3958676007acadd7e166956" +checksum = "fbeae19f6716841636c28d695375df17562ca208b2b7d0dc47635a50ae6c5de7" [[package]] name = "windows_i686_msvc" @@ -9656,16 +9697,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2e7917148b2812d1eeafaeb22a97e4813dfa60a3f8f78ebe204bcc88f12f024" [[package]] -name = "windows_x86_64_gnu" -version = "0.30.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ca64fcb0220d58db4c119e050e7af03c69e6f4f415ef69ec1773d9aab422d5a" - -[[package]] -name = "windows_x86_64_gnu" -version = "0.34.0" +name = "windows_i686_msvc" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cfce6deae227ee8d356d19effc141a509cc503dfd1f850622ec4b0f84428e1f4" +checksum = "84c12f65daa39dd2babe6e442988fc329d6243fdce47d7d2d155b8d874862246" [[package]] name = "windows_x86_64_gnu" @@ -9674,16 +9709,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4dcd171b8776c41b97521e5da127a2d86ad280114807d0b2ab1e462bc764d9e1" [[package]] -name = "windows_x86_64_msvc" -version = "0.30.0" +name = "windows_x86_64_gnu" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08cabc9f0066848fef4bc6a1c1668e6efce38b661d2aeec75d18d8617eebb5f1" +checksum = "bf7b1b21b5362cbc318f686150e5bcea75ecedc74dd157d874d754a2ca44b0ed" [[package]] -name = "windows_x86_64_msvc" -version = "0.34.0" +name = "windows_x86_64_gnullvm" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d19538ccc21819d01deaf88d6a17eae6596a12e9aafdbb97916fb49896d89de9" +checksum = "09d525d2ba30eeb3297665bd434a54297e4170c7f1a44cad4ef58095b4cd2028" [[package]] name = "windows_x86_64_msvc" @@ -9692,13 +9727,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c811ca4a8c853ef420abd8592ba53ddbbac90410fab6903b3e79972a631f7680" [[package]] -name = "winreg" -version = "0.7.0" +name = "windows_x86_64_msvc" +version = "0.42.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0120db82e8a1e0b9fb3345a539c478767c0048d842860994d96113d5b667bd69" -dependencies = [ - "winapi", -] +checksum = "f40009d85759725a34da6d89a94e63d7bdc50a862acf0dbc7c8e488f1edcb6f5" [[package]] name = "winreg" @@ -9785,9 +9817,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.5.4" +version = "1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eb5728b8afd3f280a869ce1d4c554ffaed35f45c231fc41bfbd0381bef50317" +checksum = "c394b5bd0c6f669e7275d9c20aa90ae064cb22e75a1cad54e1b34088034b149f" dependencies = [ "zeroize_derive", ] @@ -9832,3 +9864,8 @@ dependencies = [ "cc", "libc", ] + +[[patch.unused]] +name = "tokio-util" +version = "0.7.1" +source = "git+https://github.com/vectordotdev/tokio?rev=3aa231cf6f33f74ca29077163879f0de9a207ad8#3aa231cf6f33f74ca29077163879f0de9a207ad8" From 38af926fdbb5829053e56c356fbb7ee65c45ddc6 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Mon, 21 Nov 2022 19:17:06 -0700 Subject: [PATCH 13/48] address pr feedback --- src/internal_events/pulsar.rs | 6 +- src/sinks/pulsar/integration_tests.rs | 101 ++++++++++++++++++++ src/sinks/pulsar/mod.rs | 20 ++-- src/sinks/pulsar/tests.rs | 131 +++++--------------------- src/sinks/pulsar/util.rs | 26 +---- 5 files changed, 141 insertions(+), 143 deletions(-) create mode 100644 src/sinks/pulsar/integration_tests.rs diff --git a/src/internal_events/pulsar.rs b/src/internal_events/pulsar.rs index eef7166ae18f7..21188ef5420be 100644 --- a/src/internal_events/pulsar.rs +++ b/src/internal_events/pulsar.rs @@ -42,17 +42,17 @@ impl InternalEvent for PulsarPropertyExtractionError<'_> { fn emit(self) { error!( message = "Failed to extract properties. Value should be a map of String -> Bytes.", - error_code = "extracing_property", + error_code = "extracting_property", error_type = error_type::PARSER_FAILED, stage = error_stage::RECEIVING, property_field = self.property_field, - internal_log_rate_secs = 10, + internal_log_rate_limit = true, ); counter!( "component_errors_total", 1, "error_code" => "extracing_property", "error_type" => error_type::PARSER_FAILED, - "stage" => error_stage::RECEIVING, + "stage" => error_stage::PROCESSING, ); } } diff --git a/src/sinks/pulsar/integration_tests.rs b/src/sinks/pulsar/integration_tests.rs new file mode 100644 index 0000000000000..f35c4eae88e92 --- /dev/null +++ b/src/sinks/pulsar/integration_tests.rs @@ -0,0 +1,101 @@ +use crate::sinks::pulsar::config::PulsarSinkConfig; +use crate::sinks::pulsar::sink::PulsarSink; +use crate::sinks::util::TowerRequestConfig; +use codecs::TextSerializerConfig; +use futures::StreamExt; +use pulsar::SubType; +use std::collections::BTreeMap; + +use crate::event::Value; +use crate::sinks::VectorSink; +use crate::test_util::components::assert_sink_compliance; +use crate::test_util::{ + components::SINK_TAGS, random_lines_with_stream, random_string, trace_init, +}; +use bytes::Bytes; + +fn pulsar_address() -> String { + std::env::var("PULSAR_ADDRESS").unwrap_or_else(|_| "pulsar://127.0.0.1:6650".into()) +} + +async fn pulsar_happy_reuse(mut cnf: PulsarSinkConfig) { + trace_init(); + + let prop_1_key = "prop-1-key"; + let prop_1_value = "prop-1-value"; + let num_events = 1_000; + let (input, events) = random_lines_with_stream(100, num_events, None); + let prop_key_opt = cnf.properties_key.clone(); + let input_events = events.map(move |mut events| { + // if a property_key is defined, add some properties! + if let Some(prop_key) = &prop_key_opt { + let properties_key = prop_key; + let mut property_values = BTreeMap::new(); + property_values.insert( + prop_1_key.to_owned(), + Value::Bytes(Bytes::from(prop_1_value)), + ); + events.iter_logs_mut().for_each(move |log| { + log.insert(properties_key.as_str(), property_values.clone()); + }); + events + } else { + events + } + }); + + let topic = format!("test-{}", random_string(10)); + + cnf.topic = topic.clone(); + + let pulsar = cnf.create_pulsar_client().await.unwrap(); + let mut consumer = pulsar + .consumer() + .with_topic(&topic) + .with_consumer_name("VectorTestConsumer") + .with_subscription_type(SubType::Shared) + .with_subscription("VectorTestSub") + .with_options(pulsar::consumer::ConsumerOptions { + read_compacted: Some(false), + ..Default::default() + }) + .build::() + .await + .unwrap(); + + assert_sink_compliance(&SINK_TAGS, async move { + let sink = PulsarSink::new(pulsar, cnf).unwrap(); + let sink = VectorSink::from_event_streamsink(sink); + sink.run(input_events).await + }) + .await + .expect("Running sink failed"); + + for line in input { + let msg = match consumer.next().await.unwrap() { + Ok(msg) => msg, + Err(error) => panic!("{:?}", error), + }; + consumer.ack(&msg).await.unwrap(); + assert_eq!(String::from_utf8_lossy(&msg.payload.data), line); + } +} + +#[tokio::test] +async fn pulsar_happy() { + let cnf = PulsarSinkConfig { + endpoint: pulsar_address(), + // overriden by test + topic: "".to_string(), + key_field: None, + properties_key: None, + batch_size: None, + compression: None, + encoding: TextSerializerConfig::new().into(), + auth: None, + acknowledgements: Default::default(), + request: TowerRequestConfig::default(), + }; + + pulsar_happy_reuse(cnf).await +} diff --git a/src/sinks/pulsar/mod.rs b/src/sinks/pulsar/mod.rs index 49dfd20acbd9e..bbf8bf4130e40 100644 --- a/src/sinks/pulsar/mod.rs +++ b/src/sinks/pulsar/mod.rs @@ -1,7 +1,13 @@ -pub(crate) mod config; -pub(crate) mod encoder; -pub(crate) mod request_builder; -pub(crate) mod service; -pub(crate) mod sink; -pub(crate) mod tests; -mod util; +pub(super) mod config; +mod encoder; +mod request_builder; +mod service; +mod sink; +pub(super) mod util; + +#[cfg(test)] +mod tests; + +#[cfg(all(test, feature = "pulsar-integration-tests"))] +mod integration_tests; + diff --git a/src/sinks/pulsar/tests.rs b/src/sinks/pulsar/tests.rs index c0dc7647abd9e..ace462670b19e 100644 --- a/src/sinks/pulsar/tests.rs +++ b/src/sinks/pulsar/tests.rs @@ -1,116 +1,29 @@ -#[cfg(test)] -mod config { - use crate::sinks::pulsar::config::PulsarSinkConfig; - use vector_config::component::GenerateConfig; +use crate::sinks::pulsar::config::PulsarSinkConfig; +use std::collections::BTreeMap; +use vector_config::component::GenerateConfig; +use value::Value; +use crate::event::Event; - #[test] - fn generate_config() { - PulsarSinkConfig::generate_config(); - } -} - -#[cfg(feature = "pulsar-integration-tests")] -#[cfg(test)] -mod integration_tests { - use crate::sinks::pulsar::config::PulsarSinkConfig; - use crate::sinks::pulsar::sink::PulsarSink; - use crate::sinks::util::TowerRequestConfig; - use codecs::TextSerializerConfig; - use futures::StreamExt; - use pulsar::SubType; - use std::collections::BTreeMap; - - use crate::event::Value; - use crate::sinks::VectorSink; - use crate::test_util::components::assert_sink_compliance; - use crate::test_util::{ - components::SINK_TAGS, random_lines_with_stream, random_string, trace_init, - }; - use bytes::Bytes; - - fn pulsar_address() -> String { - std::env::var("PULSAR_ADDRESS").unwrap_or_else(|_| "pulsar://127.0.0.1:6650".into()) - } - - async fn pulsar_happy_reuse(mut cnf: PulsarSinkConfig) { - trace_init(); +use bytes::Bytes; - let prop_1_key = "prop-1-key"; - let prop_1_value = "prop-1-value"; - let num_events = 1_000; - let (input, events) = random_lines_with_stream(100, num_events, None); - let prop_key_opt = cnf.properties_key.clone(); - let input_events = events.map(move |mut events| { - // if a property_key is defined, add some properties! - if let Some(prop_key) = &prop_key_opt { - let properties_key = prop_key; - let mut property_values = BTreeMap::new(); - property_values.insert( - prop_1_key.to_owned(), - Value::Bytes(Bytes::from(prop_1_value)), - ); - events.iter_logs_mut().for_each(move |log| { - log.insert(properties_key.as_str(), property_values.clone()); - }); - events - } else { - events - } - }); +use crate::event::LogEvent; - let topic = format!("test-{}", random_string(10)); - - cnf.topic = topic.clone(); - - let pulsar = cnf.create_pulsar_client().await.unwrap(); - let mut consumer = pulsar - .consumer() - .with_topic(&topic) - .with_consumer_name("VectorTestConsumer") - .with_subscription_type(SubType::Shared) - .with_subscription("VectorTestSub") - .with_options(pulsar::consumer::ConsumerOptions { - read_compacted: Some(false), - ..Default::default() - }) - .build::() - .await - .unwrap(); - - assert_sink_compliance(&SINK_TAGS, async move { - let sink = PulsarSink::new(pulsar, cnf).unwrap(); - let sink = VectorSink::from_event_streamsink(sink); - sink.run(input_events).await - }) - .await - .expect("Running sink failed"); +#[test] +fn generate_config() { + PulsarSinkConfig::generate_config(); +} - for line in input { - let msg = match consumer.next().await.unwrap() { - Ok(msg) => msg, - Err(error) => panic!("{:?}", error), - }; - consumer.ack(&msg).await.unwrap(); - assert_eq!(String::from_utf8_lossy(&msg.payload.data), line); - } - } +#[test] +fn pulsar_get_headers() { + let properties_key = "properties"; + let mut property_values = BTreeMap::new(); + property_values.insert("a-key".to_string(), Value::Bytes(Bytes::from("a-value"))); + property_values.insert("b-key".to_string(), Value::Bytes(Bytes::from("b-value"))); - #[tokio::test] - async fn pulsar_happy() { - let cnf = PulsarSinkConfig { - endpoint: pulsar_address(), - // overriden by test - topic: "".to_string(), - key_field: None, - properties_key: None, - batch_size: None, - compression: None, - encoding: TextSerializerConfig::new().into(), - auth: None, - acknowledgements: Default::default(), - request: TowerRequestConfig::default(), - }; + let mut event = Event::Log(LogEvent::from("hello")); + event.as_mut_log().insert(properties_key, property_values); - pulsar_happy_reuse(cnf).await - } + let properties = super::util::get_properties(&event, &Some(properties_key.to_string())).unwrap(); + assert_eq!(properties.get("a-key").unwrap(), "a-value".as_bytes()); + assert_eq!(properties.get("b-key").unwrap(), "b-value".as_bytes()); } diff --git a/src/sinks/pulsar/util.rs b/src/sinks/pulsar/util.rs index cd65fe88b75b7..b6d5facc7b152 100644 --- a/src/sinks/pulsar/util.rs +++ b/src/sinks/pulsar/util.rs @@ -54,7 +54,7 @@ fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option .map(|ts| ts.timestamp_millis()) } -fn get_properties( +pub(super) fn get_properties( event: &Event, properties_key: &Option, ) -> Option> { @@ -88,26 +88,4 @@ fn get_properties( } #[cfg(test)] -mod tests { - use std::collections::BTreeMap; - - use bytes::Bytes; - - use super::*; - use crate::event::LogEvent; - - #[test] - fn pulsar_get_headers() { - let properties_key = "properties"; - let mut property_values = BTreeMap::new(); - property_values.insert("a-key".to_string(), Value::Bytes(Bytes::from("a-value"))); - property_values.insert("b-key".to_string(), Value::Bytes(Bytes::from("b-value"))); - - let mut event = Event::Log(LogEvent::from("hello")); - event.as_mut_log().insert(properties_key, property_values); - - let properties = get_properties(&event, &Some(properties_key.to_string())).unwrap(); - assert_eq!(properties.get("a-key").unwrap(), "a-value".as_bytes()); - assert_eq!(properties.get("b-key").unwrap(), "b-value".as_bytes()); - } -} +mod tests {} From 638c1fc45d8c85c35633df54c1087f6861200b54 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Wed, 23 Nov 2022 10:51:22 -0700 Subject: [PATCH 14/48] fix fmt --- src/sinks/pulsar/mod.rs | 1 - src/sinks/pulsar/tests.rs | 7 ++++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/sinks/pulsar/mod.rs b/src/sinks/pulsar/mod.rs index bbf8bf4130e40..3a4fe63a621d7 100644 --- a/src/sinks/pulsar/mod.rs +++ b/src/sinks/pulsar/mod.rs @@ -10,4 +10,3 @@ mod tests; #[cfg(all(test, feature = "pulsar-integration-tests"))] mod integration_tests; - diff --git a/src/sinks/pulsar/tests.rs b/src/sinks/pulsar/tests.rs index ace462670b19e..c78ce04f50f77 100644 --- a/src/sinks/pulsar/tests.rs +++ b/src/sinks/pulsar/tests.rs @@ -1,8 +1,8 @@ +use crate::event::Event; use crate::sinks::pulsar::config::PulsarSinkConfig; use std::collections::BTreeMap; -use vector_config::component::GenerateConfig; use value::Value; -use crate::event::Event; +use vector_config::component::GenerateConfig; use bytes::Bytes; @@ -23,7 +23,8 @@ fn pulsar_get_headers() { let mut event = Event::Log(LogEvent::from("hello")); event.as_mut_log().insert(properties_key, property_values); - let properties = super::util::get_properties(&event, &Some(properties_key.to_string())).unwrap(); + let properties = + super::util::get_properties(&event, &Some(properties_key.to_string())).unwrap(); assert_eq!(properties.get("a-key").unwrap(), "a-value".as_bytes()); assert_eq!(properties.get("b-key").unwrap(), "b-value".as_bytes()); } From 69e60deb83cba99ae1e93f7f66cd20bc24adc13b Mon Sep 17 00:00:00 2001 From: kyle criddle Date: Wed, 23 Nov 2022 16:11:42 -0700 Subject: [PATCH 15/48] update cargo.lock --- Cargo.lock | 51 +++++++++++++++++++++++---------------------------- 1 file changed, 23 insertions(+), 28 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e1260bcd5df47..c7450f98a0ba9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -66,9 +66,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "0.7.19" +version = "0.7.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4f55bd91a0978cbfd91c457a164bab8b4001c833b7f323132c0a4e1922dd44e" +checksum = "cc936419f96fa211c1b9166887b38e5e40b19958e5b895be7c1f93adec7071ac" dependencies = [ "memchr", ] @@ -1505,9 +1505,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.76" +version = "1.0.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76a284da2e6fe2092f2353e51713435363112dfd60030e22add80be333fb928f" +checksum = "e9f73505338f7d905b19d18738976aae232eb46b8efc15554ffc56deb5d9ebe4" dependencies = [ "jobserver", ] @@ -2016,9 +2016,9 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.12" +version = "0.9.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96bf8df95e795db1a4aca2957ad884a2df35413b24bbeb3114422f3cc21498e8" +checksum = "01a9af1f4c2ef74bb8aa1f7e19706bc72d03598c8a570bb5de72243c7a9d9d5a" dependencies = [ "autocfg", "cfg-if", @@ -2029,9 +2029,9 @@ dependencies = [ [[package]] name = "crossbeam-queue" -version = "0.3.7" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ebb3d1683412e9be6a15533314f00ec223c0762c522a3f77f048b265aab4470c" +checksum = "d1cfb3ea8a53f37c40dea2c7bedcbd88bdfae54f5e2175d6ecaff1c988353add" dependencies = [ "cfg-if", "crossbeam-utils", @@ -2805,7 +2805,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bb21c69b9fea5e15dbc1049e4b77145dd0ba1c84019c488102de0dc4ea4b0a27" dependencies = [ "cfg-if", - "rustix 0.36.2", + "rustix 0.36.3", "windows-sys 0.42.0", ] @@ -4288,9 +4288,9 @@ checksum = "d4d2456c373231a208ad294c33dc5bff30051eafd954cd4caae83a712b12854d" [[package]] name = "linux-raw-sys" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb68f22743a3fb35785f1e7f844ca5a3de2dde5bd0c0ef5b372065814699b121" +checksum = "8f9f08d8963a6c613f4b1a78f4f4a4dbfadf8e6545b2d72861731e4858b8b47f" [[package]] name = "listenfd" @@ -5294,9 +5294,9 @@ dependencies = [ [[package]] name = "os_str_bytes" -version = "6.4.0" +version = "6.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b5bf27447411e9ee3ff51186bf7a08e16c341efdde93f4d823e8844429bed7e" +checksum = "9b7820b9daea5457c9f21c69448905d723fbd21136ccf521748f23fd49e723ee" [[package]] name = "overload" @@ -5861,7 +5861,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a0841812012b2d4a6145fae9a6af1534873c32aa67fff26bd09f8fa42c83f95a" dependencies = [ "bytes 1.3.0", - "prost-derive 0.11.0", + "prost-derive 0.11.2", ] [[package]] @@ -6274,13 +6274,8 @@ checksum = "513b3649f1a111c17954296e4a3b9eecb108b766c803e2b99f179ebe27005985" dependencies = [ "arc-swap", "async-trait", -<<<<<<< HEAD - "bytes 1.2.1", - "combine 4.6.6", -======= "bytes 1.3.0", - "combine 4.6.4", ->>>>>>> master + "combine 4.6.6", "futures 0.3.25", "futures-util", "itoa 1.0.4", @@ -6607,15 +6602,15 @@ dependencies = [ [[package]] name = "rustix" -version = "0.36.2" +version = "0.36.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "203974af07ea769452490ee8de3e5947971efc3a090dca8a779dd432d3fa46a7" +checksum = "0b1fbb4dfc4eb1d390c02df47760bb19a84bb80b301ecc947ab5406394d8223e" dependencies = [ "bitflags", "errno", "io-lifetimes 1.0.1", "libc", - "linux-raw-sys 0.1.2", + "linux-raw-sys 0.1.3", "windows-sys 0.42.0", ] @@ -6991,9 +6986,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.87" +version = "1.0.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ce777b7b150d76b9cf60d28b55f5847135a003f7d7350c6be7a773508ce7d45" +checksum = "020ff22c755c2ed3f8cf162dbb41a7268d934702f3ed3631656ea597e08fc3db" dependencies = [ "indexmap", "itoa 1.0.4", @@ -8005,7 +8000,7 @@ dependencies = [ "percent-encoding", "pin-project", "prost 0.11.2", - "prost-derive 0.11.0", + "prost-derive 0.11.2", "rustls-native-certs 0.6.2", "rustls-pemfile 1.0.1", "tokio", @@ -9862,9 +9857,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.1+zstd.1.5.2" +version = "2.0.2+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fd07cbbc53846d9145dbffdf6dd09a7a0aa52be46741825f5c97bdd4f73f12b" +checksum = "24faa29d97c8ddca9b37b680e3bd2d5439d864a9cac3a0640d086b71c908bb83" dependencies = [ "cc", "libc", From 289cf6b458965f84c180b556f1f409d5dc3bc32e Mon Sep 17 00:00:00 2001 From: kyle criddle Date: Tue, 29 Nov 2022 09:04:38 -0700 Subject: [PATCH 16/48] update cargo.lock --- Cargo.lock | 174 ++++++++++++++++++++--------------------------------- 1 file changed, 66 insertions(+), 108 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ceea72c400ddd..daf5b8d5b4454 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -261,11 +261,11 @@ dependencies = [ [[package]] name = "async-channel" -version = "1.7.1" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e14485364214912d3b19cc3435dde4df66065127f05fa0d75c712f36f12c2f28" +checksum = "cf46fee83e5ccffc220104713af3292ff9bc7c64c7de289f66dae8e38d826833" dependencies = [ - "concurrent-queue 1.2.4", + "concurrent-queue", "event-listener", "futures-core", ] @@ -293,7 +293,7 @@ checksum = "17adb73da160dfb475c183343c8cccd80721ea5a605d3eb57125f0a7b7a92d0b" dependencies = [ "async-lock", "async-task", - "concurrent-queue 2.0.0", + "concurrent-queue", "fastrand", "futures-lite", "slab", @@ -429,7 +429,7 @@ checksum = "8c374dda1ed3e7d8f0d9ba58715f924862c63eae6849c92d3a18e7fbde9e2794" dependencies = [ "async-lock", "autocfg", - "concurrent-queue 2.0.0", + "concurrent-queue", "futures-lite", "libc", "log", @@ -533,9 +533,9 @@ checksum = "7a40729d2133846d9ed0ea60a8b9541bccddab49cd30f0715a1da672fe9a2524" [[package]] name = "async-trait" -version = "0.1.58" +version = "0.1.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e805d94e6b5001b651426cf4cd446b1ab5f319d27bab5c644f61de0a804360c" +checksum = "31e6e93155431f3931513b243d371981bb2770112b370c82745a1d19d2f99364" dependencies = [ "proc-macro2", "quote", @@ -1076,9 +1076,9 @@ dependencies = [ [[package]] name = "axum" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "744864363a200a5e724a7e61bc8c11b6628cf2e3ec519c8a1a48e609a8156b40" +checksum = "08b108ad2665fa3f6e6a517c3d80ec3e77d224c47d605167aefaa5d7ef97fa48" dependencies = [ "async-trait", "axum-core 0.3.0", @@ -1089,7 +1089,7 @@ dependencies = [ "http-body", "hyper", "itoa 1.0.4", - "matchit 0.6.0", + "matchit 0.7.0", "memchr", "mime", "percent-encoding", @@ -1567,12 +1567,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c58ec36aac5066d5ca17df51b3e70279f5670a72102f5752cb7e7c856adfc70" -[[package]] -name = "cache-padded" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1db59621ec70f09c5e9b597b220c7a2b43611f4710dc03ceb8748637775692c" - [[package]] name = "cassowary" version = "0.3.0" @@ -1750,9 +1744,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.0.27" +version = "4.0.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0acbd8d28a0a60d7108d7ae850af6ba34cf2d1257fc646980e5f97ce14275966" +checksum = "4d63b9e9c07271b9957ad22c173bae2a4d9a81127680962039296abcd2f8251d" dependencies = [ "bitflags", "clap_derive", @@ -1761,7 +1755,7 @@ dependencies = [ "once_cell", "strsim 0.10.0", "termcolor", - "terminal_size 0.2.2", + "terminal_size 0.2.3", ] [[package]] @@ -1830,7 +1824,7 @@ dependencies = [ "memchr", "once_cell", "ordered-float 3.4.0", - "prost 0.11.2", + "prost 0.11.3", "regex", "serde", "serde_json", @@ -1897,15 +1891,6 @@ dependencies = [ "tokio-util", ] -[[package]] -name = "concurrent-queue" -version = "1.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af4780a44ab5696ea9e28294517f1fffb421a83a25af521333c838635509db9c" -dependencies = [ - "cache-padded", -] - [[package]] name = "concurrent-queue" version = "2.0.0" @@ -1934,7 +1919,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e57ff02e8ad8e06ab9731d5dc72dc23bef9200778eae1a89d555d8c42e5d4a86" dependencies = [ - "prost 0.11.2", + "prost 0.11.3", "prost-types 0.11.2", "tonic", "tracing-core 0.1.30", @@ -2593,9 +2578,9 @@ checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" [[package]] name = "duct" -version = "0.13.5" +version = "0.13.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fc6a0a59ed0888e0041cf708e66357b7ae1a82f1c67247e1f93b5e0818f7d8d" +checksum = "37ae3fc31835f74c2a7ceda3aeede378b0ae2e74c8f1c36559fcc9ae2a4e7d3e" dependencies = [ "libc", "once_cell", @@ -2896,7 +2881,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bb21c69b9fea5e15dbc1049e4b77145dd0ba1c84019c488102de0dc4ea4b0a27" dependencies = [ "cfg-if", - "rustix 0.36.4", + "rustix", "windows-sys 0.42.0", ] @@ -3224,9 +3209,9 @@ checksum = "9b919933a397b79c37e33b77bb2aa3dc8eb6e165ad809e58ff75bc7db2e34574" [[package]] name = "gloo-utils" -version = "0.1.5" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40913a05c8297adca04392f707b1e73b12ba7b8eab7244a4961580b1fd34063c" +checksum = "a8e8fc851e9c7b9852508bc6e3f690f452f474417e8545ec9857b7f7377036b5" dependencies = [ "js-sys", "serde", @@ -3928,12 +3913,6 @@ dependencies = [ "ghost", ] -[[package]] -name = "io-lifetimes" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59ce5ef949d49ee85593fc4d3f3f95ad61657076395cbbce23e2121fc5542074" - [[package]] name = "io-lifetimes" version = "1.0.3" @@ -3987,8 +3966,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "927609f78c2913a6f6ac3c27a4fe87f43e2a35367c0c4b0f8265e8f49a104330" dependencies = [ "hermit-abi 0.2.6", - "io-lifetimes 1.0.3", - "rustix 0.36.4", + "io-lifetimes", + "rustix", "windows-sys 0.42.0", ] @@ -4401,12 +4380,6 @@ dependencies = [ "linked-hash-map", ] -[[package]] -name = "linux-raw-sys" -version = "0.0.46" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4d2456c373231a208ad294c33dc5bff30051eafd954cd4caae83a712b12854d" - [[package]] name = "linux-raw-sys" version = "0.1.3" @@ -4581,9 +4554,9 @@ checksum = "73cbba799671b762df5a175adf59ce145165747bb891505c43d09aefbbf38beb" [[package]] name = "matchit" -version = "0.6.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dfc802da7b1cf80aefffa0c7b2f77247c8b32206cc83c270b61264f5b360a80" +checksum = "b87248edafb776e59e6ee64a79086f65890d3510f2c656c000bf2a7e8a0aea40" [[package]] name = "matrixmultiply" @@ -5373,8 +5346,8 @@ dependencies = [ "chrono", "hex", "ordered-float 3.4.0", - "prost 0.11.2", - "prost-build 0.11.2", + "prost 0.11.3", + "prost-build 0.11.3", "tonic", "tonic-build", "value", @@ -5411,12 +5384,12 @@ dependencies = [ [[package]] name = "os_pipe" -version = "0.9.2" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb233f06c2307e1f5ce2ecad9f8121cffbbee2c95428f44ea85222e460d0d213" +checksum = "c6a252f1f8c11e84b3ab59d7a488e48e4478a93937e027076638c49536204639" dependencies = [ "libc", - "winapi", + "windows-sys 0.42.0", ] [[package]] @@ -5449,9 +5422,9 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.4" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dc9e0dc2adc1c69d09143aff38d3d30c5c3f0df0dad82e6d25547af174ebec0" +checksum = "7ff9f3fef3968a3ec5945535ed654cb38ff72d7495a25619e2247fb15a2ed9ba" dependencies = [ "cfg-if", "libc", @@ -5719,9 +5692,9 @@ dependencies = [ [[package]] name = "polling" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f7d73f1eaed1ca1fb37b54dcc9b38e3b17d6c7b8ecb7abfffcac8d0351f17d4" +checksum = "166ca89eb77fd403230b9c156612965a81e094ec6ec3aa13663d4c8b113fa748" dependencies = [ "autocfg", "cfg-if", @@ -5940,8 +5913,8 @@ dependencies = [ "indexmap", "nom", "num_enum", - "prost 0.11.2", - "prost-build 0.11.2", + "prost 0.11.3", + "prost-build 0.11.3", "prost-types 0.11.2", "snafu", "value", @@ -5992,9 +5965,9 @@ dependencies = [ [[package]] name = "prost" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0841812012b2d4a6145fae9a6af1534873c32aa67fff26bd09f8fa42c83f95a" +checksum = "c0b18e655c21ff5ac2084a5ad0611e827b3f92badf79f4910b5a5c58f4d87ff0" dependencies = [ "bytes 1.3.0", "prost-derive 0.11.2", @@ -6024,9 +5997,9 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d8b442418ea0822409d9e7d047cbf1e7e9e1760b172bf9982cf29d517c93511" +checksum = "e330bf1316db56b12c2bcfa399e8edddd4821965ea25ddb2c134b610b1c1c604" dependencies = [ "bytes 1.3.0", "heck 0.4.0", @@ -6036,7 +6009,7 @@ dependencies = [ "multimap", "petgraph", "prettyplease", - "prost 0.11.2", + "prost 0.11.3", "prost-types 0.11.2", "regex", "syn", @@ -6087,7 +6060,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "747761bc3dc48f9a34553bf65605cf6cb6288ba219f3450b4275dbd81539551a" dependencies = [ "bytes 1.3.0", - "prost 0.11.2", + "prost 0.11.3", ] [[package]] @@ -6131,8 +6104,8 @@ dependencies = [ "oauth2", "openidconnect", "pem", - "prost 0.11.2", - "prost-build 0.11.2", + "prost 0.11.3", + "prost-build 0.11.3", "prost-derive 0.11.2", "rand 0.8.5", "regex", @@ -6729,20 +6702,6 @@ dependencies = [ "semver 0.9.0", ] -[[package]] -name = "rustix" -version = "0.35.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "727a1a6d65f786ec22df8a81ca3121107f235970dc1705ed681d3e6e8b9cd5f9" -dependencies = [ - "bitflags", - "errno", - "io-lifetimes 0.7.5", - "libc", - "linux-raw-sys 0.0.46", - "windows-sys 0.42.0", -] - [[package]] name = "rustix" version = "0.36.4" @@ -6751,9 +6710,9 @@ checksum = "cb93e85278e08bb5788653183213d3a60fc242b10cb9be96586f5a73dcb67c23" dependencies = [ "bitflags", "errno", - "io-lifetimes 1.0.3", + "io-lifetimes", "libc", - "linux-raw-sys 0.1.3", + "linux-raw-sys", "windows-sys 0.42.0", ] @@ -7333,9 +7292,9 @@ dependencies = [ [[package]] name = "shared_child" -version = "0.3.5" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6be9f7d5565b1483af3e72975e2dee33879b3b86bd48c0929fccf6585d79e65a" +checksum = "b0d94659ad3c2137fef23ae75b03d5241d633f8acded53d672decfa0e6e0caef" dependencies = [ "libc", "winapi", @@ -7444,9 +7403,9 @@ checksum = "a507befe795404456341dfab10cef66ead4c041f62b8b11bbb92bffe5d0953e0" [[package]] name = "smol" -version = "1.2.5" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85cf3b5351f3e783c1d79ab5fc604eeed8b8ae9abd36b166e8b87a089efd85e4" +checksum = "13f2b548cd8447f8de0fdf1c592929f70f4fc7039a05e47404b0d096ec6987a1" dependencies = [ "async-channel", "async-executor", @@ -7457,7 +7416,6 @@ dependencies = [ "async-process", "blocking", "futures-lite", - "once_cell", ] [[package]] @@ -7785,11 +7743,11 @@ dependencies = [ [[package]] name = "terminal_size" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40ca90c434fd12083d1a6bdcbe9f92a14f96c8a1ba600ba451734ac334521f7a" +checksum = "cb20089a8ba2b69debd491f8d2d023761cbf196e999218c591fa1e7e15a21907" dependencies = [ - "rustix 0.35.13", + "rustix", "windows-sys 0.42.0", ] @@ -7993,9 +7951,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9724f9a975fb987ef7a3cd9be0350edcbe130698af5b8f7a631e23d42d052484" +checksum = "5ec8ecbbb74d42a8dd0ce8ea9bfc38ad13e2ed5203c4f272dbe144f4e17d70ac" dependencies = [ "proc-macro2", "quote", @@ -8128,7 +8086,7 @@ checksum = "8f219fad3b929bef19b1f86fbc0358d35daed8f2cac972037ac0dc10bbb8d5fb" dependencies = [ "async-stream", "async-trait", - "axum 0.6.0", + "axum 0.6.1", "base64", "bytes 1.3.0", "flate2", @@ -8141,7 +8099,7 @@ dependencies = [ "hyper-timeout", "percent-encoding", "pin-project", - "prost 0.11.2", + "prost 0.11.3", "prost-derive 0.11.2", "rustls-native-certs 0.6.2", "rustls-pemfile 1.0.1", @@ -8164,7 +8122,7 @@ checksum = "31fa2c5e870bdce133847d15e075333e6e1ca3fff913001fede6754f3060e367" dependencies = [ "prettyplease", "proc-macro2", - "prost-build 0.11.2", + "prost-build 0.11.3", "quote", "syn", ] @@ -8869,7 +8827,7 @@ dependencies = [ "bytesize", "chrono", "cidr-utils", - "clap 4.0.27", + "clap 4.0.29", "codecs", "colored", "console-subscriber", @@ -8946,8 +8904,8 @@ dependencies = [ "postgres-openssl", "prometheus-parser", "proptest", - "prost 0.11.2", - "prost-build 0.11.2", + "prost 0.11.3", + "prost-build 0.11.3", "prost-types 0.11.2", "pulsar", "quickcheck", @@ -9028,7 +8986,7 @@ dependencies = [ "anyhow", "async-trait", "chrono", - "clap 4.0.27", + "clap 4.0.29", "futures 0.3.25", "graphql_client", "indoc", @@ -9051,7 +9009,7 @@ dependencies = [ "async-trait", "bytecheck", "bytes 1.3.0", - "clap 4.0.27", + "clap 4.0.29", "crc32fast", "criterion", "crossbeam-queue", @@ -9208,8 +9166,8 @@ dependencies = [ "parking_lot", "pin-project", "proptest", - "prost 0.11.2", - "prost-build 0.11.2", + "prost 0.11.3", + "prost-build 0.11.3", "prost-types 0.11.2", "quanta", "quickcheck", @@ -9293,7 +9251,7 @@ dependencies = [ name = "vrl-cli" version = "0.1.0" dependencies = [ - "clap 4.0.27", + "clap 4.0.29", "exitcode", "indoc", "lookup", @@ -9436,7 +9394,7 @@ dependencies = [ "ansi_term", "chrono", "chrono-tz", - "clap 4.0.27", + "clap 4.0.29", "enrichment", "glob", "lookup", From c151b06617c82dbc58d4fa8e05b402cb29a7acec Mon Sep 17 00:00:00 2001 From: kyle criddle Date: Tue, 29 Nov 2022 10:06:49 -0700 Subject: [PATCH 17/48] generate-component-docs --- .../components/sinks/base/pulsar.cue | 178 +++++++++++++++++- 1 file changed, 175 insertions(+), 3 deletions(-) diff --git a/website/cue/reference/components/sinks/base/pulsar.cue b/website/cue/reference/components/sinks/base/pulsar.cue index 731747769d626..37103ff09d188 100644 --- a/website/cue/reference/components/sinks/base/pulsar.cue +++ b/website/cue/reference/components/sinks/base/pulsar.cue @@ -83,6 +83,42 @@ base: components: sinks: pulsar: configuration: { } } } + batch_size: { + description: """ + Determines the batch size. + + Defaults to 1000. + """ + required: false + type: uint: {} + } + compression: { + description: "Identifies the compression options that are available within Pulsar." + required: false + type: string: enum: { + Lz4: """ + [LZ4][lz4] compression. + + [lz4]: https://lz4.github.io/lz4/ + """ + None: "No compression." + Snappy: """ + [Snappy][snappy] compression. + + [snappy]: https://google.github.io/snappy/ + """ + Zlib: """ + [Zlib][zlib] compression. + + [zlib]: https://www.zlib.net + """ + Zstd: """ + [Zstd][zstd] compression. + + [zstd]: https://zstd.net + """ + } + } encoding: { description: "Encoding configuration." required: true @@ -149,11 +185,147 @@ base: components: sinks: pulsar: configuration: { required: true type: string: syntax: "literal" } - partition_key_field: { - description: "Log field to use as Pulsar message key" - required: false + key_field: { + description: """ + The log field name or tags key to use for the topic key. + + If the field does not exist in the log or in tags, a blank value will be used. If unspecified, the key is not sent. + + Pulsar uses a hash of the key to choose the topic-partition or uses round-robin if the record has no key. + """ + required: false + type: string: syntax: "literal" + } + properties_key: { + description: """ + The log field name to use for the Pulsar properties. + + If omitted, no properties will be written. + """ + required: false type: string: syntax: "literal" } + request: { + description: """ + Middleware settings for outbound requests. + + Various settings can be configured, such as concurrency and rate limits, timeouts, etc. + """ + required: false + type: object: options: { + adaptive_concurrency: { + description: """ + Configuration of adaptive concurrency parameters. + + These parameters typically do not require changes from the default, and incorrect values can lead to meta-stable or + unstable performance and sink behavior. Proceed with caution. + """ + required: false + type: object: { + default: { + decrease_ratio: 0.9 + ewma_alpha: 0.4 + rtt_deviation_scale: 2.5 + } + options: { + decrease_ratio: { + description: """ + The fraction of the current value to set the new concurrency limit when decreasing the limit. + + Valid values are greater than `0` and less than `1`. Smaller values cause the algorithm to scale back rapidly + when latency increases. + + Note that the new limit is rounded down after applying this ratio. + """ + required: false + type: float: default: 0.9 + } + ewma_alpha: { + description: """ + The weighting of new measurements compared to older measurements. + + Valid values are greater than `0` and less than `1`. + + ARC uses an exponentially weighted moving average (EWMA) of past RTT measurements as a reference to compare with + the current RTT. Smaller values cause this reference to adjust more slowly, which may be useful if a service has + unusually high response variability. + """ + required: false + type: float: default: 0.4 + } + rtt_deviation_scale: { + description: """ + Scale of RTT deviations which are not considered anomalous. + + Valid values are greater than or equal to `0`, and we expect reasonable values to range from `1.0` to `3.0`. + + When calculating the past RTT average, we also compute a secondary “deviation” value that indicates how variable + those values are. We use that deviation when comparing the past RTT average to the current measurements, so we + can ignore increases in RTT that are within an expected range. This factor is used to scale up the deviation to + an appropriate range. Larger values cause the algorithm to ignore larger increases in the RTT. + """ + required: false + type: float: default: 2.5 + } + } + } + } + concurrency: { + description: "Configuration for outbound request concurrency." + required: false + type: { + number: {} + string: { + const: "adaptive" + default: "none" + } + } + } + rate_limit_duration_secs: { + description: "The time window, in seconds, used for the `rate_limit_num` option." + required: false + type: uint: default: 1 + } + rate_limit_num: { + description: "The maximum number of requests allowed within the `rate_limit_duration_secs` time window." + required: false + type: uint: default: 9223372036854775807 + } + retry_attempts: { + description: """ + The maximum number of retries to make for failed requests. + + The default, for all intents and purposes, represents an infinite number of retries. + """ + required: false + type: uint: default: 9223372036854775807 + } + retry_initial_backoff_secs: { + description: """ + The amount of time to wait before attempting the first retry for a failed request. + + After the first retry has failed, the fibonacci sequence will be used to select future backoffs. + """ + required: false + type: uint: default: 1 + } + retry_max_duration_secs: { + description: "The maximum amount of time, in seconds, to wait between retries." + required: false + type: uint: default: 3600 + } + timeout_secs: { + description: """ + The maximum time a request can take before being aborted. + + It is highly recommended that you do not lower this value below the service’s internal timeout, as this could + create orphaned requests, pile on retries, and result in duplicate data downstream. + """ + required: false + type: uint: default: 60 + } + } + } topic: { description: "The Pulsar topic name to write events to." required: true From af47c41b0323fadf67d93670e503b9a1a3e63ed7 Mon Sep 17 00:00:00 2001 From: kyle criddle Date: Fri, 9 Dec 2022 10:08:05 -0700 Subject: [PATCH 18/48] changes to adapt to pulsar crate 5.0.0 --- Cargo.lock | 182 +++++++++++++++++++++---------------- Cargo.toml | 2 +- src/sinks/pulsar/config.rs | 31 +++++-- 3 files changed, 127 insertions(+), 88 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 291558efb49db..cc5ccacfef0c2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -247,9 +247,9 @@ dependencies = [ [[package]] name = "assert_cmd" -version = "2.0.6" +version = "2.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba45b8163c49ab5f972e59a8a5a03b6d2972619d486e19ec9fe744f7c2753d3c" +checksum = "fa3d466004a8b4cb1bc34044240a2fd29d17607e2e3bd613eb44fd48e8100da3" dependencies = [ "bstr 1.0.1", "doc-comment", @@ -1644,9 +1644,9 @@ dependencies = [ [[package]] name = "chrono-tz" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a87b30366b6766751277791b473b674f3bf7fb75696841c784a3eb7e7fbf44ee" +checksum = "fa48fa079165080f11d7753fd0bc175b7d391f276b965fe4b55bfad67856e463" dependencies = [ "chrono", "chrono-tz-build", @@ -2226,9 +2226,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.82" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4a41a86530d0fe7f5d9ea779916b7cadd2d4f9add748b99c2c029cbbdfaf453" +checksum = "bdf07d07d6531bfcdbe9b8b739b104610c6508dcc4d63b410585faf338241daf" dependencies = [ "cc", "cxxbridge-flags", @@ -2238,9 +2238,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.82" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06416d667ff3e3ad2df1cd8cd8afae5da26cf9cec4d0825040f88b5ca659a2f0" +checksum = "d2eb5b96ecdc99f72657332953d4d9c50135af1bac34277801cc3937906ebd39" dependencies = [ "cc", "codespan-reporting", @@ -2253,15 +2253,15 @@ dependencies = [ [[package]] name = "cxxbridge-flags" -version = "1.0.82" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "820a9a2af1669deeef27cb271f476ffd196a2c4b6731336011e0ba63e2c7cf71" +checksum = "ac040a39517fd1674e0f32177648334b0f4074625b5588a64519804ba0553b12" [[package]] name = "cxxbridge-macro" -version = "1.0.82" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a08a6e2fcc370a089ad3b4aaf54db3b1b4cee38ddabce5896b33eb693275f470" +checksum = "1362b0ddcfc4eb0a1f57b68bd77dd99f0e826958a96abd0ae9bd092e114ffed6" dependencies = [ "proc-macro2", "quote", @@ -2353,9 +2353,9 @@ dependencies = [ [[package]] name = "data-encoding" -version = "2.3.2" +version = "2.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ee2393c4a91429dffb4bedf19f4d6abf27d8a732c8ce4980305d782e5426d57" +checksum = "23d8666cb01533c39dde32bcbab8e227b4ed6679b2c925eba05feabea39508fb" [[package]] name = "data-url" @@ -2462,9 +2462,9 @@ dependencies = [ [[package]] name = "derive_arbitrary" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4903dff04948f22033ca30232ab8eca2c3fc4c913a8b6a34ee5199699814817f" +checksum = "f8a16495aeb28047bb1185fca837baf755e7d71ed3aeed7f8504654ffa927208" dependencies = [ "proc-macro2", "quote", @@ -2919,9 +2919,9 @@ dependencies = [ [[package]] name = "filetime" -version = "0.2.18" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b9663d381d07ae25dc88dbdf27df458faa83a9b25336bcac83d5e452b5fc9d3" +checksum = "4e884668cd0c7480504233e951174ddc3b382f7c2666e3b7310b5c4e7b0c37f9" dependencies = [ "cfg-if", "libc", @@ -3241,9 +3241,9 @@ dependencies = [ [[package]] name = "governor" -version = "0.5.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de1b4626e87b9eb1d603ed23067ba1e29ec1d0b35325a2b96c3fe1cf20871f56" +checksum = "c390a940a5d157878dd057c78680a33ce3415bcd05b4799509ea44210914b4d5" dependencies = [ "cfg-if", "dashmap", @@ -3452,7 +3452,7 @@ dependencies = [ "lazy_static", "libc", "mach", - "nix 0.23.1", + "nix 0.23.2", "pin-utils", "uom", "winapi", @@ -3534,7 +3534,7 @@ dependencies = [ "heim-runtime", "libc", "macaddr", - "nix 0.23.1", + "nix 0.23.2", "widestring 0.4.3", "winapi", ] @@ -3722,9 +3722,9 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.1" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59df7c4e19c950e6e0e868dcc0a300b09a9b88e9ec55bd879ca819087a77355d" +checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" dependencies = [ "http", "hyper", @@ -3946,9 +3946,9 @@ dependencies = [ [[package]] name = "ipnet" -version = "2.5.1" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f88c5561171189e69df9d98bcf18fd5f9558300f7ea7b801eb8a0fd748bd8745" +checksum = "ec947b7a4ce12e3b87e353abae7ce124d025b6c7d6c5aea5cc0bcf92e9510ded" [[package]] name = "ipnetwork" @@ -4038,9 +4038,9 @@ checksum = "078e285eafdfb6c4b434e0d31e8cfcb5115b651496faca5749b88fafd4f23bfd" [[package]] name = "json-patch" -version = "0.2.6" +version = "0.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f995a3c8f2bc3dd52a18a583e90f9ec109c047fa1603a853e46bcda14d2e279d" +checksum = "eb3fa5a61630976fc4c353c70297f2e93f1930e3ccee574d59d618ccbd5154ce" dependencies = [ "serde", "serde_json", @@ -4314,9 +4314,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.137" +version = "0.2.138" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc7fcc620a3bff7cdd7a365be3376c97191aeaccc2a603e600951e452615bf89" +checksum = "db6d7e329c562c5dfab7a46a2afabc8b987ab9a4834c9d1ca04dc54c1546cef8" [[package]] name = "libflate" @@ -4492,6 +4492,26 @@ dependencies = [ "cc", ] +[[package]] +name = "lz4" +version = "1.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" +dependencies = [ + "libc", + "lz4-sys", +] + +[[package]] +name = "lz4-sys" +version = "1.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" +dependencies = [ + "cc", + "libc", +] + [[package]] name = "macaddr" version = "1.0.1" @@ -4926,9 +4946,9 @@ dependencies = [ [[package]] name = "nix" -version = "0.23.1" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f866317acbd3a240710c63f065ffb1e4fd466259045ccb504130b7f668f35c6" +checksum = "8f3790c00a0150112de0f4cd161e3d7fc4b2d8a5542ffc35f099a2562aecb35c" dependencies = [ "bitflags", "cc", @@ -4939,9 +4959,9 @@ dependencies = [ [[package]] name = "nix" -version = "0.24.2" +version = "0.24.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "195cdbc1741b8134346d515b3a56a1c94b0912758009cfd53f99ea0f57b065fc" +checksum = "fa52e972a9a719cecb6864fb88568781eb706bac2cd1d4f04a648542dbf78069" dependencies = [ "bitflags", "cfg-if", @@ -4950,9 +4970,9 @@ dependencies = [ [[package]] name = "nix" -version = "0.25.0" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e322c04a9e3440c327fca7b6c8a63e6890a32fa2ad689db972425f07e0d22abb" +checksum = "f346ff70e7dbfd675fe90590b92d59ef2de15a8779ae305ebcbfd3f0caf59be4" dependencies = [ "autocfg", "bitflags", @@ -5096,9 +5116,9 @@ dependencies = [ [[package]] name = "num-format" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54b862ff8df690cf089058c98b183676a7ed0f974cc08b426800093227cbff3b" +checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" dependencies = [ "arrayvec 0.7.2", "itoa 1.0.4", @@ -5286,9 +5306,9 @@ dependencies = [ [[package]] name = "openssl" -version = "0.10.43" +version = "0.10.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "020433887e44c27ff16365eaa2d380547a94544ad509aff6eb5b6e3e0b27b376" +checksum = "29d971fd5722fec23977260f6e81aa67d2f22cadbdc2aa049f1022d9a3be1566" dependencies = [ "bitflags", "cfg-if", @@ -5326,9 +5346,9 @@ dependencies = [ [[package]] name = "openssl-sys" -version = "0.9.78" +version = "0.9.79" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07d5c8cb6e57b3a3612064d7b18b117912b4ce70955c2504d4b741c9e244b132" +checksum = "5454462c0eced1e97f2ec09036abc8da362e66802f66fd20f86854d9d8cbcbc4" dependencies = [ "autocfg", "cc", @@ -5489,9 +5509,9 @@ checksum = "478c572c3d73181ff3c2539045f6eb99e5491218eae919370993b890cdbdd98e" [[package]] name = "pest" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f400b0f7905bf702f9f3dc3df5a121b16c54e9e8012c082905fdf09a931861a" +checksum = "cc8bed3549e0f9b0a2a78bf7c0018237a2cdf085eecbbc048e52612438e4e9d0" dependencies = [ "thiserror", "ucd-trie", @@ -5499,9 +5519,9 @@ dependencies = [ [[package]] name = "pest_derive" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "423c2ba011d6e27b02b482a3707c773d19aec65cc024637aec44e19652e66f63" +checksum = "cdc078600d06ff90d4ed238f0119d84ab5d43dbaad278b0e33a8820293b32344" dependencies = [ "pest", "pest_generator", @@ -5509,9 +5529,9 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e64e6c2c85031c02fdbd9e5c72845445ca0a724d419aa0bc068ac620c9935c1" +checksum = "28a1af60b1c4148bb269006a750cff8e2ea36aff34d2d96cf7be0b14d1bed23c" dependencies = [ "pest", "pest_meta", @@ -5522,9 +5542,9 @@ dependencies = [ [[package]] name = "pest_meta" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57959b91f0a133f89a68be874a5c88ed689c19cd729ecdb5d762ebf16c64d662" +checksum = "fec8605d59fc2ae0c6c1aefc0c7c7a9769732017c0ce07f7a9cfffa7b4404f20" dependencies = [ "once_cell", "pest", @@ -5706,9 +5726,9 @@ dependencies = [ [[package]] name = "portable-atomic" -version = "0.3.15" +version = "0.3.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15eb2c6e362923af47e13c23ca5afb859e83d54452c55b0b9ac763b8f7c1ac16" +checksum = "ac662b3a6490de378b0ee15cf2dfff7127aebfe0b19acc65e7fbca3d299c3788" [[package]] name = "portpicker" @@ -5774,9 +5794,9 @@ checksum = "925383efa346730478fb4838dbe9137d2a47675ad789c546d150a6e1dd4ab31c" [[package]] name = "predicates" -version = "2.1.3" +version = "2.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed6bd09a7f7e68f3f0bf710fb7ab9c4615a488b58b5f653382a687701e458c92" +checksum = "f54fc5dc63ed3bbf19494623db4f3af16842c0d975818e469022d09e53f0aa05" dependencies = [ "difflib", "itertools", @@ -6095,10 +6115,12 @@ dependencies = [ "chrono", "crc", "data-url", + "flate2", "futures 0.3.25", "futures-io", "futures-timer", "log", + "lz4", "native-tls", "nom", "oauth2", @@ -6111,11 +6133,13 @@ dependencies = [ "regex", "serde", "serde_json", + "snap", "tokio", "tokio-native-tls", "tokio-util", "url", "uuid 1.2.2", + "zstd", ] [[package]] @@ -6179,9 +6203,9 @@ dependencies = [ [[package]] name = "quoted_printable" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fee2dce59f7a43418e3382c766554c614e06a552d53a8f07ef499ea4b332c0f" +checksum = "20f14e071918cbeefc5edc986a7aa92c425dae244e003a35e1cdddb5ca39b5cb" [[package]] name = "radix_trie" @@ -6704,9 +6728,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.36.4" +version = "0.36.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb93e85278e08bb5788653183213d3a60fc242b10cb9be96586f5a73dcb67c23" +checksum = "a3807b5d10909833d3e9acd1eb5fb988f79376ff10fce42937de71a449c4c588" dependencies = [ "bitflags", "errno", @@ -6823,7 +6847,7 @@ dependencies = [ "libc", "log", "memchr", - "nix 0.24.2", + "nix 0.24.3", "scopeguard", "unicode-segmentation", "unicode-width", @@ -7006,9 +7030,9 @@ checksum = "388a1df253eca08550bef6c72392cfe7c30914bf41df5269b68cbd6ff8f570a3" [[package]] name = "serde" -version = "1.0.148" +version = "1.0.149" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e53f64bb4ba0191d6d0676e1b141ca55047d83b74f5607e6d8eb88126c52c2dc" +checksum = "256b9932320c590e707b94576e3cc1f7c9024d0ee6612dfbcf1cb106cbe8e055" dependencies = [ "serde_derive", ] @@ -7066,9 +7090,9 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.148" +version = "1.0.149" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a55492425aa53521babf6137309e7d34c20bbfbbfcfe2c7f3a047fd1f6b92c0c" +checksum = "b4eae9b04cbffdfd550eb462ed33bc6a1b68c935127d008b27444d08380f94e4" dependencies = [ "proc-macro2", "quote", @@ -7615,9 +7639,9 @@ checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" [[package]] name = "syn" -version = "1.0.104" +version = "1.0.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ae548ec36cf198c0ef7710d3c230987c2d6d7bd98ad6edc0274462724c585ce" +checksum = "60b9b43d45702de4c839cb9b51d9f529c5dd26a4aff255b42b1ebc03e88ee908" dependencies = [ "proc-macro2", "quote", @@ -7909,9 +7933,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.22.0" +version = "1.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d76ce4a75fb488c605c54bf610f221cea8b0dafb53333c1a67e8ee199dcd2ae3" +checksum = "eab6d665857cc6ca78d6e80303a02cea7a7851e85dfbd77cbdc09bd129f1ef46" dependencies = [ "autocfg", "bytes 1.3.0", @@ -7925,7 +7949,7 @@ dependencies = [ "socket2", "tokio-macros", "tracing 0.1.37", - "winapi", + "windows-sys 0.42.0", ] [[package]] @@ -7951,9 +7975,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ec8ecbbb74d42a8dd0ce8ea9bfc38ad13e2ed5203c4f272dbe144f4e17d70ac" +checksum = "d266c00fde287f55d3f1c3e96c500c362a2b8c695076ec180f27918820bc6df8" dependencies = [ "proc-macro2", "quote", @@ -8128,9 +8152,9 @@ dependencies = [ [[package]] name = "tonic-build" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31fa2c5e870bdce133847d15e075333e6e1ca3fff913001fede6754f3060e367" +checksum = "5bf5e9b9c0f7e0a7c027dcfaba7b2c60816c7049171f679d99ee2ff65d0de8c4" dependencies = [ "prettyplease", "proc-macro2", @@ -8161,9 +8185,9 @@ dependencies = [ [[package]] name = "tower-http" -version = "0.3.4" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c530c8675c1dbf98facee631536fa116b5fb6382d7dd6dc1b118d970eafe3ba" +checksum = "f873044bf02dd1e8239e9c1293ea39dad76dc594ec16185d0a1bf31d8dc8d858" dependencies = [ "base64", "bitflags", @@ -8593,9 +8617,9 @@ dependencies = [ [[package]] name = "typenum" -version = "1.15.0" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dcf81ac59edc17cc8697ff311e8f5ef2d99fcbd9817b34cec66f90b6c3dfd987" +checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba" [[package]] name = "typetag" @@ -8917,7 +8941,7 @@ dependencies = [ "mlua", "mongodb", "nats", - "nix 0.25.0", + "nix 0.25.1", "nkeys", "nom", "notify", @@ -9957,9 +9981,9 @@ dependencies = [ [[package]] name = "zeroize_derive" -version = "1.3.2" +version = "1.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f8f187641dad4f680d25c4bfc4225b418165984179f26ca76ec4fb6441d3a17" +checksum = "44bf07cb3e50ea2003396695d58bf46bc9887a1f362260446fad6bc4e79bd36c" dependencies = [ "proc-macro2", "quote", diff --git a/Cargo.toml b/Cargo.toml index 3acd3fe525b03..47ee0eca22b6b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -285,7 +285,7 @@ ordered-float = { version = "3.4.0", default-features = false } percent-encoding = { version = "2.2.0", default-features = false } pin-project = { version = "1.0.12", default-features = false } postgres-openssl = { version = "0.5.0", default-features = false, features = ["runtime"], optional = true } -pulsar = { version = "5.0.0", default-features = false, features = ["tokio-runtime", "auth-oauth2"], optional = true } +pulsar = { version = "5.0.0", default-features = false, features = ["tokio-runtime", "auth-oauth2", "lz4", "flate2", "zstd", "snap"], optional = true } rand = { version = "0.8.5", default-features = false, features = ["small_rng"] } rand_distr = { version = "0.4.3", default-features = false } rdkafka = { version = "0.29.0", default-features = false, features = ["tokio", "libz", "ssl", "zstd"], optional = true } diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 62ae8c82f1b20..053f7a5882d44 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -12,7 +12,8 @@ use futures_util::FutureExt; use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; use pulsar::error::AuthenticationError; use pulsar::{ - message::proto, Authentication, Error as PulsarError, ProducerOptions, Pulsar, TokioExecutor, + compression, message::proto, Authentication, Error as PulsarError, ProducerOptions, Pulsar, + TokioExecutor, }; use snafu::ResultExt; use vector_config::configurable_component; @@ -179,14 +180,28 @@ impl PulsarSinkConfig { batch_size: None, compression: None, }; - if let Some(compression) = &self.compression { - match compression { - PulsarCompression::None => opts.compression = Some(proto::CompressionType::None), - PulsarCompression::Lz4 => opts.compression = Some(proto::CompressionType::Lz4), - PulsarCompression::Zlib => opts.compression = Some(proto::CompressionType::Zlib), - PulsarCompression::Zstd => opts.compression = Some(proto::CompressionType::Zstd), + if let Some(config_compression) = &self.compression { + match config_compression { + PulsarCompression::None => opts.compression = Some(compression::Compression::None), + PulsarCompression::Lz4 => { + opts.compression = Some(compression::Compression::Lz4( + compression::CompressionLz4::default(), + )) + } + PulsarCompression::Zlib => { + opts.compression = Some(compression::Compression::Zlib( + compression::CompressionZlib::default(), + )) + } + PulsarCompression::Zstd => { + opts.compression = Some(compression::Compression::Zstd( + compression::CompressionZstd::default(), + )) + } PulsarCompression::Snappy => { - opts.compression = Some(proto::CompressionType::Snappy) + opts.compression = Some(compression::Compression::Snappy( + compression::CompressionSnappy::default(), + )) } } } From 6b182e633c8293fc891fc59d6f78398fde793bb8 Mon Sep 17 00:00:00 2001 From: kyle criddle Date: Fri, 9 Dec 2022 10:52:28 -0700 Subject: [PATCH 19/48] remove unused module --- src/sinks/pulsar/util.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/sinks/pulsar/util.rs b/src/sinks/pulsar/util.rs index b6d5facc7b152..ddc34ed0ae1de 100644 --- a/src/sinks/pulsar/util.rs +++ b/src/sinks/pulsar/util.rs @@ -86,6 +86,3 @@ pub(super) fn get_properties( None }) } - -#[cfg(test)] -mod tests {} From 4463a795e540a18a93817c1ad38d100f59ace646 Mon Sep 17 00:00:00 2001 From: Kyle Criddle Date: Tue, 13 Dec 2022 16:59:29 -0700 Subject: [PATCH 20/48] restore Cargo.lock --- Cargo.lock | 3688 +++++++++++++++++++++++----------------------------- 1 file changed, 1626 insertions(+), 2062 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cc5ccacfef0c2..ec2f11e43967f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -32,11 +32,11 @@ checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" [[package]] name = "aes" -version = "0.8.2" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "433cfd6710c9986c576a25ca913c39d66a6474107b406f34f91d4a8923395241" +checksum = "bfe0133578c0986e1fe3dfcd4af1cc5b2dd6c3dbf534d69916ce16a2701d40ba" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "cipher", "cpufeatures", ] @@ -47,109 +47,34 @@ version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" dependencies = [ - "getrandom 0.2.8", - "once_cell", - "version_check", -] - -[[package]] -name = "ahash" -version = "0.8.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf6ccdb167abbf410dcb915cabd428929d7f6a04980b54a11f26a39f1c7f7107" -dependencies = [ - "cfg-if", - "getrandom 0.2.8", + "getrandom 0.2.6", "once_cell", "version_check", ] [[package]] name = "aho-corasick" -version = "0.7.20" +version = "0.7.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc936419f96fa211c1b9166887b38e5e40b19958e5b895be7c1f93adec7071ac" +checksum = "1e37cfd5e7657ada45f742d6e99ca5788580b5c529dc78faf11ece6dc702656f" dependencies = [ "memchr", ] -[[package]] -name = "amq-protocol" -version = "7.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acc7cad07d1b4533fcb46f0819a6126fa201fd0385469aba75e405424f3fe009" -dependencies = [ - "amq-protocol-tcp", - "amq-protocol-types", - "amq-protocol-uri", - "cookie-factory", - "nom", - "serde", -] - -[[package]] -name = "amq-protocol-tcp" -version = "7.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8b20aba8c35a0b885e1e978eff456ced925730a4e012e63e4ff89a1deb602b" -dependencies = [ - "amq-protocol-uri", - "tcp-stream", - "tracing 0.1.37", -] - -[[package]] -name = "amq-protocol-types" -version = "7.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e245e0e9083b6a6db5f8c10013074cb382266eb9e2a37204d19c651b8d3b8114" -dependencies = [ - "cookie-factory", - "nom", - "serde", - "serde_json", -] - -[[package]] -name = "amq-protocol-uri" -version = "7.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56987108bf48d2eb500cae8896cd9291564eedd8744776ecc5c3338a8b2ca5f8" -dependencies = [ - "amq-protocol-types", - "percent-encoding", - "url", -] - -[[package]] -name = "android_system_properties" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" -dependencies = [ - "libc", -] - -[[package]] -name = "anes" -version = "0.1.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" - [[package]] name = "ansi_term" version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d52a9bb7ec0cf484c551830a7ce27bd20d67eac647e1befb56b0be4ee39a55d2" dependencies = [ - "winapi", + "winapi 0.3.9", ] [[package]] name = "anyhow" -version = "1.0.66" +version = "1.0.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "216261ddc8289130e551ddcd5ce8a064710c0d064a4d2895c67151c92b5443f6" +checksum = "b9a8f622bcf6ff3df478e9deba3e03e4e04b300f8e6a139e192c05fa3490afc7" [[package]] name = "anymap" @@ -168,9 +93,9 @@ dependencies = [ [[package]] name = "arbitrary" -version = "1.2.0" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29d47fbf90d5149a107494b15a7dc8d69b351be2db3bb9691740e88ec17fd880" +checksum = "8931eb436ab9bf1980c6cb2b9d1ba5390cd6793b2c6e2d2ea8147da3570c2a2e" dependencies = [ "derive_arbitrary", ] @@ -182,32 +107,20 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "983cd8b9d4b02a6dc6ffa557262eb5858a27a0038ffffe21a0f133eaa819a164" [[package]] -name = "arr_macro" -version = "0.1.3" +name = "arrayref" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a105bfda48707cf19220129e78fca01e9639433ffaef4163546ed8fb04120a5" -dependencies = [ - "arr_macro_impl", - "proc-macro-hack", -] +checksum = "a4c527152e37cf757a3f78aae5a06fbeefdb07ccc535c980a3208ee3060dd544" [[package]] -name = "arr_macro_impl" -version = "0.1.3" +name = "arrayvec" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0609c78bd572f4edc74310dfb63a01f5609d53fa8b4dd7c4d98aef3b3e8d72d1" +checksum = "cd9fd44efafa8690358b7408d253adf110036b88f55672a933f01d616ad9b1b9" dependencies = [ - "proc-macro-hack", - "quote", - "syn", + "nodrop", ] -[[package]] -name = "arrayref" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4c527152e37cf757a3f78aae5a06fbeefdb07ccc535c980a3208ee3060dd544" - [[package]] name = "arrayvec" version = "0.5.2" @@ -237,9 +150,9 @@ dependencies = [ [[package]] name = "assert-json-diff" -version = "2.0.2" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "47e4f2b81832e72834d7518d8487a0396a28cc408186a2e8854c0f98011faf12" +checksum = "50f1c3703dd33532d7f0ca049168930e9099ecac238e23cf932f3a69c42f06da" dependencies = [ "serde", "serde_json", @@ -247,11 +160,11 @@ dependencies = [ [[package]] name = "assert_cmd" -version = "2.0.7" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa3d466004a8b4cb1bc34044240a2fd29d17607e2e3bd613eb44fd48e8100da3" +checksum = "93ae1ddd39efd67689deb1979d80bad3bf7f2b09c6e6117c8d1f2443b5e2f83e" dependencies = [ - "bstr 1.0.1", + "bstr", "doc-comment", "predicates", "predicates-core", @@ -261,9 +174,9 @@ dependencies = [ [[package]] name = "async-channel" -version = "1.8.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf46fee83e5ccffc220104713af3292ff9bc7c64c7de289f66dae8e38d826833" +checksum = "2114d64672151c0c5eaa5e131ec84a74f06e1e559830dabba01ca30605d66319" dependencies = [ "concurrent-queue", "event-listener", @@ -272,9 +185,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.3.15" +version = "0.3.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "942c7cd7ae39e91bde4820d74132e9862e62c2f386c3aa90ccf55949f5bad63a" +checksum = "f2bf394cfbbe876f0ac67b13b6ca819f9c9f2fb9ec67223cceb1555fbab1c31a" dependencies = [ "flate2", "futures-core", @@ -287,61 +200,34 @@ dependencies = [ [[package]] name = "async-executor" -version = "1.5.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17adb73da160dfb475c183343c8cccd80721ea5a605d3eb57125f0a7b7a92d0b" +checksum = "871f9bb5e0a22eeb7e8cf16641feb87c9dc67032ccf8ff49e772eb9941d3a965" dependencies = [ - "async-lock", "async-task", "concurrent-queue", "fastrand", "futures-lite", + "once_cell", "slab", ] [[package]] name = "async-fs" -version = "1.6.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "279cf904654eeebfa37ac9bb1598880884924aab82e290aa65c9e77a0e142e06" +checksum = "8b3ca4f8ff117c37c278a2f7415ce9be55560b846b5bc4412aaa5d29c1c3dae2" dependencies = [ "async-lock", - "autocfg", "blocking", "futures-lite", ] -[[package]] -name = "async-global-executor" -version = "2.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1b6f5d7df27bd294849f8eec66ecfc63d11814df7a4f5d74168a2394467b776" -dependencies = [ - "async-channel", - "async-executor", - "async-io", - "async-lock", - "blocking", - "futures-lite", - "once_cell", -] - -[[package]] -name = "async-global-executor-trait" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33dd14c5a15affd2abcff50d84efd4009ada28a860f01c14f9d654f3e81b3f75" -dependencies = [ - "async-global-executor", - "async-trait", - "executor-trait", -] - [[package]] name = "async-graphql" -version = "4.0.16" +version = "4.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9ed522678d412d77effe47b3c82314ac36952a35e6e852093dd48287c421f80" +checksum = "097754075ec057cf7d218886882026905e5f8be3a31572a2ffb1be1ef38136c8" dependencies = [ "async-graphql-derive", "async-graphql-parser", @@ -349,7 +235,7 @@ dependencies = [ "async-stream", "async-trait", "base64", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "fnv", "futures-util", @@ -363,7 +249,6 @@ dependencies = [ "regex", "serde", "serde_json", - "serde_urlencoded", "static_assertions", "tempfile", "thiserror", @@ -371,14 +256,14 @@ dependencies = [ [[package]] name = "async-graphql-derive" -version = "4.0.16" +version = "4.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c121a894495d7d3fc3d4e15e0a9843e422e4d1d9e3c514d8062a1c94b35b005d" +checksum = "e7fa9b19ad10364c364a46847c7cc869992a3f8515105c76fa4fcda543787336" dependencies = [ "Inflector", "async-graphql-parser", - "darling 0.14.2", - "proc-macro-crate 1.2.1", + "darling 0.14.1", + "proc-macro-crate", "proc-macro2", "quote", "syn", @@ -387,23 +272,25 @@ dependencies = [ [[package]] name = "async-graphql-parser" -version = "4.0.16" +version = "4.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b6c386f398145c6180206c1869c2279f5a3d45db5be4e0266148c6ac5c6ad68" +checksum = "52bedfd99a0ddc329585274774cd637271eef9b4d6148f3bbb50b0ae1373ecf3" dependencies = [ "async-graphql-value", "pest", + "pest_generator", + "proc-macro2", "serde", "serde_json", ] [[package]] name = "async-graphql-value" -version = "4.0.16" +version = "4.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a941b499fead4a3fb5392cabf42446566d18c86313f69f2deab69560394d65f" +checksum = "0f7b95ca40977a0b78089f20e80ecd34a564ed07d4e6a8641e85e652f7db0a12" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "indexmap", "serde", "serde_json", @@ -411,9 +298,9 @@ dependencies = [ [[package]] name = "async-graphql-warp" -version = "4.0.16" +version = "4.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c512820fb4259389aa5af696cb78297ef93d8d0f25f62007705de1aa7343253b" +checksum = "1bc6fd192f0f2c11fb6756101f8ae17172bebcfa7521e8e486b43b32551ec57c" dependencies = [ "async-graphql", "futures-util", @@ -423,74 +310,58 @@ dependencies = [ [[package]] name = "async-io" -version = "1.12.0" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c374dda1ed3e7d8f0d9ba58715f924862c63eae6849c92d3a18e7fbde9e2794" +checksum = "a811e6a479f2439f0c04038796b5cfb3d2ad56c230e0f2d3f7b04d68cfee607b" dependencies = [ - "async-lock", - "autocfg", "concurrent-queue", "futures-lite", "libc", "log", + "once_cell", "parking", "polling", "slab", "socket2", "waker-fn", - "windows-sys 0.42.0", + "winapi 0.3.9", ] [[package]] name = "async-lock" -version = "2.6.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8101efe8695a6c17e02911402145357e718ac92d3ff88ae8419e84b1707b685" +checksum = "e97a171d191782fba31bb902b14ad94e24a68145032b7eedf871ab0bc0d077b6" dependencies = [ "event-listener", - "futures-lite", ] [[package]] name = "async-net" -version = "1.7.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4051e67316bc7eff608fe723df5d32ed639946adcd69e07df41fd42a7b411f1f" +checksum = "5373304df79b9b4395068fb080369ec7178608827306ce4d081cba51cac551df" dependencies = [ "async-io", - "autocfg", "blocking", "futures-lite", ] [[package]] name = "async-process" -version = "1.6.0" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6381ead98388605d0d9ff86371043b5aa922a3905824244de40dc263a14fcba4" +checksum = "83137067e3a2a6a06d67168e49e68a0957d215410473a740cea95a2425c0b7c6" dependencies = [ "async-io", - "async-lock", - "autocfg", "blocking", - "cfg-if", + "cfg-if 1.0.0", "event-listener", "futures-lite", "libc", + "once_cell", "signal-hook", - "windows-sys 0.42.0", -] - -[[package]] -name = "async-reactor-trait" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a6012d170ad00de56c9ee354aef2e358359deb1ec504254e0e5a3774771de0e" -dependencies = [ - "async-io", - "async-trait", - "futures-core", - "reactor-trait", + "winapi 0.3.9", ] [[package]] @@ -527,15 +398,27 @@ dependencies = [ [[package]] name = "async-task" -version = "4.3.0" +version = "4.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a40729d2133846d9ed0ea60a8b9541bccddab49cd30f0715a1da672fe9a2524" +checksum = "30696a84d817107fc028e049980e09d5e140e8da8f1caeb17e8e950658a3cea9" + +[[package]] +name = "async-timer" +version = "1.0.0-beta.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "faacdfdccd10db54656717fddcd1a2ab6cd1ab16c0d6e7d89ec365b885fc9844" +dependencies = [ + "error-code", + "libc", + "wasm-bindgen", + "winapi 0.3.9", +] [[package]] name = "async-trait" -version = "0.1.59" +version = "0.1.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31e6e93155431f3931513b243d371981bb2770112b370c82745a1d19d2f99364" +checksum = "76464446b8bc32758d7e88ee1a804d9914cd9b1cb264c029899680b0be29826f" dependencies = [ "proc-macro2", "quote", @@ -554,9 +437,9 @@ version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" dependencies = [ - "hermit-abi 0.1.19", + "hermit-abi", "libc", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -589,9 +472,9 @@ dependencies = [ [[package]] name = "aws-config" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56a636c44c77fa18bdba56126a34d30cfe5538fe88f7d34988fa731fee143ddd" +checksum = "7d4cf4608abd7c8038a4c609a1270e61b73c86550f5655654ca28322e0a2e2c1" dependencies = [ "aws-http", "aws-sdk-sso", @@ -603,7 +486,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "hex", "http", "hyper", @@ -611,47 +494,47 @@ dependencies = [ "time", "tokio", "tower", - "tracing 0.1.37", + "tracing 0.1.34", "zeroize", ] [[package]] name = "aws-endpoint" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ca8f374874f6459aaa88dc861d7f5d834ca1ff97668eae190e97266b5f6c3fb" +checksum = "7ffaf1da7a11d38a5afe7cdd202ab2e25528de7cf38c47b571c0dde4008d98ae" dependencies = [ "aws-smithy-http", "aws-smithy-types", "aws-types", "http", "regex", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-http" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78d41e19e779b73463f5f0c21b3aacc995f4ba783ab13a7ae9f5dfb159a551b4" +checksum = "8309108743e2e74f249ff29a7c7be79c6343ea649dd8c31e4c0e07ca6946d8ed" dependencies = [ "aws-smithy-http", "aws-smithy-types", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "http-body", "lazy_static", "percent-encoding", "pin-project-lite", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-sdk-cloudwatch" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "520b1ac14f0850d0d6a69136d15ba7702d41ee7f4014a5d2d1bf4a86e74f7a6b" +checksum = "08ca16408172d748ece9e5a4cac14fe41b92013e532df7b21dacd03fb26f9cc5" dependencies = [ "aws-endpoint", "aws-http", @@ -664,7 +547,7 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tokio-stream", "tower", @@ -672,9 +555,9 @@ dependencies = [ [[package]] name = "aws-sdk-cloudwatchlogs" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89415e55b57044a09a7eb0a885c2d0af1aa7f95b373e0e898f71a28d7e7d10f9" +checksum = "c6ac6c1554a99b275931e2f923ec0e3370127488bc365445c530e687fa3ada2d" dependencies = [ "aws-endpoint", "aws-http", @@ -686,7 +569,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tokio-stream", "tower", @@ -694,9 +577,9 @@ dependencies = [ [[package]] name = "aws-sdk-elasticsearch" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9f4cc10278701dbc0d386ddd8cddfda2695eae7103a54eae11b981f28779ff2" +checksum = "a05dcdc71560f9e2885d66099c04532651e9646cfca225eb97d50bb14db88ced" dependencies = [ "aws-endpoint", "aws-http", @@ -708,7 +591,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tokio-stream", "tower", @@ -716,9 +599,9 @@ dependencies = [ [[package]] name = "aws-sdk-firehose" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c68310f9d7860b4fe73c58e5cec4d7a310a658d1a983fdf176eb35149939896a" +checksum = "7e57a640e35af65b906161060cd6f951aa014e2858bb25f64ea85eb4b1c3e0a9" dependencies = [ "aws-endpoint", "aws-http", @@ -730,16 +613,16 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tower", ] [[package]] name = "aws-sdk-kinesis" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37766fdf50feab317b4f939b1c9ee58a2a1c51785974328ce84cff1eea7a1bb8" +checksum = "c2dc213d616547cf0c47a735a40e6162252d559e1745f65212b0b58b8c161545" dependencies = [ "aws-endpoint", "aws-http", @@ -751,7 +634,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tokio-stream", "tower", @@ -759,9 +642,9 @@ dependencies = [ [[package]] name = "aws-sdk-s3" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9f08665c8e03aca8cb092ef01e617436ebfa977fddc1240e1b062488ab5d48a" +checksum = "323b9107094fc396a0116326b577af48d9cfb26ec7c09588584ec82cee057b81" dependencies = [ "aws-endpoint", "aws-http", @@ -776,20 +659,20 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "bytes-utils", "http", "http-body", "tokio-stream", "tower", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-sdk-sqs" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b26bb3d12238492cb12bde0de8486679b007daada21fdb110913b32a2a38275" +checksum = "098bf8f363dd36a892b8ce8f013a2d57c15de869647e9da9a10fe4e9dcbf318b" dependencies = [ "aws-endpoint", "aws-http", @@ -802,7 +685,7 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tokio-stream", "tower", @@ -810,9 +693,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86dcb1cb71aa8763b327542ead410424515cff0cde5b753eedd2917e09c63734" +checksum = "f7a0659e5269f8c4bd06f362ec7e35b4f55956c4d60e0ca177b575db80584a45" dependencies = [ "aws-endpoint", "aws-http", @@ -824,7 +707,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tokio-stream", "tower", @@ -832,9 +715,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "0.21.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdfcf584297c666f6b472d5368a78de3bc714b6e0a53d7fbf76c3e347c292ab1" +checksum = "edc795c7851c0e9bcefde5e6bb610c16a9e03220e0336fc12f75bb80d9ce7e80" dependencies = [ "aws-endpoint", "aws-http", @@ -847,34 +730,34 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.3.0", + "bytes 1.2.1", "http", "tower", ] [[package]] name = "aws-sig-auth" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12cbe7b2be9e185c1fbce27fc9c41c66b195b32d89aa099f98768d9544221308" +checksum = "0ee4bf20136757fd9f606bb4adafe6d19fb02bc48033a8d4f205f21d56fa783a" dependencies = [ "aws-sigv4", "aws-smithy-eventstream", "aws-smithy-http", "aws-types", "http", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-sigv4" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03ff4cff8c4a101962d593ba94e72cd83891aecd423f0c6e3146bff6fb92c9e3" +checksum = "b99b21b3aceaf224cccd693b353e1f38af4ede8c5fc618b97dd458bb63238efc" dependencies = [ "aws-smithy-eventstream", "aws-smithy-http", - "bytes 1.3.0", + "bytes 1.2.1", "form_urlencoded", "hex", "http", @@ -883,14 +766,14 @@ dependencies = [ "regex", "ring", "time", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-smithy-async" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b3442b4c5d3fc39891a2e5e625735fba6b24694887d49c6518460fde98247a9" +checksum = "ef79062cf5fa881dd156938ca438ec2de0f7ec9342c2f84fa6303274e1484b43" dependencies = [ "futures-util", "pin-project-lite", @@ -900,13 +783,13 @@ dependencies = [ [[package]] name = "aws-smithy-checksums" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc227e36e346f45298288359f37123e1a92628d1cec6b11b5eb335553278bd9e" +checksum = "d6face4c12b335ba734a4416c15d5eeb0af88aa61182a84ff50db62bfa261183" dependencies = [ "aws-smithy-http", "aws-smithy-types", - "bytes 1.3.0", + "bytes 1.2.1", "crc32c", "crc32fast", "hex", @@ -915,53 +798,53 @@ dependencies = [ "md-5", "pin-project-lite", "sha1", - "sha2 0.10.6", - "tracing 0.1.37", + "sha2 0.10.5", + "tracing 0.1.34", ] [[package]] name = "aws-smithy-client" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff28d553714f8f54cd921227934fc13a536a1c03f106e56b362fd57e16d450ad" +checksum = "13f402fa9a45353f7f02f8046a6a568143844d201c5b4cc3bedb6442058538c8" dependencies = [ "aws-smithy-async", "aws-smithy-http", "aws-smithy-http-tower", "aws-smithy-types", - "bytes 1.3.0", + "bytes 1.2.1", "fastrand", "http", "http-body", "hyper", - "hyper-rustls", + "hyper-rustls 0.22.1", "lazy_static", "pin-project-lite", "tokio", "tower", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-smithy-eventstream" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7ea0df7161ce65b5c8ca6eb709a1a907376fa18226976e41c748ce02ccccf24" +checksum = "9b959c2c1752c2afbd863953046c06f7ee592f68d64719b7bab3193ac3b0fa77" dependencies = [ "aws-smithy-types", - "bytes 1.3.0", + "bytes 1.2.1", "crc32fast", ] [[package]] name = "aws-smithy-http" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf58ed4fefa61dbf038e5421a521cbc2c448ef69deff0ab1d915d8a10eda5664" +checksum = "23861d0b53a1369eab1e8d48c8bb3492eb3def1c2f2222dfb1bad58dd03914a5" dependencies = [ "aws-smithy-eventstream", "aws-smithy-types", - "bytes 1.3.0", + "bytes 1.2.1", "bytes-utils", "futures-core", "http", @@ -970,39 +853,38 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "pin-utils", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-smithy-http-tower" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20c96d7bd35e7cf96aca1134b2f81b1b59ffe493f7c6539c051791cbbf7a42d3" +checksum = "04f6b3ae42d5c52bbaadfdd31c09fd11c92b823d329915dedbb08c0e9525755c" dependencies = [ "aws-smithy-http", - "bytes 1.3.0", + "bytes 1.2.1", "http", "http-body", "pin-project-lite", "tower", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "aws-smithy-json" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8324ba98c8a94187723cc16c37aefa09504646ee65c3d2c3af495bab5ea701b" +checksum = "5048b693643803c001f88fad36c5a7aa1159e56b0025527fadc57e830aa48b11" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-query" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83834ed2ff69ea6f6657baf205267dc2c0abe940703503a3e5d60ce23be3d306" +checksum = "b317cd3b326444e659a2f287f67e8c72903495c71a3473b0764880454b3aa25c" dependencies = [ "aws-smithy-types", "urlencoding", @@ -1010,11 +892,11 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b02e06ea63498c43bc0217ea4d16605d4e58d85c12fc23f6572ff6d0a840c61" +checksum = "4149b09b9d8cf37f0afc390144f5d71b8f4daadfd9540ddf43ad27b54d407470" dependencies = [ - "itoa 1.0.4", + "itoa 1.0.1", "num-integer", "ryu", "time", @@ -1022,18 +904,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "246e9f83dd1fdf5d347fa30ae4ad30a9d1d42ce4cd74a93d94afa874646f94cd" +checksum = "2c6d8e7a15feb04f041cf0ede8f6c16e03fe5a4b03e164ae3a090e829404d925" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "0.51.0" +version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05701d32da168b44f7ee63147781aed8723e792cc131cb9b18363b5393f17f70" +checksum = "1bba03e59e1a0223a2bd3567da2b07a458b067ccf7846996b82406e80008ebc1" dependencies = [ "aws-smithy-async", "aws-smithy-client", @@ -1041,26 +923,26 @@ dependencies = [ "aws-smithy-types", "http", "rustc_version 0.4.0", - "tracing 0.1.37", + "tracing 0.1.34", "zeroize", ] [[package]] name = "axum" -version = "0.5.17" +version = "0.5.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acee9fd5073ab6b045a275b3e709c163dd36c90685219cb21804a147b58dba43" +checksum = "9de18bc5f2e9df8f52da03856bf40e29b747de5a84e43aefff90e3dc4a21529b" dependencies = [ "async-trait", - "axum-core 0.2.9", + "axum-core", "bitflags", - "bytes 1.3.0", + "bytes 1.2.1", "futures-util", "http", "http-body", "hyper", - "itoa 1.0.4", - "matchit 0.5.0", + "itoa 1.0.1", + "matchit", "memchr", "mime", "percent-encoding", @@ -1074,158 +956,117 @@ dependencies = [ "tower-service", ] -[[package]] -name = "axum" -version = "0.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08b108ad2665fa3f6e6a517c3d80ec3e77d224c47d605167aefaa5d7ef97fa48" -dependencies = [ - "async-trait", - "axum-core 0.3.0", - "bitflags", - "bytes 1.3.0", - "futures-util", - "http", - "http-body", - "hyper", - "itoa 1.0.4", - "matchit 0.7.0", - "memchr", - "mime", - "percent-encoding", - "pin-project-lite", - "rustversion", - "serde", - "sync_wrapper", - "tower", - "tower-http", - "tower-layer", - "tower-service", -] - -[[package]] -name = "axum-core" -version = "0.2.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37e5939e02c56fecd5c017c37df4238c0a839fa76b7f97acdd7efb804fd181cc" -dependencies = [ - "async-trait", - "bytes 1.3.0", - "futures-util", - "http", - "http-body", - "mime", - "tower-layer", - "tower-service", -] - [[package]] name = "axum-core" -version = "0.3.0" +version = "0.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79b8558f5a0581152dc94dcd289132a1d377494bdeafcd41869b3258e3e2ad92" +checksum = "e4f44a0e6200e9d11a1cdc989e4b358f6e3d354fbf48478f345a17f4e43f8635" dependencies = [ "async-trait", - "bytes 1.3.0", + "bytes 1.2.1", "futures-util", "http", "http-body", "mime", - "rustversion", - "tower-layer", - "tower-service", ] [[package]] name = "azure_core" -version = "0.5.0" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" +version = "0.2.2" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" dependencies = [ "async-trait", "base64", - "bytes 1.3.0", + "bytes 1.2.1", + "chrono", "dyn-clone", - "futures 0.3.25", - "getrandom 0.2.8", - "http-types", + "futures 0.3.24", + "getrandom 0.2.6", + "http", "log", - "paste", + "oauth2", "pin-project", "rand 0.8.5", "reqwest", "rustc_version 0.4.0", "serde", - "serde-xml-rs", + "serde_derive", "serde_json", - "time", + "thiserror", "url", - "uuid 1.2.2", + "uuid 1.1.2", ] [[package]] name = "azure_identity" -version = "0.6.0" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" +version = "0.3.0" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" dependencies = [ "async-lock", + "async-timer", "async-trait", "azure_core", "base64", - "fix-hidden-lifetime-bug", - "futures 0.3.25", + "chrono", + "futures 0.3.24", "log", "oauth2", + "reqwest", "serde", "serde_json", - "time", + "thiserror", "url", - "uuid 1.2.2", + "uuid 1.1.2", ] [[package]] name = "azure_storage" -version = "0.6.0" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" +version = "0.2.0" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" dependencies = [ "RustyXML", "async-trait", "azure_core", "base64", - "bytes 1.3.0", - "futures 0.3.25", + "bytes 1.2.1", + "chrono", + "futures 0.3.24", "hmac", + "http", "log", "once_cell", "serde", "serde-xml-rs", "serde_derive", "serde_json", - "sha2 0.10.6", - "time", + "sha2 0.10.5", + "thiserror", "url", - "uuid 1.2.2", + "uuid 1.1.2", ] [[package]] name = "azure_storage_blobs" -version = "0.6.0" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" +version = "0.2.0" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" dependencies = [ "RustyXML", "azure_core", "azure_storage", "base64", - "bytes 1.3.0", - "futures 0.3.25", + "bytes 1.2.1", + "chrono", + "futures 0.3.24", + "http", "log", "md5", "serde", "serde-xml-rs", "serde_derive", "serde_json", - "time", + "thiserror", "url", - "uuid 1.2.2", + "uuid 1.1.2", ] [[package]] @@ -1234,16 +1075,16 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b62ddb9cb1ec0a098ad4bbf9344d0713fa193ae1a80af55febcff2627b6a00c1" dependencies = [ - "getrandom 0.2.8", + "getrandom 0.2.6", "instant", "rand 0.8.5", ] [[package]] name = "base64" -version = "0.13.1" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" +checksum = "904dfeac50f3cdaba28fc6f57fdcddb75f49ed61346676a78c4ffe55877802fd" [[package]] name = "base64-url" @@ -1262,9 +1103,9 @@ checksum = "e6b4d9b1225d28d360ec6a231d65af1fd99a2a095154c8040689617290569c5c" [[package]] name = "bit-set" -version = "0.5.3" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0700ddab506f33b20a03b13996eccd309a48e5ff77d0d95926aa0210fb4e95f1" +checksum = "6e11e16035ea35e4e5997b393eacbf6f63983188f7a2ad25bfb13465f5ad59de" dependencies = [ "bit-vec 0.6.3", ] @@ -1289,9 +1130,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitmask-enum" -version = "2.1.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd9e32d7420c85055e8107e5b2463c4eeefeaac18b52359fe9f9c08a18f342b2" +checksum = "76487de46597d345d040a1be49a6fb636b71d0abab4696b7f3492e0cd4639c73" dependencies = [ "quote", "syn", @@ -1319,9 +1160,9 @@ dependencies = [ [[package]] name = "block-buffer" -version = "0.10.3" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69cce20737498f97b993470a6e536b8523f0af7892a4f928cceb1ac5e52ebe7e" +checksum = "0bf7fe51849ea569fd452f37822f606a5cabb684dc918707a0193fd4664ff324" dependencies = [ "generic-array", ] @@ -1337,16 +1178,16 @@ dependencies = [ [[package]] name = "blocking" -version = "1.3.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c67b173a56acffd6d2326fb7ab938ba0b00a71480e14902b2591c87bc5741e8" +checksum = "c6ccb65d468978a086b69884437ded69a90faab3bbe6e67f242173ea728acccc" dependencies = [ "async-channel", - "async-lock", "async-task", "atomic-waker", "fastrand", "futures-lite", + "once_cell", ] [[package]] @@ -1366,7 +1207,7 @@ checksum = "d82e7850583ead5f8bbef247e2a3c37a19bd576e8420cd262a6711921827e1e5" dependencies = [ "base64", "bollard-stubs", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "dirs-next", "futures-core", @@ -1374,24 +1215,24 @@ dependencies = [ "hex", "http", "hyper", - "hyper-rustls", + "hyper-rustls 0.23.0", "hyperlocal", "log", "pin-project-lite", - "rustls 0.20.7", + "rustls 0.20.4", "rustls-native-certs 0.6.2", - "rustls-pemfile 1.0.1", + "rustls-pemfile 1.0.0", "serde", "serde_derive", "serde_json", "serde_urlencoded", "thiserror", "tokio", - "tokio-util", + "tokio-util 0.7.1", "url", "webpki 0.22.0", "webpki-roots", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -1405,58 +1246,13 @@ dependencies = [ "serde_with 1.14.0", ] -[[package]] -name = "borsh" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15bf3650200d8bffa99015595e10f1fbd17de07abbc25bb067da79e769939bfa" -dependencies = [ - "borsh-derive", - "hashbrown 0.11.2", -] - -[[package]] -name = "borsh-derive" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6441c552f230375d18e3cc377677914d2ca2b0d36e52129fe15450a2dce46775" -dependencies = [ - "borsh-derive-internal", - "borsh-schema-derive-internal", - "proc-macro-crate 0.1.5", - "proc-macro2", - "syn", -] - -[[package]] -name = "borsh-derive-internal" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5449c28a7b352f2d1e592a8a28bf139bc71afb0764a14f3c02500935d8c44065" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "borsh-schema-derive-internal" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdbd5696d8bfa21d53d9fe39a714a18538bad11492a42d066dbbc395fb1951c0" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "bson" -version = "2.4.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99d76085681585d39016f4d3841eb019201fc54d2dd0d92ad1e4fab3bfb32754" +checksum = "a24ecf39f5a314493ede1bb015984735d41aa6aedb59cafb95492d40cd893330" dependencies = [ - "ahash 0.7.6", + "ahash", "base64", "hex", "indexmap", @@ -1466,7 +1262,7 @@ dependencies = [ "serde_bytes", "serde_json", "time", - "uuid 1.2.2", + "uuid 0.8.2", ] [[package]] @@ -1481,23 +1277,11 @@ dependencies = [ "serde", ] -[[package]] -name = "bstr" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fca0852af221f458706eb0725c03e4ed6c46af9ac98e6a689d5e634215d594dd" -dependencies = [ - "memchr", - "once_cell", - "regex-automata", - "serde", -] - [[package]] name = "bumpalo" -version = "3.11.1" +version = "3.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "572f695136211188308f16ad2ca5c851a712c464060ae6974944458eb83880ba" +checksum = "a4a45a46ab1f2412e53d3a0ade76ffad2025804294569aae387231a0cd6e0899" [[package]] name = "bytecheck" @@ -1522,9 +1306,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.12.3" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aaa3a8d9a1ca92e282c96a32d6511b695d7d994d1d102ba85d279f9b2756947f" +checksum = "cdead85bdec19c194affaeeb670c0e41fe23de31459efd1c174d049269cf02cc" [[package]] name = "byteorder" @@ -1544,20 +1328,20 @@ dependencies = [ [[package]] name = "bytes" -version = "1.3.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfb24e866b15a1af2a1b663f10c6b6b8f397a84aadb828f12e5b289ec23a3a3c" +checksum = "ec8a7b6a70fde80372154c65702f00a0f56f3e1c36abbc6c440484be248856db" dependencies = [ "serde", ] [[package]] name = "bytes-utils" -version = "0.1.3" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e47d3a8076e283f3acd27400535992edb3ba4b5bb72f8891ad8fbe7932a7d4b9" +checksum = "1934a3ef9cac8efde4966a92781e77713e1ba329f1d42e446c7d7eba340d8ef1" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "either", ] @@ -1567,12 +1351,27 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c58ec36aac5066d5ca17df51b3e70279f5670a72102f5752cb7e7c856adfc70" +[[package]] +name = "cache-padded" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1db59621ec70f09c5e9b597b220c7a2b43611f4710dc03ceb8748637775692c" + [[package]] name = "cassowary" version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "df8670b8c7b9dae1793364eafadf7239c40d669904660c5960d74cfd80b46a53" +[[package]] +name = "cast" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c24dab4283a142afa2fdca129b80ad2c6284e073930f964c3a1293c225ee39a" +dependencies = [ + "rustc_version 0.4.0", +] + [[package]] name = "cast" version = "0.3.0" @@ -1590,9 +1389,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.77" +version = "1.0.73" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9f73505338f7d905b19d18738976aae232eb46b8efc15554ffc56deb5d9ebe4" +checksum = "2fff2a6927b3bb87f9595d67196a70493f627687a71d87a0d692242c33f58c11" dependencies = [ "jobserver", ] @@ -1605,48 +1404,42 @@ checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" [[package]] name = "cfb-mode" -version = "0.8.2" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "738b8d467867f80a71351933f70461f5b56f24d5c93e0cf216e59229c968d330" +checksum = "b258c89d55316f606923b53511d0d41c20c4a2d440e5e56dd7185267aa98bd0e" dependencies = [ "cipher", ] [[package]] name = "cfg-if" -version = "1.0.0" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" +checksum = "4785bdd1c96b2a846b2bd7cc02e86b6b3dbf14e7e53446c4f54c92a361040822" [[package]] -name = "charset" -version = "0.1.3" +name = "cfg-if" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18e9079d1a12a2cc2bffb5db039c43661836ead4082120d5844f02555aca2d46" -dependencies = [ - "base64", - "encoding_rs", -] +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.22" -source = "git+https://github.com/vectordotdev/chrono.git?branch=no-default-time-v0.4.22-1#920ff24cabedcec0f8459b8d9681cbac186dfb58" +version = "0.4.19" +source = "git+https://github.com/vectordotdev/chrono.git?branch=no-default-time#b7abfc3fcd7ffc95ad340f584fbcbe0c2d61e1e9" dependencies = [ - "iana-time-zone", - "js-sys", + "libc", "num-integer", "num-traits", "serde", - "wasm-bindgen", - "winapi", + "winapi 0.3.9", ] [[package]] name = "chrono-tz" -version = "0.8.1" +version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa48fa079165080f11d7753fd0bc175b7d391f276b965fe4b55bfad67856e463" +checksum = "29c39203181991a7dd4343b8005bd804e7a9a37afb8ac070e43771e8c820bbde" dependencies = [ "chrono", "chrono-tz-build", @@ -1656,47 +1449,20 @@ dependencies = [ [[package]] name = "chrono-tz-build" -version = "0.1.0" +version = "0.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9998fb9f7e9b2111641485bf8beb32f92945f97f92a3d061f744cfef335f751" +checksum = "6f509c3a87b33437b05e2458750a0700e5bdd6956176773e6c7d6dd15a283a0c" dependencies = [ "parse-zoneinfo", "phf", "phf_codegen", ] -[[package]] -name = "ciborium" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0c137568cc60b904a7724001b35ce2630fd00d5d84805fbb608ab89509d788f" -dependencies = [ - "ciborium-io", - "ciborium-ll", - "serde", -] - -[[package]] -name = "ciborium-io" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "346de753af073cc87b52b2083a506b38ac176a44cfb05497b622e27be899b369" - -[[package]] -name = "ciborium-ll" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "213030a2b5a4e0c0892b6652260cf6ccac84827b83a85a534e178e3906c4cf1b" -dependencies = [ - "ciborium-io", - "half", -] - [[package]] name = "cidr-utils" -version = "0.5.9" +version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355d5b5df67e58b523953d0c1a8d3d2c05f5af51f1332b0199b9c92263614ed0" +checksum = "9a0c49d94395dc29876e75966ca2176e6c7c2ab117ca99d06937faae6ca89c7b" dependencies = [ "debug-helper", "num-bigint 0.4.3", @@ -1732,79 +1498,59 @@ dependencies = [ [[package]] name = "clap" -version = "3.2.23" +version = "3.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71655c45cb9845d3270c9d6df84ebe72b4dad3c2ba3f7023ad47c144e4e473a5" -dependencies = [ - "bitflags", - "clap_lex 0.2.4", - "indexmap", - "textwrap 0.16.0", -] - -[[package]] -name = "clap" -version = "4.0.29" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d63b9e9c07271b9957ad22c173bae2a4d9a81127680962039296abcd2f8251d" +checksum = "23b71c3ce99b7611011217b366d923f1d0a7e07a92bb2dbf1e84508c673ca3bd" dependencies = [ + "atty", "bitflags", "clap_derive", - "clap_lex 0.3.0", - "is-terminal", + "clap_lex", + "indexmap", "once_cell", "strsim 0.10.0", "termcolor", - "terminal_size 0.2.3", + "textwrap 0.15.0", ] [[package]] name = "clap_derive" -version = "4.0.21" +version = "3.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0177313f9f02afc995627906bbd8967e2be069f5261954222dac78290c2b9014" +checksum = "ea0c8bce528c4be4da13ea6fead8965e95b6073585a2f05204bd8f4119f82a65" dependencies = [ "heck 0.4.0", "proc-macro-error", "proc-macro2", "quote", - "syn", -] - -[[package]] -name = "clap_lex" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2850f2f5a82cbf437dd5af4d49848fbdfc27c157c3d010345776f952765261c5" -dependencies = [ - "os_str_bytes", + "syn", ] [[package]] name = "clap_lex" -version = "0.3.0" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d4198f73e42b4936b35b5bb248d81d2b595ecb170da0bac7655c54eedfa8da8" +checksum = "87eba3c8c7f42ef17f6c659fc7416d0f4758cd3e58861ee63c5fa4a4dde649e4" dependencies = [ "os_str_bytes", ] [[package]] name = "clipboard-win" -version = "4.4.2" +version = "4.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4ab1b92798304eedc095b53942963240037c0516452cb11aeba709d420b2219" +checksum = "2f3e1238132dc01f081e1cbb9dace14e5ef4c3a51ee244bd982275fb514605db" dependencies = [ "error-code", "str-buf", - "winapi", + "winapi 0.3.9", ] [[package]] name = "cmake" -version = "0.1.49" +version = "0.1.48" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db34956e100b30725f2eb215f90d4871051239535632f84fea3bc92722c66b7c" +checksum = "e8ad8cef104ac57b68b89df3208164d228503abbdce70f6880ffa3d970e7443a" dependencies = [ "cc", ] @@ -1814,31 +1560,30 @@ name = "codecs" version = "0.1.0" dependencies = [ "avro-rs", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "derivative", "dyn-clone", - "futures 0.3.25", + "futures 0.3.24", "indoc", "lookup", "memchr", "once_cell", - "ordered-float 3.4.0", - "prost 0.11.3", + "ordered-float 3.0.0", + "pretty_assertions", + "prost 0.10.4", "regex", "serde", "serde_json", - "similar-asserts", "smallvec", "snafu", "syslog_loose", "tokio", - "tokio-util", - "tracing 0.1.37", + "tokio-util 0.7.1", + "tracing 0.1.34", "value", "vector-common", "vector-config", - "vector-config-common", "vector-config-macros", "vector-core", ] @@ -1861,7 +1606,7 @@ checksum = "b3616f750b84d8f0de8a58bda93e08e2a81ad3f523089b05f1dffecab48c6cbd" dependencies = [ "atty", "lazy_static", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -1879,38 +1624,25 @@ dependencies = [ [[package]] name = "combine" -version = "4.6.6" +version = "4.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35ed6e9d84f0b51a7f52daf1c7d71dd136fd7a3f41a8462b8cdb8c78d920fad4" +checksum = "2a604e93b79d1808327a6fca85a6f2d69de66461e7620f5a4cbf5fb4d1d7c948" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "futures-core", "memchr", "pin-project-lite", "tokio", - "tokio-util", + "tokio-util 0.7.1", ] [[package]] name = "concurrent-queue" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd7bef69dc86e3c610e4e7aed41035e2a7ed12e72dd7530f61327a6579a4390b" -dependencies = [ - "crossbeam-utils", -] - -[[package]] -name = "console" -version = "0.15.2" +version = "1.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c050367d967ced717c04b65d8c619d863ef9292ce0c5760028655a2fb298718c" +checksum = "30ed07550be01594c6026cff2a1d7fe9c8f683caa798e12b68694ac9e88286a3" dependencies = [ - "encode_unicode 0.3.6", - "lazy_static", - "libc", - "terminal_size 0.1.17", - "winapi", + "cache-padded", ] [[package]] @@ -1919,10 +1651,10 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e57ff02e8ad8e06ab9731d5dc72dc23bef9200778eae1a89d555d8c42e5d4a86" dependencies = [ - "prost 0.11.3", - "prost-types 0.11.2", - "tonic", - "tracing-core 0.1.30", + "prost 0.11.0", + "prost-types 0.11.1", + "tonic 0.8.0", + "tracing-core 0.1.28", ] [[package]] @@ -1934,19 +1666,19 @@ dependencies = [ "console-api", "crossbeam-channel", "crossbeam-utils", - "futures 0.3.25", + "futures 0.3.24", "hdrhistogram", "humantime", - "prost-types 0.11.2", + "prost-types 0.11.1", "serde", "serde_json", "thread_local", "tokio", "tokio-stream", - "tonic", - "tracing 0.1.37", - "tracing-core 0.1.30", - "tracing-subscriber 0.3.16", + "tonic 0.8.0", + "tracing 0.1.34", + "tracing-core 0.1.28", + "tracing-subscriber 0.3.15", ] [[package]] @@ -1967,12 +1699,6 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" -[[package]] -name = "cookie-factory" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "396de984970346b0d9e93d1415082923c679e5ae5c3ee3dcbd104f5610af126b" - [[package]] name = "core-foundation" version = "0.9.3" @@ -1991,9 +1717,9 @@ checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" [[package]] name = "cpufeatures" -version = "0.2.5" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28d997bd5e24a5928dd43e46dc529867e207907fe0b239c3477d924f7f2ca320" +checksum = "59a6001667ab124aebae2a495118e11d30984c3a653e99d86d58971708cf5e4b" dependencies = [ "libc", ] @@ -2028,23 +1754,22 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b540bd8bc810d3885c6ea91e2018302f68baba2129ab3e88f32389ee9370880d" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", ] [[package]] name = "criterion" -version = "0.4.0" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7c76e09c1aae2bc52b3d2f29e13c6572553b30c4aa1b8a49fd70de6412654cb" +checksum = "b01d6de93b2b6c65e17c634a26653a29d107b3c98c607c765bf38d041531cd8f" dependencies = [ - "anes", "atty", - "cast", - "ciborium", - "clap 3.2.23", + "cast 0.3.0", + "clap 2.34.0", "criterion-plot", - "futures 0.3.25", - "itertools", + "csv", + "futures 0.3.24", + "itertools 0.10.3", "lazy_static", "num-traits", "oorandom", @@ -2052,6 +1777,7 @@ dependencies = [ "rayon", "regex", "serde", + "serde_cbor", "serde_derive", "serde_json", "tinytemplate", @@ -2061,65 +1787,67 @@ dependencies = [ [[package]] name = "criterion-plot" -version = "0.5.0" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +checksum = "d00996de9f2f7559f7f4dc286073197f83e92256a59ed395f9aac01fe717da57" dependencies = [ - "cast", - "itertools", + "cast 0.2.7", + "itertools 0.10.3", ] [[package]] name = "crossbeam-channel" -version = "0.5.6" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" +checksum = "5aaa7bd5fb665c6864b5f963dd9097905c54125909c7aa94c9e18507cdbe6c53" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "crossbeam-utils", ] [[package]] name = "crossbeam-deque" -version = "0.8.2" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "715e8152b692bba2d374b53d4875445368fdf21a94751410af607a5ac677d1fc" +checksum = "6455c0ca19f0d2fbf751b908d5c55c1f5cbc65e03c4225427254b46890bdde1e" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "crossbeam-epoch", "crossbeam-utils", ] [[package]] name = "crossbeam-epoch" -version = "0.9.13" +version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01a9af1f4c2ef74bb8aa1f7e19706bc72d03598c8a570bb5de72243c7a9d9d5a" +checksum = "1145cf131a2c6ba0615079ab6a638f7e1973ac9c2634fcbeaaad6114246efe8c" dependencies = [ "autocfg", - "cfg-if", + "cfg-if 1.0.0", "crossbeam-utils", - "memoffset 0.7.1", + "lazy_static", + "memoffset", "scopeguard", ] [[package]] name = "crossbeam-queue" -version = "0.3.8" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1cfb3ea8a53f37c40dea2c7bedcbd88bdfae54f5e2175d6ecaff1c988353add" +checksum = "1cd42583b04998a5363558e5f9291ee5a5ff6b49944332103f251e7479a82aa7" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "crossbeam-utils", ] [[package]] name = "crossbeam-utils" -version = "0.8.14" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fb766fa798726286dbbb842f174001dab8abc7b627a1dd86e0b7222a95d929f" +checksum = "51887d4adc7b564537b15adcfb307936f8075dfcd5f00dde9a9f1d29383682bc" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", + "once_cell", ] [[package]] @@ -2132,11 +1860,11 @@ dependencies = [ "crossterm_winapi", "futures-core", "libc", - "mio", + "mio 0.8.2", "parking_lot", "signal-hook", "signal-hook-mio", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -2145,7 +1873,7 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2ae1b35a484aa10e07fe0638d02301c5ad24de82d310ccbd2f3693da5f09bf1c" dependencies = [ - "winapi", + "winapi 0.3.9", ] [[package]] @@ -2156,9 +1884,9 @@ checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" [[package]] name = "crypto-common" -version = "0.1.6" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +checksum = "57952ca27b5e3606ff4dd79b0020231aaf9d6aa76dc05fd30137538c50bd3ce8" dependencies = [ "generic-array", "typenum", @@ -2170,7 +1898,7 @@ version = "1.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "22813a6dc45b335f9bade10bf7271dc477e81113e89eb251a0bc2a8a81c536e1" dependencies = [ - "bstr 0.2.17", + "bstr", "csv-core", "itoa 0.4.8", "ryu", @@ -2186,11 +1914,20 @@ dependencies = [ "memchr", ] +[[package]] +name = "ct-logs" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1a816186fa68d9e426e3cb4ae4dff1fcd8e4a2c34b781bf7a822574a0d0aac8" +dependencies = [ + "sct 0.6.1", +] + [[package]] name = "ctor" -version = "0.1.26" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d2301688392eb071b0bf1a37be05c469d3cc4dbbd95df672fe28ab021e6a096" +checksum = "f877be4f7c9f246b183111634f75baa039715e3f46ce860677d3b19a69fb229c" dependencies = [ "quote", "syn", @@ -2198,19 +1935,13 @@ dependencies = [ [[package]] name = "ctr" -version = "0.9.2" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0369ee1ad671834580515889b80f2ea915f23b8be8d0daa4bbaf2ac5c7590835" +checksum = "0d14f329cfbaf5d0e06b5e87fff7e265d2673c5ea7d2c27691a2c107db1442a0" dependencies = [ "cipher", ] -[[package]] -name = "cty" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b365fabc795046672053e29c954733ec3b05e4be654ab130fe8f1f94d7051f35" - [[package]] name = "curve25519-dalek" version = "3.2.0" @@ -2224,50 +1955,6 @@ dependencies = [ "zeroize", ] -[[package]] -name = "cxx" -version = "1.0.83" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdf07d07d6531bfcdbe9b8b739b104610c6508dcc4d63b410585faf338241daf" -dependencies = [ - "cc", - "cxxbridge-flags", - "cxxbridge-macro", - "link-cplusplus", -] - -[[package]] -name = "cxx-build" -version = "1.0.83" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2eb5b96ecdc99f72657332953d4d9c50135af1bac34277801cc3937906ebd39" -dependencies = [ - "cc", - "codespan-reporting", - "once_cell", - "proc-macro2", - "quote", - "scratch", - "syn", -] - -[[package]] -name = "cxxbridge-flags" -version = "1.0.83" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac040a39517fd1674e0f32177648334b0f4074625b5588a64519804ba0553b12" - -[[package]] -name = "cxxbridge-macro" -version = "1.0.83" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1362b0ddcfc4eb0a1f57b68bd77dd99f0e826958a96abd0ae9bd092e114ffed6" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "darling" version = "0.13.4" @@ -2280,12 +1967,12 @@ dependencies = [ [[package]] name = "darling" -version = "0.14.2" +version = "0.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0dd3cd20dc6b5a876612a6e5accfe7f3dd883db6d07acfbf14c128f61550dfa" +checksum = "4529658bdda7fd6769b8614be250cdcfc3aeb0ee72fe66f9e41e5e5eb73eac02" dependencies = [ - "darling_core 0.14.2", - "darling_macro 0.14.2", + "darling_core 0.14.1", + "darling_macro 0.14.1", ] [[package]] @@ -2304,9 +1991,9 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.14.2" +version = "0.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a784d2ccaf7c98501746bf0be29b2022ba41fd62a2e622af997a03e9f972859f" +checksum = "649c91bc01e8b1eac09fb91e8dbc7d517684ca6be8ebc75bb9cafc894f9fdb6f" dependencies = [ "fnv", "ident_case", @@ -2329,39 +2016,40 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.14.2" +version = "0.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7618812407e9402654622dd402b0a89dff9ba93badd6540781526117b92aab7e" +checksum = "ddfc69c5bfcbd2fc09a0f38451d2daf0e372e367986a83906d1b0dbc88134fb5" dependencies = [ - "darling_core 0.14.2", + "darling_core 0.14.1", "quote", "syn", ] [[package]] name = "dashmap" -version = "5.4.0" +version = "5.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "907076dfda823b0b36d2a1bb5f90c96660a5bbcd7729e10727f07858f22c4edc" +checksum = "4c8858831f7781322e539ea39e72449c46b059638250c14344fec8d0aa6e539c" dependencies = [ - "cfg-if", - "hashbrown 0.12.3", - "lock_api", - "once_cell", - "parking_lot_core", + "cfg-if 1.0.0", + "num_cpus", + "parking_lot", ] [[package]] name = "data-encoding" -version = "2.3.3" +version = "2.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23d8666cb01533c39dde32bcbab8e227b4ed6679b2c925eba05feabea39508fb" +checksum = "3ee2393c4a91429dffb4bedf19f4d6abf27d8a732c8ce4980305d782e5426d57" [[package]] name = "data-url" -version = "0.2.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d7439c3735f405729d52c3fbbe4de140eaf938a1fe47d227c27f8254d4302a5" +checksum = "3a30bfce702bcfa94e906ef82421f2c0e61c076ad76030c16ee5d2e9a32fe193" +dependencies = [ + "matches", +] [[package]] name = "datadog-filter" @@ -2376,7 +2064,7 @@ dependencies = [ name = "datadog-grok" version = "0.1.0" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "chrono-tz", "criterion", @@ -2386,13 +2074,13 @@ dependencies = [ "nom", "once_cell", "onig", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "peeking_take_while", "regex", "serde_json", "thiserror", - "tracing 0.1.37", - "tracing-test 0.2.3", + "tracing 0.1.34", + "tracing-test 0.2.1", "value", "vector-common", "vrl-compiler", @@ -2402,7 +2090,7 @@ dependencies = [ name = "datadog-search-syntax" version = "0.1.0" dependencies = [ - "itertools", + "itertools 0.10.3", "once_cell", "pest", "pest_derive", @@ -2417,9 +2105,9 @@ checksum = "b72465f46d518f6015d9cf07f7f3013a95dd6b9c2747c3d65ae0cce43929d14f" [[package]] name = "deadpool" -version = "0.9.5" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "421fe0f90f2ab22016f32a9881be5134fdd71c65298917084b0c7477cbc3856e" +checksum = "a90d58a15f5acfe41afcac9775d8e92f2338d14482220c778c6e42aa77778182" dependencies = [ "async-trait", "deadpool-runtime", @@ -2462,9 +2150,9 @@ dependencies = [ [[package]] name = "derive_arbitrary" -version = "1.2.1" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8a16495aeb28047bb1185fca837baf755e7d71ed3aeed7f8504654ffa927208" +checksum = "d749b611838b7e92cba33b8552187059f2dc32f91d4d66bdc4fe5da9526b4e07" dependencies = [ "proc-macro2", "quote", @@ -2486,9 +2174,9 @@ dependencies = [ [[package]] name = "diff" -version = "0.1.13" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56254986775e3233ffa9c4d7d3faaf6d36a2c09d30b20687e9f88bc8bafc16c8" +checksum = "0e25ea47919b1560c4e3b7fe0aaab9becf5b84a10325ddf7db0f0ba5e1026499" [[package]] name = "difflib" @@ -2507,11 +2195,11 @@ dependencies = [ [[package]] name = "digest" -version = "0.10.6" +version = "0.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" +checksum = "f2fb860ca6fafa5552fb6d0e816a69c8e49f0908bf524e30a90d97c85892d506" dependencies = [ - "block-buffer 0.10.3", + "block-buffer 0.10.2", "crypto-common", "subtle", ] @@ -2524,7 +2212,7 @@ checksum = "3fd78930633bd1c6e35c4b42b1df7b0cbc6bc191146e512bb3bedf243fcc3901" dependencies = [ "libc", "redox_users 0.3.5", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -2533,7 +2221,7 @@ version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b98cf8ebf19c3d1b223e151f99a4f9f0690dca41414773390fc824184ac833e1" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "dirs-sys-next", ] @@ -2545,7 +2233,7 @@ checksum = "4ebda144c4fe02d1f7ea1a7d9641b6fc6b580adcfa024ae48797ecdeb6825b4d" dependencies = [ "libc", "redox_users 0.4.3", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -2554,10 +2242,10 @@ version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "53ecafc952c4528d9b51a458d1a8904b81783feff9fde08ab6ed2545ff396872" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "libc", "socket2", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -2567,7 +2255,7 @@ dependencies = [ "criterion", "data-encoding", "thiserror", - "trust-dns-proto 0.22.0", + "trust-dns-proto", ] [[package]] @@ -2578,9 +2266,9 @@ checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" [[package]] name = "duct" -version = "0.13.6" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ae3fc31835f74c2a7ceda3aeede378b0ae2e74c8f1c36559fcc9ae2a4e7d3e" +checksum = "0fc6a0a59ed0888e0041cf708e66357b7ae1a82f1c67247e1f93b5e0818f7d8d" dependencies = [ "libc", "once_cell", @@ -2596,9 +2284,9 @@ checksum = "4f94fa09c2aeea5b8839e414b7b841bf429fd25b9c522116ac97ee87856d88b2" [[package]] name = "ed25519" -version = "1.5.2" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e9c280362032ea4203659fc489832d0204ef09f247a0506f170dafcac08c369" +checksum = "3d5c4b5e5959dc2c2b89918d8e2cc40fcdd623cef026ed09d2f0ee05199dc8e4" dependencies = [ "signature", ] @@ -2617,9 +2305,9 @@ dependencies = [ [[package]] name = "either" -version = "1.8.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90e5c1c8368803113bf0c9584fc495a58b86dc8a29edbf8fe877d21d9507e797" +checksum = "e78d4f1cc4ae33bbfc157ed5d5a5ef3bc29227303d595861deb238fcec4e9457" [[package]] name = "ena" @@ -2648,7 +2336,7 @@ version = "0.8.31" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9852635589dc9f9ea1b6fe9f05b50ef208c85c834a562f0c6abb1c475736ec2b" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "serde", ] @@ -2682,18 +2370,6 @@ dependencies = [ "syn", ] -[[package]] -name = "enum-as-inner" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" -dependencies = [ - "heck 0.4.0", - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "enum_dispatch" version = "0.3.8" @@ -2744,12 +2420,12 @@ dependencies = [ [[package]] name = "env_logger" -version = "0.10.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85cdab6a89accf66733ad5a1693a4dcced6aeff64602b634530dd73c1f3ee9f0" +checksum = "0b2cf0344971ee6c64c31be0d530793fba457d322dfec2810c453d0ef228f9c3" dependencies = [ + "atty", "humantime", - "is-terminal", "log", "regex", "termcolor", @@ -2757,9 +2433,9 @@ dependencies = [ [[package]] name = "erased-serde" -version = "0.3.23" +version = "0.3.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54558e0ba96fbe24280072642eceb9d7d442e32c7ec0ea9e7ecd7b4ea2cf4e11" +checksum = "ad132dd8d0d0b546348d7d86cb3191aad14b34e5f979781fc005c80d4ac67ffd" dependencies = [ "serde", ] @@ -2786,7 +2462,7 @@ checksum = "f639046355ee4f37944e44f60642c6f3a7efa3cf6b78c78a0d989a8ce6c396a1" dependencies = [ "errno-dragonfly", "libc", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -2820,18 +2496,9 @@ dependencies = [ [[package]] name = "event-listener" -version = "2.5.3" +version = "2.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" - -[[package]] -name = "executor-trait" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a1052dd43212a7777ec6a69b117da52f5e52f07aec47d00c1a2b33b85d06b08" -dependencies = [ - "async-trait", -] +checksum = "77f3309417938f28bf8228fcff79a4a37103981e3e186d2ccd19c74b38f4eb71" [[package]] name = "exitcode" @@ -2867,22 +2534,22 @@ checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" [[package]] name = "fastrand" -version = "1.8.0" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7a407cfaa3385c4ae6b23e84623d48c2798d06e3e6a1878f7f59f17b3f86499" +checksum = "c3fcf0cee53519c866c09b5de1f6c56ff9d647101f81c1964fa632e148896cdf" dependencies = [ "instant", ] [[package]] name = "fd-lock" -version = "3.0.8" +version = "3.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb21c69b9fea5e15dbc1049e4b77145dd0ba1c84019c488102de0dc4ea4b0a27" +checksum = "46e245f4c8ec30c6415c56cb132c07e69e74f1942f6b4a4061da748b49f486ca" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "rustix", - "windows-sys 0.42.0", + "windows-sys 0.30.0", ] [[package]] @@ -2895,71 +2562,51 @@ checksum = "534be3985e262585e2a8bb8322969e2115f0eff293b5bf522bef397fff54c9bc" name = "file-source" version = "0.1.0" dependencies = [ - "bstr 1.0.1", - "bytes 1.3.0", + "bstr", + "bytes 1.2.1", "chrono", "crc", "criterion", "dashmap", "flate2", - "futures 0.3.25", + "futures 0.3.24", "glob", "indexmap", "libc", + "pretty_assertions", "quickcheck", "scan_fmt", "serde", "serde_json", - "similar-asserts", "tempfile", "tokio", - "tracing 0.1.37", - "winapi", + "tracing 0.1.34", + "winapi 0.3.9", ] [[package]] name = "filetime" -version = "0.2.19" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e884668cd0c7480504233e951174ddc3b382f7c2666e3b7310b5c4e7b0c37f9" +checksum = "c0408e2626025178a6a7f7ffc05a25bc47103229f19c113755de7bf63816290c" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "libc", - "redox_syscall 0.2.16", - "windows-sys 0.42.0", -] - -[[package]] -name = "fix-hidden-lifetime-bug" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4ae9c2016a663983d4e40a9ff967d6dcac59819672f0b47f2b17574e99c33c8" -dependencies = [ - "fix-hidden-lifetime-bug-proc_macros", -] - -[[package]] -name = "fix-hidden-lifetime-bug-proc_macros" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4c81935e123ab0741c4c4f0d9b8377e5fb21d3de7e062fa4b1263b1fbcba1ea" -dependencies = [ - "proc-macro2", - "quote", - "syn", + "redox_syscall 0.2.13", + "winapi 0.3.9", ] [[package]] name = "fixedbitset" -version = "0.4.2" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" +checksum = "279fb028e20b3c4c320317955b77c5e0c9701f05a1d309905d6fc702cdc5053e" [[package]] name = "flate2" -version = "1.0.25" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8a2db397cb1c8772f31494cb8917e48cd1e64f0fa7efac59fbd741a0a8ce841" +checksum = "f82b0f4c27ad9f8bfd1f3208d882da2b09c301bc1c828fd3a00d0216d2fbbff6" dependencies = [ "crc32fast", "miniz_oxide", @@ -2967,21 +2614,9 @@ dependencies = [ [[package]] name = "float_eq" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28a80e3145d8ad11ba0995949bbcf48b9df2be62772b3d351ef017dff6ecb853" - -[[package]] -name = "flume" -version = "0.10.14" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1657b4441c3403d9f7b3409e47575237dac27b1b5726df654a6ecbf92f0f7577" -dependencies = [ - "futures-core", - "futures-sink", - "pin-project", - "spin 0.9.4", -] +checksum = "b59b6469c35ab601d6487d28879bccfbe8c896c33a3fe699c4d29817e552cc58" [[package]] name = "fnv" @@ -3006,10 +2641,11 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "form_urlencoded" -version = "1.1.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9c384f161156f5260c24a097c56119f9be8c798586aecc13afbcbe7b7e26bf8" +checksum = "5fc25a87fa4fd2094bffb06925852034d90a17f0d1e05197d4956d3555752191" dependencies = [ + "matches", "percent-encoding", ] @@ -3019,11 +2655,21 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2022715d62ab30faffd124d40b76f4134a550a87792276512b18d63272333394" +[[package]] +name = "fsevent" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ab7d1bd1bd33cc98b0889831b72da23c0aa4df9cec7e0702f46ecea04b35db6" +dependencies = [ + "bitflags", + "fsevent-sys", +] + [[package]] name = "fsevent-sys" -version = "4.1.0" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76ee7a02da4d231650c7cea31349b889be2f45ddb3ef3032d2ec8185f6313fd2" +checksum = "f41b048a94555da0f42f1d632e2e19510084fb8e303b0daa2816e733fb3644a0" dependencies = [ "libc", ] @@ -3035,9 +2681,25 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04412b8935272e3a9bae6f48c7bfff74c2911f60525404edfdd28e49884c3bfb" dependencies = [ "libc", - "winapi", + "winapi 0.3.9", +] + +[[package]] +name = "fuchsia-zircon" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e9763c69ebaae630ba35f74888db465e49e259ba1bc0eda7d06f4a067615d82" +dependencies = [ + "bitflags", + "fuchsia-zircon-sys", ] +[[package]] +name = "fuchsia-zircon-sys" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3dcaa9ae7725d12cdb85b3ad99a434db70b468c09ded17e012d86b5c1010f7a7" + [[package]] name = "futures" version = "0.1.31" @@ -3046,9 +2708,9 @@ checksum = "3a471a38ef8ed83cd6e40aa59c1ffe17db6855c18e3604d9c4ed8c08ebc28678" [[package]] name = "futures" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38390104763dc37a5145a53c29c63c1290b5d316d6086ec32c293f6736051bb0" +checksum = "7f21eda599937fba36daeb58a22e8f5cee2d14c4a17b5b7739c7c8e5e3b8230c" dependencies = [ "futures-channel", "futures-core", @@ -3061,9 +2723,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52ba265a92256105f45b719605a571ffe2d1f0fea3807304b522c1d778f79eed" +checksum = "30bdd20c28fadd505d0fd6712cdfcb0d4b5648baf45faef7f852afb2399bb050" dependencies = [ "futures-core", "futures-sink", @@ -3071,15 +2733,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04909a7a7e4633ae6c4a9ab280aeb86da1236243a77b694a49eacd659a4bd3ac" +checksum = "4e5aa3de05362c3fb88de6531e6296e85cde7739cccad4b9dfeeb7f6ebce56bf" [[package]] name = "futures-executor" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7acc85df6714c176ab5edf386123fafe217be88c0840ec11f199441134a074e2" +checksum = "9ff63c23854bee61b6e9cd331d523909f238fc7636290b96826e9cfa5faa00ab" dependencies = [ "futures-core", "futures-task", @@ -3088,9 +2750,9 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00f5fb52a06bdcadeb54e8d3671f8888a39697dcb0b81b23b55174030427f4eb" +checksum = "bbf4d2a7a308fd4578637c0b17c7e1c7ba127b8f6ba00b29f717e9655d85eb68" [[package]] name = "futures-lite" @@ -3109,9 +2771,9 @@ dependencies = [ [[package]] name = "futures-macro" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdfb8ce053d86b91919aad980c220b1fb8401a9394410e1c289ed7e66b61835d" +checksum = "42cd15d1c7456c04dbdf7e88bcd69760d74f3a798d6444e16974b505b0e62f17" dependencies = [ "proc-macro2", "quote", @@ -3120,15 +2782,15 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39c15cf1a4aa79df40f1bb462fb39676d0ad9e366c2a33b590d7c66f4f81fcf9" +checksum = "21b20ba5a92e727ba30e72834706623d94ac93a725410b6a6b6fbc1b07f7ba56" [[package]] name = "futures-task" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ffb393ac5d9a6eaa9d3fdf37ae2776656b706e200c8e16b1bdb227f5198e6ea" +checksum = "a6508c467c73851293f390476d4491cf4d227dbabcd4170f3bb6044959b294f1" [[package]] name = "futures-timer" @@ -3138,9 +2800,9 @@ checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" [[package]] name = "futures-util" -version = "0.3.25" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "197676987abd2f9cadff84926f410af1c183608d36641465df73ae8211dc65d6" +checksum = "44fb6cb1be61cc1d2e43b262516aafcf63b241cffdb1d3fa115f91d9c7b09c90" dependencies = [ "futures 0.1.31", "futures-channel", @@ -3158,9 +2820,9 @@ dependencies = [ [[package]] name = "generic-array" -version = "0.14.6" +version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bff49e947297f3312447abdca79f45f4738097cc82b06e72054d2223f601f1b9" +checksum = "fd48d33ec7f05fbfa152300fdad764757cbded343c1aa1cff2fbaf4134851803" dependencies = [ "typenum", "version_check", @@ -3172,29 +2834,29 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fc3cb4d91f53b50155bdcfd23f6a4c39ae1969c2ae85982b135750cccaf5fce" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "libc", "wasi 0.9.0+wasi-snapshot-preview1", ] [[package]] name = "getrandom" -version = "0.2.8" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c05aeb6a22b8f62540c194aac980f2115af067bfe15a0734d7277a768d396b31" +checksum = "9be70c98951c83b8d2f8f60d7065fa6d5146873094452a1008da8c2f1e4205ad" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "js-sys", "libc", - "wasi 0.11.0+wasi-snapshot-preview1", + "wasi 0.10.2+wasi-snapshot-preview1", "wasm-bindgen", ] [[package]] name = "ghost" -version = "0.1.6" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb19fe8de3ea0920d282f7b77dd4227aea6b8b999b42cdf0ca41b2472b14443a" +checksum = "76c813ffb63e8fd3df6f1ac3cc1ea392c7612ac2de4d0b44dcbfe03e5c4bf94a" dependencies = [ "proc-macro2", "quote", @@ -3207,19 +2869,6 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b919933a397b79c37e33b77bb2aa3dc8eb6e165ad809e58ff75bc7db2e34574" -[[package]] -name = "gloo-utils" -version = "0.1.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8e8fc851e9c7b9852508bc6e3f690f452f474417e8545ec9857b7f7377036b5" -dependencies = [ - "js-sys", - "serde", - "serde_json", - "wasm-bindgen", - "web-sys", -] - [[package]] name = "goauth" version = "0.13.1" @@ -3227,7 +2876,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f8af59a261bcf42f45d1b261232847b9b850ba0a1419d6100698246fb66e9240" dependencies = [ "arc-swap", - "futures 0.3.25", + "futures 0.3.24", "log", "reqwest", "serde", @@ -3241,13 +2890,12 @@ dependencies = [ [[package]] name = "governor" -version = "0.5.1" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c390a940a5d157878dd057c78680a33ce3415bcd05b4799509ea44210914b4d5" +checksum = "19775995ee20209163239355bc3ad2f33f83da35d9ef72dea26e5af753552c87" dependencies = [ - "cfg-if", "dashmap", - "futures 0.3.25", + "futures 0.3.24", "futures-timer", "no-std-compat", "nonzero_ext", @@ -3329,7 +2977,7 @@ name = "h2" version = "0.3.13" source = "git+https://github.com/hyperium/h2.git?rev=f6aa3be6719270cd7b4094ee1940751b5f4ec88e#f6aa3be6719270cd7b4094ee1940751b5f4ec88e" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "fnv", "futures-core", "futures-sink", @@ -3338,8 +2986,8 @@ dependencies = [ "indexmap", "slab", "tokio", - "tokio-util", - "tracing 0.1.37", + "tokio-util 0.7.1", + "tracing 0.1.34", ] [[package]] @@ -3356,27 +3004,18 @@ checksum = "74721d007512d0cb3338cd20f0654ac913920061a4c4d0d8708edb3f2a698c0c" [[package]] name = "hashbrown" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab5ef0d4909ef3724cc8cce6ccc8572c5c817592e9285f5464f8e86f8bd3726e" -dependencies = [ - "ahash 0.7.6", -] - -[[package]] -name = "hashbrown" -version = "0.12.3" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +checksum = "8c21d40587b92fa6a6c6e3c1bdbf87d75511db5672f9c93175574b3a00df1758" dependencies = [ - "ahash 0.7.6", + "ahash", ] [[package]] name = "hdrhistogram" -version = "7.5.2" +version = "7.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f19b9f54f7c7f55e31401bb647626ce0cf0f67b0004982ce815b3ee72a02aa8" +checksum = "6ea9fe3952d32674a14e0975009a3547af9ea364995b5ec1add2e23c2ae523ab" dependencies = [ "base64", "byteorder", @@ -3394,7 +3033,7 @@ checksum = "f3e372db8e5c0d213e0cd0b9be18be2aca3d44cf2fe30a9d46a65581cd454584" dependencies = [ "base64", "bitflags", - "bytes 1.3.0", + "bytes 1.2.1", "headers-core", "http", "httpdate", @@ -3445,17 +3084,17 @@ name = "heim-common" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "core-foundation", "futures-core", "futures-util", "lazy_static", "libc", "mach", - "nix 0.23.2", + "nix 0.23.1", "pin-utils", "uom", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -3463,8 +3102,8 @@ name = "heim-cpu" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if", - "futures 0.3.25", + "cfg-if 1.0.0", + "futures 0.3.24", "glob", "heim-common", "heim-runtime", @@ -3473,7 +3112,7 @@ dependencies = [ "mach", "ntapi", "smol", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -3482,14 +3121,14 @@ version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ "bitflags", - "cfg-if", + "cfg-if 1.0.0", "core-foundation", "heim-common", "heim-runtime", "libc", "mach", "widestring 0.4.3", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -3497,7 +3136,7 @@ name = "heim-host" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "heim-common", "heim-runtime", "lazy_static", @@ -3506,7 +3145,7 @@ dependencies = [ "mach", "ntapi", "platforms", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -3514,13 +3153,13 @@ name = "heim-memory" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "heim-common", "heim-runtime", "lazy_static", "libc", "mach", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -3529,14 +3168,14 @@ version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ "bitflags", - "cfg-if", + "cfg-if 1.0.0", "heim-common", "heim-runtime", "libc", "macaddr", - "nix 0.23.2", + "nix 0.23.1", "widestring 0.4.3", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -3544,7 +3183,7 @@ name = "heim-runtime" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "futures 0.3.25", + "futures 0.3.24", "futures-timer", "once_cell", "smol", @@ -3559,15 +3198,6 @@ dependencies = [ "libc", ] -[[package]] -name = "hermit-abi" -version = "0.2.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee512640fe35acbfb4bb779db6f0d80704c2cacfa2e39b601ef3e3f47d1ae4c7" -dependencies = [ - "libc", -] - [[package]] name = "hex" version = "0.4.3" @@ -3580,7 +3210,7 @@ version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e" dependencies = [ - "digest 0.10.6", + "digest 0.10.3", ] [[package]] @@ -3591,7 +3221,7 @@ checksum = "3c731c3e10504cc8ed35cfe2f1db4c9274c3d35fa486e3b31df46f068ef3e867" dependencies = [ "libc", "match_cfg", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -3600,9 +3230,9 @@ version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "75f43d41e26995c17e71ee126451dd3941010b0514a81a9d11f3b341debc2399" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "fnv", - "itoa 1.0.4", + "itoa 1.0.1", ] [[package]] @@ -3611,7 +3241,7 @@ version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d5f38f16d184e36f2408a55281cd658ecbd3ca05cce6d6510a176eca393e26d1" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "http", "pin-project-lite", ] @@ -3645,9 +3275,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.8.0" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" +checksum = "496ce29bb5a52785b44e0f7ca2847ae0bb839c9bd28f69acac9b99d461c0c04c" [[package]] name = "httpdate" @@ -3663,11 +3293,11 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.23" +version = "0.14.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "034711faac9d2166cb1baf1a2fb0b60b1f277f8492fd72176c17f3515e1abd3c" +checksum = "02c929dc5c39e335a03c405292728118860721b10190d98c2a0f0efd5baafbac" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "futures-channel", "futures-core", "futures-util", @@ -3676,12 +3306,12 @@ dependencies = [ "http-body", "httparse", "httpdate", - "itoa 1.0.4", + "itoa 1.0.1", "pin-project-lite", "socket2", "tokio", "tower-service", - "tracing 0.1.37", + "tracing 0.1.34", "want", ] @@ -3709,8 +3339,8 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ca815a891b24fdfb243fa3239c86154392b0953ee584aa1a2a1f66d20cbe75cc" dependencies = [ - "bytes 1.3.0", - "futures 0.3.25", + "bytes 1.2.1", + "futures 0.3.24", "headers", "http", "hyper", @@ -3722,17 +3352,34 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.2" +version = "0.22.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f9f7a97316d44c0af9b0301e65010573a853a9fc97046d7331d7f6bc0fd5a64" +dependencies = [ + "ct-logs", + "futures-util", + "hyper", + "log", + "rustls 0.19.1", + "rustls-native-certs 0.5.0", + "tokio", + "tokio-rustls 0.22.0", + "webpki 0.21.4", +] + +[[package]] +name = "hyper-rustls" +version = "0.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" +checksum = "d87c48c02e0dc5e3b849a2041db3029fd066650f8f717c07bf8ed78ccb895cac" dependencies = [ "http", "hyper", "log", - "rustls 0.20.7", + "rustls 0.20.4", "rustls-native-certs 0.6.2", "tokio", - "tokio-rustls", + "tokio-rustls 0.23.3", ] [[package]] @@ -3753,7 +3400,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d6183ddfa99b85da61a140bea0efc93fdf56ceaa041b37d553518030827f9905" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "hyper", "native-tls", "tokio", @@ -3773,30 +3420,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "iana-time-zone" -version = "0.1.53" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64c122667b287044802d6ce17ee2ddf13207ed924c712de9a66a5814d5b64765" -dependencies = [ - "android_system_properties", - "core-foundation-sys", - "iana-time-zone-haiku", - "js-sys", - "wasm-bindgen", - "winapi", -] - -[[package]] -name = "iana-time-zone-haiku" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" -dependencies = [ - "cxx", - "cxx-build", -] - [[package]] name = "ident_case" version = "1.0.1" @@ -3814,24 +3437,14 @@ dependencies = [ "unicode-normalization", ] -[[package]] -name = "idna" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e14ddfc70884202db2244c223200c204c2bda1bc6e0998d11b5e024d657209e6" -dependencies = [ - "unicode-bidi", - "unicode-normalization", -] - [[package]] name = "indexmap" -version = "1.9.2" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885e79c1fc4b10f0e172c475f458b7f7b93061064d98c3293e98c5ba0c8b399" +checksum = "10a35a97730320ffe8e2d410b5d3b69279b98d2c14bdb8b70ea89ecf7888d41e" dependencies = [ "autocfg", - "hashbrown 0.12.3", + "hashbrown", "serde", ] @@ -3849,9 +3462,9 @@ checksum = "64e9829a50b42bb782c1df523f78d332fe371b10c661e78b7a3c34b0198e9fac" [[package]] name = "infer" -version = "0.11.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a6c16b11a665b26aeeb9b1d7f954cdeb034be38dd00adab4f2ae921a8fee804" +checksum = "f178e61cdbfe084aa75a2f4f7a25a5bb09701a47ae1753608f194b15783c937a" [[package]] name = "inherent" @@ -3866,9 +3479,9 @@ dependencies = [ [[package]] name = "inotify" -version = "0.9.6" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8069d3ec154eb856955c1c0fbffefbf5f3c40a104ec912d4797314c1801abff" +checksum = "4816c66d2c8ae673df83366c18341538f234a26d65a9ecea5c348b453ac1d02f" dependencies = [ "bitflags", "inotify-sys", @@ -3900,14 +3513,14 @@ version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", ] [[package]] name = "inventory" -version = "0.3.2" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e21e0a36a4dc4b469422ee17f715e8313f4a637675656d6a13637954278c6f55" +checksum = "30a61b8101d87996f82d725ba701b1987b7afc72f481c13513a30b855b9c9133" dependencies = [ "ctor", "ghost", @@ -3915,13 +3528,9 @@ dependencies = [ [[package]] name = "io-lifetimes" -version = "1.0.3" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46112a93252b123d31a119a8d1a1ac19deac4fac6e0e8b0df58f0d4e5870e63c" -dependencies = [ - "libc", - "windows-sys 0.42.0", -] +checksum = "9448015e586b611e5d322f6703812bbca2f1e709d5773ecd38ddb4e3bb649504" [[package]] name = "iovec" @@ -3934,21 +3543,21 @@ dependencies = [ [[package]] name = "ipconfig" -version = "0.3.1" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd302af1b90f2463a98fa5ad469fc212c8e3175a41c3068601bfa2727591c5be" +checksum = "723519edce41262b05d4143ceb95050e4c614f483e78e9fd9e39a8275a84ad98" dependencies = [ "socket2", "widestring 0.5.1", - "winapi", - "winreg", + "winapi 0.3.9", + "winreg 0.7.0", ] [[package]] name = "ipnet" -version = "2.6.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec947b7a4ce12e3b87e353abae7ce124d025b6c7d6c5aea5cc0bcf92e9510ded" +checksum = "879d54834c8c76457ef4293a689b2a8c59b076067ad77b15efafbb05f92a592b" [[package]] name = "ipnetwork" @@ -3960,22 +3569,19 @@ dependencies = [ ] [[package]] -name = "is-terminal" -version = "0.4.1" +name = "itertools" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "927609f78c2913a6f6ac3c27a4fe87f43e2a35367c0c4b0f8265e8f49a104330" +checksum = "284f18f85651fe11e8a991b2adb42cb078325c996ed026d994719efcfca1d54b" dependencies = [ - "hermit-abi 0.2.6", - "io-lifetimes", - "rustix", - "windows-sys 0.42.0", + "either", ] [[package]] name = "itertools" -version = "0.10.5" +version = "0.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +checksum = "a9a9d19fa1e79b6215ff29b9d6880b706147f16e9b1dbb1e4e5947b5b02bc5e3" dependencies = [ "either", ] @@ -3988,18 +3594,18 @@ checksum = "b71991ff56294aa922b450139ee08b3bfc70982c6b2c7562771375cf73542dd4" [[package]] name = "itoa" -version = "1.0.4" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4217ad341ebadf8d8e724e264f13e593e0648f5b3e94b3896a5df283be015ecc" +checksum = "1aab8fc367588b89dcee83ab0fd66b72b50b72fa1904d7095045ace2b0c81c35" [[package]] name = "jni" -version = "0.20.0" +version = "0.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "039022cdf4d7b1cf548d31f60ae783138e5fd42013f6271049d7df7afadef96c" +checksum = "c6df18c2e3db7e453d3c6ac5b3e9d5182664d28788126d39b91f2d1e22b017ec" dependencies = [ "cesu8", - "combine 4.6.6", + "combine 4.6.4", "jni-sys", "log", "thiserror", @@ -4014,18 +3620,18 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.25" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "068b1ee6743e4d11fb9c6a1e6064b3693a1b600e7f5f5988047d98b3dc9fb90b" +checksum = "af25a77299a7f711a01975c35a6a424eb6862092cc2d6c72c4ed6cbc56dfc1fa" dependencies = [ "libc", ] [[package]] name = "js-sys" -version = "0.3.60" +version = "0.3.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49409df3e3bf0856b916e2ceaca09ee28e6871cf7d9ce97a692cacfdb2a25a47" +checksum = "671a26f820db17c2a2750743f1dd03bafd15b98c9f30c7c2628c024c05d73397" dependencies = [ "wasm-bindgen", ] @@ -4038,9 +3644,9 @@ checksum = "078e285eafdfb6c4b434e0d31e8cfcb5115b651496faca5749b88fafd4f23bfd" [[package]] name = "json-patch" -version = "0.2.7" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3fa5a61630976fc4c353c70297f2e93f1930e3ccee574d59d618ccbd5154ce" +checksum = "f995a3c8f2bc3dd52a18a583e90f9ec109c047fa1603a853e46bcda14d2e279d" dependencies = [ "serde", "serde_json", @@ -4062,8 +3668,8 @@ dependencies = [ name = "k8s-e2e-tests" version = "0.1.0" dependencies = [ - "env_logger 0.10.0", - "futures 0.3.25", + "env_logger 0.9.0", + "futures 0.3.24", "indoc", "k8s-openapi", "k8s-test-framework", @@ -4072,22 +3678,22 @@ dependencies = [ "reqwest", "serde_json", "tokio", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "k8s-openapi" -version = "0.16.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d9455388f4977de4d0934efa9f7d36296295537d774574113a20f6082de03da" +checksum = "d2ae2c04fcee6b01b04e3aadd56bb418932c8e0a9d8a93f48bc68c6bdcdb559d" dependencies = [ "base64", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "http", "percent-encoding", "serde", - "serde-value", + "serde-value 0.7.0", "serde_json", "url", ] @@ -4105,31 +3711,18 @@ dependencies = [ [[package]] name = "keccak" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3afef3b6eff9ce9d8ff9b3601125eec7f0c8cbac7abd14f355d053fa56c98768" -dependencies = [ - "cpufeatures", -] - -[[package]] -name = "kqueue" -version = "1.0.7" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c8fc60ba15bf51257aa9807a48a61013db043fcf3a78cb0d916e8e396dcad98" -dependencies = [ - "kqueue-sys", - "libc", -] +checksum = "67c21572b4949434e4fc1e1978b99c5f77064153c59d998bf13ecd96fb5ecba7" [[package]] -name = "kqueue-sys" -version = "1.0.3" +name = "kernel32-sys" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8367585489f01bc55dd27404dcf56b95e6da061a256a666ab23be9ba96a2e587" +checksum = "7507624b29483431c0ba2d82aece8ca6cdba9382bff4ddd0f7490560c056098d" dependencies = [ - "bitflags", - "libc", + "winapi 0.2.8", + "winapi-build", ] [[package]] @@ -4143,9 +3736,9 @@ dependencies = [ [[package]] name = "kube" -version = "0.75.0" +version = "0.73.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9bb19108692aeafebb108fd0a1c381c06ac4c03859652599420975165e939b8a" +checksum = "f68b954ea9ad888de953fb1488bd8f377c4c78d82d4642efa5925189210b50b7" dependencies = [ "k8s-openapi", "kube-client", @@ -4155,16 +3748,16 @@ dependencies = [ [[package]] name = "kube-client" -version = "0.75.0" +version = "0.73.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97e1a80ecd1b1438a2fc004549e155d47250b9e01fbfcf4cfbe9c8b56a085593" +checksum = "9150dc7107d9acf4986088f284a0a6dddc5ae37ef1ffdf142f6811dc5998dd58" dependencies = [ "base64", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "dirs-next", "either", - "futures 0.3.25", + "futures 0.3.24", "http", "http-body", "hyper", @@ -4183,17 +3776,17 @@ dependencies = [ "thiserror", "tokio", "tokio-native-tls", - "tokio-util", + "tokio-util 0.7.1", "tower", "tower-http", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "kube-core" -version = "0.75.0" +version = "0.73.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4d780f2bb048eeef64a4c6b2582d26a0fe19e30b4d3cc9e081616e1779c5d47" +checksum = "bc8c429676abe6a73b374438d5ca02caaf9ae7a635441253c589b779fa5d0622" dependencies = [ "chrono", "form_urlencoded", @@ -4208,14 +3801,14 @@ dependencies = [ [[package]] name = "kube-runtime" -version = "0.75.0" +version = "0.73.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7769af142ee2e46bfa44bd393cf7f40b9d8b80d2e11f6317399551ed17760beb" +checksum = "b6e9e9da456f0101b77f864a9da44866b9891ad4740db508b4b269343ebeb01d" dependencies = [ - "ahash 0.8.2", + "ahash", "backoff", "derivative", - "futures 0.3.25", + "futures 0.3.24", "json-patch", "k8s-openapi", "kube-client", @@ -4226,8 +3819,8 @@ dependencies = [ "smallvec", "thiserror", "tokio", - "tokio-util", - "tracing 0.1.37", + "tokio-util 0.7.1", + "tracing 0.1.34", ] [[package]] @@ -4241,7 +3834,7 @@ dependencies = [ "bit-set", "diff", "ena", - "itertools", + "itertools 0.10.3", "lalrpop-util", "petgraph", "pico-args", @@ -4262,34 +3855,18 @@ dependencies = [ "regex", ] -[[package]] -name = "lapin" -version = "2.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd03ea5831b44775e296239a64851e2fd14a80a363d202ba147009ffc994ff0f" -dependencies = [ - "amq-protocol", - "async-global-executor-trait", - "async-reactor-trait", - "async-trait", - "executor-trait", - "flume", - "futures-core", - "futures-io", - "parking_lot", - "pinky-swear", - "reactor-trait", - "serde", - "tracing 0.1.37", - "waker-fn", -] - [[package]] name = "lazy_static" version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +[[package]] +name = "lazycell" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" + [[package]] name = "leveldb" version = "0.8.6" @@ -4314,9 +3891,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.138" +version = "0.2.132" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db6d7e329c562c5dfab7a46a2afabc8b987ab9a4834c9d1ca04dc54c1546cef8" +checksum = "8371e4e5341c3a96db127eb2465ac681ced4c433e01dd0e938adbef26ba93ba5" [[package]] name = "libflate" @@ -4340,15 +3917,15 @@ dependencies = [ [[package]] name = "libm" -version = "0.2.6" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "348108ab3fba42ec82ff6e9564fc4ca0247bdccdc68dd8af9764bbc79c3c8ffb" +checksum = "33a33a362ce288760ec6a508b94caaec573ae7d3bbbd91b87aa0bad4456839db" [[package]] name = "libz-sys" -version = "1.1.8" +version = "1.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9702761c3935f8cc2f101793272e202c72b99da8f4224a19ddcf1279a6450bbf" +checksum = "92e7e15d7610cce1d9752e137625f14e61a28cd45929b6e12e47b50fe154ee2e" dependencies = [ "cc", "libc", @@ -4356,20 +3933,11 @@ dependencies = [ "vcpkg", ] -[[package]] -name = "link-cplusplus" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9272ab7b96c9046fbc5bc56c06c117cb639fe2d509df0c421cad82d2915cf369" -dependencies = [ - "cc", -] - [[package]] name = "linked-hash-map" -version = "0.5.6" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" +checksum = "7fb9b38af92608140b86b693604b9ffcc5824240a484d1ecd4795bacb2fe88f3" [[package]] name = "linked_hash_set" @@ -4382,9 +3950,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.1.3" +version = "0.0.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f9f08d8963a6c613f4b1a78f4f4a4dbfadf8e6545b2d72861731e4858b8b47f" +checksum = "5284f00d480e1c39af34e72f8ad60b94f47007e3481cd3b731c1d67190ddc7b7" [[package]] name = "listenfd" @@ -4393,15 +3961,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "14e4fcc00ff6731d94b70e16e71f43bda62883461f31230742e3bc6dddf12988" dependencies = [ "libc", - "uuid 1.2.2", - "winapi", + "uuid 1.1.2", + "winapi 0.3.9", ] [[package]] name = "lock_api" -version = "0.4.9" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "435011366fe56583b16cf956f9df0095b405b82d76425bc8981c0e22e60ec4df" +checksum = "327fa5b6a6940e4699ec49a9beae1ea4845c6bab9314e4f84ac68742139d8c53" dependencies = [ "autocfg", "scopeguard", @@ -4419,7 +3987,7 @@ version = "0.4.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "abb12e687cfb44aa40f41fc3978ef76448f9b6038cad6aef4259d3c095a2382e" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", ] [[package]] @@ -4428,18 +3996,6 @@ version = "0.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "879777f0cc6f3646a044de60e4ab98c75617e3f9580f7a2032e6ad7ea0cd3054" -[[package]] -name = "loki-logproto" -version = "0.1.0" -dependencies = [ - "bytes 1.3.0", - "chrono", - "prost 0.10.4", - "prost-build 0.10.4", - "prost-types 0.10.1", - "snap", -] - [[package]] name = "lookup" version = "0.1.0" @@ -4454,16 +4010,16 @@ dependencies = [ "serde", "serde_json", "snafu", - "tracing 0.1.37", + "tracing 0.1.34", "vector-config", "vector-config-macros", ] [[package]] name = "lru" -version = "0.8.1" +version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6e8aaa3f231bb4bd57b84b2d5dc3ae7f350265df8aa96492e0bc394a1571909" +checksum = "e999beba7b6e8345721bd280141ed958096a2e4abdf74f67ff4ce49b4b54e47a" [[package]] name = "lru-cache" @@ -4485,31 +4041,11 @@ dependencies = [ [[package]] name = "luajit-src" -version = "210.4.3+resty8384278" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19ee5d5afddf1ec76ffa55ca7c3001f2f8a703834beba53c56a38ea6641cef44" -dependencies = [ - "cc", -] - -[[package]] -name = "lz4" -version = "1.24.0" +version = "210.4.0+resty124ff8d" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" -dependencies = [ - "libc", - "lz4-sys", -] - -[[package]] -name = "lz4-sys" -version = "1.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" +checksum = "f76fb2e2c0c7192e18719d321c9a148f7625c4dcbe3df5f4c19e685e4c286f6c" dependencies = [ "cc", - "libc", ] [[package]] @@ -4527,15 +4063,6 @@ dependencies = [ "libc", ] -[[package]] -name = "malloc_buf" -version = "0.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62bb907fe88d54d8d9ce32a3cceab4218ed2f6b7d35617cafe9adf84e43919cb" -dependencies = [ - "libc", -] - [[package]] name = "match_cfg" version = "0.1.0" @@ -4572,12 +4099,6 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73cbba799671b762df5a175adf59ce145165747bb891505c43d09aefbbf38beb" -[[package]] -name = "matchit" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b87248edafb776e59e6ee64a79086f65890d3510f2c656c000bf2a7e8a0aea40" - [[package]] name = "matrixmultiply" version = "0.3.2" @@ -4601,11 +4122,11 @@ dependencies = [ [[package]] name = "md-5" -version = "0.10.5" +version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6365506850d44bff6e2fbcb5176cf63650e48bd45ef2fe2665ae1570e0f4b9ca" +checksum = "66b48670c893079d3c2ed79114e3644b7004df1c361a4e0ad52e2e6940d07c3d" dependencies = [ - "digest 0.10.6", + "digest 0.10.3", ] [[package]] @@ -4622,9 +4143,9 @@ checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" [[package]] name = "memmap2" -version = "0.5.8" +version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b182332558b18d807c4ce1ca8ca983b34c3ee32765e47b3f0f69b90355cc1dc" +checksum = "95af15f345b17af2efc8ead6080fb8bc376f8cec1b35277b935637595fe77498" dependencies = [ "libc", ] @@ -4638,22 +4159,13 @@ dependencies = [ "autocfg", ] -[[package]] -name = "memoffset" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5de893c32cde5f383baa4c04c5d6dbdd735cfd4a794b0debdb2bb1b421da5ff4" -dependencies = [ - "autocfg", -] - [[package]] name = "metrics" version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b9b8653cec6897f73b519a43fba5ee3d50f62fe9af80b428accdcc093b4a849" dependencies = [ - "ahash 0.7.6", + "ahash", "metrics-macros", "portable-atomic", ] @@ -4675,14 +4187,14 @@ version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6097e2772147f332c9aedba572e9cd334b7946e1762d8ae4d05db0faf962542a" dependencies = [ - "itoa 1.0.4", + "itoa 1.0.1", "lockfree-object-pool", "metrics", "metrics-util", "once_cell", - "tracing 0.1.37", - "tracing-core 0.1.30", - "tracing-subscriber 0.3.16", + "tracing 0.1.34", + "tracing-core 0.1.28", + "tracing-subscriber 0.3.15", ] [[package]] @@ -4694,7 +4206,7 @@ dependencies = [ "aho-corasick", "crossbeam-epoch", "crossbeam-utils", - "hashbrown 0.12.3", + "hashbrown", "indexmap", "metrics", "num_cpus", @@ -4730,32 +4242,86 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.6.2" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b275950c28b37e794e8c55d88aeb5e139d0ce23fdbbeda68f8d7174abdf9e8fa" +checksum = "d2b29bd4bc3f33391105ebee3589c19197c4271e3e5a9ec9bfe8127eeff8f082" dependencies = [ "adler", ] [[package]] name = "mio" -version = "0.8.5" +version = "0.6.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4afd66f5b91bf2a3bc13fad0e21caedac168ca4c707504e75585648ae80e4cc4" +dependencies = [ + "cfg-if 0.1.10", + "fuchsia-zircon", + "fuchsia-zircon-sys", + "iovec", + "kernel32-sys", + "libc", + "log", + "miow 0.2.2", + "net2", + "slab", + "winapi 0.2.8", +] + +[[package]] +name = "mio" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5d732bc30207a6423068df043e3d02e0735b155ad7ce1a6f76fe2baa5b158de" +checksum = "52da4364ffb0e4fe33a9841a98a3f3014fb964045ce4f7a45a398243c8d6b0c9" dependencies = [ "libc", "log", + "miow 0.3.7", + "ntapi", "wasi 0.11.0+wasi-snapshot-preview1", - "windows-sys 0.42.0", + "winapi 0.3.9", +] + +[[package]] +name = "mio-extras" +version = "2.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52403fe290012ce777c4626790c8951324a2b9e3316b3143779c72b029742f19" +dependencies = [ + "lazycell", + "log", + "mio 0.6.23", + "slab", +] + +[[package]] +name = "miow" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebd808424166322d4a38da87083bfddd3ac4c131334ed55856112eb06d46944d" +dependencies = [ + "kernel32-sys", + "net2", + "winapi 0.2.8", + "ws2_32-sys", +] + +[[package]] +name = "miow" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9f1c5b025cda876f66ef43a113f91ebc9f4ccef34843000e0adf6ebbab84e21" +dependencies = [ + "winapi 0.3.9", ] [[package]] name = "mlua" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4351dbcc863fb6249c81b3bd0c8001214e9d4d44d22cabda17026353a77fe612" +checksum = "10277581090f5cb7ecf814bc611152ce4db6dc8deffcaa08e24ed4c5197d9186" dependencies = [ - "bstr 0.2.17", + "bstr", "cc", "lua-src", "luajit-src", @@ -4773,9 +4339,9 @@ checksum = "717e29a243b81f8130e31e24e04fb151b04a44b5a7d05370935f7d937e9de06d" [[package]] name = "mongodb" -version = "2.3.1" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5a1df476ac9541b0e4fdc8e2cc48884e66c92c933cd17a1fd75e68caf75752e" +checksum = "b95afe97b0c799fdf69cd960272a2cb9662d077bd6efd84eb722bb9805d47554" dependencies = [ "async-trait", "base64", @@ -4795,22 +4361,22 @@ dependencies = [ "percent-encoding", "rand 0.8.5", "rustc_version_runtime", - "rustls 0.20.7", + "rustls 0.20.4", "rustls-pemfile 0.3.0", "serde", "serde_bytes", "serde_with 1.14.0", - "sha-1", - "sha2 0.10.6", + "sha-1 0.10.0", + "sha2 0.10.5", "socket2", "stringprep", "strsim 0.10.0", "take_mut", "thiserror", "tokio", - "tokio-rustls", - "tokio-util", - "trust-dns-proto 0.21.2", + "tokio-rustls 0.23.3", + "tokio-util 0.7.1", + "trust-dns-proto", "trust-dns-resolver", "typed-builder 0.10.0", "uuid 0.8.2", @@ -4819,11 +4385,11 @@ dependencies = [ [[package]] name = "multer" -version = "2.0.4" +version = "2.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ed4198ce7a4cbd2a57af78d28c6fbb57d81ac5f1d6ad79ac6c5587419cbdf22" +checksum = "5f8f35e687561d5c1667590911e6698a8cb714a134a7505718a182e7bc9d3836" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "encoding_rs", "futures-util", "http", @@ -4831,7 +4397,7 @@ dependencies = [ "log", "memchr", "mime", - "spin 0.9.4", + "spin 0.9.3", "version_check", ] @@ -4843,9 +4409,9 @@ checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" [[package]] name = "native-tls" -version = "0.2.11" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07226173c32f2926027b63cce4bcd8076c3552846cbe7925f3aaffeac0a3b92e" +checksum = "fd7e2f3618557f980e0b17e8856252eee3c97fa12c54dff0ca290fb6266ca4a9" dependencies = [ "lazy_static", "libc", @@ -4861,16 +4427,16 @@ dependencies = [ [[package]] name = "nats" -version = "0.23.1" +version = "0.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3d877cd2e71146efa7065300fc5f5da967f938694b4d65e8bc64cc4a409092c" +checksum = "3d61e51453596562c82656b4fb004071b5b0c214183be3d02fbb7b16ba2370df" dependencies = [ "base64", "base64-url", "blocking", "crossbeam-channel", "fastrand", - "itoa 1.0.4", + "itoa 1.0.1", "json", "lazy_static", "libc", @@ -4881,7 +4447,6 @@ dependencies = [ "once_cell", "parking_lot", "regex", - "ring", "rustls 0.19.1", "rustls-native-certs 0.5.0", "rustls-pemfile 0.2.1", @@ -4892,7 +4457,7 @@ dependencies = [ "time", "url", "webpki 0.21.4", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -4915,7 +4480,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af5a8477ac96877b5bd1fd67e0c28736c12943aba24eda92b127e036b0c8f400" dependencies = [ "indexmap", - "itertools", + "itertools 0.10.3", "ndarray", "noisy_float", "num-integer", @@ -4923,12 +4488,73 @@ dependencies = [ "rand 0.8.5", ] +[[package]] +name = "ndk" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2032c77e030ddee34a6787a64166008da93f6a352b629261d0fee232b8742dd4" +dependencies = [ + "bitflags", + "jni-sys", + "ndk-sys", + "num_enum", + "thiserror", +] + [[package]] name = "ndk-context" version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "27b02d87554356db9e9a873add8782d4ea6e3e58ea071a9adb9a2e8ddb884a8b" +[[package]] +name = "ndk-glue" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d0c4a7b83860226e6b4183edac21851f05d5a51756e97a1144b7f5a6b63e65f" +dependencies = [ + "lazy_static", + "libc", + "log", + "ndk", + "ndk-context", + "ndk-macro", + "ndk-sys", +] + +[[package]] +name = "ndk-macro" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0df7ac00c4672f9d5aece54ee3347520b7e20f158656c7db2e6de01902eb7a6c" +dependencies = [ + "darling 0.13.4", + "proc-macro-crate", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "ndk-sys" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e5a6ae77c8ee183dcbbba6150e2e6b9f3f4196a7666c02a715a95692ec1fa97" +dependencies = [ + "jni-sys", +] + +[[package]] +name = "net2" +version = "0.2.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "391630d12b68002ae1e25e8f974306474966550ad82dac6886fb8910c19568ae" +dependencies = [ + "cfg-if 0.1.10", + "libc", + "winapi 0.3.9", +] + [[package]] name = "new_debug_unreachable" version = "1.0.4" @@ -4946,39 +4572,39 @@ dependencies = [ [[package]] name = "nix" -version = "0.23.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f3790c00a0150112de0f4cd161e3d7fc4b2d8a5542ffc35f099a2562aecb35c" +checksum = "9f866317acbd3a240710c63f065ffb1e4fd466259045ccb504130b7f668f35c6" dependencies = [ "bitflags", "cc", - "cfg-if", + "cfg-if 1.0.0", "libc", - "memoffset 0.6.5", + "memoffset", ] [[package]] name = "nix" -version = "0.24.3" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa52e972a9a719cecb6864fb88568781eb706bac2cd1d4f04a648542dbf78069" +checksum = "195cdbc1741b8134346d515b3a56a1c94b0912758009cfd53f99ea0f57b065fc" dependencies = [ "bitflags", - "cfg-if", + "cfg-if 1.0.0", "libc", ] [[package]] name = "nix" -version = "0.25.1" +version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f346ff70e7dbfd675fe90590b92d59ef2de15a8779ae305ebcbfd3f0caf59be4" +checksum = "e322c04a9e3440c327fca7b6c8a63e6890a32fa2ad689db972425f07e0d22abb" dependencies = [ "autocfg", "bitflags", - "cfg-if", + "cfg-if 1.0.0", "libc", - "memoffset 0.6.5", + "memoffset", ] [[package]] @@ -4990,7 +4616,7 @@ dependencies = [ "byteorder", "data-encoding", "ed25519-dalek", - "getrandom 0.2.8", + "getrandom 0.2.6", "log", "rand 0.8.5", "signatory", @@ -5012,6 +4638,12 @@ version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b93853da6d84c2e3c7d730d6473e8817692dd89be387eb01b94d7f108ecb5b8c" +[[package]] +name = "nodrop" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72ef4a56884ca558e5ddb05a1d1e7e1bfd9a68d9ed024c21704cc98872dae1bb" + [[package]] name = "noisy_float" version = "0.2.0" @@ -5039,19 +4671,20 @@ checksum = "38bf9645c8b145698bb0b18a4637dcacbc421ea49bef2317e4fd8065a387cf21" [[package]] name = "notify" -version = "5.0.0" +version = "4.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed2c66da08abae1c024c01d635253e402341b4060a12e99b31c7594063bf490a" +checksum = "ae03c8c853dba7bfd23e571ff0cff7bc9dceb40a4cd684cd1681824183f45257" dependencies = [ "bitflags", "filetime", + "fsevent", "fsevent-sys", "inotify", - "kqueue", "libc", - "mio", + "mio 0.6.23", + "mio-extras", "walkdir", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -5060,17 +4693,7 @@ version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28774a7fd2fbb4f0babd8237ce554b73af68021b5f695a3cebd6c59bac0980f" dependencies = [ - "winapi", -] - -[[package]] -name = "nu-ansi-term" -version = "0.46.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" -dependencies = [ - "overload", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -5107,29 +4730,29 @@ dependencies = [ [[package]] name = "num-complex" -version = "0.4.2" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ae39348c8bc5fbd7f40c727a9925f03517afd2ab27d46702108b6a7e5414c19" +checksum = "26873667bbbb7c5182d4a37c1add32cdf09f841af72da53318fdb81543c15085" dependencies = [ "num-traits", ] [[package]] name = "num-format" -version = "0.4.4" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" +checksum = "bafe4179722c2894288ee77a9f044f02811c86af699344c498b0840c698a2465" dependencies = [ - "arrayvec 0.7.2", - "itoa 1.0.4", - "num-bigint 0.4.3", + "arrayvec 0.4.12", + "itoa 0.4.8", + "num-bigint 0.2.6", ] [[package]] name = "num-integer" -version = "0.1.45" +version = "0.1.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" +checksum = "d2cc698a63b549a70bc047073d2949cce27cd1c7b0a4a862d08a8031bc2801db" dependencies = [ "autocfg", "num-traits", @@ -5158,11 +4781,11 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.14.0" +version = "1.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6058e64324c71e02bc2b150e4f3bc8286db6c83092132ffa3f6b1eab0f9def5" +checksum = "19e64526ebdee182341572e50e9ad03965aa510cd94427a4549448f285e957a1" dependencies = [ - "hermit-abi 0.1.19", + "hermit-abi", "libc", ] @@ -5181,7 +4804,7 @@ version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3b0498641e53dd6ac1a4f22547548caa6864cc4933784319cd1775271c5a46ce" dependencies = [ - "proc-macro-crate 1.2.1", + "proc-macro-crate", "proc-macro2", "quote", "syn", @@ -5189,9 +4812,9 @@ dependencies = [ [[package]] name = "num_threads" -version = "0.1.6" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2819ce041d2ee131036f4fc9d6ae7ae125a3a40e97ba64d04fe799ad9dabbb44" +checksum = "aba1801fb138d8e85e11d0fc70baf4fe1cdfffda7c6cd34a854905df588e5ed0" dependencies = [ "libc", ] @@ -5204,33 +4827,24 @@ checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" [[package]] name = "oauth2" -version = "4.3.0" +version = "4.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eeaf26a72311c087f8c5ba617c96fac67a5c04f430e716ac8d8ab2de62e23368" +checksum = "80e47cfc4c0a1a519d9a025ebfbac3a2439d1b5cdf397d72dcb79b11d9920dab" dependencies = [ "base64", "chrono", - "getrandom 0.2.8", + "getrandom 0.2.6", "http", "rand 0.8.5", "reqwest", "serde", "serde_json", "serde_path_to_error", - "sha2 0.10.6", + "sha2 0.9.9", "thiserror", "url", ] -[[package]] -name = "objc" -version = "0.2.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "915b1b472bc21c53464d6c8461c9d3af805ba1ef837e1cac254428f4a77177b1" -dependencies = [ - "malloc_buf", -] - [[package]] name = "ofb" version = "0.6.1" @@ -5242,9 +4856,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.16.0" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86f0b0d4bf799edbc74508c1e8bf170ff5f41238e5f8225603ca7caaae2b7860" +checksum = "2f7254b99e31cad77da24b08ebf628882739a608578bb1bcdfc1f9c21260d7c0" [[package]] name = "onig" @@ -5282,36 +4896,37 @@ checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" [[package]] name = "openidconnect" -version = "2.4.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87af7097640fedbe64718ac1c9b0549d72da747a3f527cd089215f96c6f691d5" +checksum = "691c1ba89b0a112f3062b946ef160711c3aea33e1476b6877a904f2f83856781" dependencies = [ "base64", "chrono", "http", - "itertools", + "itertools 0.9.0", "log", "num-bigint 0.4.3", "oauth2", "rand 0.8.5", "ring", "serde", - "serde-value", + "serde-value 0.6.0", "serde_derive", "serde_json", "serde_path_to_error", "thiserror", + "untrusted", "url", ] [[package]] name = "openssl" -version = "0.10.44" +version = "0.10.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29d971fd5722fec23977260f6e81aa67d2f22cadbdc2aa049f1022d9a3be1566" +checksum = "618febf65336490dfcf20b73f885f5651a0c89c64c2d4a8c3662585a70bf5bd0" dependencies = [ "bitflags", - "cfg-if", + "cfg-if 1.0.0", "foreign-types", "libc", "once_cell", @@ -5346,9 +4961,9 @@ dependencies = [ [[package]] name = "openssl-sys" -version = "0.9.79" +version = "0.9.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5454462c0eced1e97f2ec09036abc8da362e66802f66fd20f86854d9d8cbcbc4" +checksum = "e5f9bd0c2710541a3cda73d6f9ac4f1b240de4ae261065d309dbe73d9dceb42f" dependencies = [ "autocfg", "cc", @@ -5362,18 +4977,27 @@ dependencies = [ name = "opentelemetry-proto" version = "0.0.0" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "hex", - "ordered-float 3.4.0", - "prost 0.11.3", - "prost-build 0.11.3", - "tonic", + "ordered-float 3.0.0", + "prost 0.10.4", + "prost-build", + "tonic 0.7.2", "tonic-build", "value", "vector-core", ] +[[package]] +name = "ordered-float" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" +dependencies = [ + "num-traits", +] + [[package]] name = "ordered-float" version = "2.10.0" @@ -5385,44 +5009,47 @@ dependencies = [ [[package]] name = "ordered-float" -version = "3.4.0" +version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d84eb1409416d254e4a9c8fa56cc24701755025b458f0fcd8e59e1f5f40c23bf" +checksum = "96bcbab4bfea7a59c2c0fe47211a1ac4e3e96bea6eb446d704f310bc5c732ae2" dependencies = [ "num-traits", ] [[package]] name = "os_info" -version = "3.5.1" +version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4750134fb6a5d49afc80777394ad5d95b04bc12068c6abb92fae8f43817270f" +checksum = "023df84d545ef479cf67fd2f4459a613585c9db4852c2fad12ab70587859d340" dependencies = [ "log", - "winapi", + "winapi 0.3.9", ] [[package]] name = "os_pipe" -version = "1.1.2" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6a252f1f8c11e84b3ab59d7a488e48e4478a93937e027076638c49536204639" +checksum = "fb233f06c2307e1f5ce2ecad9f8121cffbbee2c95428f44ea85222e460d0d213" dependencies = [ "libc", - "windows-sys 0.42.0", + "winapi 0.3.9", ] [[package]] name = "os_str_bytes" -version = "6.4.1" +version = "6.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b7820b9daea5457c9f21c69448905d723fbd21136ccf521748f23fd49e723ee" +checksum = "8e22443d1643a904602595ba1cd8f7d896afe56d26712531c5ff73a15b2fbf64" [[package]] -name = "overload" -version = "0.1.1" +name = "output_vt100" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" +checksum = "628223faebab4e3e40667ee0b2336d34a5b960ff60ea743ddfdbcf7770bcfb66" +dependencies = [ + "winapi 0.3.9", +] [[package]] name = "parking" @@ -5442,15 +5069,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.5" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ff9f3fef3968a3ec5945535ed654cb38ff72d7495a25619e2247fb15a2ed9ba" +checksum = "995f667a6c822200b0433ac218e05582f0e2efa1b922a3fd2fbaadc5f87bab37" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "libc", - "redox_syscall 0.2.16", + "redox_syscall 0.2.13", "smallvec", - "windows-sys 0.42.0", + "windows-sys 0.34.0", ] [[package]] @@ -5474,7 +5101,7 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271779f35b581956db91a3e55737327a03aa051e90b1c47aeb189508533adfd7" dependencies = [ - "digest 0.10.6", + "digest 0.10.3", ] [[package]] @@ -5485,9 +5112,9 @@ checksum = "9e9ed2178b0575fff8e1b83b58ba6f75e727aafac2e1b6c795169ad3b17eb518" [[package]] name = "pem" -version = "1.1.0" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c64931a1a212348ec4f3b4362585eca7159d0d09cbdf4a7f74f02173596fd4" +checksum = "e9a3b09a20e374558580a4914d3b7d89bd61b954a5a5e1dcbea98753addb1947" dependencies = [ "base64", ] @@ -5503,15 +5130,15 @@ dependencies = [ [[package]] name = "percent-encoding" -version = "2.2.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "478c572c3d73181ff3c2539045f6eb99e5491218eae919370993b890cdbdd98e" +checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" [[package]] name = "pest" -version = "2.5.1" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc8bed3549e0f9b0a2a78bf7c0018237a2cdf085eecbbc048e52612438e4e9d0" +checksum = "4b0560d531d1febc25a3c9398a62a71256c0178f2e3443baedd9ad4bb8c9deb4" dependencies = [ "thiserror", "ucd-trie", @@ -5519,9 +5146,9 @@ dependencies = [ [[package]] name = "pest_derive" -version = "2.5.1" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdc078600d06ff90d4ed238f0119d84ab5d43dbaad278b0e33a8820293b32344" +checksum = "905708f7f674518498c1f8d644481440f476d39ca6ecae83319bba7c6c12da91" dependencies = [ "pest", "pest_generator", @@ -5529,9 +5156,9 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.5.1" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28a1af60b1c4148bb269006a750cff8e2ea36aff34d2d96cf7be0b14d1bed23c" +checksum = "5803d8284a629cc999094ecd630f55e91b561a1d1ba75e233b00ae13b91a69ad" dependencies = [ "pest", "pest_meta", @@ -5542,20 +5169,20 @@ dependencies = [ [[package]] name = "pest_meta" -version = "2.5.1" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fec8605d59fc2ae0c6c1aefc0c7c7a9769732017c0ce07f7a9cfffa7b4404f20" +checksum = "1538eb784f07615c6d9a8ab061089c6c54a344c5b4301db51990ca1c241e8c04" dependencies = [ "once_cell", "pest", - "sha1", + "sha-1 0.10.0", ] [[package]] name = "petgraph" -version = "0.6.2" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6d5014253a1331579ce62aa67443b4a658c5e7dd03d4bc6d302b94474888143" +checksum = "4a13a2fa9d0b63e5f22328828741e523766fff0ee9e779316902290dff3f824f" dependencies = [ "fixedbitset", "indexmap", @@ -5563,30 +5190,30 @@ dependencies = [ [[package]] name = "phf" -version = "0.11.1" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "928c6535de93548188ef63bb7c4036bd415cd8f36ad25af44b9789b2ee72a48c" +checksum = "4724fa946c8d1e7cd881bd3dbee63ce32fc1e9e191e35786b3dc1320a3f68131" dependencies = [ - "phf_shared 0.11.1", + "phf_shared 0.11.0", ] [[package]] name = "phf_codegen" -version = "0.11.1" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a56ac890c5e3ca598bbdeaa99964edb5b0258a583a9eb6ef4e89fc85d9224770" +checksum = "32ba0c43d7a1b6492b2924a62290cfd83987828af037b0743b38e6ab092aee58" dependencies = [ "phf_generator", - "phf_shared 0.11.1", + "phf_shared 0.11.0", ] [[package]] name = "phf_generator" -version = "0.11.1" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1181c94580fa345f50f19d738aaa39c0ed30a600d95cb2d3e23f94266f14fbf" +checksum = "5b450720b6f75cfbfabc195814bd3765f337a4f9a83186f8537297cac12f6705" dependencies = [ - "phf_shared 0.11.1", + "phf_shared 0.11.0", "rand 0.8.5", ] @@ -5601,11 +5228,12 @@ dependencies = [ [[package]] name = "phf_shared" -version = "0.11.1" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1fb5f6f826b772a8d4c0394209441e7d37cbbb967ae9c7e0e8134365c9ee676" +checksum = "9dd5609d4b2df87167f908a32e1b146ce309c16cf35df76bc11f440b756048e4" dependencies = [ "siphasher", + "uncased", ] [[package]] @@ -5646,18 +5274,6 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" -[[package]] -name = "pinky-swear" -version = "6.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d894b67aa7a4bf295db5e85349078c604edaa6fa5c8721e8eca3c7729a27f2ac" -dependencies = [ - "doc-comment", - "flume", - "parking_lot", - "tracing 0.1.37", -] - [[package]] name = "pkcs8" version = "0.7.6" @@ -5672,9 +5288,9 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.26" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ac9a59f73473f1b8d852421e59e64809f025994837ef743615c6d0c5b305160" +checksum = "1df8c4ec4b0627e53bdf214615ad287367e482558cf84b109250b37464dc03ae" [[package]] name = "platforms" @@ -5684,9 +5300,9 @@ checksum = "989d43012e2ca1c4a02507c67282691a0a3207f9dc67cec596b43fe925b3d325" [[package]] name = "plotters" -version = "0.3.4" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2538b639e642295546c50fcd545198c9d64ee2a38620a628724a3b266d5fbf97" +checksum = "32a3fd9ec30b9749ce28cd91f255d569591cdf937fe280c312143e3c4bad6f2a" dependencies = [ "num-traits", "plotters-backend", @@ -5697,38 +5313,37 @@ dependencies = [ [[package]] name = "plotters-backend" -version = "0.3.4" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "193228616381fecdc1224c62e96946dfbc73ff4384fba576e052ff8c1bea8142" +checksum = "d88417318da0eaf0fdcdb51a0ee6c3bed624333bff8f946733049380be67ac1c" [[package]] name = "plotters-svg" -version = "0.3.3" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9a81d2759aae1dae668f783c308bc5c8ebd191ff4184aaa1b37f65a6ae5a56f" +checksum = "521fa9638fa597e1dc53e9412a4f9cefb01187ee1f7413076f9e6749e2885ba9" dependencies = [ "plotters-backend", ] [[package]] name = "polling" -version = "2.5.1" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "166ca89eb77fd403230b9c156612965a81e094ec6ec3aa13663d4c8b113fa748" +checksum = "685404d509889fade3e86fe3a5803bca2ec09b0c0778d5ada6ec8bf7a8de5259" dependencies = [ - "autocfg", - "cfg-if", + "cfg-if 1.0.0", "libc", "log", "wepoll-ffi", - "windows-sys 0.42.0", + "winapi 0.3.9", ] [[package]] name = "portable-atomic" -version = "0.3.16" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac662b3a6490de378b0ee15cf2dfff7127aebfe0b19acc65e7fbca3d299c3788" +checksum = "763095e04dcbeb889b2ab35296ecb18a20fe16b4e9877ce64aab73d8fd05a8c3" [[package]] name = "portpicker" @@ -5743,7 +5358,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1de0ea6504e07ca78355a6fb88ad0f36cafe9e696cbc6717f16a207f3a60be72" dependencies = [ - "futures 0.3.25", + "futures 0.3.24", "openssl", "tokio", "tokio-openssl", @@ -5758,13 +5373,13 @@ checksum = "878c6cbf956e03af9aa8204b407b9cbf47c072164800aa918c516cd4b056c50c" dependencies = [ "base64", "byteorder", - "bytes 1.3.0", + "bytes 1.2.1", "fallible-iterator", "hmac", "md-5", "memchr", "rand 0.8.5", - "sha2 0.10.6", + "sha2 0.10.5", "stringprep", ] @@ -5774,7 +5389,7 @@ version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73d946ec7d256b04dfadc4e6a3292324e6f417124750fc5c0950f981b703a0f1" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "fallible-iterator", "postgres-protocol", @@ -5782,9 +5397,9 @@ dependencies = [ [[package]] name = "ppv-lite86" -version = "0.2.17" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" +checksum = "eb9f9e6e233e5c4a35559a617bf40a4ec447db2e84c20b55a6f83167b7e57872" [[package]] name = "precomputed-hash" @@ -5794,31 +5409,43 @@ checksum = "925383efa346730478fb4838dbe9137d2a47675ad789c546d150a6e1dd4ab31c" [[package]] name = "predicates" -version = "2.1.4" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f54fc5dc63ed3bbf19494623db4f3af16842c0d975818e469022d09e53f0aa05" +checksum = "a5aab5be6e4732b473071984b3164dbbfb7a3674d30ea5ff44410b6bcd960c3c" dependencies = [ "difflib", - "itertools", + "itertools 0.10.3", "predicates-core", ] [[package]] name = "predicates-core" -version = "1.0.5" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72f883590242d3c6fc5bf50299011695fa6590c2c70eac95ee1bdb9a733ad1a2" +checksum = "da1c2388b1513e1b605fcec39a95e0a9e8ef088f71443ef37099fa9ae6673fcb" [[package]] name = "predicates-tree" -version = "1.0.7" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54ff541861505aabf6ea722d2131ee980b8276e10a1297b94e896dd8b621850d" +checksum = "4d86de6de25020a36c6d3643a86d9a6a9f552107c0559c60ea03551b5e16c032" dependencies = [ "predicates-core", "termtree", ] +[[package]] +name = "pretty_assertions" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a25e9bcb20aa780fd0bb16b72403a9064d6b3f22f026946029acb941a50af755" +dependencies = [ + "ctor", + "diff", + "output_vt100", + "yansi", +] + [[package]] name = "prettydiff" version = "0.6.1" @@ -5832,9 +5459,9 @@ dependencies = [ [[package]] name = "prettyplease" -version = "0.1.21" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c142c0e46b57171fe0c528bee8c5b7569e80f0c17e377cd0e30ea57dbc11bb51" +checksum = "d9e07e3a46d0771a8a06b5f4441527802830b43e679ba12f44960f48dd4c6803" dependencies = [ "proc-macro2", "syn", @@ -5869,20 +5496,10 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d6ea3c4595b96363c13943497db34af4460fb474a95c43f4446ad341b8c9785" -dependencies = [ - "toml", -] - -[[package]] -name = "proc-macro-crate" -version = "1.2.1" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eda0fc3b0fb7c975631757e14d9049da17374063edb6ebbcbc54d880d4fe94e9" +checksum = "e17d47ce914bf4de440332250b0edd23ce48c005f59fab39d3335866b114f11a" dependencies = [ - "once_cell", "thiserror", "toml", ] @@ -5911,17 +5528,11 @@ dependencies = [ "version_check", ] -[[package]] -name = "proc-macro-hack" -version = "0.5.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbf0c48bc1d91375ae5c3cd81e3722dff1abcf81a30960240640d223f59fe0e5" - [[package]] name = "proc-macro2" -version = "1.0.47" +version = "1.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ea3d908b0e36316caf9e9e2c4625cdde190a7e6f440d794667ed17a1855e725" +checksum = "0a2ca2c61bc9f3d74d2886294ab7b9853abd9c1ad903a3ac7815c58989bb7bab" dependencies = [ "unicode-ident", ] @@ -5933,9 +5544,9 @@ dependencies = [ "indexmap", "nom", "num_enum", - "prost 0.11.3", - "prost-build 0.11.3", - "prost-types 0.11.2", + "prost 0.10.4", + "prost-build", + "prost-types 0.10.1", "snafu", "value", "vector-common", @@ -5967,7 +5578,7 @@ version = "0.1.0" dependencies = [ "chrono", "lookup", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "proptest", "vrl-diagnostic", "vrl-parser", @@ -5979,18 +5590,18 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71adf41db68aa0daaefc69bb30bcd68ded9b9abaad5d1fbb6304c4fb390e083e" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "prost-derive 0.10.1", ] [[package]] name = "prost" -version = "0.11.3" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0b18e655c21ff5ac2084a5ad0611e827b3f92badf79f4910b5a5c58f4d87ff0" +checksum = "399c3c31cdec40583bb68f0b18403400d01ec4289c383aa047560439952c4dd7" dependencies = [ - "bytes 1.3.0", - "prost-derive 0.11.2", + "bytes 1.2.1", + "prost-derive 0.11.0", ] [[package]] @@ -5999,11 +5610,11 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8ae5a4388762d5815a9fc0dea33c56b021cdc8dde0c55e0c9ca57197254b0cab" dependencies = [ - "bytes 1.3.0", - "cfg-if", + "bytes 1.2.1", + "cfg-if 1.0.0", "cmake", "heck 0.4.0", - "itertools", + "itertools 0.10.3", "lazy_static", "log", "multimap", @@ -6015,28 +5626,6 @@ dependencies = [ "which", ] -[[package]] -name = "prost-build" -version = "0.11.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e330bf1316db56b12c2bcfa399e8edddd4821965ea25ddb2c134b610b1c1c604" -dependencies = [ - "bytes 1.3.0", - "heck 0.4.0", - "itertools", - "lazy_static", - "log", - "multimap", - "petgraph", - "prettyplease", - "prost 0.11.3", - "prost-types 0.11.2", - "regex", - "syn", - "tempfile", - "which", -] - [[package]] name = "prost-derive" version = "0.10.1" @@ -6044,7 +5633,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b670f45da57fb8542ebdbb6105a925fe571b67f9e7ed9f47a06a84e72b4e7cc" dependencies = [ "anyhow", - "itertools", + "itertools 0.10.3", "proc-macro2", "quote", "syn", @@ -6052,12 +5641,12 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.11.2" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "164ae68b6587001ca506d3bf7f1000bfa248d0e1217b618108fba4ec1d0cc306" +checksum = "7345d5f0e08c0536d7ac7229952590239e77abf0a0100a1b1d890add6ea96364" dependencies = [ "anyhow", - "itertools", + "itertools 0.10.3", "proc-macro2", "quote", "syn", @@ -6069,18 +5658,18 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d0a014229361011dc8e69c8a1ec6c2e8d0f2af7c91e3ea3f5b2170298461e68" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "prost 0.10.4", ] [[package]] name = "prost-types" -version = "0.11.2" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "747761bc3dc48f9a34553bf65605cf6cb6288ba219f3450b4275dbd81539551a" +checksum = "4dfaa718ad76a44b3415e6c4d53b17c8f99160dcb3a99b10470fce8ad43f6e3e" dependencies = [ - "bytes 1.3.0", - "prost 0.11.3", + "bytes 1.2.1", + "prost 0.11.0", ] [[package]] @@ -6105,41 +5694,36 @@ dependencies = [ [[package]] name = "pulsar" -version = "5.0.0" +version = "4.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0efdf1fcdc925382b4a52e98a2d13f0a223116d9f458565b26cb44866b195953" +checksum = "31a5a4fbe9363c113b7e25ae76dd2d6455411c815d688977be6f0b68ae5e73b8" dependencies = [ "async-trait", "bit-vec 0.6.3", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "crc", "data-url", - "flate2", - "futures 0.3.25", + "futures 0.3.24", "futures-io", "futures-timer", "log", - "lz4", "native-tls", "nom", "oauth2", "openidconnect", "pem", - "prost 0.11.3", - "prost-build 0.11.3", - "prost-derive 0.11.2", + "prost 0.10.4", + "prost-build", + "prost-derive 0.10.1", "rand 0.8.5", "regex", "serde", "serde_json", - "snap", "tokio", "tokio-native-tls", - "tokio-util", + "tokio-util 0.7.1", "url", - "uuid 1.2.2", - "zstd", ] [[package]] @@ -6155,7 +5739,7 @@ dependencies = [ "raw-cpuid", "wasi 0.10.2+wasi-snapshot-preview1", "web-sys", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -6181,17 +5765,6 @@ dependencies = [ "rand 0.8.5", ] -[[package]] -name = "quickcheck_macros" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b22a693222d716a9587786f37ac3f6b4faedb5b80c23914e7303ff5a1d8016e9" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "quote" version = "1.0.21" @@ -6201,12 +5774,6 @@ dependencies = [ "proc-macro2", ] -[[package]] -name = "quoted_printable" -version = "0.4.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20f14e071918cbeefc5edc986a7aa92c425dae244e003a35e1cdddb5ca39b5cb" - [[package]] name = "radix_trie" version = "0.2.1" @@ -6238,7 +5805,7 @@ checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" dependencies = [ "libc", "rand_chacha 0.3.1", - "rand_core 0.6.4", + "rand_core 0.6.3", ] [[package]] @@ -6258,7 +5825,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" dependencies = [ "ppv-lite86", - "rand_core 0.6.4", + "rand_core 0.6.3", ] [[package]] @@ -6272,11 +5839,11 @@ dependencies = [ [[package]] name = "rand_core" -version = "0.6.4" +version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +checksum = "d34f1408f55294453790c48b2f1ebbb1c5b4b7563eb1f418bcfcfdbb06ebb4e7" dependencies = [ - "getrandom 0.2.8", + "getrandom 0.2.6", ] [[package]] @@ -6304,27 +5871,18 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d25bf25ec5ae4a3f1b92f929810509a2f53d7dca2f50b794ff57e3face536c8f" dependencies = [ - "rand_core 0.6.4", + "rand_core 0.6.3", ] [[package]] name = "raw-cpuid" -version = "10.6.0" +version = "10.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6823ea29436221176fe662da99998ad3b4db2c7f31e7b6f5fe43adccd6320bb" +checksum = "738bc47119e3eeccc7e94c4a506901aea5e7b4944ecd0829cbebf4af04ceda12" dependencies = [ "bitflags", ] -[[package]] -name = "raw-window-handle" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed7e3d950b66e19e0c372f3fa3fbbcf85b1746b571f74e0c2af6042a5c93420a" -dependencies = [ - "cty", -] - [[package]] name = "rawpointer" version = "0.2.1" @@ -6333,10 +5891,11 @@ checksum = "60a357793950651c4ed0f3f52338f53b2f809f32d83a07f72909fa13e4c6c1e3" [[package]] name = "rayon" -version = "1.6.0" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e060280438193c554f654141c9ea9417886713b7acd75974c85b18a69a88e0b" +checksum = "fd249e82c21598a9a426a4e00dd7adc1d640b22445ec8545feef801d1a74c221" dependencies = [ + "autocfg", "crossbeam-deque", "either", "rayon-core", @@ -6344,9 +5903,9 @@ dependencies = [ [[package]] name = "rayon-core" -version = "1.10.1" +version = "1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cac410af5d00ab6884528b4ab69d1e8e146e8d471201800fa1b4524126de6ad3" +checksum = "9f51245e1e62e1f1629cbfec37b5793bbabcaeb90f30e94d2ba03564687353e4" dependencies = [ "crossbeam-channel", "crossbeam-deque", @@ -6356,12 +5915,11 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.29.0" +version = "0.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd7c5d6d17442bcb9f943aae96d67d98c6d36af60442dd5da62aaa7fcbb25c48" +checksum = "1de127f294f2dba488ed46760b129d5ecbeabbd337ccbf3739cb29d50db2161c" dependencies = [ - "futures-channel", - "futures-util", + "futures 0.3.24", "libc", "log", "rdkafka-sys", @@ -6374,9 +5932,9 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.3.0+1.9.2" +version = "4.2.0+1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d222a401698c7f2010e3967353eae566d9934dcda49c29910da922414ab4e3f4" +checksum = "9e542c6863b04ce0fa0c5719bc6b7b348cf8dd21af1bb03c9db5f9805b2a6473" dependencies = [ "cmake", "libc", @@ -6388,37 +5946,26 @@ dependencies = [ "zstd-sys", ] -[[package]] -name = "reactor-trait" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "438a4293e4d097556730f4711998189416232f009c137389e0f961d2bc0ddc58" -dependencies = [ - "async-trait", - "futures-core", - "futures-io", -] - [[package]] name = "redis" -version = "0.22.1" +version = "0.21.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "513b3649f1a111c17954296e4a3b9eecb108b766c803e2b99f179ebe27005985" +checksum = "571c252c68d09a2ad3e49edd14e9ee48932f3e0f27b06b4ea4c9b2a706d31103" dependencies = [ "arc-swap", "async-trait", - "bytes 1.3.0", - "combine 4.6.6", - "futures 0.3.25", + "bytes 1.2.1", + "combine 4.6.4", + "futures 0.3.24", "futures-util", - "itoa 1.0.4", + "itoa 1.0.1", "native-tls", "percent-encoding", "pin-project-lite", "ryu", "tokio", "tokio-native-tls", - "tokio-util", + "tokio-util 0.7.1", "url", ] @@ -6430,9 +5977,9 @@ checksum = "41cc0f7e4d5d4544e8861606a285bb08d3e70712ccc7d2b84d7c0ccfaf4b05ce" [[package]] name = "redox_syscall" -version = "0.2.16" +version = "0.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" +checksum = "62f25bc4c7e55e0b0b7a1d43fb893f4fa1361d0abe38b9ce4f323c2adfe6ef42" dependencies = [ "bitflags", ] @@ -6454,16 +6001,16 @@ version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b033d837a7cf162d7993aded9304e30a83213c648b6e389db233191f891e5c2b" dependencies = [ - "getrandom 0.2.8", - "redox_syscall 0.2.16", + "getrandom 0.2.6", + "redox_syscall 0.2.13", "thiserror", ] [[package]] name = "regex" -version = "1.7.0" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e076559ef8e241f2ae3479e36f97bd5741c0330689e217ad51ce2c76808b868a" +checksum = "4c4eb3267174b8c6c2f654116623910a0fef09c4753f8dd83db29c48a0df988b" dependencies = [ "aho-corasick", "memchr", @@ -6481,9 +6028,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.6.28" +version = "0.6.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "456c603be3e8d448b072f410900c09faf164fbce2d480456f50eea6e25f9c848" +checksum = "a3f87b73ce11b1619a3c6332f45341e0047173771e8b8b73f87bfeefb7b56244" [[package]] name = "remove_dir_all" @@ -6491,7 +6038,7 @@ version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3acd125665422973a33ac9d3dd2df85edad0f4ae9b00dafb1a05e43a9f5ef8e7" dependencies = [ - "winapi", + "winapi 0.3.9", ] [[package]] @@ -6505,12 +6052,12 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.13" +version = "0.11.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68cc60575865c7831548863cc02356512e3f1dc2f3f82cb837d7fc4cc8f3c97c" +checksum = "b75aa69a3f06bbcc66ede33af2af253c6f7a86b1ca0033f60c580a27074fbf92" dependencies = [ "base64", - "bytes 1.3.0", + "bytes 1.2.1", "encoding_rs", "futures-core", "futures-util", @@ -6518,32 +6065,32 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls", + "hyper-rustls 0.23.0", "hyper-tls", "ipnet", "js-sys", + "lazy_static", "log", "mime", "native-tls", - "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.20.7", - "rustls-pemfile 1.0.1", + "rustls 0.20.4", + "rustls-pemfile 1.0.0", "serde", "serde_json", "serde_urlencoded", "tokio", "tokio-native-tls", - "tokio-rustls", - "tokio-util", + "tokio-rustls 0.23.3", + "tokio-util 0.7.1", "tower-service", "url", "wasm-bindgen", "wasm-bindgen-futures", "web-sys", "webpki-roots", - "winreg", + "winreg 0.10.1", ] [[package]] @@ -6574,7 +6121,7 @@ dependencies = [ "spin 0.5.2", "untrusted", "web-sys", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -6584,7 +6131,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cec2b3485b07d96ddfd3134767b8a447b45ea4eb91448d0a35180ec0ffd5ed15" dependencies = [ "bytecheck", - "hashbrown 0.12.3", + "hashbrown", "ptr_meta", "rend", "rkyv_derive", @@ -6621,9 +6168,9 @@ dependencies = [ [[package]] name = "rmp-serde" -version = "1.1.1" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5b13be192e0220b8afb7222aa5813cb62cc269ebb5cac346ca6487681d2913e" +checksum = "25786b0d276110195fa3d6f3f31299900cf71dfbd6c28450f3f58a0e7f7a347e" dependencies = [ "byteorder", "rmp", @@ -6644,9 +6191,9 @@ dependencies = [ [[package]] name = "roaring" -version = "0.10.1" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef0fb5e826a8bde011ecae6a8539dd333884335c57ff0f003fbe27c25bbe8f71" +checksum = "48de5b5febb28599c6d9d34817d8ee8c74417af507b90dde0d481b28ca55ed6f" dependencies = [ "bytemuck", "byteorder", @@ -6655,9 +6202,9 @@ dependencies = [ [[package]] name = "roxmltree" -version = "0.15.1" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b9de9831a129b122e7e61f242db509fa9d0838008bf0b29bb0624669edfe48a" +checksum = "112908c3ac4711a1554b3948432ecaf5f061a951aa326977b63f7f72a86a4c0e" dependencies = [ "xmlparser", ] @@ -6676,20 +6223,13 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.27.0" +version = "1.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33c321ee4e17d2b7abe12b5d20c1231db708dd36185c8a21e9de5fed6da4dbe9" +checksum = "ee9164faf726e4f3ece4978b25ca877ddc6802fa77f38cdccb32c7f805ecd70c" dependencies = [ "arrayvec 0.7.2", - "borsh", - "bytecheck", - "byteorder", - "bytes 1.3.0", "num-traits", - "rand 0.8.5", - "rkyv", "serde", - "serde_json", ] [[package]] @@ -6713,7 +6253,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" dependencies = [ - "semver 1.0.14", + "semver 1.0.13", ] [[package]] @@ -6728,16 +6268,16 @@ dependencies = [ [[package]] name = "rustix" -version = "0.36.5" +version = "0.34.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3807b5d10909833d3e9acd1eb5fb988f79376ff10fce42937de71a449c4c588" +checksum = "3f5d1c6ed6d1c6915aa64749b809fc1bafff49d160f5d927463658093d7d62ab" dependencies = [ "bitflags", "errno", "io-lifetimes", "libc", "linux-raw-sys", - "windows-sys 0.42.0", + "winapi 0.3.9", ] [[package]] @@ -6755,9 +6295,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.7" +version = "0.20.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "539a2bfe908f471bfa933876bd1eb6a19cf2176d375f82ef7f99530a40e48c2c" +checksum = "4fbfeb8d0ddb84706bc597a5574ab8912817c52a397f819e5b614e2265206921" dependencies = [ "log", "ring", @@ -6784,7 +6324,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0167bac7a9f490495f3c33013e7722b53cb087ecbe082fb0c6387c96f634ea50" dependencies = [ "openssl-probe", - "rustls-pemfile 1.0.1", + "rustls-pemfile 1.0.0", "schannel", "security-framework", ] @@ -6809,18 +6349,18 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "1.0.1" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0864aeff53f8c05aa08d86e5ef839d3dfcf07aeba2db32f12db0ef716e87bd55" +checksum = "e7522c9de787ff061458fe9a829dc790a3f5b22dc571694fc5883f448b94d9a9" dependencies = [ "base64", ] [[package]] name = "rustversion" -version = "1.0.9" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97477e48b4cf8603ad5f7aaf897467cf42ab4218a38ef76fb14c2d6773a6d6a8" +checksum = "f2cc38e8fa666e2de3c4aba7edeb5ffc5246c1c2ed0e3d17e560aeeba736b23f" [[package]] name = "rusty-fork" @@ -6841,25 +6381,25 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d1cd5ae51d3f7bf65d7969d579d502168ef578f289452bd8ccc91de28fda20e" dependencies = [ "bitflags", - "cfg-if", + "cfg-if 1.0.0", "clipboard-win", "fd-lock", "libc", "log", "memchr", - "nix 0.24.3", + "nix 0.24.2", "scopeguard", "unicode-segmentation", "unicode-width", "utf8parse", - "winapi", + "winapi 0.3.9", ] [[package]] name = "ryu" -version = "1.0.11" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4501abdff3ae82a1c1b477a17252eb69cee9e66eb915c1abaa4f44d873df9f09" +checksum = "73b4b750c782965c211b42f022f59af1fbceabdd026623714f104152f1ec149f" [[package]] name = "same-file" @@ -6872,9 +6412,9 @@ dependencies = [ [[package]] name = "sasl2-sys" -version = "0.1.20+2.1.28" +version = "0.1.19+2.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e645bd98535fc8fd251c43ba7c7c1f9be1e0369c99b6a5ea719052a773e655c" +checksum = "21d3579e03127aee0792cc0e2d739fe05b1652f396ee92127d15b2748be9adf7" dependencies = [ "cc", "duct", @@ -6904,9 +6444,9 @@ dependencies = [ [[package]] name = "schemars" -version = "0.8.11" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a5fb6c61f29e723026dc8e923d94c694313212abbecbbe5f55a7748eec5b307" +checksum = "1847b767a3d62d95cbf3d8a9f0e421cf57a0d8aa4f411d4b16525afb0284d4ed" dependencies = [ "dyn-clone", "indexmap", @@ -6917,9 +6457,9 @@ dependencies = [ [[package]] name = "schemars_derive" -version = "0.8.11" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f188d036977451159430f3b8dc82ec76364a42b7e289c2b18a9a18f4470058e9" +checksum = "af4d7e1b012cb3d9129567661a63755ea4b8a7386d339dc945ae187e403c6743" dependencies = [ "proc-macro2", "quote", @@ -6929,9 +6469,9 @@ dependencies = [ [[package]] name = "scoped-tls" -version = "1.0.1" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" +checksum = "ea6a9290e3c9cf0f18145ef7ffa62d68ee0bf5fcd651017e586dc7fd5da448c2" [[package]] name = "scopeguard" @@ -6939,12 +6479,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" -[[package]] -name = "scratch" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8132065adcfd6e02db789d9285a0deb2f3fcb04002865ab67d5fb103533898" - [[package]] name = "sct" version = "0.6.1" @@ -7015,9 +6549,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.14" +version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e25dfac463d778e353db5be2449d1cce89bd6fd23c9f1ea21310ce6e5a1b29c4" +checksum = "93f6841e709003d68bb2deee8c343572bf446003ec20a583e76f7b15cebf3711" dependencies = [ "serde", ] @@ -7030,9 +6564,9 @@ checksum = "388a1df253eca08550bef6c72392cfe7c30914bf41df5269b68cbd6ff8f570a3" [[package]] name = "serde" -version = "1.0.149" +version = "1.0.144" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "256b9932320c590e707b94576e3cc1f7c9024d0ee6612dfbcf1cb106cbe8e055" +checksum = "0f747710de3dcd43b88c9168773254e809d8ddbdf9653b84e2554ab219f17860" dependencies = [ "serde_derive", ] @@ -7048,30 +6582,29 @@ dependencies = [ [[package]] name = "serde-value" -version = "0.7.0" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +checksum = "5a65a7291a8a568adcae4c10a677ebcedbc6c9cec91c054dee2ce40b0e3290eb" dependencies = [ - "ordered-float 2.10.0", + "ordered-float 1.1.1", "serde", ] [[package]] -name = "serde-wasm-bindgen" -version = "0.4.5" +name = "serde-value" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3b4c031cd0d9014307d82b8abf653c0290fbdaeb4c02d00c63cf52f728628bf" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "js-sys", + "ordered-float 2.10.0", "serde", - "wasm-bindgen", ] [[package]] name = "serde-xml-rs" -version = "0.6.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb3aa78ecda1ebc9ec9847d5d3aba7d618823446a049ba2491940506da6e2782" +checksum = "65162e9059be2f6a3421ebbb4fef3e74b7d9e7c60c50a0e292c6239f19f1edfa" dependencies = [ "log", "serde", @@ -7088,11 +6621,21 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_cbor" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2bef2ebfde456fb76bbcf9f59315333decc4fda0b2b44b420243c11e0f5ec1f5" +dependencies = [ + "half", + "serde", +] + [[package]] name = "serde_derive" -version = "1.0.149" +version = "1.0.144" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4eae9b04cbffdfd550eb462ed33bc6a1b68c935127d008b27444d08380f94e4" +checksum = "94ed3a816fb1d101812f83e789f888322c34e291f894f19590dc310963e87a00" dependencies = [ "proc-macro2", "quote", @@ -7112,12 +6655,12 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.89" +version = "1.0.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "020ff22c755c2ed3f8cf162dbb41a7268d934702f3ed3631656ea597e08fc3db" +checksum = "e55a28e3aaef9d5ce0506d0a14dbba8054ddc7e499ef522dd8b26859ec9d4a44" dependencies = [ "indexmap", - "itoa 1.0.4", + "itoa 1.0.1", "ryu", "serde", ] @@ -7133,9 +6676,9 @@ dependencies = [ [[package]] name = "serde_path_to_error" -version = "0.1.8" +version = "0.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "184c643044780f7ceb59104cef98a5a6f12cb2288a7bc701ab93a362b49fd47d" +checksum = "d7868ad3b8196a8a0aea99a8220b124278ee5320a55e4fde97794b6f85b1a377" dependencies = [ "serde", ] @@ -7153,9 +6696,9 @@ dependencies = [ [[package]] name = "serde_repr" -version = "0.1.9" +version = "0.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fe39d9fbb0ebf5eb2c7cb7e2a47e4f462fad1379f1166b8ae49ad9eae89a7ca" +checksum = "98d0516900518c29efa217c298fa1f4e6c6ffc85ae29fd7f4ee48f176e1a9ed5" dependencies = [ "proc-macro2", "quote", @@ -7169,7 +6712,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3491c14715ca2294c4d6a88f15e84739788c1d030eed8c110436aafdaa2f3fd" dependencies = [ "form_urlencoded", - "itoa 1.0.4", + "itoa 1.0.1", "ryu", "serde", ] @@ -7186,9 +6729,9 @@ dependencies = [ [[package]] name = "serde_with" -version = "2.1.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25bf4a5a814902cd1014dbccfa4d4560fb8432c779471e96e035602519f82eef" +checksum = "89df7a26519371a3cce44fbb914c2819c84d9b897890987fa3ab096491cc0ea8" dependencies = [ "base64", "chrono", @@ -7196,7 +6739,7 @@ dependencies = [ "indexmap", "serde", "serde_json", - "serde_with_macros 2.1.0", + "serde_with_macros 2.0.0", "time", ] @@ -7214,11 +6757,11 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "2.1.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3452b4c0f6c1e357f73fdb87cd1efabaa12acf328c7a528e252893baeb3f4aa" +checksum = "de337f322382fcdfbb21a014f7c224ee041a23785651db67b9827403178f698f" dependencies = [ - "darling 0.14.2", + "darling 0.14.1", "proc-macro2", "quote", "syn", @@ -7238,12 +6781,12 @@ dependencies = [ [[package]] name = "serde_yaml" -version = "0.9.14" +version = "0.9.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d232d893b10de3eb7258ff01974d6ee20663d8e833263c99409d4b13a0209da" +checksum = "89f31df3f50926cdf2855da5fd8812295c34752cb20438dae42a67f79e021ac3" dependencies = [ "indexmap", - "itoa 1.0.4", + "itoa 1.0.1", "ryu", "serde", "unsafe-libyaml", @@ -7251,24 +6794,37 @@ dependencies = [ [[package]] name = "sha-1" -version = "0.10.1" +version = "0.9.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "99cd6713db3cf16b6c84e06321e049a9b9f699826e16096d23bbcc44d15d51a6" +dependencies = [ + "block-buffer 0.9.0", + "cfg-if 1.0.0", + "cpufeatures", + "digest 0.9.0", + "opaque-debug", +] + +[[package]] +name = "sha-1" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5058ada175748e33390e40e872bd0fe59a19f265d0158daa551c5a88a76009c" +checksum = "028f48d513f9678cda28f6e4064755b3fbb2af6acd672f2c209b62323f7aea0f" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "cpufeatures", - "digest 0.10.6", + "digest 0.10.3", ] [[package]] name = "sha1" -version = "0.10.5" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f04293dc80c3993519f2d7f6f511707ee7094fe0c6d3406feb330cdb3540eba3" +checksum = "c77f4e7f65455545c2153c1253d25056825e77ee2533f0e41deb65a93a34852f" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "cpufeatures", - "digest 0.10.6", + "digest 0.10.3", ] [[package]] @@ -7278,7 +6834,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4d58a1e1bf39749807d89cf2d98ac2dfa0ff1cb3faa38fbb64dd88ac8013d800" dependencies = [ "block-buffer 0.9.0", - "cfg-if", + "cfg-if 1.0.0", "cpufeatures", "digest 0.9.0", "opaque-debug", @@ -7286,22 +6842,22 @@ dependencies = [ [[package]] name = "sha2" -version = "0.10.6" +version = "0.10.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82e6b795fe2e3b1e845bafcb27aa35405c4d47cdfc92af5fc8d3002f76cebdc0" +checksum = "cf9db03534dff993187064c4e0c05a5708d2a9728ace9a8959b77bedf415dac5" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "cpufeatures", - "digest 0.10.6", + "digest 0.10.3", ] [[package]] name = "sha3" -version = "0.10.6" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdf0c33fae925bdc080598b84bc15c55e7b9a4a43b3c704da051f977469691c9" +checksum = "881bf8156c87b6301fc5ca6b27f11eeb2761224c7081e69b409d5a1951a70c86" dependencies = [ - "digest 0.10.6", + "digest 0.10.3", "keccak", ] @@ -7316,19 +6872,19 @@ dependencies = [ [[package]] name = "shared_child" -version = "1.0.0" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0d94659ad3c2137fef23ae75b03d5241d633f8acded53d672decfa0e6e0caef" +checksum = "6be9f7d5565b1483af3e72975e2dee33879b3b86bd48c0929fccf6585d79e65a" dependencies = [ "libc", - "winapi", + "winapi 0.3.9", ] [[package]] name = "signal-hook" -version = "0.3.14" +version = "0.3.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a253b5e89e2698464fc26b545c9edceb338e18a89effeeecfea192c3025be29d" +checksum = "647c97df271007dcea485bb74ffdb57f2e683f1306c854f468a0c244badabf2d" dependencies = [ "libc", "signal-hook-registry", @@ -7341,7 +6897,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "29ad2e15f37ec9a6cc544097b78a1ec90001e9f71b81338ca39f430adaca99af" dependencies = [ "libc", - "mio", + "mio 0.8.2", "signal-hook", ] @@ -7361,36 +6917,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5dfecc059e81632eef1dd9b79e22fc28b8fe69b30d3357512a77a0ad8ee3c782" dependencies = [ "pkcs8", - "rand_core 0.6.4", + "rand_core 0.6.3", "signature", "zeroize", ] [[package]] name = "signature" -version = "1.6.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74233d3b3b2f6d4b006dc19dee745e73e2a6bfb6f93607cd3b02bd5b00797d7c" - -[[package]] -name = "similar" -version = "2.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "420acb44afdae038210c99e69aae24109f32f15500aa708e81d46c9f29d55fcf" -dependencies = [ - "bstr 0.2.17", - "unicode-segmentation", -] - -[[package]] -name = "similar-asserts" -version = "1.4.2" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbf644ad016b75129f01a34a355dcb8d66a5bc803e417c7a77cc5d5ee9fa0f18" -dependencies = [ - "console", - "similar", -] +checksum = "f054c6c1a6e95179d6f23ed974060dcefb2d9388bb7256900badad682c499de4" [[package]] name = "simpl" @@ -7412,24 +6948,21 @@ checksum = "ceb945e54128e09c43d8e4f1277851bd5044c6fc540bbaa2ad888f60b3da9ae7" [[package]] name = "slab" -version = "0.4.7" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4614a76b2a8be0058caa9dbbaf66d988527d86d003c11a94fbd335d7661edcef" -dependencies = [ - "autocfg", -] +checksum = "eb703cfe953bccee95685111adeedb76fabe4e97549a58d16f03ea7b9367bb32" [[package]] name = "smallvec" -version = "1.10.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a507befe795404456341dfab10cef66ead4c041f62b8b11bbb92bffe5d0953e0" +checksum = "2fd0db749597d91ff862fd1d55ea87f7855a744a8425a64695b6fca237d1dad1" [[package]] name = "smol" -version = "1.3.0" +version = "1.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13f2b548cd8447f8de0fdf1c592929f70f4fc7039a05e47404b0d096ec6987a1" +checksum = "85cf3b5351f3e783c1d79ab5fc604eeed8b8ae9abd36b166e8b87a089efd85e4" dependencies = [ "async-channel", "async-executor", @@ -7440,6 +6973,7 @@ dependencies = [ "async-process", "blocking", "futures-lite", + "once_cell", ] [[package]] @@ -7460,9 +6994,9 @@ dependencies = [ [[package]] name = "snafu" -version = "0.7.3" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a152ba99b054b22972ee794cf04e5ef572da1229e33b65f3c57abbff0525a454" +checksum = "5177903bf45656592d9eb5c0e22f408fc023aae51dbe2088889b71633ba451f2" dependencies = [ "doc-comment", "futures-core", @@ -7472,9 +7006,9 @@ dependencies = [ [[package]] name = "snafu-derive" -version = "0.7.3" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5e79cdebbabaebb06a9bdbaedc7f159b410461f63611d4d0e3fb0fab8fed850" +checksum = "410b26ed97440d90ced3e2488c868d56a86e2064f5d7d6f417909b286afe25e5" dependencies = [ "heck 0.4.0", "proc-macro2", @@ -7484,9 +7018,9 @@ dependencies = [ [[package]] name = "snap" -version = "1.1.0" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" +checksum = "45456094d1983e2ee2a18fdfebce3189fa451699d0502cb8e3b49dba5ba41451" [[package]] name = "socket2" @@ -7495,7 +7029,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "02e2d2db9033d13a1567121ddd7a095ee144db4e1ca1b1bda3419bc0da294ebd" dependencies = [ "libc", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -7506,12 +7040,9 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "spin" -version = "0.9.4" +version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f6002a767bff9e83f8eeecf883ecb8011875a21ae8da43bffb817a57e78cc09" -dependencies = [ - "lock_api", -] +checksum = "c530c2b0d0bf8b69304b39fe2001993e267461948b890cd037d8ad4293fa1a0d" [[package]] name = "spki" @@ -7530,9 +7061,9 @@ checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" [[package]] name = "str-buf" -version = "1.0.6" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e08d8363704e6c71fc928674353e6b7c23dcea9d82d7012c8faf2a3a025f8d0" +checksum = "d44a3643b4ff9caf57abcee9c2c621d6c03d9135e0d8b589bd9afb5992cb176a" [[package]] name = "stream-cancel" @@ -7639,9 +7170,9 @@ checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" [[package]] name = "syn" -version = "1.0.105" +version = "1.0.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60b9b43d45702de4c839cb9b51d9f529c5dd26a4aff255b42b1ebc03e88ee908" +checksum = "58dbef6ec655055e20b86b15a8cc6d439cca19b667537ac6a1369572d151ab13" dependencies = [ "proc-macro2", "quote", @@ -7681,9 +7212,9 @@ dependencies = [ [[package]] name = "syslog_loose" -version = "0.18.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fb75f176928530867b2a659e470f9c9ff71904695bab6556f7ad30f9039efd" +checksum = "1c4eae4d024d7912b5bea75e54319445d0ffe7f423bb4b68a46129cdcebecaef" dependencies = [ "chrono", "nom", @@ -7695,15 +7226,6 @@ version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f764005d11ee5f36500a149ace24e00e3da98b0158b3e2d53a7495660d3f4d60" -[[package]] -name = "tcp-stream" -version = "0.24.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09a4b0a70bac0a58ca6a7659d1328e34ee462339c70b0fa49f72bad1f278910a" -dependencies = [ - "cfg-if", -] - [[package]] name = "temp-dir" version = "0.1.11" @@ -7716,12 +7238,12 @@ version = "3.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5cdb1ef4eaeeaddc8fbd371e5017057064af0911902ef36b39801f67cc6d79e4" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "fastrand", "libc", - "redox_syscall 0.2.16", + "redox_syscall 0.2.13", "remove_dir_all", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -7732,7 +7254,7 @@ checksum = "edd106a334b7657c10b7c540a0106114feadeb4dc314513e97df481d5d966f42" dependencies = [ "byteorder", "dirs", - "winapi", + "winapi 0.3.9", ] [[package]] @@ -7742,61 +7264,41 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c59df8ac95d96ff9bede18eb7300b0fda5e5d8d90960e76f8e14ae765eedbf1f" dependencies = [ "dirs-next", - "rustversion", - "winapi", -] - -[[package]] -name = "termcolor" -version = "1.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bab24d30b911b2376f3a13cc2cd443142f0c81dda04c118693e35b3835757755" -dependencies = [ - "winapi-util", -] - -[[package]] -name = "terminal_size" -version = "0.1.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "633c1a546cee861a1a6d0dc69ebeca693bf4296661ba7852b9d21d159e0506df" -dependencies = [ - "libc", - "winapi", + "rustversion", + "winapi 0.3.9", ] [[package]] -name = "terminal_size" -version = "0.2.3" +name = "termcolor" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb20089a8ba2b69debd491f8d2d023761cbf196e999218c591fa1e7e15a21907" +checksum = "bab24d30b911b2376f3a13cc2cd443142f0c81dda04c118693e35b3835757755" dependencies = [ - "rustix", - "windows-sys 0.42.0", + "winapi-util", ] [[package]] name = "termtree" -version = "0.4.0" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95059e91184749cb66be6dc994f67f182b6d897cb3df74a5bf66b5e709295fd8" +checksum = "507e9898683b6c43a9aa55b64259b721b52ba226e0f3779137e50ad114a4c90b" [[package]] name = "test-case" -version = "2.2.2" +version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21d6cf5a7dffb3f9dceec8e6b8ca528d9bd71d36c9f074defb548ce161f598c0" +checksum = "07aea929e9488998b64adc414c29fe5620398f01c2e3f58164122b17e567a6d5" dependencies = [ "test-case-macros", ] [[package]] name = "test-case-macros" -version = "2.2.2" +version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e45b7bf6e19353ddd832745c8fcf77a17a93171df7151187f26623f2b75b5b26" +checksum = "c95968eedc6fc4f5c21920e0f4264f78ec5e4c56bb394f319becc1a5830b3e54" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "proc-macro-error", "proc-macro2", "quote", @@ -7814,24 +7316,24 @@ dependencies = [ [[package]] name = "textwrap" -version = "0.16.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" +checksum = "b1141d4d61095b28419e22cb0bbf02755f5e54e0526f97f1e3d1d160e60885fb" [[package]] name = "thiserror" -version = "1.0.37" +version = "1.0.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10deb33631e3c9018b9baf9dcbbc4f737320d2b576bac10f6aefa048fa407e3e" +checksum = "8c1b05ca9d106ba7d2e31a9dab4a64e7be2cce415321966ea3132c49a656e252" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.37" +version = "1.0.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "982d17546b47146b28f7c22e3d08465f6b8903d0ea13c1660d9d84a6e7adcdbb" +checksum = "e8f2591983642de85c921015f3f070c665a197ed69e417af436115e3a1407487" dependencies = [ "proc-macro2", "quote", @@ -7849,9 +7351,9 @@ dependencies = [ [[package]] name = "tikv-jemalloc-sys" -version = "0.5.2+5.3.0-patched" +version = "0.5.0+5.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec45c14da997d0925c7835883e4d5c181f196fa142f8c19d7643d1e9af2592c3" +checksum = "aeab4310214fe0226df8bfeb893a291a58b19682e8a07e1e1d4483ad4200d315" dependencies = [ "cc", "fs_extra", @@ -7870,32 +7372,22 @@ dependencies = [ [[package]] name = "time" -version = "0.3.17" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a561bf4617eebd33bca6434b988f39ed798e527f51a1e797d0ee4f61c0a38376" +checksum = "c2702e08a7a860f005826c6815dcac101b19b5eb330c27fe4a5928fec1d20ddd" dependencies = [ - "itoa 1.0.4", + "itoa 1.0.1", "libc", "num_threads", "serde", - "time-core", "time-macros", ] -[[package]] -name = "time-core" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" - [[package]] name = "time-macros" -version = "0.2.6" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d967f99f534ca7e495c575c62638eebc2898a8c84c119b89e250477bc4ba16b2" -dependencies = [ - "time-core", -] +checksum = "42657b1a6f4d817cda8e7a0ace261fe0cc946cf3a80314390b22cc61ae080792" [[package]] name = "tiny-keccak" @@ -7933,23 +7425,24 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.23.0" +version = "1.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eab6d665857cc6ca78d6e80303a02cea7a7851e85dfbd77cbdc09bd129f1ef46" +checksum = "7a8325f63a7d4774dd041e363b2409ed1c5cbbd0f867795e661df066b2b0a581" dependencies = [ "autocfg", - "bytes 1.3.0", + "bytes 1.2.1", "libc", "memchr", - "mio", + "mio 0.8.2", "num_cpus", + "once_cell", "parking_lot", "pin-project-lite", "signal-hook-registry", "socket2", "tokio-macros", - "tracing 0.1.37", - "windows-sys 0.42.0", + "tracing 0.1.34", + "winapi 0.3.9", ] [[package]] @@ -7975,9 +7468,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.8.2" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d266c00fde287f55d3f1c3e96c500c362a2b8c695076ec180f27918820bc6df8" +checksum = "b557f72f448c511a979e2564e55d74e6c4432fc96ff4f6241bc6bded342643b7" dependencies = [ "proc-macro2", "quote", @@ -8014,7 +7507,7 @@ checksum = "29a12c1b3e0704ae7dfc25562629798b29c72e6b1d0a681b6f29ab4ae5e7f7bf" dependencies = [ "async-trait", "byteorder", - "bytes 1.3.0", + "bytes 1.2.1", "fallible-iterator", "futures-channel", "futures-util", @@ -8027,30 +7520,41 @@ dependencies = [ "postgres-types", "socket2", "tokio", - "tokio-util", + "tokio-util 0.7.1", +] + +[[package]] +name = "tokio-rustls" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc6844de72e57df1980054b38be3a9f4702aba4858be64dd700181a8a6d0e1b6" +dependencies = [ + "rustls 0.19.1", + "tokio", + "webpki 0.21.4", ] [[package]] name = "tokio-rustls" -version = "0.23.4" +version = "0.23.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" +checksum = "4151fda0cf2798550ad0b34bcfc9b9dcc2a9d2471c895c68f3a8818e54f2389e" dependencies = [ - "rustls 0.20.7", + "rustls 0.20.4", "tokio", "webpki 0.22.0", ] [[package]] name = "tokio-stream" -version = "0.1.11" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d660770404473ccd7bc9f8b28494a811bc18542b915c0855c51e8f419d5223ce" +checksum = "df54d54117d6fdc4e4fea40fe1e4e566b3505700e148a6827e59b34b0d2600d9" dependencies = [ "futures-core", "pin-project-lite", "tokio", - "tokio-util", + "tokio-util 0.7.1", ] [[package]] @@ -8060,12 +7564,25 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "53474327ae5e166530d17f2d956afcb4f8a004de581b3cae10f12006bc8163e3" dependencies = [ "async-stream", - "bytes 1.3.0", + "bytes 1.2.1", "futures-core", "tokio", "tokio-stream", ] +[[package]] +name = "tokio-tungstenite" +version = "0.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "511de3f85caf1c98983545490c3d09685fa8eb634e57eec22bb4db271f46cbd8" +dependencies = [ + "futures-util", + "log", + "pin-project", + "tokio", + "tungstenite 0.14.0", +] + [[package]] name = "tokio-tungstenite" version = "0.17.2" @@ -8074,35 +7591,37 @@ checksum = "f714dd15bead90401d77e04243611caec13726c2408afd5b31901dfcdcb3b181" dependencies = [ "futures-util", "log", + "rustls 0.20.4", "tokio", "tungstenite 0.17.3", ] [[package]] -name = "tokio-tungstenite" -version = "0.18.0" +name = "tokio-util" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54319c93411147bced34cb5609a80e0a8e44c5999c93903a81cd866630ec0bfd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ - "futures-util", + "bytes 1.2.1", + "futures-core", + "futures-sink", "log", - "rustls 0.20.7", + "pin-project-lite", "tokio", - "tungstenite 0.18.0", ] [[package]] name = "tokio-util" -version = "0.7.4" -source = "git+https://github.com/vectordotdev/tokio?branch=tokio-util-0.7.4-framed-read-continue-on-error#53a17f257b599a9d18bd75249de98d0b6fc28cfa" +version = "0.7.1" +source = "git+https://github.com/vectordotdev/tokio?rev=3aa231cf6f33f74ca29077163879f0de9a207ad8#3aa231cf6f33f74ca29077163879f0de9a207ad8" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "futures-core", "futures-sink", "pin-project-lite", "slab", "tokio", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] @@ -8116,15 +7635,15 @@ dependencies = [ [[package]] name = "tonic" -version = "0.8.3" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f219fad3b929bef19b1f86fbc0358d35daed8f2cac972037ac0dc10bbb8d5fb" +checksum = "5be9d60db39854b30b835107500cf0aca0b0d14d6e1c3de124217c23a29c2ddb" dependencies = [ "async-stream", "async-trait", - "axum 0.6.1", + "axum", "base64", - "bytes 1.3.0", + "bytes 1.2.1", "flate2", "futures-core", "futures-util", @@ -8135,30 +7654,62 @@ dependencies = [ "hyper-timeout", "percent-encoding", "pin-project", - "prost 0.11.3", - "prost-derive 0.11.2", + "prost 0.10.4", + "prost-derive 0.10.1", "rustls-native-certs 0.6.2", - "rustls-pemfile 1.0.1", + "rustls-pemfile 1.0.0", + "tokio", + "tokio-rustls 0.23.3", + "tokio-stream", + "tokio-util 0.7.1", + "tower", + "tower-layer", + "tower-service", + "tracing 0.1.34", + "tracing-futures 0.2.5", +] + +[[package]] +name = "tonic" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "498f271adc46acce75d66f639e4d35b31b2394c295c82496727dafa16d465dd2" +dependencies = [ + "async-stream", + "async-trait", + "axum", + "base64", + "bytes 1.2.1", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "hyper", + "hyper-timeout", + "percent-encoding", + "pin-project", + "prost 0.11.0", + "prost-derive 0.11.0", "tokio", - "tokio-rustls", "tokio-stream", - "tokio-util", + "tokio-util 0.7.1", "tower", "tower-layer", "tower-service", - "tracing 0.1.37", + "tracing 0.1.34", "tracing-futures 0.2.5", ] [[package]] name = "tonic-build" -version = "0.8.4" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bf5e9b9c0f7e0a7c027dcfaba7b2c60816c7049171f679d99ee2ff65d0de8c4" +checksum = "d9263bf4c9bfaae7317c1c2faf7f18491d2fe476f70c414b73bf5d445b00ffa1" dependencies = [ "prettyplease", "proc-macro2", - "prost-build 0.11.3", + "prost-build", "quote", "syn", ] @@ -8177,21 +7728,21 @@ dependencies = [ "rand 0.8.5", "slab", "tokio", - "tokio-util", + "tokio-util 0.7.1", "tower-layer", "tower-service", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "tower-http" -version = "0.3.5" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f873044bf02dd1e8239e9c1293ea39dad76dc594ec16185d0a1bf31d8dc8d858" +checksum = "7d342c6d58709c0a6d48d48dabbb62d4ef955cf5f0f3bbfd845838e7ae88dbae" dependencies = [ "base64", "bitflags", - "bytes 1.3.0", + "bytes 1.2.1", "futures-core", "futures-util", "http", @@ -8201,20 +7752,20 @@ dependencies = [ "tower", "tower-layer", "tower-service", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] name = "tower-layer" -version = "0.3.2" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c20c8dbed6283a09604c3e69b4b7eeb54e298b8a600d4d5ecb5ad39de609f1d0" +checksum = "343bc9466d3fe6b0f960ef45960509f84480bf4fd96f92901afe7ff3df9d3a62" [[package]] name = "tower-service" -version = "0.3.2" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" +checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tower-test" @@ -8232,15 +7783,15 @@ dependencies = [ [[package]] name = "tracing" -version = "0.1.37" +version = "0.1.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +checksum = "5d0ecdcb44a79f0fe9844f0c4f33a342cbcbb5117de8001e6ba0dc2351327d09" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "log", "pin-project-lite", "tracing-attributes", - "tracing-core 0.1.30", + "tracing-core 0.1.28", ] [[package]] @@ -8248,16 +7799,16 @@ name = "tracing" version = "0.2.0" source = "git+https://github.com/tokio-rs/tracing?rev=e0642d949891546a3bb7e47080365ee7274f05cd#e0642d949891546a3bb7e47080365ee7274f05cd" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "pin-project-lite", "tracing-core 0.2.0", ] [[package]] name = "tracing-attributes" -version = "0.1.23" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4017f8f45139870ca7e672686113917c71c7a6e02d4924eda67186083c03081a" +checksum = "cc6b8ad3567499f98a1db7a752b07a7c8c7c7c34c332ec00effb2b0027974b7c" dependencies = [ "proc-macro2", "quote", @@ -8266,9 +7817,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.30" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24eb03ba0eab1fd845050058ce5e616558e8f8d8fca633e6b163fe25c797213a" +checksum = "7b7358be39f2f274f322d2aaed611acc57f382e8eb1e5b48cb9ae30933495ce7" dependencies = [ "once_cell", "valuable", @@ -8288,9 +7839,9 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "12de1a8c6bcfee614305e836308b596bbac831137a04c61f7e5b0b0bf2cfeaf6" dependencies = [ - "tracing 0.1.37", - "tracing-core 0.1.30", - "tracing-subscriber 0.3.16", + "tracing 0.1.34", + "tracing-core 0.1.28", + "tracing-subscriber 0.3.15", ] [[package]] @@ -8299,10 +7850,10 @@ version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97d095ae15e245a057c8e8451bab9b3ee1e1f68e9ba2b4fbc18d0ac5237835f2" dependencies = [ - "futures 0.3.25", + "futures 0.3.24", "futures-task", "pin-project", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] @@ -8321,9 +7872,9 @@ dependencies = [ "criterion", "dashmap", "mock_instant", - "tracing 0.1.37", - "tracing-core 0.1.30", - "tracing-subscriber 0.3.16", + "tracing 0.1.34", + "tracing-core 0.1.28", + "tracing-subscriber 0.3.15", ] [[package]] @@ -8334,7 +7885,7 @@ checksum = "78ddad33d2d10b1ed7eb9d1f518a5674713876e97e5bb9b7345a7984fbb4f922" dependencies = [ "lazy_static", "log", - "tracing-core 0.1.30", + "tracing-core 0.1.28", ] [[package]] @@ -8344,7 +7895,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bc6b213177105856957181934e4920de57730fc69bf42c37ee5bb664d406d9e1" dependencies = [ "serde", - "tracing-core 0.1.30", + "tracing-core 0.1.28", ] [[package]] @@ -8363,20 +7914,20 @@ dependencies = [ "sharded-slab", "smallvec", "thread_local", - "tracing 0.1.37", - "tracing-core 0.1.30", + "tracing 0.1.34", + "tracing-core 0.1.28", "tracing-log", "tracing-serde", ] [[package]] name = "tracing-subscriber" -version = "0.3.16" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6176eae26dd70d0c919749377897b54a9276bd7061339665dd68777926b5a70" +checksum = "60db860322da191b40952ad9affe65ea23e7dd6a5c442c2c42865810c6ab8e6b" dependencies = [ + "ansi_term", "matchers 0.1.0", - "nu-ansi-term", "once_cell", "regex", "serde", @@ -8384,8 +7935,8 @@ dependencies = [ "sharded-slab", "smallvec", "thread_local", - "tracing 0.1.37", - "tracing-core 0.1.30", + "tracing 0.1.34", + "tracing-core 0.1.28", "tracing-log", "tracing-serde", ] @@ -8397,21 +7948,21 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a3b48778c2d401c6a7fcf38a0e3c55dc8e8e753cbd381044a8cdb6fd69a29f53" dependencies = [ "lazy_static", - "tracing-core 0.1.30", + "tracing-core 0.1.28", "tracing-subscriber 0.2.25", "tracing-test-macro 0.1.0", ] [[package]] name = "tracing-test" -version = "0.2.3" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e3d272c44878d2bbc9f4a20ad463724f03e19dbc667c6e84ac433ab7ffcc70b" +checksum = "3eb7bda2e93bbc9c5b247034acc6a4b3d04f033a3d4b8fc1cb87d4d1c7c7ebd7" dependencies = [ "lazy_static", - "tracing-core 0.1.30", - "tracing-subscriber 0.3.16", - "tracing-test-macro 0.2.3", + "tracing-core 0.1.28", + "tracing-subscriber 0.3.15", + "tracing-test-macro 0.2.1", ] [[package]] @@ -8427,9 +7978,9 @@ dependencies = [ [[package]] name = "tracing-test-macro" -version = "0.2.3" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "744324b12d69a9fc1edea4b38b7b1311295b662d161ad5deac17bb1358224a08" +checksum = "4801dca35e4e2cee957c469bd4a1c370fadb7894c0d50721a40eba3523e6e91c" dependencies = [ "lazy_static", "quote", @@ -8441,7 +7992,7 @@ name = "tracing-tower" version = "0.1.0" source = "git+https://github.com/tokio-rs/tracing?rev=e0642d949891546a3bb7e47080365ee7274f05cd#e0642d949891546a3bb7e47080365ee7274f05cd" dependencies = [ - "futures 0.3.25", + "futures 0.3.24", "tower-service", "tracing 0.2.0", "tracing-futures 0.3.0", @@ -8463,13 +8014,13 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c31f240f59877c3d4bb3b3ea0ec5a6a0cff07323580ff8c7a605cd7d08b255d" dependencies = [ "async-trait", - "cfg-if", + "cfg-if 1.0.0", "data-encoding", - "enum-as-inner 0.4.0", + "enum-as-inner", "futures-channel", "futures-io", "futures-util", - "idna 0.2.3", + "idna", "ipnet", "lazy_static", "log", @@ -8481,38 +8032,13 @@ dependencies = [ "url", ] -[[package]] -name = "trust-dns-proto" -version = "0.22.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f7f83d1e4a0e4358ac54c5c3681e5d7da5efc5a7a632c90bb6d6669ddd9bc26" -dependencies = [ - "async-trait", - "cfg-if", - "data-encoding", - "enum-as-inner 0.5.1", - "futures-channel", - "futures-io", - "futures-util", - "idna 0.2.3", - "ipnet", - "lazy_static", - "rand 0.8.5", - "smallvec", - "thiserror", - "tinyvec", - "tokio", - "tracing 0.1.37", - "url", -] - [[package]] name = "trust-dns-resolver" version = "0.21.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e4ba72c2ea84515690c9fcef4c6c660bb9df3036ed1051686de84605b74fd558" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "futures-util", "ipconfig", "lazy_static", @@ -8523,7 +8049,7 @@ dependencies = [ "smallvec", "thiserror", "tokio", - "trust-dns-proto 0.21.2", + "trust-dns-proto", ] [[package]] @@ -8547,18 +8073,18 @@ dependencies = [ [[package]] name = "tungstenite" -version = "0.17.3" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e27992fd6a8c29ee7eef28fc78349aa244134e10ad447ce3b9f0ac0ed0fa4ce0" +checksum = "a0b2d8558abd2e276b0a8df5c05a2ec762609344191e5fd23e292c910e9165b5" dependencies = [ "base64", "byteorder", - "bytes 1.3.0", + "bytes 1.2.1", "http", "httparse", "log", "rand 0.8.5", - "sha-1", + "sha-1 0.9.8", "thiserror", "url", "utf-8", @@ -8566,18 +8092,18 @@ dependencies = [ [[package]] name = "tungstenite" -version = "0.18.0" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30ee6ab729cd4cf0fd55218530c4522ed30b7b6081752839b68fcec8d0960788" +checksum = "e27992fd6a8c29ee7eef28fc78349aa244134e10ad447ce3b9f0ac0ed0fa4ce0" dependencies = [ "base64", "byteorder", - "bytes 1.3.0", + "bytes 1.2.1", "http", "httparse", "log", "rand 0.8.5", - "sha1", + "sha-1 0.10.0", "thiserror", "url", "utf-8", @@ -8589,7 +8115,7 @@ version = "1.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "static_assertions", ] @@ -8617,9 +8143,9 @@ dependencies = [ [[package]] name = "typenum" -version = "1.16.0" +version = "1.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba" +checksum = "dcf81ac59edc17cc8697ff311e8f5ef2d99fcbd9817b34cec66f90b6c3dfd987" [[package]] name = "typetag" @@ -8661,9 +8187,18 @@ dependencies = [ [[package]] name = "ucd-trie" -version = "0.1.5" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56dee185309b50d1f11bfedef0fe6d036842e3fb77413abef29f8f8d1c5d4c1c" + +[[package]] +name = "uncased" +version = "0.9.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e79c4d996edb816c91e4308506774452e55e95c3c9de07b6729e17e15a5ef81" +checksum = "5baeed7327e25054889b9bd4f975f32e5f4c5d434042d59ab6cd4142c0a76ed0" +dependencies = [ + "version_check", +] [[package]] name = "unicase" @@ -8682,36 +8217,36 @@ checksum = "099b7128301d285f79ddd55b9a83d5e6b9e97c92e0ea0daebee7263e932de992" [[package]] name = "unicode-ident" -version = "1.0.5" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ceab39d59e4c9499d4e5a8ee0e2735b891bb7308ac83dfb4e80cad195c9f6f3" +checksum = "d22af068fba1eb5edcb4aea19d382b2a3deb4c8f9d475c589b6ada9e0fd493ee" [[package]] name = "unicode-normalization" -version = "0.1.22" +version = "0.1.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" +checksum = "d54590932941a9e9266f0832deed84ebe1bf2e4c9e4a3554d393d18f5e854bf9" dependencies = [ "tinyvec", ] [[package]] name = "unicode-segmentation" -version = "1.10.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fdbf052a0783de01e944a6ce7a8cb939e295b1e7be835a1112c3b9a7f047a5a" +checksum = "7e8820f5d777f6224dc4be3632222971ac30164d4a258d595640799554ebfd99" [[package]] name = "unicode-width" -version = "0.1.10" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" +checksum = "3ed742d4ea2bd1176e236172c8429aaf54486e7ac098db29ffe6529e0ce50973" [[package]] name = "unicode-xid" -version = "0.2.4" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f962df74c8c05a667b5ee8bcf162993134c104e96440b663c8daa176dc772d8c" +checksum = "8ccb82d61f80a663efe1f787a51b16b5a51e3314d6ac365b08639f52387b33f3" [[package]] name = "unreachable" @@ -8724,9 +8259,9 @@ dependencies = [ [[package]] name = "unsafe-libyaml" -version = "0.2.4" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1e5fa573d8ac5f1a856f8d7be41d390ee973daf97c806b2c1a465e4e1406e68" +checksum = "931179334a56395bcf64ba5e0ff56781381c1a5832178280c7d7f91d1679aeb0" [[package]] name = "untrusted" @@ -8747,21 +8282,21 @@ dependencies = [ [[package]] name = "url" -version = "2.3.1" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d68c799ae75762b8c3fe375feb6600ef5602c883c5d21eb51c09f22b83c4643" +checksum = "22fe195a4f217c25b25cb5058ced57059824a678474874038dc88d211bf508d3" dependencies = [ "form_urlencoded", - "idna 0.3.0", + "idna", "percent-encoding", "serde", ] [[package]] name = "urlencoding" -version = "2.1.2" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8db7427f936968176eaa7cdf81b7f98b980b18495ec28f1b5791ac3bfe3eea9" +checksum = "68b90931029ab9b034b300b797048cf23723400aa757e8a2bfb9d748102f9821" [[package]] name = "utf-8" @@ -8787,18 +8322,17 @@ version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bc5cf98d8186244414c848017f0e2676b3fcb46807f6668a97dfe67359a3c4b7" dependencies = [ - "getrandom 0.2.8", + "getrandom 0.2.6", "serde", ] [[package]] name = "uuid" -version = "1.2.2" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "422ee0de9031b5b948b97a8fc04e3aa35230001a722ddd27943e0be31564ce4c" +checksum = "dd6469f4314d5f1ffec476e05f17cc9a78bc7a27a6a857842170bdf8d6f98d2f" dependencies = [ - "getrandom 0.2.8", - "rand 0.8.5", + "getrandom 0.2.6", "serde", ] @@ -8813,19 +8347,19 @@ name = "value" version = "0.1.0" dependencies = [ "async-graphql", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "indoc", "lookup", "mlua", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "quickcheck", "regex", "serde", "serde_json", "snafu", "toml", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] @@ -8842,11 +8376,10 @@ checksum = "f1bddf1187be692e79c5ffeab891132dfb0f236ed36a43c7ed39f1165ee20191" [[package]] name = "vector" -version = "0.26.0" +version = "0.25.0" dependencies = [ "approx", "arc-swap", - "arr_macro", "assert_cmd", "async-compression", "async-graphql", @@ -8870,7 +8403,7 @@ dependencies = [ "aws-smithy-http-tower", "aws-smithy-types", "aws-types", - "axum 0.5.17", + "axum", "azure_core", "azure_identity", "azure_storage", @@ -8878,11 +8411,11 @@ dependencies = [ "base64", "bloom", "bollard", - "bytes 1.3.0", + "bytes 1.2.1", "bytesize", "chrono", "cidr-utils", - "clap 4.0.29", + "clap 3.2.20", "codecs", "colored", "console-subscriber", @@ -8902,7 +8435,7 @@ dependencies = [ "fakedata", "file-source", "flate2", - "futures 0.3.25", + "futures 0.3.24", "futures-util", "glob", "goauth", @@ -8910,7 +8443,6 @@ dependencies = [ "grok", "h2", "hash_hasher", - "hashbrown 0.12.3", "headers", "heim", "hex", @@ -8922,16 +8454,14 @@ dependencies = [ "hyper-proxy", "indexmap", "indoc", - "infer 0.11.0", + "infer 0.9.0", "inventory", - "itertools", + "itertools 0.10.3", "k8s-openapi", "kube", - "lapin", "libc", "listenfd", "logfmt", - "loki-logproto", "lookup", "lru", "maxminddb", @@ -8941,7 +8471,7 @@ dependencies = [ "mlua", "mongodb", "nats", - "nix 0.25.1", + "nix 0.25.0", "nkeys", "nom", "notify", @@ -8952,16 +8482,17 @@ dependencies = [ "openssl-probe", "openssl-src", "opentelemetry-proto", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "percent-encoding", "pin-project", "portpicker", "postgres-openssl", + "pretty_assertions", "prometheus-parser", "proptest", - "prost 0.11.3", - "prost-build 0.11.3", - "prost-types 0.11.2", + "prost 0.10.4", + "prost-build", + "prost-types 0.10.1", "pulsar", "quickcheck", "rand 0.8.5", @@ -8974,15 +8505,14 @@ dependencies = [ "rmpv", "roaring", "seahash", - "semver 1.0.14", + "semver 1.0.13", "serde", "serde-toml-merge", "serde_bytes", "serde_json", - "serde_with 2.1.0", - "serde_yaml 0.9.14", - "sha2 0.10.6", - "similar-asserts", + "serde_with 2.0.0", + "serde_yaml 0.9.11", + "sha2 0.10.5", "smallvec", "smpl_jwt", "snafu", @@ -8998,24 +8528,24 @@ dependencies = [ "tokio-postgres", "tokio-stream", "tokio-test", - "tokio-tungstenite 0.18.0", - "tokio-util", + "tokio-tungstenite 0.17.2", + "tokio-util 0.7.1", "toml", - "tonic", + "tonic 0.7.2", "tonic-build", "tower", "tower-test", - "tracing 0.1.37", - "tracing-core 0.1.30", + "tracing 0.1.34", + "tracing-core 0.1.28", "tracing-futures 0.2.5", "tracing-limit", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.15", "tracing-tower", - "trust-dns-proto 0.22.0", + "trust-dns-proto", "tui", "typetag", "url", - "uuid 1.2.2", + "uuid 1.1.2", "value", "vector-api-client", "vector-buffers", @@ -9041,8 +8571,7 @@ dependencies = [ "anyhow", "async-trait", "chrono", - "clap 4.0.29", - "futures 0.3.25", + "futures 0.3.24", "graphql_client", "indoc", "reqwest", @@ -9050,9 +8579,9 @@ dependencies = [ "serde_json", "tokio", "tokio-stream", - "tokio-tungstenite 0.18.0", + "tokio-tungstenite 0.17.2", "url", - "uuid 1.2.2", + "uuid 1.1.2", ] [[package]] @@ -9063,15 +8592,15 @@ dependencies = [ "async-stream", "async-trait", "bytecheck", - "bytes 1.3.0", - "clap 4.0.29", + "bytes 1.2.1", + "clap 3.2.20", "crc32fast", "criterion", "crossbeam-queue", "crossbeam-utils", "db-key", "fslock", - "futures 0.3.25", + "futures 0.3.24", "hdrhistogram", "leveldb", "memmap2", @@ -9087,15 +8616,15 @@ dependencies = [ "rand 0.8.5", "rkyv", "serde", - "serde_yaml 0.9.14", + "serde_yaml 0.9.11", "snafu", "temp-dir", "tokio", "tokio-test", - "tokio-util", - "tracing 0.1.37", + "tokio-util 0.7.1", + "tracing 0.1.34", "tracing-fluent-assertions", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.15", "vector-common", "vector-config", "vector-config-common", @@ -9107,30 +8636,25 @@ name = "vector-common" version = "0.1.0" dependencies = [ "async-stream", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "chrono-tz", "crossbeam-utils", "derivative", - "futures 0.3.25", - "indexmap", + "futures 0.3.24", "metrics", "nom", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "pin-project", - "quickcheck", - "quickcheck_macros", - "ryu", "serde", "serde_json", "smallvec", "snafu", "stream-cancel", "tokio", - "tracing 0.1.37", + "tracing 0.1.34", "value", "vector-config", - "vector-config-common", "vector-config-macros", ] @@ -9148,7 +8672,7 @@ dependencies = [ "schemars", "serde", "serde_json", - "serde_with 2.1.0", + "serde_with 2.0.0", "snafu", "toml", "url", @@ -9189,9 +8713,8 @@ dependencies = [ "async-trait", "base64", "bitmask-enum", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", - "chrono-tz", "criterion", "crossbeam-utils", "db-key", @@ -9200,7 +8723,7 @@ dependencies = [ "enumflags2", "env-test-util", "float_eq", - "futures 0.3.25", + "futures 0.3.24", "futures-util", "headers", "http", @@ -9217,16 +8740,15 @@ dependencies = [ "noisy_float", "once_cell", "openssl", - "ordered-float 3.4.0", - "parking_lot", + "ordered-float 3.0.0", "pin-project", + "pretty_assertions", "proptest", - "prost 0.11.3", - "prost-build 0.11.3", - "prost-types 0.11.2", + "prost 0.10.4", + "prost-build", + "prost-types 0.10.1", "quanta", "quickcheck", - "quickcheck_macros", "rand 0.8.5", "rand_distr", "regex", @@ -9234,22 +8756,21 @@ dependencies = [ "security-framework", "serde", "serde_json", - "serde_with 2.1.0", - "similar-asserts", + "serde_with 2.0.0", "snafu", "socket2", "tokio", "tokio-openssl", "tokio-stream", "tokio-test", - "tokio-util", + "tokio-util 0.7.1", "toml", - "tonic", + "tonic 0.7.2", "tower", - "tracing 0.1.37", - "tracing-core 0.1.30", + "tracing 0.1.34", + "tracing-core 0.1.28", "tracing-log", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.15", "twox-hash", "typetag", "url", @@ -9257,7 +8778,6 @@ dependencies = [ "vector-buffers", "vector-common", "vector-config", - "vector-config-common", "vector-config-macros", "vrl", ] @@ -9287,11 +8807,11 @@ checksum = "6a02e4885ed3bc0f2de90ea6dd45ebcbb66dacffe03547fadbb0eeae2770887d" name = "vrl" version = "0.1.0" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "criterion", "indoc", "lookup", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "serde_json", "value", "vector-common", @@ -9306,7 +8826,7 @@ dependencies = [ name = "vrl-cli" version = "0.1.0" dependencies = [ - "clap 4.0.29", + "clap 3.2.20", "exitcode", "indoc", "lookup", @@ -9330,15 +8850,14 @@ name = "vrl-compiler" version = "0.1.0" dependencies = [ "anymap", - "bytes 1.3.0", + "bytes 1.2.1", "chrono", "criterion", "dyn-clone", - "getrandom 0.2.8", "indoc", "lalrpop-util", "lookup", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "paste", "regex", "serde", @@ -9346,7 +8865,6 @@ dependencies = [ "value", "vector-common", "vector-config", - "vector-config-common", "vector-config-macros", "vrl-core", "vrl-diagnostic", @@ -9378,7 +8896,7 @@ dependencies = [ "lalrpop", "lalrpop-util", "lookup", - "ordered-float 3.4.0", + "ordered-float 3.0.0", "paste", "test-case", "thiserror", @@ -9392,17 +8910,15 @@ dependencies = [ "aes", "anyhow", "base64", - "bytes 1.3.0", + "bytes 1.2.1", "cbc", "cfb-mode", - "charset", "chrono", "chrono-tz", "cidr-utils", "criterion", "csv", "ctr", - "data-encoding", "datadog-filter", "datadog-grok", "datadog-search-syntax", @@ -9417,24 +8933,23 @@ dependencies = [ "ofb", "once_cell", "percent-encoding", - "quoted_printable", "rand 0.8.5", "regex", "roxmltree", "rust_decimal", "serde", "serde_json", - "sha-1", - "sha2 0.10.6", + "sha-1 0.10.0", + "sha2 0.10.5", "sha3", "strip-ansi-escapes", "syslog_loose", - "tracing 0.1.37", + "tracing 0.1.34", "tracing-test 0.1.0", "uaparser", "url", "utf8-width", - "uuid 1.2.2", + "uuid 1.1.2", "value", "vector-common", "vrl", @@ -9449,7 +8964,7 @@ dependencies = [ "ansi_term", "chrono", "chrono-tz", - "clap 4.0.29", + "clap 3.2.20", "enrichment", "glob", "lookup", @@ -9458,7 +8973,7 @@ dependencies = [ "serde", "serde_json", "tikv-jemallocator", - "tracing-subscriber 0.3.16", + "tracing-subscriber 0.3.15", "value", "vector-common", "vector-vrl-functions", @@ -9466,20 +8981,6 @@ dependencies = [ "vrl-stdlib", ] -[[package]] -name = "vrl-web-playground" -version = "0.1.0" -dependencies = [ - "getrandom 0.2.8", - "gloo-utils", - "serde", - "serde-wasm-bindgen", - "value", - "vrl", - "vrl-stdlib", - "wasm-bindgen", -] - [[package]] name = "vte" version = "0.10.1" @@ -9523,7 +9024,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "808cf2735cd4b6866113f648b791c6adc5714537bc222d9347bb203386ffda56" dependencies = [ "same-file", - "winapi", + "winapi 0.3.9", "winapi-util", ] @@ -9539,11 +9040,11 @@ dependencies = [ [[package]] name = "warp" -version = "0.3.3" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed7b8be92646fc3d18b06147664ebc5f48d222686cb11a8755e561a735aacc6d" +checksum = "3cef4e1e9114a4b7f1ac799f16ce71c14de5778500c5450ec6b7b920c55b587e" dependencies = [ - "bytes 1.3.0", + "bytes 1.2.1", "futures-channel", "futures-util", "headers", @@ -9554,17 +9055,16 @@ dependencies = [ "mime_guess", "percent-encoding", "pin-project", - "rustls-pemfile 0.2.1", "scoped-tls", "serde", "serde_json", "serde_urlencoded", "tokio", "tokio-stream", - "tokio-tungstenite 0.17.2", - "tokio-util", + "tokio-tungstenite 0.15.0", + "tokio-util 0.6.9", "tower-service", - "tracing 0.1.37", + "tracing 0.1.34", ] [[package]] @@ -9587,23 +9087,23 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.83" +version = "0.2.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eaf9f5aceeec8be17c128b2e93e031fb8a4d469bb9c4ae2d7dc1888b26887268" +checksum = "27370197c907c55e3f1a9fbe26f44e937fe6451368324e009cba39e139dc08ad" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "wasm-bindgen-macro", ] [[package]] name = "wasm-bindgen-backend" -version = "0.2.83" +version = "0.2.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c8ffb332579b0557b52d268b91feab8df3615f265d5270fec2a8c95b17c1142" +checksum = "53e04185bfa3a779273da532f5025e33398409573f348985af9a1cbf3774d3f4" dependencies = [ "bumpalo", + "lazy_static", "log", - "once_cell", "proc-macro2", "quote", "syn", @@ -9612,11 +9112,11 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.33" +version = "0.4.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23639446165ca5a5de86ae1d8896b737ae80319560fbaa4c2887b7da6e7ebd7d" +checksum = "6f741de44b75e14c35df886aff5f1eb73aa114fa5d4d00dcd37b5e01259bf3b2" dependencies = [ - "cfg-if", + "cfg-if 1.0.0", "js-sys", "wasm-bindgen", "web-sys", @@ -9624,9 +9124,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.83" +version = "0.2.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "052be0f94026e6cbc75cdefc9bae13fd6052cdcaf532fa6c45e7ae33a1e6c810" +checksum = "17cae7ff784d7e83a2fe7611cfe766ecf034111b49deb850a3dc7699c08251f5" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -9634,9 +9134,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.83" +version = "0.2.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07bc0c051dc5f23e307b13285f9d75df86bfdf816c5721e573dec1f9b8aa193c" +checksum = "99ec0dc7a4756fffc231aab1b9f2f578d23cd391390ab27f952ae0c9b3ece20b" dependencies = [ "proc-macro2", "quote", @@ -9647,15 +9147,15 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.83" +version = "0.2.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c38c045535d93ec4f0b4defec448e4291638ee608530863b1e2ba115d4fff7f" +checksum = "d554b7f530dee5964d9a9468d95c1f8b8acae4f282807e7d27d4b03099a46744" [[package]] name = "web-sys" -version = "0.3.60" +version = "0.3.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcda906d8be16e728fd5adc5b729afad4e444e106ab28cd1c7256e54fa61510f" +checksum = "7b17e741662c70c8bd24ac5c5b18de314a2c26c32bf8346ee1e6f53de919c283" dependencies = [ "js-sys", "wasm-bindgen", @@ -9663,18 +9163,16 @@ dependencies = [ [[package]] name = "webbrowser" -version = "0.8.2" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a0cc7962b5aaa0dfcebaeef0161eec6edf5f4606c12e6777fd7d392f52033a5" +checksum = "fc6a3cffdb686fbb24d9fb8f03a213803277ed2300f11026a3afe1f108dc021b" dependencies = [ "jni", - "ndk-context", - "objc", - "raw-window-handle", + "ndk-glue", "url", "web-sys", - "widestring 1.0.2", - "winapi", + "widestring 0.5.1", + "winapi 0.3.9", ] [[package]] @@ -9699,9 +9197,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.22.5" +version = "0.22.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "368bfe657969fb01238bb756d351dcade285e0f6fcbd36dcb23359a5169975be" +checksum = "44d8de8415c823c8abd270ad483c6feeac771fad964890779f9a8cb24fbbc1bf" dependencies = [ "webpki 0.22.0", ] @@ -9717,13 +9215,13 @@ dependencies = [ [[package]] name = "which" -version = "4.3.0" +version = "4.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c831fbbee9e129a8cf93e7747a82da9d95ba8e16621cae60ec2cdc849bacb7b" +checksum = "5c4fb54e6113b6a8772ee41c3404fb0301ac79604489467e0a9ce1f3e97c24ae" dependencies = [ "either", + "lazy_static", "libc", - "once_cell", ] [[package]] @@ -9744,6 +9242,12 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "653f141f39ec16bba3c5abe400a0c60da7468261cc2cbf36805022876bc721a8" +[[package]] +name = "winapi" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "167dc9d6949a9b857f3451275e911c3f44255842c1f7a76f33c55103a909087a" + [[package]] name = "winapi" version = "0.3.9" @@ -9754,6 +9258,12 @@ dependencies = [ "winapi-x86_64-pc-windows-gnu", ] +[[package]] +name = "winapi-build" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d315eee3b34aca4797b2da6b13ed88266e6d612562a0c46390af8299fc699bc" + [[package]] name = "winapi-i686-pc-windows-gnu" version = "0.4.0" @@ -9766,7 +9276,7 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70ec6ce85bb158151cae5e5c87f95a8e97d2c0c4b001223f33a334e3ce5de178" dependencies = [ - "winapi", + "winapi 0.3.9", ] [[package]] @@ -9787,6 +9297,32 @@ dependencies = [ "windows-sys 0.36.1", ] +[[package]] +name = "windows-sys" +version = "0.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "030b7ff91626e57a05ca64a07c481973cbb2db774e4852c9c7ca342408c6a99a" +dependencies = [ + "windows_aarch64_msvc 0.30.0", + "windows_i686_gnu 0.30.0", + "windows_i686_msvc 0.30.0", + "windows_x86_64_gnu 0.30.0", + "windows_x86_64_msvc 0.30.0", +] + +[[package]] +name = "windows-sys" +version = "0.34.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5acdd78cb4ba54c0045ac14f62d8f94a03d10047904ae2a40afa1e99d8f70825" +dependencies = [ + "windows_aarch64_msvc 0.34.0", + "windows_i686_gnu 0.34.0", + "windows_i686_msvc 0.34.0", + "windows_x86_64_gnu 0.34.0", + "windows_x86_64_msvc 0.34.0", +] + [[package]] name = "windows-sys" version = "0.36.1" @@ -9801,25 +9337,16 @@ dependencies = [ ] [[package]] -name = "windows-sys" -version = "0.42.0" +name = "windows_aarch64_msvc" +version = "0.30.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a3e1820f08b8513f676f7ab6c1f99ff312fb97b553d30ff4dd86f9f15728aa7" -dependencies = [ - "windows_aarch64_gnullvm", - "windows_aarch64_msvc 0.42.0", - "windows_i686_gnu 0.42.0", - "windows_i686_msvc 0.42.0", - "windows_x86_64_gnu 0.42.0", - "windows_x86_64_gnullvm", - "windows_x86_64_msvc 0.42.0", -] +checksum = "29277a4435d642f775f63c7d1faeb927adba532886ce0287bd985bffb16b6bca" [[package]] -name = "windows_aarch64_gnullvm" -version = "0.42.0" +name = "windows_aarch64_msvc" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41d2aa71f6f0cbe00ae5167d90ef3cfe66527d6f613ca78ac8024c3ccab9a19e" +checksum = "17cffbe740121affb56fad0fc0e421804adf0ae00891205213b5cecd30db881d" [[package]] name = "windows_aarch64_msvc" @@ -9828,10 +9355,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9bb8c3fd39ade2d67e9874ac4f3db21f0d710bee00fe7cab16949ec184eeaa47" [[package]] -name = "windows_aarch64_msvc" -version = "0.42.0" +name = "windows_i686_gnu" +version = "0.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1145e1989da93956c68d1864f32fb97c8f561a8f89a5125f6a2b7ea75524e4b8" + +[[package]] +name = "windows_i686_gnu" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd0f252f5a35cac83d6311b2e795981f5ee6e67eb1f9a7f64eb4500fbc4dcdb4" +checksum = "2564fde759adb79129d9b4f54be42b32c89970c18ebf93124ca8870a498688ed" [[package]] name = "windows_i686_gnu" @@ -9840,10 +9373,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "180e6ccf01daf4c426b846dfc66db1fc518f074baa793aa7d9b9aaeffad6a3b6" [[package]] -name = "windows_i686_gnu" -version = "0.42.0" +name = "windows_i686_msvc" +version = "0.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4a09e3a0d4753b73019db171c1339cd4362c8c44baf1bcea336235e955954a6" + +[[package]] +name = "windows_i686_msvc" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbeae19f6716841636c28d695375df17562ca208b2b7d0dc47635a50ae6c5de7" +checksum = "9cd9d32ba70453522332c14d38814bceeb747d80b3958676007acadd7e166956" [[package]] name = "windows_i686_msvc" @@ -9852,10 +9391,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2e7917148b2812d1eeafaeb22a97e4813dfa60a3f8f78ebe204bcc88f12f024" [[package]] -name = "windows_i686_msvc" -version = "0.42.0" +name = "windows_x86_64_gnu" +version = "0.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ca64fcb0220d58db4c119e050e7af03c69e6f4f415ef69ec1773d9aab422d5a" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84c12f65daa39dd2babe6e442988fc329d6243fdce47d7d2d155b8d874862246" +checksum = "cfce6deae227ee8d356d19effc141a509cc503dfd1f850622ec4b0f84428e1f4" [[package]] name = "windows_x86_64_gnu" @@ -9864,16 +9409,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4dcd171b8776c41b97521e5da127a2d86ad280114807d0b2ab1e462bc764d9e1" [[package]] -name = "windows_x86_64_gnu" -version = "0.42.0" +name = "windows_x86_64_msvc" +version = "0.30.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf7b1b21b5362cbc318f686150e5bcea75ecedc74dd157d874d754a2ca44b0ed" +checksum = "08cabc9f0066848fef4bc6a1c1668e6efce38b661d2aeec75d18d8617eebb5f1" [[package]] -name = "windows_x86_64_gnullvm" -version = "0.42.0" +name = "windows_x86_64_msvc" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09d525d2ba30eeb3297665bd434a54297e4170c7f1a44cad4ef58095b4cd2028" +checksum = "d19538ccc21819d01deaf88d6a17eae6596a12e9aafdbb97916fb49896d89de9" [[package]] name = "windows_x86_64_msvc" @@ -9882,10 +9427,13 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c811ca4a8c853ef420abd8592ba53ddbbac90410fab6903b3e79972a631f7680" [[package]] -name = "windows_x86_64_msvc" -version = "0.42.0" +name = "winreg" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f40009d85759725a34da6d89a94e63d7bdc50a862acf0dbc7c8e488f1edcb6f5" +checksum = "0120db82e8a1e0b9fb3345a539c478767c0048d842860994d96113d5b667bd69" +dependencies = [ + "winapi 0.3.9", +] [[package]] name = "winreg" @@ -9893,20 +9441,20 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d" dependencies = [ - "winapi", + "winapi 0.3.9", ] [[package]] name = "wiremock" -version = "0.5.15" +version = "0.5.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "249dc68542861d17eae4b4e5e8fb381c2f9e8f255a84f6771d5fdf8b6c03ce3c" +checksum = "cc3c7b7557dbfdad6431b5a51196c9110cef9d83f6a9b26699f35cdc0ae113ec" dependencies = [ "assert-json-diff", "async-trait", "base64", "deadpool", - "futures 0.3.25", + "futures 0.3.24", "futures-timer", "http-types", "hyper", @@ -9928,6 +9476,16 @@ dependencies = [ "regex", ] +[[package]] +name = "ws2_32-sys" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d59cefebd0c892fa2dd6de581e937301d8552cb44489cdff035c6187cb63fa5e" +dependencies = [ + "winapi 0.2.8", + "winapi-build", +] + [[package]] name = "xml-rs" version = "0.8.4" @@ -9936,9 +9494,9 @@ checksum = "d2d7d3948613f75c98fd9328cfdcc45acc4d360655289d0a7d4ec931392200a3" [[package]] name = "xmlparser" -version = "0.13.5" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d25c75bf9ea12c4040a97f829154768bbbce366287e2dc044af160cd79a13fd" +checksum = "114ba2b24d2167ef6d67d7d04c8cc86522b87f490025f39f0303b7db5bf5e3d8" [[package]] name = "yaml-rust" @@ -9949,6 +9507,12 @@ dependencies = [ "linked-hash-map", ] +[[package]] +name = "yansi" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" + [[package]] name = "zerocopy" version = "0.3.0" @@ -9972,18 +9536,18 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.5.7" +version = "1.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c394b5bd0c6f669e7275d9c20aa90ae064cb22e75a1cad54e1b34088034b149f" +checksum = "7eb5728b8afd3f280a869ce1d4c554ffaed35f45c231fc41bfbd0381bef50317" dependencies = [ "zeroize_derive", ] [[package]] name = "zeroize_derive" -version = "1.3.3" +version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44bf07cb3e50ea2003396695d58bf46bc9887a1f362260446fad6bc4e79bd36c" +checksum = "3f8f187641dad4f680d25c4bfc4225b418165984179f26ca76ec4fb6441d3a17" dependencies = [ "proc-macro2", "quote", @@ -9993,18 +9557,18 @@ dependencies = [ [[package]] name = "zstd" -version = "0.11.2+zstd.1.5.2" +version = "0.10.2+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" +checksum = "5f4a6bd64f22b5e3e94b4e238669ff9f10815c27a5180108b849d24174a83847" dependencies = [ "zstd-safe", ] [[package]] name = "zstd-safe" -version = "5.0.2+zstd.1.5.2" +version = "4.1.6+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" +checksum = "94b61c51bb270702d6167b8ce67340d2754b088d0c091b06e593aa772c3ee9bb" dependencies = [ "libc", "zstd-sys", @@ -10012,9 +9576,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.4+zstd.1.5.2" +version = "1.6.3+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fa202f2ef00074143e219d15b62ffc317d17cc33909feac471c044087cad7b0" +checksum = "fc49afa5c8d634e75761feda8c592051e7eeb4683ba827211eb0d731d3402ea8" dependencies = [ "cc", "libc", From 89773f1b6c2bbf4297ce43a5e40598f04b80ac4e Mon Sep 17 00:00:00 2001 From: Kyle Criddle Date: Tue, 13 Dec 2022 17:02:57 -0700 Subject: [PATCH 21/48] regenerate cargo.lock --- Cargo.lock | 2773 ++++++++++++++++++++++++++++++---------------------- 1 file changed, 1604 insertions(+), 1169 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ec2f11e43967f..2bd71dde435bb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -32,11 +32,11 @@ checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" [[package]] name = "aes" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfe0133578c0986e1fe3dfcd4af1cc5b2dd6c3dbf534d69916ce16a2701d40ba" +checksum = "433cfd6710c9986c576a25ca913c39d66a6474107b406f34f91d4a8923395241" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "cipher", "cpufeatures", ] @@ -47,7 +47,19 @@ version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" dependencies = [ - "getrandom 0.2.6", + "getrandom 0.2.8", + "once_cell", + "version_check", +] + +[[package]] +name = "ahash" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf6ccdb167abbf410dcb915cabd428929d7f6a04980b54a11f26a39f1c7f7107" +dependencies = [ + "cfg-if", + "getrandom 0.2.8", "once_cell", "version_check", ] @@ -61,20 +73,83 @@ dependencies = [ "memchr", ] +[[package]] +name = "amq-protocol" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acc7cad07d1b4533fcb46f0819a6126fa201fd0385469aba75e405424f3fe009" +dependencies = [ + "amq-protocol-tcp", + "amq-protocol-types", + "amq-protocol-uri", + "cookie-factory", + "nom", + "serde", +] + +[[package]] +name = "amq-protocol-tcp" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5d8b20aba8c35a0b885e1e978eff456ced925730a4e012e63e4ff89a1deb602b" +dependencies = [ + "amq-protocol-uri", + "tcp-stream", + "tracing 0.1.37", +] + +[[package]] +name = "amq-protocol-types" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e245e0e9083b6a6db5f8c10013074cb382266eb9e2a37204d19c651b8d3b8114" +dependencies = [ + "cookie-factory", + "nom", + "serde", + "serde_json", +] + +[[package]] +name = "amq-protocol-uri" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56987108bf48d2eb500cae8896cd9291564eedd8744776ecc5c3338a8b2ca5f8" +dependencies = [ + "amq-protocol-types", + "percent-encoding", + "url", +] + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + [[package]] name = "ansi_term" version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d52a9bb7ec0cf484c551830a7ce27bd20d67eac647e1befb56b0be4ee39a55d2" dependencies = [ - "winapi 0.3.9", + "winapi", ] [[package]] name = "anyhow" -version = "1.0.64" +version = "1.0.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9a8f622bcf6ff3df478e9deba3e03e4e04b300f8e6a139e192c05fa3490afc7" +checksum = "216261ddc8289130e551ddcd5ce8a064710c0d064a4d2895c67151c92b5443f6" [[package]] name = "anymap" @@ -106,6 +181,27 @@ version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "983cd8b9d4b02a6dc6ffa557262eb5858a27a0038ffffe21a0f133eaa819a164" +[[package]] +name = "arr_macro" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a105bfda48707cf19220129e78fca01e9639433ffaef4163546ed8fb04120a5" +dependencies = [ + "arr_macro_impl", + "proc-macro-hack", +] + +[[package]] +name = "arr_macro_impl" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0609c78bd572f4edc74310dfb63a01f5609d53fa8b4dd7c4d98aef3b3e8d72d1" +dependencies = [ + "proc-macro-hack", + "quote", + "syn", +] + [[package]] name = "arrayref" version = "0.3.6" @@ -160,11 +256,11 @@ dependencies = [ [[package]] name = "assert_cmd" -version = "2.0.4" +version = "2.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93ae1ddd39efd67689deb1979d80bad3bf7f2b09c6e6117c8d1f2443b5e2f83e" +checksum = "fa3d466004a8b4cb1bc34044240a2fd29d17607e2e3bd613eb44fd48e8100da3" dependencies = [ - "bstr", + "bstr 1.0.1", "doc-comment", "predicates", "predicates-core", @@ -185,9 +281,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.3.12" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2bf394cfbbe876f0ac67b13b6ca819f9c9f2fb9ec67223cceb1555fbab1c31a" +checksum = "942c7cd7ae39e91bde4820d74132e9862e62c2f386c3aa90ccf55949f5bad63a" dependencies = [ "flate2", "futures-core", @@ -223,11 +319,37 @@ dependencies = [ "futures-lite", ] +[[package]] +name = "async-global-executor" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1b6f5d7df27bd294849f8eec66ecfc63d11814df7a4f5d74168a2394467b776" +dependencies = [ + "async-channel", + "async-executor", + "async-io", + "async-lock", + "blocking", + "futures-lite", + "once_cell", +] + +[[package]] +name = "async-global-executor-trait" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33dd14c5a15affd2abcff50d84efd4009ada28a860f01c14f9d654f3e81b3f75" +dependencies = [ + "async-global-executor", + "async-trait", + "executor-trait", +] + [[package]] name = "async-graphql" -version = "4.0.12" +version = "4.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "097754075ec057cf7d218886882026905e5f8be3a31572a2ffb1be1ef38136c8" +checksum = "d9ed522678d412d77effe47b3c82314ac36952a35e6e852093dd48287c421f80" dependencies = [ "async-graphql-derive", "async-graphql-parser", @@ -235,7 +357,7 @@ dependencies = [ "async-stream", "async-trait", "base64", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "fnv", "futures-util", @@ -249,6 +371,7 @@ dependencies = [ "regex", "serde", "serde_json", + "serde_urlencoded", "static_assertions", "tempfile", "thiserror", @@ -256,9 +379,9 @@ dependencies = [ [[package]] name = "async-graphql-derive" -version = "4.0.12" +version = "4.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7fa9b19ad10364c364a46847c7cc869992a3f8515105c76fa4fcda543787336" +checksum = "c121a894495d7d3fc3d4e15e0a9843e422e4d1d9e3c514d8062a1c94b35b005d" dependencies = [ "Inflector", "async-graphql-parser", @@ -272,25 +395,23 @@ dependencies = [ [[package]] name = "async-graphql-parser" -version = "4.0.12" +version = "4.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52bedfd99a0ddc329585274774cd637271eef9b4d6148f3bbb50b0ae1373ecf3" +checksum = "6b6c386f398145c6180206c1869c2279f5a3d45db5be4e0266148c6ac5c6ad68" dependencies = [ "async-graphql-value", "pest", - "pest_generator", - "proc-macro2", "serde", "serde_json", ] [[package]] name = "async-graphql-value" -version = "4.0.12" +version = "4.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f7b95ca40977a0b78089f20e80ecd34a564ed07d4e6a8641e85e652f7db0a12" +checksum = "7a941b499fead4a3fb5392cabf42446566d18c86313f69f2deab69560394d65f" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "indexmap", "serde", "serde_json", @@ -298,9 +419,9 @@ dependencies = [ [[package]] name = "async-graphql-warp" -version = "4.0.12" +version = "4.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bc6fd192f0f2c11fb6756101f8ae17172bebcfa7521e8e486b43b32551ec57c" +checksum = "c512820fb4259389aa5af696cb78297ef93d8d0f25f62007705de1aa7343253b" dependencies = [ "async-graphql", "futures-util", @@ -324,7 +445,7 @@ dependencies = [ "slab", "socket2", "waker-fn", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -355,13 +476,25 @@ checksum = "83137067e3a2a6a06d67168e49e68a0957d215410473a740cea95a2425c0b7c6" dependencies = [ "async-io", "blocking", - "cfg-if 1.0.0", + "cfg-if", "event-listener", "futures-lite", "libc", "once_cell", "signal-hook", - "winapi 0.3.9", + "winapi", +] + +[[package]] +name = "async-reactor-trait" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a6012d170ad00de56c9ee354aef2e358359deb1ec504254e0e5a3774771de0e" +dependencies = [ + "async-io", + "async-trait", + "futures-core", + "reactor-trait", ] [[package]] @@ -402,23 +535,11 @@ version = "4.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "30696a84d817107fc028e049980e09d5e140e8da8f1caeb17e8e950658a3cea9" -[[package]] -name = "async-timer" -version = "1.0.0-beta.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faacdfdccd10db54656717fddcd1a2ab6cd1ab16c0d6e7d89ec365b885fc9844" -dependencies = [ - "error-code", - "libc", - "wasm-bindgen", - "winapi 0.3.9", -] - [[package]] name = "async-trait" -version = "0.1.57" +version = "0.1.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76464446b8bc32758d7e88ee1a804d9914cd9b1cb264c029899680b0be29826f" +checksum = "31e6e93155431f3931513b243d371981bb2770112b370c82745a1d19d2f99364" dependencies = [ "proc-macro2", "quote", @@ -437,9 +558,9 @@ version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" dependencies = [ - "hermit-abi", + "hermit-abi 0.1.19", "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -472,9 +593,9 @@ dependencies = [ [[package]] name = "aws-config" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d4cf4608abd7c8038a4c609a1270e61b73c86550f5655654ca28322e0a2e2c1" +checksum = "56a636c44c77fa18bdba56126a34d30cfe5538fe88f7d34988fa731fee143ddd" dependencies = [ "aws-http", "aws-sdk-sso", @@ -486,7 +607,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "hex", "http", "hyper", @@ -494,47 +615,47 @@ dependencies = [ "time", "tokio", "tower", - "tracing 0.1.34", + "tracing 0.1.37", "zeroize", ] [[package]] name = "aws-endpoint" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ffaf1da7a11d38a5afe7cdd202ab2e25528de7cf38c47b571c0dde4008d98ae" +checksum = "6ca8f374874f6459aaa88dc861d7f5d834ca1ff97668eae190e97266b5f6c3fb" dependencies = [ "aws-smithy-http", "aws-smithy-types", "aws-types", "http", "regex", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "aws-http" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8309108743e2e74f249ff29a7c7be79c6343ea649dd8c31e4c0e07ca6946d8ed" +checksum = "78d41e19e779b73463f5f0c21b3aacc995f4ba783ab13a7ae9f5dfb159a551b4" dependencies = [ "aws-smithy-http", "aws-smithy-types", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "http-body", "lazy_static", "percent-encoding", "pin-project-lite", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "aws-sdk-cloudwatch" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08ca16408172d748ece9e5a4cac14fe41b92013e532df7b21dacd03fb26f9cc5" +checksum = "520b1ac14f0850d0d6a69136d15ba7702d41ee7f4014a5d2d1bf4a86e74f7a6b" dependencies = [ "aws-endpoint", "aws-http", @@ -547,7 +668,7 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tokio-stream", "tower", @@ -555,9 +676,9 @@ dependencies = [ [[package]] name = "aws-sdk-cloudwatchlogs" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6ac6c1554a99b275931e2f923ec0e3370127488bc365445c530e687fa3ada2d" +checksum = "89415e55b57044a09a7eb0a885c2d0af1aa7f95b373e0e898f71a28d7e7d10f9" dependencies = [ "aws-endpoint", "aws-http", @@ -569,7 +690,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tokio-stream", "tower", @@ -577,9 +698,9 @@ dependencies = [ [[package]] name = "aws-sdk-elasticsearch" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a05dcdc71560f9e2885d66099c04532651e9646cfca225eb97d50bb14db88ced" +checksum = "e9f4cc10278701dbc0d386ddd8cddfda2695eae7103a54eae11b981f28779ff2" dependencies = [ "aws-endpoint", "aws-http", @@ -591,7 +712,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tokio-stream", "tower", @@ -599,9 +720,9 @@ dependencies = [ [[package]] name = "aws-sdk-firehose" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e57a640e35af65b906161060cd6f951aa014e2858bb25f64ea85eb4b1c3e0a9" +checksum = "c68310f9d7860b4fe73c58e5cec4d7a310a658d1a983fdf176eb35149939896a" dependencies = [ "aws-endpoint", "aws-http", @@ -613,16 +734,16 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tower", ] [[package]] name = "aws-sdk-kinesis" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2dc213d616547cf0c47a735a40e6162252d559e1745f65212b0b58b8c161545" +checksum = "37766fdf50feab317b4f939b1c9ee58a2a1c51785974328ce84cff1eea7a1bb8" dependencies = [ "aws-endpoint", "aws-http", @@ -634,7 +755,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tokio-stream", "tower", @@ -642,9 +763,9 @@ dependencies = [ [[package]] name = "aws-sdk-s3" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "323b9107094fc396a0116326b577af48d9cfb26ec7c09588584ec82cee057b81" +checksum = "a9f08665c8e03aca8cb092ef01e617436ebfa977fddc1240e1b062488ab5d48a" dependencies = [ "aws-endpoint", "aws-http", @@ -659,20 +780,20 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "bytes-utils", "http", "http-body", "tokio-stream", "tower", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "aws-sdk-sqs" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "098bf8f363dd36a892b8ce8f013a2d57c15de869647e9da9a10fe4e9dcbf318b" +checksum = "8b26bb3d12238492cb12bde0de8486679b007daada21fdb110913b32a2a38275" dependencies = [ "aws-endpoint", "aws-http", @@ -685,7 +806,7 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tokio-stream", "tower", @@ -693,9 +814,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7a0659e5269f8c4bd06f362ec7e35b4f55956c4d60e0ca177b575db80584a45" +checksum = "86dcb1cb71aa8763b327542ead410424515cff0cde5b753eedd2917e09c63734" dependencies = [ "aws-endpoint", "aws-http", @@ -707,7 +828,7 @@ dependencies = [ "aws-smithy-json", "aws-smithy-types", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tokio-stream", "tower", @@ -715,9 +836,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "0.18.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edc795c7851c0e9bcefde5e6bb610c16a9e03220e0336fc12f75bb80d9ce7e80" +checksum = "fdfcf584297c666f6b472d5368a78de3bc714b6e0a53d7fbf76c3e347c292ab1" dependencies = [ "aws-endpoint", "aws-http", @@ -730,34 +851,34 @@ dependencies = [ "aws-smithy-types", "aws-smithy-xml", "aws-types", - "bytes 1.2.1", + "bytes 1.3.0", "http", "tower", ] [[package]] name = "aws-sig-auth" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ee4bf20136757fd9f606bb4adafe6d19fb02bc48033a8d4f205f21d56fa783a" +checksum = "12cbe7b2be9e185c1fbce27fc9c41c66b195b32d89aa099f98768d9544221308" dependencies = [ "aws-sigv4", "aws-smithy-eventstream", "aws-smithy-http", "aws-types", "http", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "aws-sigv4" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b99b21b3aceaf224cccd693b353e1f38af4ede8c5fc618b97dd458bb63238efc" +checksum = "03ff4cff8c4a101962d593ba94e72cd83891aecd423f0c6e3146bff6fb92c9e3" dependencies = [ "aws-smithy-eventstream", "aws-smithy-http", - "bytes 1.2.1", + "bytes 1.3.0", "form_urlencoded", "hex", "http", @@ -766,14 +887,14 @@ dependencies = [ "regex", "ring", "time", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "aws-smithy-async" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef79062cf5fa881dd156938ca438ec2de0f7ec9342c2f84fa6303274e1484b43" +checksum = "7b3442b4c5d3fc39891a2e5e625735fba6b24694887d49c6518460fde98247a9" dependencies = [ "futures-util", "pin-project-lite", @@ -783,13 +904,13 @@ dependencies = [ [[package]] name = "aws-smithy-checksums" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6face4c12b335ba734a4416c15d5eeb0af88aa61182a84ff50db62bfa261183" +checksum = "cc227e36e346f45298288359f37123e1a92628d1cec6b11b5eb335553278bd9e" dependencies = [ "aws-smithy-http", "aws-smithy-types", - "bytes 1.2.1", + "bytes 1.3.0", "crc32c", "crc32fast", "hex", @@ -798,53 +919,53 @@ dependencies = [ "md-5", "pin-project-lite", "sha1", - "sha2 0.10.5", - "tracing 0.1.34", + "sha2 0.10.6", + "tracing 0.1.37", ] [[package]] name = "aws-smithy-client" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13f402fa9a45353f7f02f8046a6a568143844d201c5b4cc3bedb6442058538c8" +checksum = "ff28d553714f8f54cd921227934fc13a536a1c03f106e56b362fd57e16d450ad" dependencies = [ "aws-smithy-async", "aws-smithy-http", "aws-smithy-http-tower", "aws-smithy-types", - "bytes 1.2.1", + "bytes 1.3.0", "fastrand", "http", "http-body", "hyper", - "hyper-rustls 0.22.1", + "hyper-rustls", "lazy_static", "pin-project-lite", "tokio", "tower", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "aws-smithy-eventstream" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b959c2c1752c2afbd863953046c06f7ee592f68d64719b7bab3193ac3b0fa77" +checksum = "d7ea0df7161ce65b5c8ca6eb709a1a907376fa18226976e41c748ce02ccccf24" dependencies = [ "aws-smithy-types", - "bytes 1.2.1", + "bytes 1.3.0", "crc32fast", ] [[package]] name = "aws-smithy-http" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23861d0b53a1369eab1e8d48c8bb3492eb3def1c2f2222dfb1bad58dd03914a5" +checksum = "bf58ed4fefa61dbf038e5421a521cbc2c448ef69deff0ab1d915d8a10eda5664" dependencies = [ "aws-smithy-eventstream", "aws-smithy-types", - "bytes 1.2.1", + "bytes 1.3.0", "bytes-utils", "futures-core", "http", @@ -853,38 +974,39 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "tracing 0.1.34", + "pin-utils", + "tracing 0.1.37", ] [[package]] name = "aws-smithy-http-tower" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04f6b3ae42d5c52bbaadfdd31c09fd11c92b823d329915dedbb08c0e9525755c" +checksum = "20c96d7bd35e7cf96aca1134b2f81b1b59ffe493f7c6539c051791cbbf7a42d3" dependencies = [ "aws-smithy-http", - "bytes 1.2.1", + "bytes 1.3.0", "http", "http-body", "pin-project-lite", "tower", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "aws-smithy-json" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5048b693643803c001f88fad36c5a7aa1159e56b0025527fadc57e830aa48b11" +checksum = "d8324ba98c8a94187723cc16c37aefa09504646ee65c3d2c3af495bab5ea701b" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-query" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b317cd3b326444e659a2f287f67e8c72903495c71a3473b0764880454b3aa25c" +checksum = "83834ed2ff69ea6f6657baf205267dc2c0abe940703503a3e5d60ce23be3d306" dependencies = [ "aws-smithy-types", "urlencoding", @@ -892,9 +1014,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4149b09b9d8cf37f0afc390144f5d71b8f4daadfd9540ddf43ad27b54d407470" +checksum = "8b02e06ea63498c43bc0217ea4d16605d4e58d85c12fc23f6572ff6d0a840c61" dependencies = [ "itoa 1.0.1", "num-integer", @@ -904,18 +1026,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c6d8e7a15feb04f041cf0ede8f6c16e03fe5a4b03e164ae3a090e829404d925" +checksum = "246e9f83dd1fdf5d347fa30ae4ad30a9d1d42ce4cd74a93d94afa874646f94cd" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "0.48.0" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bba03e59e1a0223a2bd3567da2b07a458b067ccf7846996b82406e80008ebc1" +checksum = "05701d32da168b44f7ee63147781aed8723e792cc131cb9b18363b5393f17f70" dependencies = [ "aws-smithy-async", "aws-smithy-client", @@ -923,20 +1045,20 @@ dependencies = [ "aws-smithy-types", "http", "rustc_version 0.4.0", - "tracing 0.1.34", + "tracing 0.1.37", "zeroize", ] [[package]] name = "axum" -version = "0.5.15" +version = "0.5.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9de18bc5f2e9df8f52da03856bf40e29b747de5a84e43aefff90e3dc4a21529b" +checksum = "acee9fd5073ab6b045a275b3e709c163dd36c90685219cb21804a147b58dba43" dependencies = [ "async-trait", "axum-core", "bitflags", - "bytes 1.2.1", + "bytes 1.3.0", "futures-util", "http", "http-body", @@ -958,115 +1080,110 @@ dependencies = [ [[package]] name = "axum-core" -version = "0.2.7" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4f44a0e6200e9d11a1cdc989e4b358f6e3d354fbf48478f345a17f4e43f8635" +checksum = "37e5939e02c56fecd5c017c37df4238c0a839fa76b7f97acdd7efb804fd181cc" dependencies = [ "async-trait", - "bytes 1.2.1", + "bytes 1.3.0", "futures-util", "http", "http-body", "mime", + "tower-layer", + "tower-service", ] [[package]] name = "azure_core" -version = "0.2.2" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" +version = "0.5.0" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" dependencies = [ "async-trait", "base64", - "bytes 1.2.1", - "chrono", + "bytes 1.3.0", "dyn-clone", - "futures 0.3.24", - "getrandom 0.2.6", - "http", + "futures 0.3.25", + "getrandom 0.2.8", + "http-types", "log", - "oauth2", + "paste", "pin-project", "rand 0.8.5", "reqwest", "rustc_version 0.4.0", "serde", - "serde_derive", + "serde-xml-rs", "serde_json", - "thiserror", + "time", "url", - "uuid 1.1.2", + "uuid 1.2.2", ] [[package]] name = "azure_identity" -version = "0.3.0" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" +version = "0.6.0" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" dependencies = [ "async-lock", - "async-timer", "async-trait", "azure_core", "base64", - "chrono", - "futures 0.3.24", + "fix-hidden-lifetime-bug", + "futures 0.3.25", "log", "oauth2", - "reqwest", "serde", "serde_json", - "thiserror", + "time", "url", - "uuid 1.1.2", + "uuid 1.2.2", ] [[package]] name = "azure_storage" -version = "0.2.0" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" +version = "0.6.0" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" dependencies = [ "RustyXML", "async-trait", "azure_core", "base64", - "bytes 1.2.1", - "chrono", - "futures 0.3.24", + "bytes 1.3.0", + "futures 0.3.25", "hmac", - "http", "log", "once_cell", "serde", "serde-xml-rs", "serde_derive", "serde_json", - "sha2 0.10.5", - "thiserror", + "sha2 0.10.6", + "time", "url", - "uuid 1.1.2", + "uuid 1.2.2", ] [[package]] name = "azure_storage_blobs" -version = "0.2.0" -source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b7171eb40909f7f2805f4622e076f8a6dbbe2d98#b7171eb40909f7f2805f4622e076f8a6dbbe2d98" +version = "0.6.0" +source = "git+https://github.com/Azure/azure-sdk-for-rust.git?rev=b4544d4920fa3064eb921340054cd9cc130b7664#b4544d4920fa3064eb921340054cd9cc130b7664" dependencies = [ "RustyXML", "azure_core", "azure_storage", "base64", - "bytes 1.2.1", - "chrono", - "futures 0.3.24", - "http", + "bytes 1.3.0", + "futures 0.3.25", "log", "md5", "serde", "serde-xml-rs", "serde_derive", "serde_json", - "thiserror", + "time", "url", - "uuid 1.1.2", + "uuid 1.2.2", ] [[package]] @@ -1075,16 +1192,16 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b62ddb9cb1ec0a098ad4bbf9344d0713fa193ae1a80af55febcff2627b6a00c1" dependencies = [ - "getrandom 0.2.6", + "getrandom 0.2.8", "instant", "rand 0.8.5", ] [[package]] name = "base64" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "904dfeac50f3cdaba28fc6f57fdcddb75f49ed61346676a78c4ffe55877802fd" +checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" [[package]] name = "base64-url" @@ -1130,9 +1247,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitmask-enum" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76487de46597d345d040a1be49a6fb636b71d0abab4696b7f3492e0cd4639c73" +checksum = "fd9e32d7420c85055e8107e5b2463c4eeefeaac18b52359fe9f9c08a18f342b2" dependencies = [ "quote", "syn", @@ -1207,7 +1324,7 @@ checksum = "d82e7850583ead5f8bbef247e2a3c37a19bd576e8420cd262a6711921827e1e5" dependencies = [ "base64", "bollard-stubs", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "dirs-next", "futures-core", @@ -1215,7 +1332,7 @@ dependencies = [ "hex", "http", "hyper", - "hyper-rustls 0.23.0", + "hyper-rustls", "hyperlocal", "log", "pin-project-lite", @@ -1228,11 +1345,11 @@ dependencies = [ "serde_urlencoded", "thiserror", "tokio", - "tokio-util 0.7.1", + "tokio-util", "url", "webpki 0.22.0", "webpki-roots", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1252,7 +1369,7 @@ version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a24ecf39f5a314493ede1bb015984735d41aa6aedb59cafb95492d40cd893330" dependencies = [ - "ahash", + "ahash 0.7.6", "base64", "hex", "indexmap", @@ -1277,6 +1394,18 @@ dependencies = [ "serde", ] +[[package]] +name = "bstr" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fca0852af221f458706eb0725c03e4ed6c46af9ac98e6a689d5e634215d594dd" +dependencies = [ + "memchr", + "once_cell", + "regex-automata", + "serde", +] + [[package]] name = "bumpalo" version = "3.9.1" @@ -1328,9 +1457,9 @@ dependencies = [ [[package]] name = "bytes" -version = "1.2.1" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec8a7b6a70fde80372154c65702f00a0f56f3e1c36abbc6c440484be248856db" +checksum = "dfb24e866b15a1af2a1b663f10c6b6b8f397a84aadb828f12e5b289ec23a3a3c" dependencies = [ "serde", ] @@ -1341,7 +1470,7 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1934a3ef9cac8efde4966a92781e77713e1ba329f1d42e446c7d7eba340d8ef1" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "either", ] @@ -1363,15 +1492,6 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "df8670b8c7b9dae1793364eafadf7239c40d669904660c5960d74cfd80b46a53" -[[package]] -name = "cast" -version = "0.2.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c24dab4283a142afa2fdca129b80ad2c6284e073930f964c3a1293c225ee39a" -dependencies = [ - "rustc_version 0.4.0", -] - [[package]] name = "cast" version = "0.3.0" @@ -1404,42 +1524,48 @@ checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" [[package]] name = "cfb-mode" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b258c89d55316f606923b53511d0d41c20c4a2d440e5e56dd7185267aa98bd0e" +checksum = "738b8d467867f80a71351933f70461f5b56f24d5c93e0cf216e59229c968d330" dependencies = [ "cipher", ] [[package]] name = "cfg-if" -version = "0.1.10" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4785bdd1c96b2a846b2bd7cc02e86b6b3dbf14e7e53446c4f54c92a361040822" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] -name = "cfg-if" -version = "1.0.0" +name = "charset" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" +checksum = "18e9079d1a12a2cc2bffb5db039c43661836ead4082120d5844f02555aca2d46" +dependencies = [ + "base64", + "encoding_rs", +] [[package]] name = "chrono" -version = "0.4.19" -source = "git+https://github.com/vectordotdev/chrono.git?branch=no-default-time#b7abfc3fcd7ffc95ad340f584fbcbe0c2d61e1e9" +version = "0.4.22" +source = "git+https://github.com/vectordotdev/chrono.git?branch=no-default-time-v0.4.22-1#920ff24cabedcec0f8459b8d9681cbac186dfb58" dependencies = [ - "libc", + "iana-time-zone", + "js-sys", "num-integer", "num-traits", "serde", - "winapi 0.3.9", + "wasm-bindgen", + "winapi", ] [[package]] name = "chrono-tz" -version = "0.6.3" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c39203181991a7dd4343b8005bd804e7a9a37afb8ac070e43771e8c820bbde" +checksum = "fa48fa079165080f11d7753fd0bc175b7d391f276b965fe4b55bfad67856e463" dependencies = [ "chrono", "chrono-tz-build", @@ -1449,20 +1575,47 @@ dependencies = [ [[package]] name = "chrono-tz-build" -version = "0.0.3" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f509c3a87b33437b05e2458750a0700e5bdd6956176773e6c7d6dd15a283a0c" +checksum = "d9998fb9f7e9b2111641485bf8beb32f92945f97f92a3d061f744cfef335f751" dependencies = [ "parse-zoneinfo", "phf", "phf_codegen", ] +[[package]] +name = "ciborium" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0c137568cc60b904a7724001b35ce2630fd00d5d84805fbb608ab89509d788f" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "346de753af073cc87b52b2083a506b38ac176a44cfb05497b622e27be899b369" + +[[package]] +name = "ciborium-ll" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "213030a2b5a4e0c0892b6652260cf6ccac84827b83a85a534e178e3906c4cf1b" +dependencies = [ + "ciborium-io", + "half", +] + [[package]] name = "cidr-utils" -version = "0.5.7" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a0c49d94395dc29876e75966ca2176e6c7c2ab117ca99d06937faae6ca89c7b" +checksum = "355d5b5df67e58b523953d0c1a8d3d2c05f5af51f1332b0199b9c92263614ed0" dependencies = [ "debug-helper", "num-bigint 0.4.3", @@ -1502,22 +1655,33 @@ version = "3.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "23b71c3ce99b7611011217b366d923f1d0a7e07a92bb2dbf1e84508c673ca3bd" dependencies = [ - "atty", "bitflags", - "clap_derive", - "clap_lex", + "clap_lex 0.2.3", "indexmap", + "textwrap 0.15.0", +] + +[[package]] +name = "clap" +version = "4.0.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d63b9e9c07271b9957ad22c173bae2a4d9a81127680962039296abcd2f8251d" +dependencies = [ + "bitflags", + "clap_derive", + "clap_lex 0.3.0", + "is-terminal", "once_cell", "strsim 0.10.0", "termcolor", - "textwrap 0.15.0", + "terminal_size 0.2.3", ] [[package]] name = "clap_derive" -version = "3.2.18" +version = "4.0.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea0c8bce528c4be4da13ea6fead8965e95b6073585a2f05204bd8f4119f82a65" +checksum = "0177313f9f02afc995627906bbd8967e2be069f5261954222dac78290c2b9014" dependencies = [ "heck 0.4.0", "proc-macro-error", @@ -1535,6 +1699,15 @@ dependencies = [ "os_str_bytes", ] +[[package]] +name = "clap_lex" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d4198f73e42b4936b35b5bb248d81d2b595ecb170da0bac7655c54eedfa8da8" +dependencies = [ + "os_str_bytes", +] + [[package]] name = "clipboard-win" version = "4.4.1" @@ -1543,7 +1716,7 @@ checksum = "2f3e1238132dc01f081e1cbb9dace14e5ef4c3a51ee244bd982275fb514605db" dependencies = [ "error-code", "str-buf", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1560,30 +1733,31 @@ name = "codecs" version = "0.1.0" dependencies = [ "avro-rs", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "derivative", "dyn-clone", - "futures 0.3.24", + "futures 0.3.25", "indoc", "lookup", "memchr", "once_cell", - "ordered-float 3.0.0", - "pretty_assertions", - "prost 0.10.4", + "ordered-float 3.4.0", + "prost 0.11.3", "regex", "serde", "serde_json", + "similar-asserts", "smallvec", "snafu", "syslog_loose", "tokio", - "tokio-util 0.7.1", - "tracing 0.1.34", + "tokio-util", + "tracing 0.1.37", "value", "vector-common", "vector-config", + "vector-config-common", "vector-config-macros", "vector-core", ] @@ -1606,7 +1780,7 @@ checksum = "b3616f750b84d8f0de8a58bda93e08e2a81ad3f523089b05f1dffecab48c6cbd" dependencies = [ "atty", "lazy_static", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1628,12 +1802,12 @@ version = "4.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2a604e93b79d1808327a6fca85a6f2d69de66461e7620f5a4cbf5fb4d1d7c948" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "futures-core", "memchr", "pin-project-lite", "tokio", - "tokio-util 0.7.1", + "tokio-util", ] [[package]] @@ -1645,16 +1819,29 @@ dependencies = [ "cache-padded", ] +[[package]] +name = "console" +version = "0.15.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c050367d967ced717c04b65d8c619d863ef9292ce0c5760028655a2fb298718c" +dependencies = [ + "encode_unicode 0.3.6", + "lazy_static", + "libc", + "terminal_size 0.1.17", + "winapi", +] + [[package]] name = "console-api" version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e57ff02e8ad8e06ab9731d5dc72dc23bef9200778eae1a89d555d8c42e5d4a86" dependencies = [ - "prost 0.11.0", - "prost-types 0.11.1", - "tonic 0.8.0", - "tracing-core 0.1.28", + "prost 0.11.3", + "prost-types 0.11.2", + "tonic", + "tracing-core 0.1.30", ] [[package]] @@ -1666,19 +1853,19 @@ dependencies = [ "console-api", "crossbeam-channel", "crossbeam-utils", - "futures 0.3.24", + "futures 0.3.25", "hdrhistogram", "humantime", - "prost-types 0.11.1", + "prost-types 0.11.2", "serde", "serde_json", "thread_local", "tokio", "tokio-stream", - "tonic 0.8.0", - "tracing 0.1.34", - "tracing-core 0.1.28", - "tracing-subscriber 0.3.15", + "tonic", + "tracing 0.1.37", + "tracing-core 0.1.30", + "tracing-subscriber 0.3.16", ] [[package]] @@ -1699,6 +1886,12 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" +[[package]] +name = "cookie-factory" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "396de984970346b0d9e93d1415082923c679e5ae5c3ee3dcbd104f5610af126b" + [[package]] name = "core-foundation" version = "0.9.3" @@ -1754,22 +1947,23 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b540bd8bc810d3885c6ea91e2018302f68baba2129ab3e88f32389ee9370880d" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", ] [[package]] name = "criterion" -version = "0.3.6" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b01d6de93b2b6c65e17c634a26653a29d107b3c98c607c765bf38d041531cd8f" +checksum = "e7c76e09c1aae2bc52b3d2f29e13c6572553b30c4aa1b8a49fd70de6412654cb" dependencies = [ + "anes", "atty", - "cast 0.3.0", - "clap 2.34.0", + "cast", + "ciborium", + "clap 3.2.20", "criterion-plot", - "csv", - "futures 0.3.24", - "itertools 0.10.3", + "futures 0.3.25", + "itertools", "lazy_static", "num-traits", "oorandom", @@ -1777,7 +1971,6 @@ dependencies = [ "rayon", "regex", "serde", - "serde_cbor", "serde_derive", "serde_json", "tinytemplate", @@ -1787,12 +1980,12 @@ dependencies = [ [[package]] name = "criterion-plot" -version = "0.4.4" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d00996de9f2f7559f7f4dc286073197f83e92256a59ed395f9aac01fe717da57" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" dependencies = [ - "cast 0.2.7", - "itertools 0.10.3", + "cast", + "itertools", ] [[package]] @@ -1801,7 +1994,7 @@ version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5aaa7bd5fb665c6864b5f963dd9097905c54125909c7aa94c9e18507cdbe6c53" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "crossbeam-utils", ] @@ -1811,7 +2004,7 @@ version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6455c0ca19f0d2fbf751b908d5c55c1f5cbc65e03c4225427254b46890bdde1e" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "crossbeam-epoch", "crossbeam-utils", ] @@ -1823,7 +2016,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1145cf131a2c6ba0615079ab6a638f7e1973ac9c2634fcbeaaad6114246efe8c" dependencies = [ "autocfg", - "cfg-if 1.0.0", + "cfg-if", "crossbeam-utils", "lazy_static", "memoffset", @@ -1832,22 +2025,21 @@ dependencies = [ [[package]] name = "crossbeam-queue" -version = "0.3.6" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd42583b04998a5363558e5f9291ee5a5ff6b49944332103f251e7479a82aa7" +checksum = "d1cfb3ea8a53f37c40dea2c7bedcbd88bdfae54f5e2175d6ecaff1c988353add" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "crossbeam-utils", ] [[package]] name = "crossbeam-utils" -version = "0.8.11" +version = "0.8.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51887d4adc7b564537b15adcfb307936f8075dfcd5f00dde9a9f1d29383682bc" +checksum = "4fb766fa798726286dbbb842f174001dab8abc7b627a1dd86e0b7222a95d929f" dependencies = [ - "cfg-if 1.0.0", - "once_cell", + "cfg-if", ] [[package]] @@ -1860,11 +2052,11 @@ dependencies = [ "crossterm_winapi", "futures-core", "libc", - "mio 0.8.2", + "mio", "parking_lot", "signal-hook", "signal-hook-mio", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1873,7 +2065,7 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2ae1b35a484aa10e07fe0638d02301c5ad24de82d310ccbd2f3693da5f09bf1c" dependencies = [ - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1898,7 +2090,7 @@ version = "1.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "22813a6dc45b335f9bade10bf7271dc477e81113e89eb251a0bc2a8a81c536e1" dependencies = [ - "bstr", + "bstr 0.2.17", "csv-core", "itoa 0.4.8", "ryu", @@ -1914,15 +2106,6 @@ dependencies = [ "memchr", ] -[[package]] -name = "ct-logs" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1a816186fa68d9e426e3cb4ae4dff1fcd8e4a2c34b781bf7a822574a0d0aac8" -dependencies = [ - "sct 0.6.1", -] - [[package]] name = "ctor" version = "0.1.22" @@ -1935,13 +2118,19 @@ dependencies = [ [[package]] name = "ctr" -version = "0.9.1" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d14f329cfbaf5d0e06b5e87fff7e265d2673c5ea7d2c27691a2c107db1442a0" +checksum = "0369ee1ad671834580515889b80f2ea915f23b8be8d0daa4bbaf2ac5c7590835" dependencies = [ "cipher", ] +[[package]] +name = "cty" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b365fabc795046672053e29c954733ec3b05e4be654ab130fe8f1f94d7051f35" + [[package]] name = "curve25519-dalek" version = "3.2.0" @@ -1955,6 +2144,50 @@ dependencies = [ "zeroize", ] +[[package]] +name = "cxx" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bdf07d07d6531bfcdbe9b8b739b104610c6508dcc4d63b410585faf338241daf" +dependencies = [ + "cc", + "cxxbridge-flags", + "cxxbridge-macro", + "link-cplusplus", +] + +[[package]] +name = "cxx-build" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2eb5b96ecdc99f72657332953d4d9c50135af1bac34277801cc3937906ebd39" +dependencies = [ + "cc", + "codespan-reporting", + "once_cell", + "proc-macro2", + "quote", + "scratch", + "syn", +] + +[[package]] +name = "cxxbridge-flags" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac040a39517fd1674e0f32177648334b0f4074625b5588a64519804ba0553b12" + +[[package]] +name = "cxxbridge-macro" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1362b0ddcfc4eb0a1f57b68bd77dd99f0e826958a96abd0ae9bd092e114ffed6" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "darling" version = "0.13.4" @@ -2031,7 +2264,7 @@ version = "5.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c8858831f7781322e539ea39e72449c46b059638250c14344fec8d0aa6e539c" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "num_cpus", "parking_lot", ] @@ -2044,12 +2277,9 @@ checksum = "3ee2393c4a91429dffb4bedf19f4d6abf27d8a732c8ce4980305d782e5426d57" [[package]] name = "data-url" -version = "0.1.1" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a30bfce702bcfa94e906ef82421f2c0e61c076ad76030c16ee5d2e9a32fe193" -dependencies = [ - "matches", -] +checksum = "8d7439c3735f405729d52c3fbbe4de140eaf938a1fe47d227c27f8254d4302a5" [[package]] name = "datadog-filter" @@ -2064,7 +2294,7 @@ dependencies = [ name = "datadog-grok" version = "0.1.0" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "chrono-tz", "criterion", @@ -2074,12 +2304,12 @@ dependencies = [ "nom", "once_cell", "onig", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "peeking_take_while", "regex", "serde_json", "thiserror", - "tracing 0.1.34", + "tracing 0.1.37", "tracing-test 0.2.1", "value", "vector-common", @@ -2090,7 +2320,7 @@ dependencies = [ name = "datadog-search-syntax" version = "0.1.0" dependencies = [ - "itertools 0.10.3", + "itertools", "once_cell", "pest", "pest_derive", @@ -2195,9 +2425,9 @@ dependencies = [ [[package]] name = "digest" -version = "0.10.3" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2fb860ca6fafa5552fb6d0e816a69c8e49f0908bf524e30a90d97c85892d506" +checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" dependencies = [ "block-buffer 0.10.2", "crypto-common", @@ -2212,7 +2442,7 @@ checksum = "3fd78930633bd1c6e35c4b42b1df7b0cbc6bc191146e512bb3bedf243fcc3901" dependencies = [ "libc", "redox_users 0.3.5", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -2221,7 +2451,7 @@ version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b98cf8ebf19c3d1b223e151f99a4f9f0690dca41414773390fc824184ac833e1" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "dirs-sys-next", ] @@ -2233,7 +2463,7 @@ checksum = "4ebda144c4fe02d1f7ea1a7d9641b6fc6b580adcfa024ae48797ecdeb6825b4d" dependencies = [ "libc", "redox_users 0.4.3", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -2242,10 +2472,10 @@ version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "53ecafc952c4528d9b51a458d1a8904b81783feff9fde08ab6ed2545ff396872" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "libc", "socket2", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -2255,7 +2485,7 @@ dependencies = [ "criterion", "data-encoding", "thiserror", - "trust-dns-proto", + "trust-dns-proto 0.22.0", ] [[package]] @@ -2336,7 +2566,7 @@ version = "0.8.31" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9852635589dc9f9ea1b6fe9f05b50ef208c85c834a562f0c6abb1c475736ec2b" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "serde", ] @@ -2370,6 +2600,18 @@ dependencies = [ "syn", ] +[[package]] +name = "enum-as-inner" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +dependencies = [ + "heck 0.4.0", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "enum_dispatch" version = "0.3.8" @@ -2420,12 +2662,12 @@ dependencies = [ [[package]] name = "env_logger" -version = "0.9.0" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b2cf0344971ee6c64c31be0d530793fba457d322dfec2810c453d0ef228f9c3" +checksum = "85cdab6a89accf66733ad5a1693a4dcced6aeff64602b634530dd73c1f3ee9f0" dependencies = [ - "atty", "humantime", + "is-terminal", "log", "regex", "termcolor", @@ -2462,7 +2704,7 @@ checksum = "f639046355ee4f37944e44f60642c6f3a7efa3cf6b78c78a0d989a8ce6c396a1" dependencies = [ "errno-dragonfly", "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -2500,6 +2742,15 @@ version = "2.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "77f3309417938f28bf8228fcff79a4a37103981e3e186d2ccd19c74b38f4eb71" +[[package]] +name = "executor-trait" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a1052dd43212a7777ec6a69b117da52f5e52f07aec47d00c1a2b33b85d06b08" +dependencies = [ + "async-trait", +] + [[package]] name = "exitcode" version = "1.1.2" @@ -2547,8 +2798,8 @@ version = "3.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "46e245f4c8ec30c6415c56cb132c07e69e74f1942f6b4a4061da748b49f486ca" dependencies = [ - "cfg-if 1.0.0", - "rustix", + "cfg-if", + "rustix 0.34.4", "windows-sys 0.30.0", ] @@ -2562,26 +2813,26 @@ checksum = "534be3985e262585e2a8bb8322969e2115f0eff293b5bf522bef397fff54c9bc" name = "file-source" version = "0.1.0" dependencies = [ - "bstr", - "bytes 1.2.1", + "bstr 1.0.1", + "bytes 1.3.0", "chrono", "crc", "criterion", "dashmap", "flate2", - "futures 0.3.24", + "futures 0.3.25", "glob", "indexmap", "libc", - "pretty_assertions", "quickcheck", "scan_fmt", "serde", "serde_json", + "similar-asserts", "tempfile", "tokio", - "tracing 0.1.34", - "winapi 0.3.9", + "tracing 0.1.37", + "winapi", ] [[package]] @@ -2590,10 +2841,30 @@ version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0408e2626025178a6a7f7ffc05a25bc47103229f19c113755de7bf63816290c" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "libc", "redox_syscall 0.2.13", - "winapi 0.3.9", + "winapi", +] + +[[package]] +name = "fix-hidden-lifetime-bug" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4ae9c2016a663983d4e40a9ff967d6dcac59819672f0b47f2b17574e99c33c8" +dependencies = [ + "fix-hidden-lifetime-bug-proc_macros", +] + +[[package]] +name = "fix-hidden-lifetime-bug-proc_macros" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4c81935e123ab0741c4c4f0d9b8377e5fb21d3de7e062fa4b1263b1fbcba1ea" +dependencies = [ + "proc-macro2", + "quote", + "syn", ] [[package]] @@ -2604,9 +2875,9 @@ checksum = "279fb028e20b3c4c320317955b77c5e0c9701f05a1d309905d6fc702cdc5053e" [[package]] name = "flate2" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f82b0f4c27ad9f8bfd1f3208d882da2b09c301bc1c828fd3a00d0216d2fbbff6" +checksum = "a8a2db397cb1c8772f31494cb8917e48cd1e64f0fa7efac59fbd741a0a8ce841" dependencies = [ "crc32fast", "miniz_oxide", @@ -2618,6 +2889,18 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b59b6469c35ab601d6487d28879bccfbe8c896c33a3fe699c4d29817e552cc58" +[[package]] +name = "flume" +version = "0.10.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1657b4441c3403d9f7b3409e47575237dac27b1b5726df654a6ecbf92f0f7577" +dependencies = [ + "futures-core", + "futures-sink", + "pin-project", + "spin 0.9.3", +] + [[package]] name = "fnv" version = "1.0.7" @@ -2641,11 +2924,10 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "form_urlencoded" -version = "1.0.1" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fc25a87fa4fd2094bffb06925852034d90a17f0d1e05197d4956d3555752191" +checksum = "a9c384f161156f5260c24a097c56119f9be8c798586aecc13afbcbe7b7e26bf8" dependencies = [ - "matches", "percent-encoding", ] @@ -2655,21 +2937,11 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2022715d62ab30faffd124d40b76f4134a550a87792276512b18d63272333394" -[[package]] -name = "fsevent" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ab7d1bd1bd33cc98b0889831b72da23c0aa4df9cec7e0702f46ecea04b35db6" -dependencies = [ - "bitflags", - "fsevent-sys", -] - [[package]] name = "fsevent-sys" -version = "2.0.1" +version = "4.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f41b048a94555da0f42f1d632e2e19510084fb8e303b0daa2816e733fb3644a0" +checksum = "76ee7a02da4d231650c7cea31349b889be2f45ddb3ef3032d2ec8185f6313fd2" dependencies = [ "libc", ] @@ -2681,25 +2953,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04412b8935272e3a9bae6f48c7bfff74c2911f60525404edfdd28e49884c3bfb" dependencies = [ "libc", - "winapi 0.3.9", -] - -[[package]] -name = "fuchsia-zircon" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e9763c69ebaae630ba35f74888db465e49e259ba1bc0eda7d06f4a067615d82" -dependencies = [ - "bitflags", - "fuchsia-zircon-sys", + "winapi", ] -[[package]] -name = "fuchsia-zircon-sys" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dcaa9ae7725d12cdb85b3ad99a434db70b468c09ded17e012d86b5c1010f7a7" - [[package]] name = "futures" version = "0.1.31" @@ -2708,9 +2964,9 @@ checksum = "3a471a38ef8ed83cd6e40aa59c1ffe17db6855c18e3604d9c4ed8c08ebc28678" [[package]] name = "futures" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f21eda599937fba36daeb58a22e8f5cee2d14c4a17b5b7739c7c8e5e3b8230c" +checksum = "38390104763dc37a5145a53c29c63c1290b5d316d6086ec32c293f6736051bb0" dependencies = [ "futures-channel", "futures-core", @@ -2723,9 +2979,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30bdd20c28fadd505d0fd6712cdfcb0d4b5648baf45faef7f852afb2399bb050" +checksum = "52ba265a92256105f45b719605a571ffe2d1f0fea3807304b522c1d778f79eed" dependencies = [ "futures-core", "futures-sink", @@ -2733,15 +2989,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e5aa3de05362c3fb88de6531e6296e85cde7739cccad4b9dfeeb7f6ebce56bf" +checksum = "04909a7a7e4633ae6c4a9ab280aeb86da1236243a77b694a49eacd659a4bd3ac" [[package]] name = "futures-executor" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ff63c23854bee61b6e9cd331d523909f238fc7636290b96826e9cfa5faa00ab" +checksum = "7acc85df6714c176ab5edf386123fafe217be88c0840ec11f199441134a074e2" dependencies = [ "futures-core", "futures-task", @@ -2750,9 +3006,9 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbf4d2a7a308fd4578637c0b17c7e1c7ba127b8f6ba00b29f717e9655d85eb68" +checksum = "00f5fb52a06bdcadeb54e8d3671f8888a39697dcb0b81b23b55174030427f4eb" [[package]] name = "futures-lite" @@ -2771,9 +3027,9 @@ dependencies = [ [[package]] name = "futures-macro" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42cd15d1c7456c04dbdf7e88bcd69760d74f3a798d6444e16974b505b0e62f17" +checksum = "bdfb8ce053d86b91919aad980c220b1fb8401a9394410e1c289ed7e66b61835d" dependencies = [ "proc-macro2", "quote", @@ -2782,15 +3038,15 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21b20ba5a92e727ba30e72834706623d94ac93a725410b6a6b6fbc1b07f7ba56" +checksum = "39c15cf1a4aa79df40f1bb462fb39676d0ad9e366c2a33b590d7c66f4f81fcf9" [[package]] name = "futures-task" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6508c467c73851293f390476d4491cf4d227dbabcd4170f3bb6044959b294f1" +checksum = "2ffb393ac5d9a6eaa9d3fdf37ae2776656b706e200c8e16b1bdb227f5198e6ea" [[package]] name = "futures-timer" @@ -2800,9 +3056,9 @@ checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" [[package]] name = "futures-util" -version = "0.3.24" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44fb6cb1be61cc1d2e43b262516aafcf63b241cffdb1d3fa115f91d9c7b09c90" +checksum = "197676987abd2f9cadff84926f410af1c183608d36641465df73ae8211dc65d6" dependencies = [ "futures 0.1.31", "futures-channel", @@ -2834,21 +3090,21 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fc3cb4d91f53b50155bdcfd23f6a4c39ae1969c2ae85982b135750cccaf5fce" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "libc", "wasi 0.9.0+wasi-snapshot-preview1", ] [[package]] name = "getrandom" -version = "0.2.6" +version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9be70c98951c83b8d2f8f60d7065fa6d5146873094452a1008da8c2f1e4205ad" +checksum = "c05aeb6a22b8f62540c194aac980f2115af067bfe15a0734d7277a768d396b31" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "js-sys", "libc", - "wasi 0.10.2+wasi-snapshot-preview1", + "wasi 0.11.0+wasi-snapshot-preview1", "wasm-bindgen", ] @@ -2869,6 +3125,19 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b919933a397b79c37e33b77bb2aa3dc8eb6e165ad809e58ff75bc7db2e34574" +[[package]] +name = "gloo-utils" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8e8fc851e9c7b9852508bc6e3f690f452f474417e8545ec9857b7f7377036b5" +dependencies = [ + "js-sys", + "serde", + "serde_json", + "wasm-bindgen", + "web-sys", +] + [[package]] name = "goauth" version = "0.13.1" @@ -2876,7 +3145,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f8af59a261bcf42f45d1b261232847b9b850ba0a1419d6100698246fb66e9240" dependencies = [ "arc-swap", - "futures 0.3.24", + "futures 0.3.25", "log", "reqwest", "serde", @@ -2890,12 +3159,13 @@ dependencies = [ [[package]] name = "governor" -version = "0.4.2" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19775995ee20209163239355bc3ad2f33f83da35d9ef72dea26e5af753552c87" +checksum = "c390a940a5d157878dd057c78680a33ce3415bcd05b4799509ea44210914b4d5" dependencies = [ + "cfg-if", "dashmap", - "futures 0.3.24", + "futures 0.3.25", "futures-timer", "no-std-compat", "nonzero_ext", @@ -2977,7 +3247,7 @@ name = "h2" version = "0.3.13" source = "git+https://github.com/hyperium/h2.git?rev=f6aa3be6719270cd7b4094ee1940751b5f4ec88e#f6aa3be6719270cd7b4094ee1940751b5f4ec88e" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "fnv", "futures-core", "futures-sink", @@ -2986,8 +3256,8 @@ dependencies = [ "indexmap", "slab", "tokio", - "tokio-util 0.7.1", - "tracing 0.1.34", + "tokio-util", + "tracing 0.1.37", ] [[package]] @@ -3004,18 +3274,18 @@ checksum = "74721d007512d0cb3338cd20f0654ac913920061a4c4d0d8708edb3f2a698c0c" [[package]] name = "hashbrown" -version = "0.12.0" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c21d40587b92fa6a6c6e3c1bdbf87d75511db5672f9c93175574b3a00df1758" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" dependencies = [ - "ahash", + "ahash 0.7.6", ] [[package]] name = "hdrhistogram" -version = "7.5.1" +version = "7.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ea9fe3952d32674a14e0975009a3547af9ea364995b5ec1add2e23c2ae523ab" +checksum = "7f19b9f54f7c7f55e31401bb647626ce0cf0f67b0004982ce815b3ee72a02aa8" dependencies = [ "base64", "byteorder", @@ -3033,7 +3303,7 @@ checksum = "f3e372db8e5c0d213e0cd0b9be18be2aca3d44cf2fe30a9d46a65581cd454584" dependencies = [ "base64", "bitflags", - "bytes 1.2.1", + "bytes 1.3.0", "headers-core", "http", "httpdate", @@ -3084,7 +3354,7 @@ name = "heim-common" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "core-foundation", "futures-core", "futures-util", @@ -3094,7 +3364,7 @@ dependencies = [ "nix 0.23.1", "pin-utils", "uom", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -3102,8 +3372,8 @@ name = "heim-cpu" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if 1.0.0", - "futures 0.3.24", + "cfg-if", + "futures 0.3.25", "glob", "heim-common", "heim-runtime", @@ -3112,7 +3382,7 @@ dependencies = [ "mach", "ntapi", "smol", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -3121,14 +3391,14 @@ version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ "bitflags", - "cfg-if 1.0.0", + "cfg-if", "core-foundation", "heim-common", "heim-runtime", "libc", "mach", "widestring 0.4.3", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -3136,7 +3406,7 @@ name = "heim-host" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "heim-common", "heim-runtime", "lazy_static", @@ -3145,7 +3415,7 @@ dependencies = [ "mach", "ntapi", "platforms", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -3153,13 +3423,13 @@ name = "heim-memory" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "heim-common", "heim-runtime", "lazy_static", "libc", "mach", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -3168,14 +3438,14 @@ version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ "bitflags", - "cfg-if 1.0.0", + "cfg-if", "heim-common", "heim-runtime", "libc", "macaddr", "nix 0.23.1", "widestring 0.4.3", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -3183,7 +3453,7 @@ name = "heim-runtime" version = "0.1.0-rc.1" source = "git+https://github.com/vectordotdev/heim.git?branch=update-nix#76fa765c7ed7fbe43d1465bf52da6b8d19f2d2a9" dependencies = [ - "futures 0.3.24", + "futures 0.3.25", "futures-timer", "once_cell", "smol", @@ -3198,6 +3468,15 @@ dependencies = [ "libc", ] +[[package]] +name = "hermit-abi" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee512640fe35acbfb4bb779db6f0d80704c2cacfa2e39b601ef3e3f47d1ae4c7" +dependencies = [ + "libc", +] + [[package]] name = "hex" version = "0.4.3" @@ -3210,7 +3489,7 @@ version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e" dependencies = [ - "digest 0.10.3", + "digest 0.10.6", ] [[package]] @@ -3221,7 +3500,7 @@ checksum = "3c731c3e10504cc8ed35cfe2f1db4c9274c3d35fa486e3b31df46f068ef3e867" dependencies = [ "libc", "match_cfg", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -3230,7 +3509,7 @@ version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "75f43d41e26995c17e71ee126451dd3941010b0514a81a9d11f3b341debc2399" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "fnv", "itoa 1.0.1", ] @@ -3241,7 +3520,7 @@ version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d5f38f16d184e36f2408a55281cd658ecbd3ca05cce6d6510a176eca393e26d1" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "http", "pin-project-lite", ] @@ -3275,9 +3554,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.7.1" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "496ce29bb5a52785b44e0f7ca2847ae0bb839c9bd28f69acac9b99d461c0c04c" +checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" [[package]] name = "httpdate" @@ -3293,11 +3572,11 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.20" +version = "0.14.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02c929dc5c39e335a03c405292728118860721b10190d98c2a0f0efd5baafbac" +checksum = "034711faac9d2166cb1baf1a2fb0b60b1f277f8492fd72176c17f3515e1abd3c" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "futures-channel", "futures-core", "futures-util", @@ -3311,7 +3590,7 @@ dependencies = [ "socket2", "tokio", "tower-service", - "tracing 0.1.34", + "tracing 0.1.37", "want", ] @@ -3339,8 +3618,8 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ca815a891b24fdfb243fa3239c86154392b0953ee584aa1a2a1f66d20cbe75cc" dependencies = [ - "bytes 1.2.1", - "futures 0.3.24", + "bytes 1.3.0", + "futures 0.3.25", "headers", "http", "hyper", @@ -3350,23 +3629,6 @@ dependencies = [ "tower-service", ] -[[package]] -name = "hyper-rustls" -version = "0.22.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f9f7a97316d44c0af9b0301e65010573a853a9fc97046d7331d7f6bc0fd5a64" -dependencies = [ - "ct-logs", - "futures-util", - "hyper", - "log", - "rustls 0.19.1", - "rustls-native-certs 0.5.0", - "tokio", - "tokio-rustls 0.22.0", - "webpki 0.21.4", -] - [[package]] name = "hyper-rustls" version = "0.23.0" @@ -3379,7 +3641,7 @@ dependencies = [ "rustls 0.20.4", "rustls-native-certs 0.6.2", "tokio", - "tokio-rustls 0.23.3", + "tokio-rustls", ] [[package]] @@ -3400,7 +3662,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d6183ddfa99b85da61a140bea0efc93fdf56ceaa041b37d553518030827f9905" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "hyper", "native-tls", "tokio", @@ -3420,6 +3682,30 @@ dependencies = [ "tokio", ] +[[package]] +name = "iana-time-zone" +version = "0.1.53" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64c122667b287044802d6ce17ee2ddf13207ed924c712de9a66a5814d5b64765" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "wasm-bindgen", + "winapi", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" +dependencies = [ + "cxx", + "cxx-build", +] + [[package]] name = "ident_case" version = "1.0.1" @@ -3437,11 +3723,21 @@ dependencies = [ "unicode-normalization", ] +[[package]] +name = "idna" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e14ddfc70884202db2244c223200c204c2bda1bc6e0998d11b5e024d657209e6" +dependencies = [ + "unicode-bidi", + "unicode-normalization", +] + [[package]] name = "indexmap" -version = "1.9.1" +version = "1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10a35a97730320ffe8e2d410b5d3b69279b98d2c14bdb8b70ea89ecf7888d41e" +checksum = "1885e79c1fc4b10f0e172c475f458b7f7b93061064d98c3293e98c5ba0c8b399" dependencies = [ "autocfg", "hashbrown", @@ -3462,9 +3758,9 @@ checksum = "64e9829a50b42bb782c1df523f78d332fe371b10c661e78b7a3c34b0198e9fac" [[package]] name = "infer" -version = "0.9.0" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f178e61cdbfe084aa75a2f4f7a25a5bb09701a47ae1753608f194b15783c937a" +checksum = "0a6c16b11a665b26aeeb9b1d7f954cdeb034be38dd00adab4f2ae921a8fee804" [[package]] name = "inherent" @@ -3479,9 +3775,9 @@ dependencies = [ [[package]] name = "inotify" -version = "0.7.1" +version = "0.9.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4816c66d2c8ae673df83366c18341538f234a26d65a9ecea5c348b453ac1d02f" +checksum = "f8069d3ec154eb856955c1c0fbffefbf5f3c40a104ec912d4797314c1801abff" dependencies = [ "bitflags", "inotify-sys", @@ -3513,14 +3809,14 @@ version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", ] [[package]] name = "inventory" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30a61b8101d87996f82d725ba701b1987b7afc72f481c13513a30b855b9c9133" +checksum = "e21e0a36a4dc4b469422ee17f715e8313f4a637675656d6a13637954278c6f55" dependencies = [ "ctor", "ghost", @@ -3532,6 +3828,16 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9448015e586b611e5d322f6703812bbca2f1e709d5773ecd38ddb4e3bb649504" +[[package]] +name = "io-lifetimes" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46112a93252b123d31a119a8d1a1ac19deac4fac6e0e8b0df58f0d4e5870e63c" +dependencies = [ + "libc", + "windows-sys 0.42.0", +] + [[package]] name = "iovec" version = "0.1.4" @@ -3549,7 +3855,7 @@ checksum = "723519edce41262b05d4143ceb95050e4c614f483e78e9fd9e39a8275a84ad98" dependencies = [ "socket2", "widestring 0.5.1", - "winapi 0.3.9", + "winapi", "winreg 0.7.0", ] @@ -3569,19 +3875,22 @@ dependencies = [ ] [[package]] -name = "itertools" -version = "0.9.0" +name = "is-terminal" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "284f18f85651fe11e8a991b2adb42cb078325c996ed026d994719efcfca1d54b" +checksum = "927609f78c2913a6f6ac3c27a4fe87f43e2a35367c0c4b0f8265e8f49a104330" dependencies = [ - "either", + "hermit-abi 0.2.6", + "io-lifetimes 1.0.3", + "rustix 0.36.5", + "windows-sys 0.42.0", ] [[package]] name = "itertools" -version = "0.10.3" +version = "0.10.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9a9d19fa1e79b6215ff29b9d6880b706147f16e9b1dbb1e4e5947b5b02bc5e3" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" dependencies = [ "either", ] @@ -3600,9 +3909,9 @@ checksum = "1aab8fc367588b89dcee83ab0fd66b72b50b72fa1904d7095045ace2b0c81c35" [[package]] name = "jni" -version = "0.19.0" +version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6df18c2e3db7e453d3c6ac5b3e9d5182664d28788126d39b91f2d1e22b017ec" +checksum = "039022cdf4d7b1cf548d31f60ae783138e5fd42013f6271049d7df7afadef96c" dependencies = [ "cesu8", "combine 4.6.4", @@ -3668,8 +3977,8 @@ dependencies = [ name = "k8s-e2e-tests" version = "0.1.0" dependencies = [ - "env_logger 0.9.0", - "futures 0.3.24", + "env_logger 0.10.0", + "futures 0.3.25", "indoc", "k8s-openapi", "k8s-test-framework", @@ -3678,22 +3987,22 @@ dependencies = [ "reqwest", "serde_json", "tokio", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "k8s-openapi" -version = "0.15.0" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2ae2c04fcee6b01b04e3aadd56bb418932c8e0a9d8a93f48bc68c6bdcdb559d" +checksum = "6d9455388f4977de4d0934efa9f7d36296295537d774574113a20f6082de03da" dependencies = [ "base64", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "http", "percent-encoding", "serde", - "serde-value 0.7.0", + "serde-value", "serde_json", "url", ] @@ -3716,13 +4025,23 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "67c21572b4949434e4fc1e1978b99c5f77064153c59d998bf13ecd96fb5ecba7" [[package]] -name = "kernel32-sys" -version = "0.2.2" +name = "kqueue" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c8fc60ba15bf51257aa9807a48a61013db043fcf3a78cb0d916e8e396dcad98" +dependencies = [ + "kqueue-sys", + "libc", +] + +[[package]] +name = "kqueue-sys" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7507624b29483431c0ba2d82aece8ca6cdba9382bff4ddd0f7490560c056098d" +checksum = "8367585489f01bc55dd27404dcf56b95e6da061a256a666ab23be9ba96a2e587" dependencies = [ - "winapi 0.2.8", - "winapi-build", + "bitflags", + "libc", ] [[package]] @@ -3736,9 +4055,9 @@ dependencies = [ [[package]] name = "kube" -version = "0.73.1" +version = "0.75.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f68b954ea9ad888de953fb1488bd8f377c4c78d82d4642efa5925189210b50b7" +checksum = "9bb19108692aeafebb108fd0a1c381c06ac4c03859652599420975165e939b8a" dependencies = [ "k8s-openapi", "kube-client", @@ -3748,16 +4067,16 @@ dependencies = [ [[package]] name = "kube-client" -version = "0.73.1" +version = "0.75.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9150dc7107d9acf4986088f284a0a6dddc5ae37ef1ffdf142f6811dc5998dd58" +checksum = "97e1a80ecd1b1438a2fc004549e155d47250b9e01fbfcf4cfbe9c8b56a085593" dependencies = [ "base64", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "dirs-next", "either", - "futures 0.3.24", + "futures 0.3.25", "http", "http-body", "hyper", @@ -3776,17 +4095,17 @@ dependencies = [ "thiserror", "tokio", "tokio-native-tls", - "tokio-util 0.7.1", + "tokio-util", "tower", "tower-http", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] name = "kube-core" -version = "0.73.1" +version = "0.75.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc8c429676abe6a73b374438d5ca02caaf9ae7a635441253c589b779fa5d0622" +checksum = "f4d780f2bb048eeef64a4c6b2582d26a0fe19e30b4d3cc9e081616e1779c5d47" dependencies = [ "chrono", "form_urlencoded", @@ -3801,14 +4120,14 @@ dependencies = [ [[package]] name = "kube-runtime" -version = "0.73.1" +version = "0.75.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6e9e9da456f0101b77f864a9da44866b9891ad4740db508b4b269343ebeb01d" +checksum = "7769af142ee2e46bfa44bd393cf7f40b9d8b80d2e11f6317399551ed17760beb" dependencies = [ - "ahash", + "ahash 0.8.2", "backoff", "derivative", - "futures 0.3.24", + "futures 0.3.25", "json-patch", "k8s-openapi", "kube-client", @@ -3819,8 +4138,8 @@ dependencies = [ "smallvec", "thiserror", "tokio", - "tokio-util 0.7.1", - "tracing 0.1.34", + "tokio-util", + "tracing 0.1.37", ] [[package]] @@ -3834,7 +4153,7 @@ dependencies = [ "bit-set", "diff", "ena", - "itertools 0.10.3", + "itertools", "lalrpop-util", "petgraph", "pico-args", @@ -3856,16 +4175,32 @@ dependencies = [ ] [[package]] -name = "lazy_static" -version = "1.4.0" +name = "lapin" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +checksum = "bd03ea5831b44775e296239a64851e2fd14a80a363d202ba147009ffc994ff0f" +dependencies = [ + "amq-protocol", + "async-global-executor-trait", + "async-reactor-trait", + "async-trait", + "executor-trait", + "flume", + "futures-core", + "futures-io", + "parking_lot", + "pinky-swear", + "reactor-trait", + "serde", + "tracing 0.1.37", + "waker-fn", +] [[package]] -name = "lazycell" -version = "1.3.0" +name = "lazy_static" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "leveldb" @@ -3891,9 +4226,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.132" +version = "0.2.138" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8371e4e5341c3a96db127eb2465ac681ced4c433e01dd0e938adbef26ba93ba5" +checksum = "db6d7e329c562c5dfab7a46a2afabc8b987ab9a4834c9d1ca04dc54c1546cef8" [[package]] name = "libflate" @@ -3933,6 +4268,15 @@ dependencies = [ "vcpkg", ] +[[package]] +name = "link-cplusplus" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9272ab7b96c9046fbc5bc56c06c117cb639fe2d509df0c421cad82d2915cf369" +dependencies = [ + "cc", +] + [[package]] name = "linked-hash-map" version = "0.5.4" @@ -3954,6 +4298,12 @@ version = "0.0.42" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5284f00d480e1c39af34e72f8ad60b94f47007e3481cd3b731c1d67190ddc7b7" +[[package]] +name = "linux-raw-sys" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f051f77a7c8e6957c0696eac88f26b0117e54f52d3fc682ab19397a8812846a4" + [[package]] name = "listenfd" version = "1.0.0" @@ -3961,8 +4311,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "14e4fcc00ff6731d94b70e16e71f43bda62883461f31230742e3bc6dddf12988" dependencies = [ "libc", - "uuid 1.1.2", - "winapi 0.3.9", + "uuid 1.2.2", + "winapi", ] [[package]] @@ -3987,7 +4337,7 @@ version = "0.4.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "abb12e687cfb44aa40f41fc3978ef76448f9b6038cad6aef4259d3c095a2382e" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", ] [[package]] @@ -3996,6 +4346,18 @@ version = "0.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "879777f0cc6f3646a044de60e4ab98c75617e3f9580f7a2032e6ad7ea0cd3054" +[[package]] +name = "loki-logproto" +version = "0.1.0" +dependencies = [ + "bytes 1.3.0", + "chrono", + "prost 0.10.4", + "prost-build 0.10.4", + "prost-types 0.10.1", + "snap", +] + [[package]] name = "lookup" version = "0.1.0" @@ -4010,16 +4372,16 @@ dependencies = [ "serde", "serde_json", "snafu", - "tracing 0.1.34", + "tracing 0.1.37", "vector-config", "vector-config-macros", ] [[package]] name = "lru" -version = "0.7.8" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e999beba7b6e8345721bd280141ed958096a2e4abdf74f67ff4ce49b4b54e47a" +checksum = "b6e8aaa3f231bb4bd57b84b2d5dc3ae7f350265df8aa96492e0bc394a1571909" [[package]] name = "lru-cache" @@ -4048,6 +4410,26 @@ dependencies = [ "cc", ] +[[package]] +name = "lz4" +version = "1.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" +dependencies = [ + "libc", + "lz4-sys", +] + +[[package]] +name = "lz4-sys" +version = "1.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" +dependencies = [ + "cc", + "libc", +] + [[package]] name = "macaddr" version = "1.0.1" @@ -4063,6 +4445,15 @@ dependencies = [ "libc", ] +[[package]] +name = "malloc_buf" +version = "0.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62bb907fe88d54d8d9ce32a3cceab4218ed2f6b7d35617cafe9adf84e43919cb" +dependencies = [ + "libc", +] + [[package]] name = "match_cfg" version = "0.1.0" @@ -4126,7 +4517,7 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "66b48670c893079d3c2ed79114e3644b7004df1c361a4e0ad52e2e6940d07c3d" dependencies = [ - "digest 0.10.3", + "digest 0.10.6", ] [[package]] @@ -4143,9 +4534,9 @@ checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" [[package]] name = "memmap2" -version = "0.5.7" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95af15f345b17af2efc8ead6080fb8bc376f8cec1b35277b935637595fe77498" +checksum = "4b182332558b18d807c4ce1ca8ca983b34c3ee32765e47b3f0f69b90355cc1dc" dependencies = [ "libc", ] @@ -4165,7 +4556,7 @@ version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b9b8653cec6897f73b519a43fba5ee3d50f62fe9af80b428accdcc093b4a849" dependencies = [ - "ahash", + "ahash 0.7.6", "metrics-macros", "portable-atomic", ] @@ -4192,9 +4583,9 @@ dependencies = [ "metrics", "metrics-util", "once_cell", - "tracing 0.1.34", - "tracing-core 0.1.28", - "tracing-subscriber 0.3.15", + "tracing 0.1.37", + "tracing-core 0.1.30", + "tracing-subscriber 0.3.16", ] [[package]] @@ -4235,93 +4626,39 @@ dependencies = [ ] [[package]] -name = "minimal-lexical" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" - -[[package]] -name = "miniz_oxide" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b29bd4bc3f33391105ebee3589c19197c4271e3e5a9ec9bfe8127eeff8f082" -dependencies = [ - "adler", -] - -[[package]] -name = "mio" -version = "0.6.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4afd66f5b91bf2a3bc13fad0e21caedac168ca4c707504e75585648ae80e4cc4" -dependencies = [ - "cfg-if 0.1.10", - "fuchsia-zircon", - "fuchsia-zircon-sys", - "iovec", - "kernel32-sys", - "libc", - "log", - "miow 0.2.2", - "net2", - "slab", - "winapi 0.2.8", -] - -[[package]] -name = "mio" -version = "0.8.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52da4364ffb0e4fe33a9841a98a3f3014fb964045ce4f7a45a398243c8d6b0c9" -dependencies = [ - "libc", - "log", - "miow 0.3.7", - "ntapi", - "wasi 0.11.0+wasi-snapshot-preview1", - "winapi 0.3.9", -] - -[[package]] -name = "mio-extras" -version = "2.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52403fe290012ce777c4626790c8951324a2b9e3316b3143779c72b029742f19" -dependencies = [ - "lazycell", - "log", - "mio 0.6.23", - "slab", -] - -[[package]] -name = "miow" -version = "0.2.2" +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ebd808424166322d4a38da87083bfddd3ac4c131334ed55856112eb06d46944d" +checksum = "b275950c28b37e794e8c55d88aeb5e139d0ce23fdbbeda68f8d7174abdf9e8fa" dependencies = [ - "kernel32-sys", - "net2", - "winapi 0.2.8", - "ws2_32-sys", + "adler", ] [[package]] -name = "miow" -version = "0.3.7" +name = "mio" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f1c5b025cda876f66ef43a113f91ebc9f4ccef34843000e0adf6ebbab84e21" +checksum = "e5d732bc30207a6423068df043e3d02e0735b155ad7ce1a6f76fe2baa5b158de" dependencies = [ - "winapi 0.3.9", + "libc", + "log", + "wasi 0.11.0+wasi-snapshot-preview1", + "windows-sys 0.42.0", ] [[package]] name = "mlua" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10277581090f5cb7ecf814bc611152ce4db6dc8deffcaa08e24ed4c5197d9186" +checksum = "4351dbcc863fb6249c81b3bd0c8001214e9d4d44d22cabda17026353a77fe612" dependencies = [ - "bstr", + "bstr 0.2.17", "cc", "lua-src", "luajit-src", @@ -4339,9 +4676,9 @@ checksum = "717e29a243b81f8130e31e24e04fb151b04a44b5a7d05370935f7d937e9de06d" [[package]] name = "mongodb" -version = "2.3.0" +version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b95afe97b0c799fdf69cd960272a2cb9662d077bd6efd84eb722bb9805d47554" +checksum = "b5a1df476ac9541b0e4fdc8e2cc48884e66c92c933cd17a1fd75e68caf75752e" dependencies = [ "async-trait", "base64", @@ -4366,17 +4703,17 @@ dependencies = [ "serde", "serde_bytes", "serde_with 1.14.0", - "sha-1 0.10.0", - "sha2 0.10.5", + "sha-1", + "sha2 0.10.6", "socket2", "stringprep", "strsim 0.10.0", "take_mut", "thiserror", "tokio", - "tokio-rustls 0.23.3", - "tokio-util 0.7.1", - "trust-dns-proto", + "tokio-rustls", + "tokio-util", + "trust-dns-proto 0.21.2", "trust-dns-resolver", "typed-builder 0.10.0", "uuid 0.8.2", @@ -4389,7 +4726,7 @@ version = "2.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f8f35e687561d5c1667590911e6698a8cb714a134a7505718a182e7bc9d3836" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "encoding_rs", "futures-util", "http", @@ -4427,9 +4764,9 @@ dependencies = [ [[package]] name = "nats" -version = "0.23.0" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d61e51453596562c82656b4fb004071b5b0c214183be3d02fbb7b16ba2370df" +checksum = "b3d877cd2e71146efa7065300fc5f5da967f938694b4d65e8bc64cc4a409092c" dependencies = [ "base64", "base64-url", @@ -4447,6 +4784,7 @@ dependencies = [ "once_cell", "parking_lot", "regex", + "ring", "rustls 0.19.1", "rustls-native-certs 0.5.0", "rustls-pemfile 0.2.1", @@ -4457,7 +4795,7 @@ dependencies = [ "time", "url", "webpki 0.21.4", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -4480,7 +4818,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af5a8477ac96877b5bd1fd67e0c28736c12943aba24eda92b127e036b0c8f400" dependencies = [ "indexmap", - "itertools 0.10.3", + "itertools", "ndarray", "noisy_float", "num-integer", @@ -4488,73 +4826,12 @@ dependencies = [ "rand 0.8.5", ] -[[package]] -name = "ndk" -version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2032c77e030ddee34a6787a64166008da93f6a352b629261d0fee232b8742dd4" -dependencies = [ - "bitflags", - "jni-sys", - "ndk-sys", - "num_enum", - "thiserror", -] - [[package]] name = "ndk-context" version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "27b02d87554356db9e9a873add8782d4ea6e3e58ea071a9adb9a2e8ddb884a8b" -[[package]] -name = "ndk-glue" -version = "0.6.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d0c4a7b83860226e6b4183edac21851f05d5a51756e97a1144b7f5a6b63e65f" -dependencies = [ - "lazy_static", - "libc", - "log", - "ndk", - "ndk-context", - "ndk-macro", - "ndk-sys", -] - -[[package]] -name = "ndk-macro" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0df7ac00c4672f9d5aece54ee3347520b7e20f158656c7db2e6de01902eb7a6c" -dependencies = [ - "darling 0.13.4", - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "ndk-sys" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e5a6ae77c8ee183dcbbba6150e2e6b9f3f4196a7666c02a715a95692ec1fa97" -dependencies = [ - "jni-sys", -] - -[[package]] -name = "net2" -version = "0.2.37" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "391630d12b68002ae1e25e8f974306474966550ad82dac6886fb8910c19568ae" -dependencies = [ - "cfg-if 0.1.10", - "libc", - "winapi 0.3.9", -] - [[package]] name = "new_debug_unreachable" version = "1.0.4" @@ -4578,7 +4855,7 @@ checksum = "9f866317acbd3a240710c63f065ffb1e4fd466259045ccb504130b7f668f35c6" dependencies = [ "bitflags", "cc", - "cfg-if 1.0.0", + "cfg-if", "libc", "memoffset", ] @@ -4590,7 +4867,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "195cdbc1741b8134346d515b3a56a1c94b0912758009cfd53f99ea0f57b065fc" dependencies = [ "bitflags", - "cfg-if 1.0.0", + "cfg-if", "libc", ] @@ -4602,7 +4879,7 @@ checksum = "e322c04a9e3440c327fca7b6c8a63e6890a32fa2ad689db972425f07e0d22abb" dependencies = [ "autocfg", "bitflags", - "cfg-if 1.0.0", + "cfg-if", "libc", "memoffset", ] @@ -4616,7 +4893,7 @@ dependencies = [ "byteorder", "data-encoding", "ed25519-dalek", - "getrandom 0.2.6", + "getrandom 0.2.8", "log", "rand 0.8.5", "signatory", @@ -4671,20 +4948,19 @@ checksum = "38bf9645c8b145698bb0b18a4637dcacbc421ea49bef2317e4fd8065a387cf21" [[package]] name = "notify" -version = "4.0.17" +version = "5.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae03c8c853dba7bfd23e571ff0cff7bc9dceb40a4cd684cd1681824183f45257" +checksum = "ed2c66da08abae1c024c01d635253e402341b4060a12e99b31c7594063bf490a" dependencies = [ "bitflags", "filetime", - "fsevent", "fsevent-sys", "inotify", + "kqueue", "libc", - "mio 0.6.23", - "mio-extras", + "mio", "walkdir", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -4693,7 +4969,17 @@ version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28774a7fd2fbb4f0babd8237ce554b73af68021b5f695a3cebd6c59bac0980f" dependencies = [ - "winapi 0.3.9", + "winapi", +] + +[[package]] +name = "nu-ansi-term" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +dependencies = [ + "overload", + "winapi", ] [[package]] @@ -4785,7 +5071,7 @@ version = "1.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "19e64526ebdee182341572e50e9ad03965aa510cd94427a4549448f285e957a1" dependencies = [ - "hermit-abi", + "hermit-abi 0.1.19", "libc", ] @@ -4827,24 +5113,33 @@ checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" [[package]] name = "oauth2" -version = "4.1.0" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80e47cfc4c0a1a519d9a025ebfbac3a2439d1b5cdf397d72dcb79b11d9920dab" +checksum = "eeaf26a72311c087f8c5ba617c96fac67a5c04f430e716ac8d8ab2de62e23368" dependencies = [ "base64", "chrono", - "getrandom 0.2.6", + "getrandom 0.2.8", "http", "rand 0.8.5", "reqwest", "serde", "serde_json", "serde_path_to_error", - "sha2 0.9.9", + "sha2 0.10.6", "thiserror", "url", ] +[[package]] +name = "objc" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "915b1b472bc21c53464d6c8461c9d3af805ba1ef837e1cac254428f4a77177b1" +dependencies = [ + "malloc_buf", +] + [[package]] name = "ofb" version = "0.6.1" @@ -4856,9 +5151,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.14.0" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f7254b99e31cad77da24b08ebf628882739a608578bb1bcdfc1f9c21260d7c0" +checksum = "86f0b0d4bf799edbc74508c1e8bf170ff5f41238e5f8225603ca7caaae2b7860" [[package]] name = "onig" @@ -4896,37 +5191,36 @@ checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" [[package]] name = "openidconnect" -version = "2.3.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "691c1ba89b0a112f3062b946ef160711c3aea33e1476b6877a904f2f83856781" +checksum = "87af7097640fedbe64718ac1c9b0549d72da747a3f527cd089215f96c6f691d5" dependencies = [ "base64", "chrono", "http", - "itertools 0.9.0", + "itertools", "log", "num-bigint 0.4.3", "oauth2", "rand 0.8.5", "ring", "serde", - "serde-value 0.6.0", + "serde-value", "serde_derive", "serde_json", "serde_path_to_error", "thiserror", - "untrusted", "url", ] [[package]] name = "openssl" -version = "0.10.41" +version = "0.10.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "618febf65336490dfcf20b73f885f5651a0c89c64c2d4a8c3662585a70bf5bd0" +checksum = "29d971fd5722fec23977260f6e81aa67d2f22cadbdc2aa049f1022d9a3be1566" dependencies = [ "bitflags", - "cfg-if 1.0.0", + "cfg-if", "foreign-types", "libc", "once_cell", @@ -4961,9 +5255,9 @@ dependencies = [ [[package]] name = "openssl-sys" -version = "0.9.75" +version = "0.9.79" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5f9bd0c2710541a3cda73d6f9ac4f1b240de4ae261065d309dbe73d9dceb42f" +checksum = "5454462c0eced1e97f2ec09036abc8da362e66802f66fd20f86854d9d8cbcbc4" dependencies = [ "autocfg", "cc", @@ -4977,27 +5271,18 @@ dependencies = [ name = "opentelemetry-proto" version = "0.0.0" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "hex", - "ordered-float 3.0.0", - "prost 0.10.4", - "prost-build", - "tonic 0.7.2", + "ordered-float 3.4.0", + "prost 0.11.3", + "prost-build 0.11.4", + "tonic", "tonic-build", "value", "vector-core", ] -[[package]] -name = "ordered-float" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" -dependencies = [ - "num-traits", -] - [[package]] name = "ordered-float" version = "2.10.0" @@ -5009,9 +5294,9 @@ dependencies = [ [[package]] name = "ordered-float" -version = "3.0.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96bcbab4bfea7a59c2c0fe47211a1ac4e3e96bea6eb446d704f310bc5c732ae2" +checksum = "d84eb1409416d254e4a9c8fa56cc24701755025b458f0fcd8e59e1f5f40c23bf" dependencies = [ "num-traits", ] @@ -5023,7 +5308,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "023df84d545ef479cf67fd2f4459a613585c9db4852c2fad12ab70587859d340" dependencies = [ "log", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -5033,7 +5318,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fb233f06c2307e1f5ce2ecad9f8121cffbbee2c95428f44ea85222e460d0d213" dependencies = [ "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -5043,13 +5328,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e22443d1643a904602595ba1cd8f7d896afe56d26712531c5ff73a15b2fbf64" [[package]] -name = "output_vt100" -version = "0.1.3" +name = "overload" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "628223faebab4e3e40667ee0b2336d34a5b960ff60ea743ddfdbcf7770bcfb66" -dependencies = [ - "winapi 0.3.9", -] +checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" [[package]] name = "parking" @@ -5073,7 +5355,7 @@ version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "995f667a6c822200b0433ac218e05582f0e2efa1b922a3fd2fbaadc5f87bab37" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "libc", "redox_syscall 0.2.13", "smallvec", @@ -5101,7 +5383,7 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271779f35b581956db91a3e55737327a03aa051e90b1c47aeb189508533adfd7" dependencies = [ - "digest 0.10.3", + "digest 0.10.6", ] [[package]] @@ -5112,9 +5394,9 @@ checksum = "9e9ed2178b0575fff8e1b83b58ba6f75e727aafac2e1b6c795169ad3b17eb518" [[package]] name = "pem" -version = "1.0.2" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9a3b09a20e374558580a4914d3b7d89bd61b954a5a5e1dcbea98753addb1947" +checksum = "03c64931a1a212348ec4f3b4362585eca7159d0d09cbdf4a7f74f02173596fd4" dependencies = [ "base64", ] @@ -5130,15 +5412,15 @@ dependencies = [ [[package]] name = "percent-encoding" -version = "2.1.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" +checksum = "478c572c3d73181ff3c2539045f6eb99e5491218eae919370993b890cdbdd98e" [[package]] name = "pest" -version = "2.3.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b0560d531d1febc25a3c9398a62a71256c0178f2e3443baedd9ad4bb8c9deb4" +checksum = "cc8bed3549e0f9b0a2a78bf7c0018237a2cdf085eecbbc048e52612438e4e9d0" dependencies = [ "thiserror", "ucd-trie", @@ -5146,9 +5428,9 @@ dependencies = [ [[package]] name = "pest_derive" -version = "2.3.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "905708f7f674518498c1f8d644481440f476d39ca6ecae83319bba7c6c12da91" +checksum = "cdc078600d06ff90d4ed238f0119d84ab5d43dbaad278b0e33a8820293b32344" dependencies = [ "pest", "pest_generator", @@ -5156,9 +5438,9 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.3.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5803d8284a629cc999094ecd630f55e91b561a1d1ba75e233b00ae13b91a69ad" +checksum = "28a1af60b1c4148bb269006a750cff8e2ea36aff34d2d96cf7be0b14d1bed23c" dependencies = [ "pest", "pest_meta", @@ -5169,13 +5451,13 @@ dependencies = [ [[package]] name = "pest_meta" -version = "2.3.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1538eb784f07615c6d9a8ab061089c6c54a344c5b4301db51990ca1c241e8c04" +checksum = "fec8605d59fc2ae0c6c1aefc0c7c7a9769732017c0ce07f7a9cfffa7b4404f20" dependencies = [ "once_cell", "pest", - "sha-1 0.10.0", + "sha1", ] [[package]] @@ -5233,7 +5515,6 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9dd5609d4b2df87167f908a32e1b146ce309c16cf35df76bc11f440b756048e4" dependencies = [ "siphasher", - "uncased", ] [[package]] @@ -5274,6 +5555,18 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pinky-swear" +version = "6.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d894b67aa7a4bf295db5e85349078c604edaa6fa5c8721e8eca3c7729a27f2ac" +dependencies = [ + "doc-comment", + "flume", + "parking_lot", + "tracing 0.1.37", +] + [[package]] name = "pkcs8" version = "0.7.6" @@ -5332,11 +5625,11 @@ version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "685404d509889fade3e86fe3a5803bca2ec09b0c0778d5ada6ec8bf7a8de5259" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "libc", "log", "wepoll-ffi", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -5358,7 +5651,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1de0ea6504e07ca78355a6fb88ad0f36cafe9e696cbc6717f16a207f3a60be72" dependencies = [ - "futures 0.3.24", + "futures 0.3.25", "openssl", "tokio", "tokio-openssl", @@ -5373,13 +5666,13 @@ checksum = "878c6cbf956e03af9aa8204b407b9cbf47c072164800aa918c516cd4b056c50c" dependencies = [ "base64", "byteorder", - "bytes 1.2.1", + "bytes 1.3.0", "fallible-iterator", "hmac", "md-5", "memchr", "rand 0.8.5", - "sha2 0.10.5", + "sha2 0.10.6", "stringprep", ] @@ -5389,7 +5682,7 @@ version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73d946ec7d256b04dfadc4e6a3292324e6f417124750fc5c0950f981b703a0f1" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "fallible-iterator", "postgres-protocol", @@ -5414,7 +5707,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a5aab5be6e4732b473071984b3164dbbfb7a3674d30ea5ff44410b6bcd960c3c" dependencies = [ "difflib", - "itertools 0.10.3", + "itertools", "predicates-core", ] @@ -5434,18 +5727,6 @@ dependencies = [ "termtree", ] -[[package]] -name = "pretty_assertions" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a25e9bcb20aa780fd0bb16b72403a9064d6b3f22f026946029acb941a50af755" -dependencies = [ - "ctor", - "diff", - "output_vt100", - "yansi", -] - [[package]] name = "prettydiff" version = "0.6.1" @@ -5528,11 +5809,17 @@ dependencies = [ "version_check", ] +[[package]] +name = "proc-macro-hack" +version = "0.5.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbf0c48bc1d91375ae5c3cd81e3722dff1abcf81a30960240640d223f59fe0e5" + [[package]] name = "proc-macro2" -version = "1.0.43" +version = "1.0.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a2ca2c61bc9f3d74d2886294ab7b9853abd9c1ad903a3ac7815c58989bb7bab" +checksum = "5ea3d908b0e36316caf9e9e2c4625cdde190a7e6f440d794667ed17a1855e725" dependencies = [ "unicode-ident", ] @@ -5544,9 +5831,9 @@ dependencies = [ "indexmap", "nom", "num_enum", - "prost 0.10.4", - "prost-build", - "prost-types 0.10.1", + "prost 0.11.3", + "prost-build 0.11.4", + "prost-types 0.11.2", "snafu", "value", "vector-common", @@ -5578,7 +5865,7 @@ version = "0.1.0" dependencies = [ "chrono", "lookup", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "proptest", "vrl-diagnostic", "vrl-parser", @@ -5590,17 +5877,17 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71adf41db68aa0daaefc69bb30bcd68ded9b9abaad5d1fbb6304c4fb390e083e" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "prost-derive 0.10.1", ] [[package]] name = "prost" -version = "0.11.0" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c3c31cdec40583bb68f0b18403400d01ec4289c383aa047560439952c4dd7" +checksum = "c0b18e655c21ff5ac2084a5ad0611e827b3f92badf79f4910b5a5c58f4d87ff0" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "prost-derive 0.11.0", ] @@ -5610,11 +5897,11 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8ae5a4388762d5815a9fc0dea33c56b021cdc8dde0c55e0c9ca57197254b0cab" dependencies = [ - "bytes 1.2.1", - "cfg-if 1.0.0", + "bytes 1.3.0", + "cfg-if", "cmake", "heck 0.4.0", - "itertools 0.10.3", + "itertools", "lazy_static", "log", "multimap", @@ -5626,6 +5913,28 @@ dependencies = [ "which", ] +[[package]] +name = "prost-build" +version = "0.11.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "276470f7f281b0ed53d2ae42dd52b4a8d08853a3c70e7fe95882acbb98a6ae94" +dependencies = [ + "bytes 1.3.0", + "heck 0.4.0", + "itertools", + "lazy_static", + "log", + "multimap", + "petgraph", + "prettyplease", + "prost 0.11.3", + "prost-types 0.11.2", + "regex", + "syn", + "tempfile", + "which", +] + [[package]] name = "prost-derive" version = "0.10.1" @@ -5633,7 +5942,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b670f45da57fb8542ebdbb6105a925fe571b67f9e7ed9f47a06a84e72b4e7cc" dependencies = [ "anyhow", - "itertools 0.10.3", + "itertools", "proc-macro2", "quote", "syn", @@ -5646,7 +5955,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7345d5f0e08c0536d7ac7229952590239e77abf0a0100a1b1d890add6ea96364" dependencies = [ "anyhow", - "itertools 0.10.3", + "itertools", "proc-macro2", "quote", "syn", @@ -5658,18 +5967,18 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d0a014229361011dc8e69c8a1ec6c2e8d0f2af7c91e3ea3f5b2170298461e68" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "prost 0.10.4", ] [[package]] name = "prost-types" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dfaa718ad76a44b3415e6c4d53b17c8f99160dcb3a99b10470fce8ad43f6e3e" +checksum = "747761bc3dc48f9a34553bf65605cf6cb6288ba219f3450b4275dbd81539551a" dependencies = [ - "bytes 1.2.1", - "prost 0.11.0", + "bytes 1.3.0", + "prost 0.11.3", ] [[package]] @@ -5694,36 +6003,41 @@ dependencies = [ [[package]] name = "pulsar" -version = "4.1.2" +version = "5.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31a5a4fbe9363c113b7e25ae76dd2d6455411c815d688977be6f0b68ae5e73b8" +checksum = "0efdf1fcdc925382b4a52e98a2d13f0a223116d9f458565b26cb44866b195953" dependencies = [ "async-trait", "bit-vec 0.6.3", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "crc", "data-url", - "futures 0.3.24", + "flate2", + "futures 0.3.25", "futures-io", "futures-timer", "log", + "lz4", "native-tls", "nom", "oauth2", "openidconnect", "pem", - "prost 0.10.4", - "prost-build", - "prost-derive 0.10.1", + "prost 0.11.3", + "prost-build 0.11.4", + "prost-derive 0.11.0", "rand 0.8.5", "regex", "serde", "serde_json", + "snap", "tokio", "tokio-native-tls", - "tokio-util 0.7.1", + "tokio-util", "url", + "uuid 1.2.2", + "zstd", ] [[package]] @@ -5739,7 +6053,7 @@ dependencies = [ "raw-cpuid", "wasi 0.10.2+wasi-snapshot-preview1", "web-sys", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -5765,6 +6079,17 @@ dependencies = [ "rand 0.8.5", ] +[[package]] +name = "quickcheck_macros" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b22a693222d716a9587786f37ac3f6b4faedb5b80c23914e7303ff5a1d8016e9" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "quote" version = "1.0.21" @@ -5774,6 +6099,12 @@ dependencies = [ "proc-macro2", ] +[[package]] +name = "quoted_printable" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20f14e071918cbeefc5edc986a7aa92c425dae244e003a35e1cdddb5ca39b5cb" + [[package]] name = "radix_trie" version = "0.2.1" @@ -5843,7 +6174,7 @@ version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d34f1408f55294453790c48b2f1ebbb1c5b4b7563eb1f418bcfcfdbb06ebb4e7" dependencies = [ - "getrandom 0.2.6", + "getrandom 0.2.8", ] [[package]] @@ -5883,6 +6214,15 @@ dependencies = [ "bitflags", ] +[[package]] +name = "raw-window-handle" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed7e3d950b66e19e0c372f3fa3fbbcf85b1746b571f74e0c2af6042a5c93420a" +dependencies = [ + "cty", +] + [[package]] name = "rawpointer" version = "0.2.1" @@ -5915,11 +6255,12 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.28.0" +version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1de127f294f2dba488ed46760b129d5ecbeabbd337ccbf3739cb29d50db2161c" +checksum = "bd7c5d6d17442bcb9f943aae96d67d98c6d36af60442dd5da62aaa7fcbb25c48" dependencies = [ - "futures 0.3.24", + "futures-channel", + "futures-util", "libc", "log", "rdkafka-sys", @@ -5932,9 +6273,9 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.2.0+1.8.2" +version = "4.3.0+1.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e542c6863b04ce0fa0c5719bc6b7b348cf8dd21af1bb03c9db5f9805b2a6473" +checksum = "d222a401698c7f2010e3967353eae566d9934dcda49c29910da922414ab4e3f4" dependencies = [ "cmake", "libc", @@ -5946,17 +6287,28 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "reactor-trait" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "438a4293e4d097556730f4711998189416232f009c137389e0f961d2bc0ddc58" +dependencies = [ + "async-trait", + "futures-core", + "futures-io", +] + [[package]] name = "redis" -version = "0.21.6" +version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "571c252c68d09a2ad3e49edd14e9ee48932f3e0f27b06b4ea4c9b2a706d31103" +checksum = "513b3649f1a111c17954296e4a3b9eecb108b766c803e2b99f179ebe27005985" dependencies = [ "arc-swap", "async-trait", - "bytes 1.2.1", + "bytes 1.3.0", "combine 4.6.4", - "futures 0.3.24", + "futures 0.3.25", "futures-util", "itoa 1.0.1", "native-tls", @@ -5965,7 +6317,7 @@ dependencies = [ "ryu", "tokio", "tokio-native-tls", - "tokio-util 0.7.1", + "tokio-util", "url", ] @@ -6001,16 +6353,16 @@ version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b033d837a7cf162d7993aded9304e30a83213c648b6e389db233191f891e5c2b" dependencies = [ - "getrandom 0.2.6", + "getrandom 0.2.8", "redox_syscall 0.2.13", "thiserror", ] [[package]] name = "regex" -version = "1.6.0" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c4eb3267174b8c6c2f654116623910a0fef09c4753f8dd83db29c48a0df988b" +checksum = "e076559ef8e241f2ae3479e36f97bd5741c0330689e217ad51ce2c76808b868a" dependencies = [ "aho-corasick", "memchr", @@ -6038,7 +6390,7 @@ version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3acd125665422973a33ac9d3dd2df85edad0f4ae9b00dafb1a05e43a9f5ef8e7" dependencies = [ - "winapi 0.3.9", + "winapi", ] [[package]] @@ -6052,12 +6404,12 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.11" +version = "0.11.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b75aa69a3f06bbcc66ede33af2af253c6f7a86b1ca0033f60c580a27074fbf92" +checksum = "68cc60575865c7831548863cc02356512e3f1dc2f3f82cb837d7fc4cc8f3c97c" dependencies = [ "base64", - "bytes 1.2.1", + "bytes 1.3.0", "encoding_rs", "futures-core", "futures-util", @@ -6065,14 +6417,14 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls 0.23.0", + "hyper-rustls", "hyper-tls", "ipnet", "js-sys", - "lazy_static", "log", "mime", "native-tls", + "once_cell", "percent-encoding", "pin-project-lite", "rustls 0.20.4", @@ -6082,8 +6434,8 @@ dependencies = [ "serde_urlencoded", "tokio", "tokio-native-tls", - "tokio-rustls 0.23.3", - "tokio-util 0.7.1", + "tokio-rustls", + "tokio-util", "tower-service", "url", "wasm-bindgen", @@ -6121,7 +6473,7 @@ dependencies = [ "spin 0.5.2", "untrusted", "web-sys", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -6168,9 +6520,9 @@ dependencies = [ [[package]] name = "rmp-serde" -version = "1.1.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25786b0d276110195fa3d6f3f31299900cf71dfbd6c28450f3f58a0e7f7a347e" +checksum = "c5b13be192e0220b8afb7222aa5813cb62cc269ebb5cac346ca6487681d2913e" dependencies = [ "byteorder", "rmp", @@ -6191,9 +6543,9 @@ dependencies = [ [[package]] name = "roaring" -version = "0.10.0" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48de5b5febb28599c6d9d34817d8ee8c74417af507b90dde0d481b28ca55ed6f" +checksum = "ef0fb5e826a8bde011ecae6a8539dd333884335c57ff0f003fbe27c25bbe8f71" dependencies = [ "bytemuck", "byteorder", @@ -6202,9 +6554,9 @@ dependencies = [ [[package]] name = "roxmltree" -version = "0.15.0" +version = "0.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "112908c3ac4711a1554b3948432ecaf5f061a951aa326977b63f7f72a86a4c0e" +checksum = "6b9de9831a129b122e7e61f242db509fa9d0838008bf0b29bb0624669edfe48a" dependencies = [ "xmlparser", ] @@ -6253,7 +6605,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" dependencies = [ - "semver 1.0.13", + "semver 1.0.14", ] [[package]] @@ -6274,10 +6626,24 @@ checksum = "3f5d1c6ed6d1c6915aa64749b809fc1bafff49d160f5d927463658093d7d62ab" dependencies = [ "bitflags", "errno", - "io-lifetimes", + "io-lifetimes 0.6.1", "libc", - "linux-raw-sys", - "winapi 0.3.9", + "linux-raw-sys 0.0.42", + "winapi", +] + +[[package]] +name = "rustix" +version = "0.36.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3807b5d10909833d3e9acd1eb5fb988f79376ff10fce42937de71a449c4c588" +dependencies = [ + "bitflags", + "errno", + "io-lifetimes 1.0.3", + "libc", + "linux-raw-sys 0.1.4", + "windows-sys 0.42.0", ] [[package]] @@ -6381,7 +6747,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d1cd5ae51d3f7bf65d7969d579d502168ef578f289452bd8ccc91de28fda20e" dependencies = [ "bitflags", - "cfg-if 1.0.0", + "cfg-if", "clipboard-win", "fd-lock", "libc", @@ -6392,7 +6758,7 @@ dependencies = [ "unicode-segmentation", "unicode-width", "utf8parse", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -6444,9 +6810,9 @@ dependencies = [ [[package]] name = "schemars" -version = "0.8.10" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1847b767a3d62d95cbf3d8a9f0e421cf57a0d8aa4f411d4b16525afb0284d4ed" +checksum = "2a5fb6c61f29e723026dc8e923d94c694313212abbecbbe5f55a7748eec5b307" dependencies = [ "dyn-clone", "indexmap", @@ -6457,9 +6823,9 @@ dependencies = [ [[package]] name = "schemars_derive" -version = "0.8.10" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af4d7e1b012cb3d9129567661a63755ea4b8a7386d339dc945ae187e403c6743" +checksum = "f188d036977451159430f3b8dc82ec76364a42b7e289c2b18a9a18f4470058e9" dependencies = [ "proc-macro2", "quote", @@ -6479,6 +6845,12 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" +[[package]] +name = "scratch" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8132065adcfd6e02db789d9285a0deb2f3fcb04002865ab67d5fb103533898" + [[package]] name = "sct" version = "0.6.1" @@ -6549,9 +6921,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.13" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93f6841e709003d68bb2deee8c343572bf446003ec20a583e76f7b15cebf3711" +checksum = "e25dfac463d778e353db5be2449d1cce89bd6fd23c9f1ea21310ce6e5a1b29c4" dependencies = [ "serde", ] @@ -6564,9 +6936,9 @@ checksum = "388a1df253eca08550bef6c72392cfe7c30914bf41df5269b68cbd6ff8f570a3" [[package]] name = "serde" -version = "1.0.144" +version = "1.0.150" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f747710de3dcd43b88c9168773254e809d8ddbdf9653b84e2554ab219f17860" +checksum = "e326c9ec8042f1b5da33252c8a37e9ffbd2c9bef0155215b6e6c80c790e05f91" dependencies = [ "serde_derive", ] @@ -6582,29 +6954,30 @@ dependencies = [ [[package]] name = "serde-value" -version = "0.6.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a65a7291a8a568adcae4c10a677ebcedbc6c9cec91c054dee2ce40b0e3290eb" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float 1.1.1", + "ordered-float 2.10.0", "serde", ] [[package]] -name = "serde-value" -version = "0.7.0" +name = "serde-wasm-bindgen" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +checksum = "e3b4c031cd0d9014307d82b8abf653c0290fbdaeb4c02d00c63cf52f728628bf" dependencies = [ - "ordered-float 2.10.0", + "js-sys", "serde", + "wasm-bindgen", ] [[package]] name = "serde-xml-rs" -version = "0.5.1" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65162e9059be2f6a3421ebbb4fef3e74b7d9e7c60c50a0e292c6239f19f1edfa" +checksum = "fb3aa78ecda1ebc9ec9847d5d3aba7d618823446a049ba2491940506da6e2782" dependencies = [ "log", "serde", @@ -6621,21 +6994,11 @@ dependencies = [ "serde", ] -[[package]] -name = "serde_cbor" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2bef2ebfde456fb76bbcf9f59315333decc4fda0b2b44b420243c11e0f5ec1f5" -dependencies = [ - "half", - "serde", -] - [[package]] name = "serde_derive" -version = "1.0.144" +version = "1.0.150" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94ed3a816fb1d101812f83e789f888322c34e291f894f19590dc310963e87a00" +checksum = "42a3df25b0713732468deadad63ab9da1f1fd75a48a15024b50363f128db627e" dependencies = [ "proc-macro2", "quote", @@ -6655,9 +7018,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.85" +version = "1.0.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e55a28e3aaef9d5ce0506d0a14dbba8054ddc7e499ef522dd8b26859ec9d4a44" +checksum = "020ff22c755c2ed3f8cf162dbb41a7268d934702f3ed3631656ea597e08fc3db" dependencies = [ "indexmap", "itoa 1.0.1", @@ -6729,9 +7092,9 @@ dependencies = [ [[package]] name = "serde_with" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89df7a26519371a3cce44fbb914c2819c84d9b897890987fa3ab096491cc0ea8" +checksum = "25bf4a5a814902cd1014dbccfa4d4560fb8432c779471e96e035602519f82eef" dependencies = [ "base64", "chrono", @@ -6739,7 +7102,7 @@ dependencies = [ "indexmap", "serde", "serde_json", - "serde_with_macros 2.0.0", + "serde_with_macros 2.1.0", "time", ] @@ -6757,9 +7120,9 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de337f322382fcdfbb21a014f7c224ee041a23785651db67b9827403178f698f" +checksum = "e3452b4c0f6c1e357f73fdb87cd1efabaa12acf328c7a528e252893baeb3f4aa" dependencies = [ "darling 0.14.1", "proc-macro2", @@ -6781,9 +7144,9 @@ dependencies = [ [[package]] name = "serde_yaml" -version = "0.9.11" +version = "0.9.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89f31df3f50926cdf2855da5fd8812295c34752cb20438dae42a67f79e021ac3" +checksum = "6d232d893b10de3eb7258ff01974d6ee20663d8e833263c99409d4b13a0209da" dependencies = [ "indexmap", "itoa 1.0.1", @@ -6792,28 +7155,15 @@ dependencies = [ "unsafe-libyaml", ] -[[package]] -name = "sha-1" -version = "0.9.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99cd6713db3cf16b6c84e06321e049a9b9f699826e16096d23bbcc44d15d51a6" -dependencies = [ - "block-buffer 0.9.0", - "cfg-if 1.0.0", - "cpufeatures", - "digest 0.9.0", - "opaque-debug", -] - [[package]] name = "sha-1" version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "028f48d513f9678cda28f6e4064755b3fbb2af6acd672f2c209b62323f7aea0f" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "cpufeatures", - "digest 0.10.3", + "digest 0.10.6", ] [[package]] @@ -6822,9 +7172,9 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c77f4e7f65455545c2153c1253d25056825e77ee2533f0e41deb65a93a34852f" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "cpufeatures", - "digest 0.10.3", + "digest 0.10.6", ] [[package]] @@ -6834,7 +7184,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4d58a1e1bf39749807d89cf2d98ac2dfa0ff1cb3faa38fbb64dd88ac8013d800" dependencies = [ "block-buffer 0.9.0", - "cfg-if 1.0.0", + "cfg-if", "cpufeatures", "digest 0.9.0", "opaque-debug", @@ -6842,13 +7192,13 @@ dependencies = [ [[package]] name = "sha2" -version = "0.10.5" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf9db03534dff993187064c4e0c05a5708d2a9728ace9a8959b77bedf415dac5" +checksum = "82e6b795fe2e3b1e845bafcb27aa35405c4d47cdfc92af5fc8d3002f76cebdc0" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "cpufeatures", - "digest 0.10.3", + "digest 0.10.6", ] [[package]] @@ -6857,7 +7207,7 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "881bf8156c87b6301fc5ca6b27f11eeb2761224c7081e69b409d5a1951a70c86" dependencies = [ - "digest 0.10.3", + "digest 0.10.6", "keccak", ] @@ -6877,7 +7227,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6be9f7d5565b1483af3e72975e2dee33879b3b86bd48c0929fccf6585d79e65a" dependencies = [ "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -6897,7 +7247,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "29ad2e15f37ec9a6cc544097b78a1ec90001e9f71b81338ca39f430adaca99af" dependencies = [ "libc", - "mio 0.8.2", + "mio", "signal-hook", ] @@ -6923,10 +7273,30 @@ dependencies = [ ] [[package]] -name = "signature" -version = "1.5.0" +name = "signature" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f054c6c1a6e95179d6f23ed974060dcefb2d9388bb7256900badad682c499de4" + +[[package]] +name = "similar" +version = "2.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "420acb44afdae038210c99e69aae24109f32f15500aa708e81d46c9f29d55fcf" +dependencies = [ + "bstr 0.2.17", + "unicode-segmentation", +] + +[[package]] +name = "similar-asserts" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f054c6c1a6e95179d6f23ed974060dcefb2d9388bb7256900badad682c499de4" +checksum = "bbf644ad016b75129f01a34a355dcb8d66a5bc803e417c7a77cc5d5ee9fa0f18" +dependencies = [ + "console", + "similar", +] [[package]] name = "simpl" @@ -6994,9 +7364,9 @@ dependencies = [ [[package]] name = "snafu" -version = "0.7.1" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5177903bf45656592d9eb5c0e22f408fc023aae51dbe2088889b71633ba451f2" +checksum = "a152ba99b054b22972ee794cf04e5ef572da1229e33b65f3c57abbff0525a454" dependencies = [ "doc-comment", "futures-core", @@ -7006,9 +7376,9 @@ dependencies = [ [[package]] name = "snafu-derive" -version = "0.7.1" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "410b26ed97440d90ced3e2488c868d56a86e2064f5d7d6f417909b286afe25e5" +checksum = "d5e79cdebbabaebb06a9bdbaedc7f159b410461f63611d4d0e3fb0fab8fed850" dependencies = [ "heck 0.4.0", "proc-macro2", @@ -7018,9 +7388,9 @@ dependencies = [ [[package]] name = "snap" -version = "1.0.5" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45456094d1983e2ee2a18fdfebce3189fa451699d0502cb8e3b49dba5ba41451" +checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" [[package]] name = "socket2" @@ -7029,7 +7399,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "02e2d2db9033d13a1567121ddd7a095ee144db4e1ca1b1bda3419bc0da294ebd" dependencies = [ "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -7043,6 +7413,9 @@ name = "spin" version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c530c2b0d0bf8b69304b39fe2001993e267461948b890cd037d8ad4293fa1a0d" +dependencies = [ + "lock_api", +] [[package]] name = "spki" @@ -7170,9 +7543,9 @@ checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" [[package]] name = "syn" -version = "1.0.99" +version = "1.0.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58dbef6ec655055e20b86b15a8cc6d439cca19b667537ac6a1369572d151ab13" +checksum = "60b9b43d45702de4c839cb9b51d9f529c5dd26a4aff255b42b1ebc03e88ee908" dependencies = [ "proc-macro2", "quote", @@ -7212,9 +7585,9 @@ dependencies = [ [[package]] name = "syslog_loose" -version = "0.17.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c4eae4d024d7912b5bea75e54319445d0ffe7f423bb4b68a46129cdcebecaef" +checksum = "97fb75f176928530867b2a659e470f9c9ff71904695bab6556f7ad30f9039efd" dependencies = [ "chrono", "nom", @@ -7226,6 +7599,15 @@ version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f764005d11ee5f36500a149ace24e00e3da98b0158b3e2d53a7495660d3f4d60" +[[package]] +name = "tcp-stream" +version = "0.24.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09a4b0a70bac0a58ca6a7659d1328e34ee462339c70b0fa49f72bad1f278910a" +dependencies = [ + "cfg-if", +] + [[package]] name = "temp-dir" version = "0.1.11" @@ -7238,12 +7620,12 @@ version = "3.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5cdb1ef4eaeeaddc8fbd371e5017057064af0911902ef36b39801f67cc6d79e4" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "fastrand", "libc", "redox_syscall 0.2.13", "remove_dir_all", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -7254,7 +7636,7 @@ checksum = "edd106a334b7657c10b7c540a0106114feadeb4dc314513e97df481d5d966f42" dependencies = [ "byteorder", "dirs", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -7265,7 +7647,7 @@ checksum = "c59df8ac95d96ff9bede18eb7300b0fda5e5d8d90960e76f8e14ae765eedbf1f" dependencies = [ "dirs-next", "rustversion", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -7277,6 +7659,26 @@ dependencies = [ "winapi-util", ] +[[package]] +name = "terminal_size" +version = "0.1.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "633c1a546cee861a1a6d0dc69ebeca693bf4296661ba7852b9d21d159e0506df" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "terminal_size" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb20089a8ba2b69debd491f8d2d023761cbf196e999218c591fa1e7e15a21907" +dependencies = [ + "rustix 0.36.5", + "windows-sys 0.42.0", +] + [[package]] name = "termtree" version = "0.2.4" @@ -7298,7 +7700,7 @@ version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c95968eedc6fc4f5c21920e0f4264f78ec5e4c56bb394f319becc1a5830b3e54" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "proc-macro-error", "proc-macro2", "quote", @@ -7322,18 +7724,18 @@ checksum = "b1141d4d61095b28419e22cb0bbf02755f5e54e0526f97f1e3d1d160e60885fb" [[package]] name = "thiserror" -version = "1.0.34" +version = "1.0.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c1b05ca9d106ba7d2e31a9dab4a64e7be2cce415321966ea3132c49a656e252" +checksum = "10deb33631e3c9018b9baf9dcbbc4f737320d2b576bac10f6aefa048fa407e3e" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.34" +version = "1.0.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8f2591983642de85c921015f3f070c665a197ed69e417af436115e3a1407487" +checksum = "982d17546b47146b28f7c22e3d08465f6b8903d0ea13c1660d9d84a6e7adcdbb" dependencies = [ "proc-macro2", "quote", @@ -7372,22 +7774,32 @@ dependencies = [ [[package]] name = "time" -version = "0.3.9" +version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2702e08a7a860f005826c6815dcac101b19b5eb330c27fe4a5928fec1d20ddd" +checksum = "a561bf4617eebd33bca6434b988f39ed798e527f51a1e797d0ee4f61c0a38376" dependencies = [ "itoa 1.0.1", "libc", "num_threads", "serde", + "time-core", "time-macros", ] +[[package]] +name = "time-core" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" + [[package]] name = "time-macros" -version = "0.2.4" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42657b1a6f4d817cda8e7a0ace261fe0cc946cf3a80314390b22cc61ae080792" +checksum = "d967f99f534ca7e495c575c62638eebc2898a8c84c119b89e250477bc4ba16b2" +dependencies = [ + "time-core", +] [[package]] name = "tiny-keccak" @@ -7425,24 +7837,23 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.20.1" +version = "1.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a8325f63a7d4774dd041e363b2409ed1c5cbbd0f867795e661df066b2b0a581" +checksum = "eab6d665857cc6ca78d6e80303a02cea7a7851e85dfbd77cbdc09bd129f1ef46" dependencies = [ "autocfg", - "bytes 1.2.1", + "bytes 1.3.0", "libc", "memchr", - "mio 0.8.2", + "mio", "num_cpus", - "once_cell", "parking_lot", "pin-project-lite", "signal-hook-registry", "socket2", "tokio-macros", - "tracing 0.1.34", - "winapi 0.3.9", + "tracing 0.1.37", + "windows-sys 0.42.0", ] [[package]] @@ -7507,7 +7918,7 @@ checksum = "29a12c1b3e0704ae7dfc25562629798b29c72e6b1d0a681b6f29ab4ae5e7f7bf" dependencies = [ "async-trait", "byteorder", - "bytes 1.2.1", + "bytes 1.3.0", "fallible-iterator", "futures-channel", "futures-util", @@ -7520,18 +7931,7 @@ dependencies = [ "postgres-types", "socket2", "tokio", - "tokio-util 0.7.1", -] - -[[package]] -name = "tokio-rustls" -version = "0.22.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc6844de72e57df1980054b38be3a9f4702aba4858be64dd700181a8a6d0e1b6" -dependencies = [ - "rustls 0.19.1", - "tokio", - "webpki 0.21.4", + "tokio-util", ] [[package]] @@ -7547,14 +7947,14 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.9" +version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df54d54117d6fdc4e4fea40fe1e4e566b3505700e148a6827e59b34b0d2600d9" +checksum = "d660770404473ccd7bc9f8b28494a811bc18542b915c0855c51e8f419d5223ce" dependencies = [ "futures-core", "pin-project-lite", "tokio", - "tokio-util 0.7.1", + "tokio-util", ] [[package]] @@ -7564,25 +7964,12 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "53474327ae5e166530d17f2d956afcb4f8a004de581b3cae10f12006bc8163e3" dependencies = [ "async-stream", - "bytes 1.2.1", + "bytes 1.3.0", "futures-core", "tokio", "tokio-stream", ] -[[package]] -name = "tokio-tungstenite" -version = "0.15.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "511de3f85caf1c98983545490c3d09685fa8eb634e57eec22bb4db271f46cbd8" -dependencies = [ - "futures-util", - "log", - "pin-project", - "tokio", - "tungstenite 0.14.0", -] - [[package]] name = "tokio-tungstenite" version = "0.17.2" @@ -7591,37 +7978,35 @@ checksum = "f714dd15bead90401d77e04243611caec13726c2408afd5b31901dfcdcb3b181" dependencies = [ "futures-util", "log", - "rustls 0.20.4", "tokio", "tungstenite 0.17.3", ] [[package]] -name = "tokio-util" -version = "0.6.9" +name = "tokio-tungstenite" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" +checksum = "54319c93411147bced34cb5609a80e0a8e44c5999c93903a81cd866630ec0bfd" dependencies = [ - "bytes 1.2.1", - "futures-core", - "futures-sink", + "futures-util", "log", - "pin-project-lite", + "rustls 0.20.4", "tokio", + "tungstenite 0.18.0", ] [[package]] name = "tokio-util" -version = "0.7.1" -source = "git+https://github.com/vectordotdev/tokio?rev=3aa231cf6f33f74ca29077163879f0de9a207ad8#3aa231cf6f33f74ca29077163879f0de9a207ad8" +version = "0.7.4" +source = "git+https://github.com/vectordotdev/tokio?branch=tokio-util-0.7.4-framed-read-continue-on-error#53a17f257b599a9d18bd75249de98d0b6fc28cfa" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "futures-core", "futures-sink", "pin-project-lite", "slab", "tokio", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] @@ -7633,42 +8018,6 @@ dependencies = [ "serde", ] -[[package]] -name = "tonic" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5be9d60db39854b30b835107500cf0aca0b0d14d6e1c3de124217c23a29c2ddb" -dependencies = [ - "async-stream", - "async-trait", - "axum", - "base64", - "bytes 1.2.1", - "flate2", - "futures-core", - "futures-util", - "h2", - "http", - "http-body", - "hyper", - "hyper-timeout", - "percent-encoding", - "pin-project", - "prost 0.10.4", - "prost-derive 0.10.1", - "rustls-native-certs 0.6.2", - "rustls-pemfile 1.0.0", - "tokio", - "tokio-rustls 0.23.3", - "tokio-stream", - "tokio-util 0.7.1", - "tower", - "tower-layer", - "tower-service", - "tracing 0.1.34", - "tracing-futures 0.2.5", -] - [[package]] name = "tonic" version = "0.8.0" @@ -7679,7 +8028,8 @@ dependencies = [ "async-trait", "axum", "base64", - "bytes 1.2.1", + "bytes 1.3.0", + "flate2", "futures-core", "futures-util", "h2", @@ -7689,27 +8039,30 @@ dependencies = [ "hyper-timeout", "percent-encoding", "pin-project", - "prost 0.11.0", + "prost 0.11.3", "prost-derive 0.11.0", + "rustls-native-certs 0.6.2", + "rustls-pemfile 1.0.0", "tokio", + "tokio-rustls", "tokio-stream", - "tokio-util 0.7.1", + "tokio-util", "tower", "tower-layer", "tower-service", - "tracing 0.1.34", + "tracing 0.1.37", "tracing-futures 0.2.5", ] [[package]] name = "tonic-build" -version = "0.7.2" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9263bf4c9bfaae7317c1c2faf7f18491d2fe476f70c414b73bf5d445b00ffa1" +checksum = "5bf5e9b9c0f7e0a7c027dcfaba7b2c60816c7049171f679d99ee2ff65d0de8c4" dependencies = [ "prettyplease", "proc-macro2", - "prost-build", + "prost-build 0.11.4", "quote", "syn", ] @@ -7728,10 +8081,10 @@ dependencies = [ "rand 0.8.5", "slab", "tokio", - "tokio-util 0.7.1", + "tokio-util", "tower-layer", "tower-service", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] @@ -7742,7 +8095,7 @@ checksum = "7d342c6d58709c0a6d48d48dabbb62d4ef955cf5f0f3bbfd845838e7ae88dbae" dependencies = [ "base64", "bitflags", - "bytes 1.2.1", + "bytes 1.3.0", "futures-core", "futures-util", "http", @@ -7752,7 +8105,7 @@ dependencies = [ "tower", "tower-layer", "tower-service", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] @@ -7783,15 +8136,15 @@ dependencies = [ [[package]] name = "tracing" -version = "0.1.34" +version = "0.1.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d0ecdcb44a79f0fe9844f0c4f33a342cbcbb5117de8001e6ba0dc2351327d09" +checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "log", "pin-project-lite", "tracing-attributes", - "tracing-core 0.1.28", + "tracing-core 0.1.30", ] [[package]] @@ -7799,16 +8152,16 @@ name = "tracing" version = "0.2.0" source = "git+https://github.com/tokio-rs/tracing?rev=e0642d949891546a3bb7e47080365ee7274f05cd#e0642d949891546a3bb7e47080365ee7274f05cd" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "pin-project-lite", "tracing-core 0.2.0", ] [[package]] name = "tracing-attributes" -version = "0.1.21" +version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6b8ad3567499f98a1db7a752b07a7c8c7c7c34c332ec00effb2b0027974b7c" +checksum = "4017f8f45139870ca7e672686113917c71c7a6e02d4924eda67186083c03081a" dependencies = [ "proc-macro2", "quote", @@ -7817,9 +8170,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.28" +version = "0.1.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b7358be39f2f274f322d2aaed611acc57f382e8eb1e5b48cb9ae30933495ce7" +checksum = "24eb03ba0eab1fd845050058ce5e616558e8f8d8fca633e6b163fe25c797213a" dependencies = [ "once_cell", "valuable", @@ -7839,9 +8192,9 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "12de1a8c6bcfee614305e836308b596bbac831137a04c61f7e5b0b0bf2cfeaf6" dependencies = [ - "tracing 0.1.34", - "tracing-core 0.1.28", - "tracing-subscriber 0.3.15", + "tracing 0.1.37", + "tracing-core 0.1.30", + "tracing-subscriber 0.3.16", ] [[package]] @@ -7850,10 +8203,10 @@ version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97d095ae15e245a057c8e8451bab9b3ee1e1f68e9ba2b4fbc18d0ac5237835f2" dependencies = [ - "futures 0.3.24", + "futures 0.3.25", "futures-task", "pin-project", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] @@ -7872,9 +8225,9 @@ dependencies = [ "criterion", "dashmap", "mock_instant", - "tracing 0.1.34", - "tracing-core 0.1.28", - "tracing-subscriber 0.3.15", + "tracing 0.1.37", + "tracing-core 0.1.30", + "tracing-subscriber 0.3.16", ] [[package]] @@ -7885,7 +8238,7 @@ checksum = "78ddad33d2d10b1ed7eb9d1f518a5674713876e97e5bb9b7345a7984fbb4f922" dependencies = [ "lazy_static", "log", - "tracing-core 0.1.28", + "tracing-core 0.1.30", ] [[package]] @@ -7895,7 +8248,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bc6b213177105856957181934e4920de57730fc69bf42c37ee5bb664d406d9e1" dependencies = [ "serde", - "tracing-core 0.1.28", + "tracing-core 0.1.30", ] [[package]] @@ -7914,20 +8267,20 @@ dependencies = [ "sharded-slab", "smallvec", "thread_local", - "tracing 0.1.34", - "tracing-core 0.1.28", + "tracing 0.1.37", + "tracing-core 0.1.30", "tracing-log", "tracing-serde", ] [[package]] name = "tracing-subscriber" -version = "0.3.15" +version = "0.3.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60db860322da191b40952ad9affe65ea23e7dd6a5c442c2c42865810c6ab8e6b" +checksum = "a6176eae26dd70d0c919749377897b54a9276bd7061339665dd68777926b5a70" dependencies = [ - "ansi_term", "matchers 0.1.0", + "nu-ansi-term", "once_cell", "regex", "serde", @@ -7935,8 +8288,8 @@ dependencies = [ "sharded-slab", "smallvec", "thread_local", - "tracing 0.1.34", - "tracing-core 0.1.28", + "tracing 0.1.37", + "tracing-core 0.1.30", "tracing-log", "tracing-serde", ] @@ -7948,7 +8301,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a3b48778c2d401c6a7fcf38a0e3c55dc8e8e753cbd381044a8cdb6fd69a29f53" dependencies = [ "lazy_static", - "tracing-core 0.1.28", + "tracing-core 0.1.30", "tracing-subscriber 0.2.25", "tracing-test-macro 0.1.0", ] @@ -7960,8 +8313,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3eb7bda2e93bbc9c5b247034acc6a4b3d04f033a3d4b8fc1cb87d4d1c7c7ebd7" dependencies = [ "lazy_static", - "tracing-core 0.1.28", - "tracing-subscriber 0.3.15", + "tracing-core 0.1.30", + "tracing-subscriber 0.3.16", "tracing-test-macro 0.2.1", ] @@ -7992,7 +8345,7 @@ name = "tracing-tower" version = "0.1.0" source = "git+https://github.com/tokio-rs/tracing?rev=e0642d949891546a3bb7e47080365ee7274f05cd#e0642d949891546a3bb7e47080365ee7274f05cd" dependencies = [ - "futures 0.3.24", + "futures 0.3.25", "tower-service", "tracing 0.2.0", "tracing-futures 0.3.0", @@ -8014,13 +8367,13 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c31f240f59877c3d4bb3b3ea0ec5a6a0cff07323580ff8c7a605cd7d08b255d" dependencies = [ "async-trait", - "cfg-if 1.0.0", + "cfg-if", "data-encoding", - "enum-as-inner", + "enum-as-inner 0.4.0", "futures-channel", "futures-io", "futures-util", - "idna", + "idna 0.2.3", "ipnet", "lazy_static", "log", @@ -8032,13 +8385,38 @@ dependencies = [ "url", ] +[[package]] +name = "trust-dns-proto" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f7f83d1e4a0e4358ac54c5c3681e5d7da5efc5a7a632c90bb6d6669ddd9bc26" +dependencies = [ + "async-trait", + "cfg-if", + "data-encoding", + "enum-as-inner 0.5.1", + "futures-channel", + "futures-io", + "futures-util", + "idna 0.2.3", + "ipnet", + "lazy_static", + "rand 0.8.5", + "smallvec", + "thiserror", + "tinyvec", + "tokio", + "tracing 0.1.37", + "url", +] + [[package]] name = "trust-dns-resolver" version = "0.21.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e4ba72c2ea84515690c9fcef4c6c660bb9df3036ed1051686de84605b74fd558" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "futures-util", "ipconfig", "lazy_static", @@ -8049,7 +8427,7 @@ dependencies = [ "smallvec", "thiserror", "tokio", - "trust-dns-proto", + "trust-dns-proto 0.21.2", ] [[package]] @@ -8073,18 +8451,18 @@ dependencies = [ [[package]] name = "tungstenite" -version = "0.14.0" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0b2d8558abd2e276b0a8df5c05a2ec762609344191e5fd23e292c910e9165b5" +checksum = "e27992fd6a8c29ee7eef28fc78349aa244134e10ad447ce3b9f0ac0ed0fa4ce0" dependencies = [ "base64", "byteorder", - "bytes 1.2.1", + "bytes 1.3.0", "http", "httparse", "log", "rand 0.8.5", - "sha-1 0.9.8", + "sha-1", "thiserror", "url", "utf-8", @@ -8092,18 +8470,18 @@ dependencies = [ [[package]] name = "tungstenite" -version = "0.17.3" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e27992fd6a8c29ee7eef28fc78349aa244134e10ad447ce3b9f0ac0ed0fa4ce0" +checksum = "30ee6ab729cd4cf0fd55218530c4522ed30b7b6081752839b68fcec8d0960788" dependencies = [ "base64", "byteorder", - "bytes 1.2.1", + "bytes 1.3.0", "http", "httparse", "log", "rand 0.8.5", - "sha-1 0.10.0", + "sha1", "thiserror", "url", "utf-8", @@ -8115,7 +8493,7 @@ version = "1.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "static_assertions", ] @@ -8187,18 +8565,9 @@ dependencies = [ [[package]] name = "ucd-trie" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56dee185309b50d1f11bfedef0fe6d036842e3fb77413abef29f8f8d1c5d4c1c" - -[[package]] -name = "uncased" -version = "0.9.6" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5baeed7327e25054889b9bd4f975f32e5f4c5d434042d59ab6cd4142c0a76ed0" -dependencies = [ - "version_check", -] +checksum = "9e79c4d996edb816c91e4308506774452e55e95c3c9de07b6729e17e15a5ef81" [[package]] name = "unicase" @@ -8259,9 +8628,9 @@ dependencies = [ [[package]] name = "unsafe-libyaml" -version = "0.2.2" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "931179334a56395bcf64ba5e0ff56781381c1a5832178280c7d7f91d1679aeb0" +checksum = "c1e5fa573d8ac5f1a856f8d7be41d390ee973daf97c806b2c1a465e4e1406e68" [[package]] name = "untrusted" @@ -8282,12 +8651,12 @@ dependencies = [ [[package]] name = "url" -version = "2.3.0" +version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22fe195a4f217c25b25cb5058ced57059824a678474874038dc88d211bf508d3" +checksum = "0d68c799ae75762b8c3fe375feb6600ef5602c883c5d21eb51c09f22b83c4643" dependencies = [ "form_urlencoded", - "idna", + "idna 0.3.0", "percent-encoding", "serde", ] @@ -8322,17 +8691,18 @@ version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bc5cf98d8186244414c848017f0e2676b3fcb46807f6668a97dfe67359a3c4b7" dependencies = [ - "getrandom 0.2.6", + "getrandom 0.2.8", "serde", ] [[package]] name = "uuid" -version = "1.1.2" +version = "1.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd6469f4314d5f1ffec476e05f17cc9a78bc7a27a6a857842170bdf8d6f98d2f" +checksum = "422ee0de9031b5b948b97a8fc04e3aa35230001a722ddd27943e0be31564ce4c" dependencies = [ - "getrandom 0.2.6", + "getrandom 0.2.8", + "rand 0.8.5", "serde", ] @@ -8347,19 +8717,19 @@ name = "value" version = "0.1.0" dependencies = [ "async-graphql", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "indoc", "lookup", "mlua", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "quickcheck", "regex", "serde", "serde_json", "snafu", "toml", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] @@ -8376,10 +8746,11 @@ checksum = "f1bddf1187be692e79c5ffeab891132dfb0f236ed36a43c7ed39f1165ee20191" [[package]] name = "vector" -version = "0.25.0" +version = "0.26.0" dependencies = [ "approx", "arc-swap", + "arr_macro", "assert_cmd", "async-compression", "async-graphql", @@ -8411,11 +8782,11 @@ dependencies = [ "base64", "bloom", "bollard", - "bytes 1.2.1", + "bytes 1.3.0", "bytesize", "chrono", "cidr-utils", - "clap 3.2.20", + "clap 4.0.29", "codecs", "colored", "console-subscriber", @@ -8435,7 +8806,7 @@ dependencies = [ "fakedata", "file-source", "flate2", - "futures 0.3.24", + "futures 0.3.25", "futures-util", "glob", "goauth", @@ -8443,6 +8814,7 @@ dependencies = [ "grok", "h2", "hash_hasher", + "hashbrown", "headers", "heim", "hex", @@ -8454,14 +8826,16 @@ dependencies = [ "hyper-proxy", "indexmap", "indoc", - "infer 0.9.0", + "infer 0.11.0", "inventory", - "itertools 0.10.3", + "itertools", "k8s-openapi", "kube", + "lapin", "libc", "listenfd", "logfmt", + "loki-logproto", "lookup", "lru", "maxminddb", @@ -8482,17 +8856,16 @@ dependencies = [ "openssl-probe", "openssl-src", "opentelemetry-proto", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "percent-encoding", "pin-project", "portpicker", "postgres-openssl", - "pretty_assertions", "prometheus-parser", "proptest", - "prost 0.10.4", - "prost-build", - "prost-types 0.10.1", + "prost 0.11.3", + "prost-build 0.11.4", + "prost-types 0.11.2", "pulsar", "quickcheck", "rand 0.8.5", @@ -8505,14 +8878,15 @@ dependencies = [ "rmpv", "roaring", "seahash", - "semver 1.0.13", + "semver 1.0.14", "serde", "serde-toml-merge", "serde_bytes", "serde_json", - "serde_with 2.0.0", - "serde_yaml 0.9.11", - "sha2 0.10.5", + "serde_with 2.1.0", + "serde_yaml 0.9.14", + "sha2 0.10.6", + "similar-asserts", "smallvec", "smpl_jwt", "snafu", @@ -8528,24 +8902,24 @@ dependencies = [ "tokio-postgres", "tokio-stream", "tokio-test", - "tokio-tungstenite 0.17.2", - "tokio-util 0.7.1", + "tokio-tungstenite 0.18.0", + "tokio-util", "toml", - "tonic 0.7.2", + "tonic", "tonic-build", "tower", "tower-test", - "tracing 0.1.34", - "tracing-core 0.1.28", + "tracing 0.1.37", + "tracing-core 0.1.30", "tracing-futures 0.2.5", "tracing-limit", - "tracing-subscriber 0.3.15", + "tracing-subscriber 0.3.16", "tracing-tower", - "trust-dns-proto", + "trust-dns-proto 0.22.0", "tui", "typetag", "url", - "uuid 1.1.2", + "uuid 1.2.2", "value", "vector-api-client", "vector-buffers", @@ -8571,7 +8945,8 @@ dependencies = [ "anyhow", "async-trait", "chrono", - "futures 0.3.24", + "clap 4.0.29", + "futures 0.3.25", "graphql_client", "indoc", "reqwest", @@ -8579,9 +8954,9 @@ dependencies = [ "serde_json", "tokio", "tokio-stream", - "tokio-tungstenite 0.17.2", + "tokio-tungstenite 0.18.0", "url", - "uuid 1.1.2", + "uuid 1.2.2", ] [[package]] @@ -8592,15 +8967,15 @@ dependencies = [ "async-stream", "async-trait", "bytecheck", - "bytes 1.2.1", - "clap 3.2.20", + "bytes 1.3.0", + "clap 4.0.29", "crc32fast", "criterion", "crossbeam-queue", "crossbeam-utils", "db-key", "fslock", - "futures 0.3.24", + "futures 0.3.25", "hdrhistogram", "leveldb", "memmap2", @@ -8616,15 +8991,15 @@ dependencies = [ "rand 0.8.5", "rkyv", "serde", - "serde_yaml 0.9.11", + "serde_yaml 0.9.14", "snafu", "temp-dir", "tokio", "tokio-test", - "tokio-util 0.7.1", - "tracing 0.1.34", + "tokio-util", + "tracing 0.1.37", "tracing-fluent-assertions", - "tracing-subscriber 0.3.15", + "tracing-subscriber 0.3.16", "vector-common", "vector-config", "vector-config-common", @@ -8636,25 +9011,30 @@ name = "vector-common" version = "0.1.0" dependencies = [ "async-stream", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "chrono-tz", "crossbeam-utils", "derivative", - "futures 0.3.24", + "futures 0.3.25", + "indexmap", "metrics", "nom", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "pin-project", + "quickcheck", + "quickcheck_macros", + "ryu", "serde", "serde_json", "smallvec", "snafu", "stream-cancel", "tokio", - "tracing 0.1.34", + "tracing 0.1.37", "value", "vector-config", + "vector-config-common", "vector-config-macros", ] @@ -8672,7 +9052,7 @@ dependencies = [ "schemars", "serde", "serde_json", - "serde_with 2.0.0", + "serde_with 2.1.0", "snafu", "toml", "url", @@ -8713,8 +9093,9 @@ dependencies = [ "async-trait", "base64", "bitmask-enum", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", + "chrono-tz", "criterion", "crossbeam-utils", "db-key", @@ -8723,7 +9104,7 @@ dependencies = [ "enumflags2", "env-test-util", "float_eq", - "futures 0.3.24", + "futures 0.3.25", "futures-util", "headers", "http", @@ -8740,15 +9121,16 @@ dependencies = [ "noisy_float", "once_cell", "openssl", - "ordered-float 3.0.0", + "ordered-float 3.4.0", + "parking_lot", "pin-project", - "pretty_assertions", "proptest", - "prost 0.10.4", - "prost-build", - "prost-types 0.10.1", + "prost 0.11.3", + "prost-build 0.11.4", + "prost-types 0.11.2", "quanta", "quickcheck", + "quickcheck_macros", "rand 0.8.5", "rand_distr", "regex", @@ -8756,21 +9138,22 @@ dependencies = [ "security-framework", "serde", "serde_json", - "serde_with 2.0.0", + "serde_with 2.1.0", + "similar-asserts", "snafu", "socket2", "tokio", "tokio-openssl", "tokio-stream", "tokio-test", - "tokio-util 0.7.1", + "tokio-util", "toml", - "tonic 0.7.2", + "tonic", "tower", - "tracing 0.1.34", - "tracing-core 0.1.28", + "tracing 0.1.37", + "tracing-core 0.1.30", "tracing-log", - "tracing-subscriber 0.3.15", + "tracing-subscriber 0.3.16", "twox-hash", "typetag", "url", @@ -8778,6 +9161,7 @@ dependencies = [ "vector-buffers", "vector-common", "vector-config", + "vector-config-common", "vector-config-macros", "vrl", ] @@ -8807,11 +9191,11 @@ checksum = "6a02e4885ed3bc0f2de90ea6dd45ebcbb66dacffe03547fadbb0eeae2770887d" name = "vrl" version = "0.1.0" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "criterion", "indoc", "lookup", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "serde_json", "value", "vector-common", @@ -8826,7 +9210,7 @@ dependencies = [ name = "vrl-cli" version = "0.1.0" dependencies = [ - "clap 3.2.20", + "clap 4.0.29", "exitcode", "indoc", "lookup", @@ -8850,14 +9234,15 @@ name = "vrl-compiler" version = "0.1.0" dependencies = [ "anymap", - "bytes 1.2.1", + "bytes 1.3.0", "chrono", "criterion", "dyn-clone", + "getrandom 0.2.8", "indoc", "lalrpop-util", "lookup", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "paste", "regex", "serde", @@ -8865,6 +9250,7 @@ dependencies = [ "value", "vector-common", "vector-config", + "vector-config-common", "vector-config-macros", "vrl-core", "vrl-diagnostic", @@ -8896,7 +9282,7 @@ dependencies = [ "lalrpop", "lalrpop-util", "lookup", - "ordered-float 3.0.0", + "ordered-float 3.4.0", "paste", "test-case", "thiserror", @@ -8910,15 +9296,17 @@ dependencies = [ "aes", "anyhow", "base64", - "bytes 1.2.1", + "bytes 1.3.0", "cbc", "cfb-mode", + "charset", "chrono", "chrono-tz", "cidr-utils", "criterion", "csv", "ctr", + "data-encoding", "datadog-filter", "datadog-grok", "datadog-search-syntax", @@ -8933,23 +9321,24 @@ dependencies = [ "ofb", "once_cell", "percent-encoding", + "quoted_printable", "rand 0.8.5", "regex", "roxmltree", "rust_decimal", "serde", "serde_json", - "sha-1 0.10.0", - "sha2 0.10.5", + "sha-1", + "sha2 0.10.6", "sha3", "strip-ansi-escapes", "syslog_loose", - "tracing 0.1.34", + "tracing 0.1.37", "tracing-test 0.1.0", "uaparser", "url", "utf8-width", - "uuid 1.1.2", + "uuid 1.2.2", "value", "vector-common", "vrl", @@ -8964,7 +9353,7 @@ dependencies = [ "ansi_term", "chrono", "chrono-tz", - "clap 3.2.20", + "clap 4.0.29", "enrichment", "glob", "lookup", @@ -8973,7 +9362,7 @@ dependencies = [ "serde", "serde_json", "tikv-jemallocator", - "tracing-subscriber 0.3.15", + "tracing-subscriber 0.3.16", "value", "vector-common", "vector-vrl-functions", @@ -8981,6 +9370,20 @@ dependencies = [ "vrl-stdlib", ] +[[package]] +name = "vrl-web-playground" +version = "0.1.0" +dependencies = [ + "getrandom 0.2.8", + "gloo-utils", + "serde", + "serde-wasm-bindgen", + "value", + "vrl", + "vrl-stdlib", + "wasm-bindgen", +] + [[package]] name = "vte" version = "0.10.1" @@ -9024,7 +9427,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "808cf2735cd4b6866113f648b791c6adc5714537bc222d9347bb203386ffda56" dependencies = [ "same-file", - "winapi 0.3.9", + "winapi", "winapi-util", ] @@ -9040,11 +9443,11 @@ dependencies = [ [[package]] name = "warp" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cef4e1e9114a4b7f1ac799f16ce71c14de5778500c5450ec6b7b920c55b587e" +checksum = "ed7b8be92646fc3d18b06147664ebc5f48d222686cb11a8755e561a735aacc6d" dependencies = [ - "bytes 1.2.1", + "bytes 1.3.0", "futures-channel", "futures-util", "headers", @@ -9055,16 +9458,17 @@ dependencies = [ "mime_guess", "percent-encoding", "pin-project", + "rustls-pemfile 0.2.1", "scoped-tls", "serde", "serde_json", "serde_urlencoded", "tokio", "tokio-stream", - "tokio-tungstenite 0.15.0", - "tokio-util 0.6.9", + "tokio-tungstenite 0.17.2", + "tokio-util", "tower-service", - "tracing 0.1.34", + "tracing 0.1.37", ] [[package]] @@ -9087,23 +9491,23 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.80" +version = "0.2.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27370197c907c55e3f1a9fbe26f44e937fe6451368324e009cba39e139dc08ad" +checksum = "eaf9f5aceeec8be17c128b2e93e031fb8a4d469bb9c4ae2d7dc1888b26887268" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "wasm-bindgen-macro", ] [[package]] name = "wasm-bindgen-backend" -version = "0.2.80" +version = "0.2.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53e04185bfa3a779273da532f5025e33398409573f348985af9a1cbf3774d3f4" +checksum = "4c8ffb332579b0557b52d268b91feab8df3615f265d5270fec2a8c95b17c1142" dependencies = [ "bumpalo", - "lazy_static", "log", + "once_cell", "proc-macro2", "quote", "syn", @@ -9116,7 +9520,7 @@ version = "0.4.30" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6f741de44b75e14c35df886aff5f1eb73aa114fa5d4d00dcd37b5e01259bf3b2" dependencies = [ - "cfg-if 1.0.0", + "cfg-if", "js-sys", "wasm-bindgen", "web-sys", @@ -9124,9 +9528,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.80" +version = "0.2.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17cae7ff784d7e83a2fe7611cfe766ecf034111b49deb850a3dc7699c08251f5" +checksum = "052be0f94026e6cbc75cdefc9bae13fd6052cdcaf532fa6c45e7ae33a1e6c810" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -9134,9 +9538,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.80" +version = "0.2.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99ec0dc7a4756fffc231aab1b9f2f578d23cd391390ab27f952ae0c9b3ece20b" +checksum = "07bc0c051dc5f23e307b13285f9d75df86bfdf816c5721e573dec1f9b8aa193c" dependencies = [ "proc-macro2", "quote", @@ -9147,9 +9551,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.80" +version = "0.2.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d554b7f530dee5964d9a9468d95c1f8b8acae4f282807e7d27d4b03099a46744" +checksum = "1c38c045535d93ec4f0b4defec448e4291638ee608530863b1e2ba115d4fff7f" [[package]] name = "web-sys" @@ -9163,16 +9567,18 @@ dependencies = [ [[package]] name = "webbrowser" -version = "0.7.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc6a3cffdb686fbb24d9fb8f03a213803277ed2300f11026a3afe1f108dc021b" +checksum = "2a0cc7962b5aaa0dfcebaeef0161eec6edf5f4606c12e6777fd7d392f52033a5" dependencies = [ "jni", - "ndk-glue", + "ndk-context", + "objc", + "raw-window-handle", "url", "web-sys", - "widestring 0.5.1", - "winapi 0.3.9", + "widestring 1.0.2", + "winapi", ] [[package]] @@ -9242,12 +9648,6 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "653f141f39ec16bba3c5abe400a0c60da7468261cc2cbf36805022876bc721a8" -[[package]] -name = "winapi" -version = "0.2.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "167dc9d6949a9b857f3451275e911c3f44255842c1f7a76f33c55103a909087a" - [[package]] name = "winapi" version = "0.3.9" @@ -9258,12 +9658,6 @@ dependencies = [ "winapi-x86_64-pc-windows-gnu", ] -[[package]] -name = "winapi-build" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d315eee3b34aca4797b2da6b13ed88266e6d612562a0c46390af8299fc699bc" - [[package]] name = "winapi-i686-pc-windows-gnu" version = "0.4.0" @@ -9276,7 +9670,7 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70ec6ce85bb158151cae5e5c87f95a8e97d2c0c4b001223f33a334e3ce5de178" dependencies = [ - "winapi 0.3.9", + "winapi", ] [[package]] @@ -9336,6 +9730,27 @@ dependencies = [ "windows_x86_64_msvc 0.36.1", ] +[[package]] +name = "windows-sys" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a3e1820f08b8513f676f7ab6c1f99ff312fb97b553d30ff4dd86f9f15728aa7" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc 0.42.0", + "windows_i686_gnu 0.42.0", + "windows_i686_msvc 0.42.0", + "windows_x86_64_gnu 0.42.0", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc 0.42.0", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41d2aa71f6f0cbe00ae5167d90ef3cfe66527d6f613ca78ac8024c3ccab9a19e" + [[package]] name = "windows_aarch64_msvc" version = "0.30.0" @@ -9354,6 +9769,12 @@ version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9bb8c3fd39ade2d67e9874ac4f3db21f0d710bee00fe7cab16949ec184eeaa47" +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd0f252f5a35cac83d6311b2e795981f5ee6e67eb1f9a7f64eb4500fbc4dcdb4" + [[package]] name = "windows_i686_gnu" version = "0.30.0" @@ -9372,6 +9793,12 @@ version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "180e6ccf01daf4c426b846dfc66db1fc518f074baa793aa7d9b9aaeffad6a3b6" +[[package]] +name = "windows_i686_gnu" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbeae19f6716841636c28d695375df17562ca208b2b7d0dc47635a50ae6c5de7" + [[package]] name = "windows_i686_msvc" version = "0.30.0" @@ -9390,6 +9817,12 @@ version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2e7917148b2812d1eeafaeb22a97e4813dfa60a3f8f78ebe204bcc88f12f024" +[[package]] +name = "windows_i686_msvc" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "84c12f65daa39dd2babe6e442988fc329d6243fdce47d7d2d155b8d874862246" + [[package]] name = "windows_x86_64_gnu" version = "0.30.0" @@ -9408,6 +9841,18 @@ version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4dcd171b8776c41b97521e5da127a2d86ad280114807d0b2ab1e462bc764d9e1" +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf7b1b21b5362cbc318f686150e5bcea75ecedc74dd157d874d754a2ca44b0ed" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09d525d2ba30eeb3297665bd434a54297e4170c7f1a44cad4ef58095b4cd2028" + [[package]] name = "windows_x86_64_msvc" version = "0.30.0" @@ -9426,13 +9871,19 @@ version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c811ca4a8c853ef420abd8592ba53ddbbac90410fab6903b3e79972a631f7680" +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f40009d85759725a34da6d89a94e63d7bdc50a862acf0dbc7c8e488f1edcb6f5" + [[package]] name = "winreg" version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0120db82e8a1e0b9fb3345a539c478767c0048d842860994d96113d5b667bd69" dependencies = [ - "winapi 0.3.9", + "winapi", ] [[package]] @@ -9441,20 +9892,20 @@ version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d" dependencies = [ - "winapi 0.3.9", + "winapi", ] [[package]] name = "wiremock" -version = "0.5.14" +version = "0.5.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc3c7b7557dbfdad6431b5a51196c9110cef9d83f6a9b26699f35cdc0ae113ec" +checksum = "249dc68542861d17eae4b4e5e8fb381c2f9e8f255a84f6771d5fdf8b6c03ce3c" dependencies = [ "assert-json-diff", "async-trait", "base64", "deadpool", - "futures 0.3.24", + "futures 0.3.25", "futures-timer", "http-types", "hyper", @@ -9476,16 +9927,6 @@ dependencies = [ "regex", ] -[[package]] -name = "ws2_32-sys" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d59cefebd0c892fa2dd6de581e937301d8552cb44489cdff035c6187cb63fa5e" -dependencies = [ - "winapi 0.2.8", - "winapi-build", -] - [[package]] name = "xml-rs" version = "0.8.4" @@ -9494,9 +9935,9 @@ checksum = "d2d7d3948613f75c98fd9328cfdcc45acc4d360655289d0a7d4ec931392200a3" [[package]] name = "xmlparser" -version = "0.13.3" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "114ba2b24d2167ef6d67d7d04c8cc86522b87f490025f39f0303b7db5bf5e3d8" +checksum = "4d25c75bf9ea12c4040a97f829154768bbbce366287e2dc044af160cd79a13fd" [[package]] name = "yaml-rust" @@ -9507,12 +9948,6 @@ dependencies = [ "linked-hash-map", ] -[[package]] -name = "yansi" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" - [[package]] name = "zerocopy" version = "0.3.0" @@ -9557,18 +9992,18 @@ dependencies = [ [[package]] name = "zstd" -version = "0.10.2+zstd.1.5.2" +version = "0.11.2+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f4a6bd64f22b5e3e94b4e238669ff9f10815c27a5180108b849d24174a83847" +checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" dependencies = [ "zstd-safe", ] [[package]] name = "zstd-safe" -version = "4.1.6+zstd.1.5.2" +version = "5.0.2+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94b61c51bb270702d6167b8ce67340d2754b088d0c091b06e593aa772c3ee9bb" +checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" dependencies = [ "libc", "zstd-sys", @@ -9576,9 +10011,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "1.6.3+zstd.1.5.2" +version = "2.0.4+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc49afa5c8d634e75761feda8c592051e7eeb4683ba827211eb0d731d3402ea8" +checksum = "4fa202f2ef00074143e219d15b62ffc317d17cc33909feac471c044087cad7b0" dependencies = [ "cc", "libc", From 500217bac44f72300444dfa55f971cceaa5af677 Mon Sep 17 00:00:00 2001 From: neuronull Date: Wed, 15 Mar 2023 13:30:15 -0600 Subject: [PATCH 22/48] start to work out compilation errors --- Cargo.lock | 828 ++++++++++++++++++------------------ src/sinks/pulsar/config.rs | 25 +- src/sinks/pulsar/service.rs | 4 +- src/sinks/pulsar/sink.rs | 27 +- 4 files changed, 449 insertions(+), 435 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6ed5723738fa6..1f0c586b4a421 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -54,9 +54,9 @@ dependencies = [ [[package]] name = "ahash" -version = "0.8.3" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" +checksum = "bf6ccdb167abbf410dcb915cabd428929d7f6a04980b54a11f26a39f1c7f7107" dependencies = [ "cfg-if", "getrandom 0.2.8", @@ -287,11 +287,11 @@ dependencies = [ [[package]] name = "async-channel" -version = "1.8.0" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf46fee83e5ccffc220104713af3292ff9bc7c64c7de289f66dae8e38d826833" +checksum = "e14485364214912d3b19cc3435dde4df66065127f05fa0d75c712f36f12c2f28" dependencies = [ - "concurrent-queue", + "concurrent-queue 1.2.4", "event-listener", "futures-core", ] @@ -332,7 +332,7 @@ checksum = "17adb73da160dfb475c183343c8cccd80721ea5a605d3eb57125f0a7b7a92d0b" dependencies = [ "async-lock", "async-task", - "concurrent-queue", + "concurrent-queue 2.0.0", "fastrand", "futures-lite", "slab", @@ -418,8 +418,8 @@ checksum = "ec10e63a513389190e9f8f32453bfcfeef271e25e841d61905985f838a5345eb" dependencies = [ "Inflector", "async-graphql-parser", - "darling 0.14.4", - "proc-macro-crate 1.3.1", + "darling 0.14.2", + "proc-macro-crate 1.2.1", "proc-macro2 1.0.52", "quote 1.0.26", "syn 1.0.109", @@ -470,7 +470,7 @@ checksum = "8c374dda1ed3e7d8f0d9ba58715f924862c63eae6849c92d3a18e7fbde9e2794" dependencies = [ "async-lock", "autocfg", - "concurrent-queue", + "concurrent-queue 2.0.0", "futures-lite", "libc", "log", @@ -484,11 +484,12 @@ dependencies = [ [[package]] name = "async-lock" -version = "2.7.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa24f727524730b077666307f2734b4a1a1c57acb79193127dcc8914d5242dd7" +checksum = "c8101efe8695a6c17e02911402145357e718ac92d3ff88ae8419e84b1707b685" dependencies = [ "event-listener", + "futures-lite", ] [[package]] @@ -591,9 +592,9 @@ checksum = "2ce4f10ea3abcd6617873bae9f91d1c5332b4a778bd9ce34d0cd517474c1de82" [[package]] name = "atomic-waker" -version = "1.1.0" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "debc29dde2e69f9e47506b525f639ed42300fc014a3e007832592448fa8e4599" +checksum = "065374052e7df7ee4047b1160cca5e1467a12351a40b3da123c870ba0b8eda2a" [[package]] name = "atty" @@ -1172,9 +1173,9 @@ dependencies = [ [[package]] name = "axum" -version = "0.6.11" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13d8068b6ccb8b34db9de397c7043f91db8b4c66414952c6db944f238c4d3db3" +checksum = "2fb79c228270dcf2426e74864cabc94babb5dbab01a4314e702d2f16540e1591" dependencies = [ "async-trait", "axum-core", @@ -1197,15 +1198,16 @@ dependencies = [ "sync_wrapper", "tokio", "tower", + "tower-http 0.3.5", "tower-layer", "tower-service", ] [[package]] name = "axum-core" -version = "0.3.3" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2f958c80c248b34b9a877a643811be8dbca03ca5ba827f2b63baf3a81e5fc4e" +checksum = "1cae3e661676ffbacb30f1a824089a8c9150e71017f7e1e38f2aa32009188d34" dependencies = [ "async-trait", "bytes 1.4.0", @@ -1381,9 +1383,9 @@ dependencies = [ [[package]] name = "base64ct" -version = "1.6.0" +version = "1.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" +checksum = "b645a089122eccb6111b4f81cbc1a49f5900ac4666bb93ac027feaecf15607bf" [[package]] name = "bit-set" @@ -1422,18 +1424,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "bitvec" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" -dependencies = [ - "funty", - "radium", - "tap", - "wyz", -] - [[package]] name = "block-buffer" version = "0.9.0" @@ -1445,9 +1435,9 @@ dependencies = [ [[package]] name = "block-buffer" -version = "0.10.4" +version = "0.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +checksum = "69cce20737498f97b993470a6e536b8523f0af7892a4f928cceb1ac5e52ebe7e" dependencies = [ "generic-array", ] @@ -1504,9 +1494,9 @@ dependencies = [ "hyperlocal", "log", "pin-project-lite", - "rustls 0.20.8", + "rustls 0.20.7", "rustls-native-certs 0.6.2", - "rustls-pemfile 1.0.2", + "rustls-pemfile 1.0.1", "serde", "serde_derive", "serde_json", @@ -1534,9 +1524,9 @@ dependencies = [ [[package]] name = "borsh" -version = "0.10.2" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40f9ca3698b2e4cb7c15571db0abc5551dca417a21ae8140460b50309bb2cc62" +checksum = "822462c1e7b17b31961798a6874b36daea6818e99e0cb7d3b7b0fa3c477751c3" dependencies = [ "borsh-derive", "hashbrown 0.13.2", @@ -1544,9 +1534,9 @@ dependencies = [ [[package]] name = "borsh-derive" -version = "0.10.2" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "598b3eacc6db9c3ee57b22707ad8f6a8d2f6d442bfe24ffeb8cbb70ca59e6a35" +checksum = "37626c9e941a687ee9abef6065b44c379478ae563b7483c613dd705ef1dff59e" dependencies = [ "borsh-derive-internal", "borsh-schema-derive-internal", @@ -1557,9 +1547,9 @@ dependencies = [ [[package]] name = "borsh-derive-internal" -version = "0.10.2" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "186b734fa1c9f6743e90c95d7233c9faab6360d1a96d4ffa19d9cfd1e9350f8a" +checksum = "61820b4c5693eafb998b1e67485423c923db4a75f72585c247bdee32bad81e7b" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -1568,9 +1558,9 @@ dependencies = [ [[package]] name = "borsh-schema-derive-internal" -version = "0.10.2" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99b7ff1008316626f485991b960ade129253d4034014616b94f309a15366cc49" +checksum = "c76cdbfa13def20d1f8af3ae7b3c6771f06352a74221d8851262ac384c122b8e" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -1579,16 +1569,14 @@ dependencies = [ [[package]] name = "bson" -version = "2.6.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6cc3e671fc2a91c8e84396cf14a8ed5a4c929739446aa6a33763b7e7d1568bf4" +checksum = "8746d07211bb12a7c34d995539b4a2acd4e0b0e757de98ce2ab99bcf17443fad" dependencies = [ "ahash 0.7.6", "base64 0.13.1", - "bitvec", "hex", "indexmap", - "js-sys", "lazy_static", "rand 0.8.5", "serde", @@ -1623,26 +1611,25 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.12.0" +version = "3.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d261e256854913907f67ed06efbc3338dfe6179796deefc1ff763fc1aee5535" +checksum = "572f695136211188308f16ad2ca5c851a712c464060ae6974944458eb83880ba" [[package]] name = "bytecheck" -version = "0.6.10" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13fe11640a23eb24562225322cd3e452b93a3d4091d62fab69c70542fcd17d1f" +checksum = "d11cac2c12b5adc6570dad2ee1b87eff4955dac476fe12d81e5fdd352e52406f" dependencies = [ "bytecheck_derive", "ptr_meta", - "simdutf8", ] [[package]] name = "bytecheck_derive" -version = "0.6.10" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e31225543cb46f81a7e224762764f4a6a0f097b1db0b175f69e8065efaa42de5" +checksum = "13e576ebe98e605500b3c8041bb888e966653577172df6dd97398714eb30b9bf" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -1651,9 +1638,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.13.1" +version = "1.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17febce684fd15d89027105661fec94afb475cb995fbc59d2865198446ba2eea" +checksum = "aaa3a8d9a1ca92e282c96a32d6511b695d7d994d1d102ba85d279f9b2756947f" [[package]] name = "byteorder" @@ -1696,6 +1683,12 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "38fcc2979eff34a4b84e1cf9a1e3da42a7d44b3b690a40cdcb23e3d556cfb2e5" +[[package]] +name = "cache-padded" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1db59621ec70f09c5e9b597b220c7a2b43611f4710dc03ceb8748637775692c" + [[package]] name = "cached" version = "0.42.0" @@ -1722,7 +1715,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e10ca87c81aaa3a949dbbe2b5e6c2c45dbc94ba4897e45ea31ff9ec5087be3dc" dependencies = [ "cached_proc_macro_types", - "darling 0.14.4", + "darling 0.14.2", "proc-macro2 1.0.52", "quote 1.0.26", "syn 1.0.109", @@ -1757,9 +1750,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.79" +version = "1.0.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" +checksum = "e9f73505338f7d905b19d18738976aae232eb46b8efc15554ffc56deb5d9ebe4" dependencies = [ "jobserver", ] @@ -1874,9 +1867,9 @@ dependencies = [ [[package]] name = "cipher" -version = "0.4.4" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "773f3b9af64447d2ce9850330c473515014aa235e6a783b02db81ff39e4a3dad" +checksum = "d1873270f8f7942c191139cb8a40fd228da6c3fd2fc376d7e92d47aa14aeb59e" dependencies = [ "crypto-common", "inout", @@ -1917,12 +1910,12 @@ checksum = "c3d7ae14b20b94cb02149ed21a86c423859cbe18dc7ed69845cace50e52b40a5" dependencies = [ "bitflags", "clap_derive", - "clap_lex 0.3.2", + "clap_lex 0.3.0", "is-terminal", "once_cell", "strsim 0.10.0", "termcolor", - "terminal_size", + "terminal_size 0.2.2", ] [[package]] @@ -1950,7 +1943,7 @@ version = "4.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "44bec8e5c9d09e439c4335b1af0abaab56dcf3b94999a936e1bb47b9134288f0" dependencies = [ - "heck 0.4.1", + "heck 0.4.0", "proc-macro-error", "proc-macro2 1.0.52", "quote 1.0.26", @@ -1968,18 +1961,18 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.3.2" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "350b9cf31731f9957399229e9b2adc51eeabdfbe9d71d9a0552275fd12710d09" +checksum = "0d4198f73e42b4936b35b5bb248d81d2b595ecb170da0bac7655c54eedfa8da8" dependencies = [ "os_str_bytes", ] [[package]] name = "clipboard-win" -version = "4.5.0" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7191c27c2357d9b7ef96baac1773290d4ca63b24205b82a3fd8a0637afcf0362" +checksum = "c4ab1b92798304eedc095b53942963240037c0516452cb11aeba709d420b2219" dependencies = [ "error-code", "str-buf", @@ -2080,9 +2073,18 @@ dependencies = [ [[package]] name = "concurrent-queue" -version = "2.1.0" +version = "1.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af4780a44ab5696ea9e28294517f1fffb421a83a25af521333c838635509db9c" +dependencies = [ + "cache-padded", +] + +[[package]] +name = "concurrent-queue" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c278839b831783b70278b14df4d45e1beb1aad306c07bb796637de9a0e323e8e" +checksum = "bd7bef69dc86e3c610e4e7aed41035e2a7ed12e72dd7530f61327a6579a4390b" dependencies = [ "crossbeam-utils", ] @@ -2101,15 +2103,16 @@ dependencies = [ [[package]] name = "console" -version = "0.15.5" +version = "0.15.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3d79fbe8970a77e3e34151cc13d3b3e248aa0faaecb9f6091fa07ebefe5ad60" +checksum = "c050367d967ced717c04b65d8c619d863ef9292ce0c5760028655a2fb298718c" dependencies = [ "encode_unicode 0.3.6", "lazy_static", "libc", + "terminal_size 0.1.17", "unicode-width", - "windows-sys 0.42.0", + "winapi", ] [[package]] @@ -2156,9 +2159,9 @@ checksum = "9d6f2aa4d0537bcc1c74df8755072bd31c1ef1a3a1b85a68e8404a8c353b7b8b" [[package]] name = "const-oid" -version = "0.9.2" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "520fbf3c07483f94e3e3ca9d0cfd913d7718ef2483d2cfd91c0d9e91474ab913" +checksum = "cec318a675afcb6a1ea1d4340e2d377e56e47c266f28043ceccbf4412ddfdd3b" [[package]] name = "convert_case" @@ -2208,9 +2211,9 @@ dependencies = [ [[package]] name = "crc-catalog" -version = "2.2.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cace84e55f07e7301bae1c519df89cdad8cc3cd868413d3fdbdeca9ff3db484" +checksum = "2d0165d2900ae6778e36e80bbc4da3b5eefccee9ba939761f9c2882a5d9af3ff" [[package]] name = "crc32c" @@ -2270,9 +2273,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.7" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf2b3e8478797446514c91ef04bafcb59faba183e621ad488df88983cc14128c" +checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" dependencies = [ "cfg-if", "crossbeam-utils", @@ -2280,9 +2283,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.3" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce6fd6f855243022dcecf8702fef0c297d4338e226845fe067f6341ad9fa0cef" +checksum = "715e8152b692bba2d374b53d4875445368fdf21a94751410af607a5ac677d1fc" dependencies = [ "cfg-if", "crossbeam-epoch", @@ -2291,14 +2294,14 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.14" +version = "0.9.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46bd5f3f85273295a9d14aedfb86f6aadbff6d8f5295c4a9edb08e819dcf5695" +checksum = "01a9af1f4c2ef74bb8aa1f7e19706bc72d03598c8a570bb5de72243c7a9d9d5a" dependencies = [ "autocfg", "cfg-if", "crossbeam-utils", - "memoffset 0.8.0", + "memoffset 0.7.1", "scopeguard", ] @@ -2419,6 +2422,12 @@ dependencies = [ "cipher", ] +[[package]] +name = "cty" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b365fabc795046672053e29c954733ec3b05e4be654ab130fe8f1f94d7051f35" + [[package]] name = "curve25519-dalek" version = "3.2.0" @@ -2434,9 +2443,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.92" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a140f260e6f3f79013b8bfc65e7ce630c9ab4388c6a89c71e07226f49487b72" +checksum = "d4a41a86530d0fe7f5d9ea779916b7cadd2d4f9add748b99c2c029cbbdfaf453" dependencies = [ "cc", "cxxbridge-flags", @@ -2446,9 +2455,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.92" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da6383f459341ea689374bf0a42979739dc421874f112ff26f829b8040b8e613" +checksum = "06416d667ff3e3ad2df1cd8cd8afae5da26cf9cec4d0825040f88b5ca659a2f0" dependencies = [ "cc", "codespan-reporting", @@ -2461,15 +2470,15 @@ dependencies = [ [[package]] name = "cxxbridge-flags" -version = "1.0.92" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90201c1a650e95ccff1c8c0bb5a343213bdd317c6e600a93075bca2eff54ec97" +checksum = "820a9a2af1669deeef27cb271f476ffd196a2c4b6731336011e0ba63e2c7cf71" [[package]] name = "cxxbridge-macro" -version = "1.0.92" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b75aed41bb2e6367cae39e6326ef817a851db13c13e4f3263714ca3cfb8de56" +checksum = "a08a6e2fcc370a089ad3b4aaf54db3b1b4cee38ddabce5896b33eb693275f470" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -2488,12 +2497,12 @@ dependencies = [ [[package]] name = "darling" -version = "0.14.4" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b750cb3417fd1b327431a470f388520309479ab0bf5e323505daf0290cd3850" +checksum = "b0dd3cd20dc6b5a876612a6e5accfe7f3dd883db6d07acfbf14c128f61550dfa" dependencies = [ - "darling_core 0.14.4", - "darling_macro 0.14.4", + "darling_core 0.14.2", + "darling_macro 0.14.2", ] [[package]] @@ -2512,9 +2521,9 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.14.4" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "109c1ca6e6b7f82cc233a97004ea8ed7ca123a9af07a8230878fcfda9b158bf0" +checksum = "a784d2ccaf7c98501746bf0be29b2022ba41fd62a2e622af997a03e9f972859f" dependencies = [ "fnv", "ident_case", @@ -2537,11 +2546,11 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.14.4" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4aab4dbc9f7611d8b55048a3a16d2d010c2c8334e46304b40ac1cc14bf3b48e" +checksum = "7618812407e9402654622dd402b0a89dff9ba93badd6540781526117b92aab7e" dependencies = [ - "darling_core 0.14.4", + "darling_core 0.14.2", "quote 1.0.26", "syn 1.0.109", ] @@ -2663,7 +2672,7 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1a467a65c5e759bce6e65eaf91cc29f466cdc57cb65777bd646872a8a1fd4de" dependencies = [ - "const-oid 0.9.2", + "const-oid 0.9.1", "pem-rfc7468 0.6.0", "zeroize", ] @@ -2730,8 +2739,8 @@ version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" dependencies = [ - "block-buffer 0.10.4", - "const-oid 0.9.2", + "block-buffer 0.10.3", + "const-oid 0.9.1", "crypto-common", "subtle", ] @@ -2822,9 +2831,9 @@ checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" [[package]] name = "duct" -version = "0.13.6" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ae3fc31835f74c2a7ceda3aeede378b0ae2e74c8f1c36559fcc9ae2a4e7d3e" +checksum = "0fc6a0a59ed0888e0041cf708e66357b7ae1a82f1c67247e1f93b5e0818f7d8d" dependencies = [ "libc", "once_cell", @@ -2867,15 +2876,15 @@ dependencies = [ [[package]] name = "either" -version = "1.8.1" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" +checksum = "90e5c1c8368803113bf0c9584fc495a58b86dc8a29edbf8fe877d21d9507e797" [[package]] name = "ena" -version = "0.14.1" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2e5d13ca2353ab7d0230988629def93914a8c4015f621f9b13ed2955614731d" +checksum = "d7402b94a93c24e742487327a7cd839dc9d36fec9de9fb25b09f2dae459f36c3" dependencies = [ "log", ] @@ -2926,7 +2935,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "21cdad81446a7f7dc43f6a77409efeb9733d2fa65553efef6018ef257c959b73" dependencies = [ - "heck 0.4.1", + "heck 0.4.0", "proc-macro2 1.0.52", "quote 1.0.26", "syn 1.0.109", @@ -2938,7 +2947,7 @@ version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" dependencies = [ - "heck 0.4.1", + "heck 0.4.0", "proc-macro2 1.0.52", "quote 1.0.26", "syn 1.0.109", @@ -3007,9 +3016,9 @@ dependencies = [ [[package]] name = "erased-serde" -version = "0.3.25" +version = "0.3.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f2b0c2380453a92ea8b6c8e5f64ecaafccddde8ceab55ff7a8ac1029f894569" +checksum = "54558e0ba96fbe24280072642eceb9d7d442e32c7ec0ea9e7ecd7b4ea2cf4e11" dependencies = [ "serde", ] @@ -3112,9 +3121,9 @@ dependencies = [ [[package]] name = "fastrand" -version = "1.9.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" +checksum = "a7a407cfaa3385c4ae6b23e84623d48c2798d06e3e6a1878f7f59f17b3f86499" dependencies = [ "instant", ] @@ -3150,14 +3159,14 @@ dependencies = [ [[package]] name = "filetime" -version = "0.2.20" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a3de6e8d11b22ff9edc6d916f890800597d60f8b2da1caf2955c274638d6412" +checksum = "4b9663d381d07ae25dc88dbdf27df458faa83a9b25336bcac83d5e452b5fc9d3" dependencies = [ "cfg-if", "libc", "redox_syscall", - "windows-sys 0.45.0", + "windows-sys 0.42.0", ] [[package]] @@ -3217,7 +3226,7 @@ dependencies = [ "futures-core", "futures-sink", "pin-project", - "spin 0.9.6", + "spin 0.9.4", ] [[package]] @@ -3250,6 +3259,12 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "fs_extra" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2022715d62ab30faffd124d40b76f4134a550a87792276512b18d63272333394" + [[package]] name = "fsevent-sys" version = "4.1.0" @@ -3269,12 +3284,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "funty" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" - [[package]] name = "futures" version = "0.1.31" @@ -3429,9 +3438,9 @@ dependencies = [ [[package]] name = "ghost" -version = "0.1.8" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69e0cd8a998937e25c6ba7cc276b96ec5cc3f4dc4ab5de9ede4fb152bdd5c5eb" +checksum = "eb19fe8de3ea0920d282f7b77dd4227aea6b8b999b42cdf0ca41b2472b14443a" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -3531,7 +3540,7 @@ checksum = "4e55df64cc702c4ad6647f8df13a799ad11688a3781fadf5045f7ba12733fa9b" dependencies = [ "graphql-introspection-query", "graphql-parser", - "heck 0.4.1", + "heck 0.4.0", "lazy_static", "proc-macro2 1.0.52", "quote 1.0.26", @@ -3621,7 +3630,7 @@ version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" dependencies = [ - "ahash 0.8.3", + "ahash 0.8.2", ] [[package]] @@ -3684,9 +3693,9 @@ dependencies = [ [[package]] name = "heck" -version = "0.4.1" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" +checksum = "2540771e65fc8cb83cd6e8a237f70c319bd5c29f78ed1084ba5d50eeac86f7f9" [[package]] name = "heim" @@ -3714,7 +3723,7 @@ dependencies = [ "lazy_static", "libc", "mach", - "nix 0.23.2", + "nix 0.23.1", "pin-utils", "uom", "winapi", @@ -3796,7 +3805,7 @@ dependencies = [ "heim-runtime", "libc", "macaddr", - "nix 0.23.2", + "nix 0.23.1", "widestring 0.4.3", "winapi", ] @@ -3830,12 +3839,6 @@ dependencies = [ "libc", ] -[[package]] -name = "hermit-abi" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" - [[package]] name = "hex" version = "0.4.3" @@ -3990,14 +3993,14 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" +checksum = "59df7c4e19c950e6e0e868dcc0a300b09a9b88e9ec55bd879ca819087a77355d" dependencies = [ "http", "hyper", "log", - "rustls 0.20.8", + "rustls 0.20.7", "rustls-native-certs 0.6.2", "tokio", "tokio-rustls", @@ -4136,9 +4139,9 @@ checksum = "f551f8c3a39f68f986517db0d1759de85881894fdc7db798bd2a9df9cb04b7fc" [[package]] name = "inherent" -version = "1.0.5" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca36b2075a0e4eaa6fa1cf0ebf4cf6fc7b6a4834783dea1c25ba68245744a053" +checksum = "a036328c11e86e024522cb1e9b78ba9df3e316995e004e98854a18e4a326d2e1" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -4196,12 +4199,18 @@ dependencies = [ [[package]] name = "io-lifetimes" -version = "1.0.6" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cfa919a82ea574332e2de6e74b4c36e74d41982b335080fa59d4ef31be20fdf3" +checksum = "59ce5ef949d49ee85593fc4d3f3f95ad61657076395cbbce23e2121fc5542074" + +[[package]] +name = "io-lifetimes" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46112a93252b123d31a119a8d1a1ac19deac4fac6e0e8b0df58f0d4e5870e63c" dependencies = [ "libc", - "windows-sys 0.45.0", + "windows-sys 0.42.0", ] [[package]] @@ -4227,9 +4236,9 @@ dependencies = [ [[package]] name = "ipnet" -version = "2.7.1" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30e22bd8629359895450b59ea7a776c850561b96a3b1d31321c1949d9e6c9146" +checksum = "f88c5561171189e69df9d98bcf18fd5f9558300f7ea7b801eb8a0fd748bd8745" [[package]] name = "ipnetwork" @@ -4242,14 +4251,14 @@ dependencies = [ [[package]] name = "is-terminal" -version = "0.4.4" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21b6b32576413a8e69b90e952e4a026476040d81017b80445deda5f2d3921857" +checksum = "927609f78c2913a6f6ac3c27a4fe87f43e2a35367c0c4b0f8265e8f49a104330" dependencies = [ - "hermit-abi 0.3.1", - "io-lifetimes", - "rustix", - "windows-sys 0.45.0", + "hermit-abi 0.2.6", + "io-lifetimes 1.0.3", + "rustix 0.36.4", + "windows-sys 0.42.0", ] [[package]] @@ -4269,9 +4278,9 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.6" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "453ad9f582a441959e5f0d088b02ce04cfe8d51a8eaf077f12ac6d3e94164ca6" +checksum = "4217ad341ebadf8d8e724e264f13e593e0648f5b3e94b3896a5df283be015ecc" [[package]] name = "jni" @@ -4295,18 +4304,18 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.26" +version = "0.1.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "936cfd212a0155903bcbc060e316fb6cc7cbf2e1907329391ebadc1fe0ce77c2" +checksum = "068b1ee6743e4d11fb9c6a1e6064b3693a1b600e7f5f5988047d98b3dc9fb90b" dependencies = [ "libc", ] [[package]] name = "js-sys" -version = "0.3.61" +version = "0.3.60" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "445dde2150c55e483f3d8416706b97ec8e8237c307e5b7b4b8dd15e6af2a0730" +checksum = "49409df3e3bf0856b916e2ceaca09ee28e6871cf7d9ce97a692cacfdb2a25a47" dependencies = [ "wasm-bindgen", ] @@ -4319,9 +4328,9 @@ checksum = "078e285eafdfb6c4b434e0d31e8cfcb5115b651496faca5749b88fafd4f23bfd" [[package]] name = "json-patch" -version = "0.2.7" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3fa5a61630976fc4c353c70297f2e93f1930e3ccee574d59d618ccbd5154ce" +checksum = "f995a3c8f2bc3dd52a18a583e90f9ec109c047fa1603a853e46bcda14d2e279d" dependencies = [ "serde", "serde_json", @@ -4507,7 +4516,7 @@ version = "0.75.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7769af142ee2e46bfa44bd393cf7f40b9d8b80d2e11f6317399551ed17760beb" dependencies = [ - "ahash 0.8.3", + "ahash 0.8.2", "backoff", "derivative", "futures 0.3.27", @@ -4596,9 +4605,9 @@ checksum = "99227334921fae1a979cf0bfdfcc6b3e5ce376ef57e16fb6fb3ea2ed6095f80c" [[package]] name = "libflate" -version = "1.3.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97822bf791bd4d5b403713886a5fbe8bf49520fe78e323b0dc480ca1a03e50b0" +checksum = "05605ab2bce11bcfc0e9c635ff29ef8b2ea83f29be257ee7d730cac3ee373093" dependencies = [ "adler32", "crc32fast", @@ -4607,9 +4616,9 @@ dependencies = [ [[package]] name = "libflate_lz77" -version = "1.2.0" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a52d3a8bfc85f250440e4424db7d857e241a3aebbbe301f3eb606ab15c39acbf" +checksum = "39a734c0493409afcd49deee13c006a04e3586b9761a03543c6272c9c51f2f5a" dependencies = [ "rle-decode-fast", ] @@ -4634,9 +4643,9 @@ dependencies = [ [[package]] name = "link-cplusplus" -version = "1.0.8" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" +checksum = "9272ab7b96c9046fbc5bc56c06c117cb639fe2d509df0c421cad82d2915cf369" dependencies = [ "cc", ] @@ -4658,9 +4667,15 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.1.4" +version = "0.0.46" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4d2456c373231a208ad294c33dc5bff30051eafd954cd4caae83a712b12854d" + +[[package]] +name = "linux-raw-sys" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f051f77a7c8e6957c0696eac88f26b0117e54f52d3fc682ab19397a8812846a4" +checksum = "8f9f08d8963a6c613f4b1a78f4f4a4dbfadf8e6545b2d72861731e4858b8b47f" [[package]] name = "listenfd" @@ -4759,9 +4774,9 @@ dependencies = [ [[package]] name = "luajit-src" -version = "210.4.5+resty2cf5186" +version = "210.4.3+resty8384278" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27b7992a40e602786272d84c6f2beca44a588ededcfd57b48ec6f82008a7cb97" +checksum = "19ee5d5afddf1ec76ffa55ca7c3001f2f8a703834beba53c56a38ea6641cef44" dependencies = [ "cc", ] @@ -4827,9 +4842,9 @@ dependencies = [ [[package]] name = "matches" -version = "0.1.10" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2532096657941c2fea9c289d370a250971c689d4f143798ff67113ec042024a5" +checksum = "a3e378b66a060d48947b590737b30a1be76706c8dd7b8ba0f2fe3989c68a853f" [[package]] name = "matchit" @@ -4906,15 +4921,6 @@ dependencies = [ "autocfg", ] -[[package]] -name = "memoffset" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d61c719bcfbcf5d62b3a09efa6088de8c54bc0bfcd3ea7ae39fcc186108b8de1" -dependencies = [ - "autocfg", -] - [[package]] name = "metrics" version = "0.20.1" @@ -5007,14 +5013,14 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.6" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b9d9a46eff5b4ff64b45a9e316a6d1e0bc719ef429cbec4dc630684212bfdf9" +checksum = "e5d732bc30207a6423068df043e3d02e0735b155ad7ce1a6f76fe2baa5b158de" dependencies = [ "libc", "log", "wasi 0.11.0+wasi-snapshot-preview1", - "windows-sys 0.45.0", + "windows-sys 0.42.0", ] [[package]] @@ -5064,8 +5070,8 @@ dependencies = [ "percent-encoding", "rand 0.8.5", "rustc_version_runtime", - "rustls 0.20.8", - "rustls-pemfile 1.0.2", + "rustls 0.20.7", + "rustls-pemfile 1.0.1", "serde", "serde_bytes", "serde_with 1.14.0", @@ -5100,7 +5106,7 @@ dependencies = [ "log", "memchr", "mime", - "spin 0.9.6", + "spin 0.9.4", "version_check", ] @@ -5215,9 +5221,9 @@ dependencies = [ [[package]] name = "nix" -version = "0.23.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f3790c00a0150112de0f4cd161e3d7fc4b2d8a5542ffc35f099a2562aecb35c" +checksum = "9f866317acbd3a240710c63f065ffb1e4fd466259045ccb504130b7f668f35c6" dependencies = [ "bitflags", "cc", @@ -5370,9 +5376,9 @@ dependencies = [ [[package]] name = "num-complex" -version = "0.4.3" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02e0d21255c828d6f128a1e41534206671e8c3ea0c62f32291e808dc82cff17d" +checksum = "7ae39348c8bc5fbd7f40c727a9925f03517afd2ab27d46702108b6a7e5414c19" dependencies = [ "num-traits", ] @@ -5432,11 +5438,11 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.15.0" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fac9e2da13b5eb447a6ce3d392f23a29d8694bff781bf03a16cd9ac8697593b" +checksum = "f6058e64324c71e02bc2b150e4f3bc8286db6c83092132ffa3f6b1eab0f9def5" dependencies = [ - "hermit-abi 0.2.6", + "hermit-abi 0.1.19", "libc", ] @@ -5455,7 +5461,7 @@ version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dcbff9bc912032c62bf65ef1d5aea88983b420f4f839db1e9b0c281a25c9c799" dependencies = [ - "proc-macro-crate 1.3.1", + "proc-macro-crate 1.2.1", "proc-macro2 1.0.52", "quote 1.0.26", "syn 1.0.109", @@ -5576,7 +5582,7 @@ dependencies = [ "parking_lot", "percent-encoding", "pin-project", - "quick-xml 0.27.1", + "quick-xml", "reqsign", "reqwest", "serde", @@ -5589,9 +5595,9 @@ dependencies = [ [[package]] name = "openidconnect" -version = "2.5.1" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98dd5b7049bac4fdd2233b8c9767d42c05da8006fdb79cc903258556d2b18009" +checksum = "87af7097640fedbe64718ac1c9b0549d72da747a3f527cd089215f96c6f691d5" dependencies = [ "base64 0.13.1", "chrono", @@ -5607,18 +5613,15 @@ dependencies = [ "serde_derive", "serde_json", "serde_path_to_error", - "serde_plain", - "serde_with 1.14.0", - "subtle", "thiserror", "url", ] [[package]] name = "openssl" -version = "0.10.46" +version = "0.10.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd2523381e46256e40930512c7fd25562b9eae4812cb52078f155e87217c9d1e" +checksum = "b102428fd03bc5edf97f62620f7298614c45cedf287c271e7ed450bbaf83f2e1" dependencies = [ "bitflags", "cfg-if", @@ -5648,18 +5651,18 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "openssl-src" -version = "111.25.1+1.1.1t" +version = "111.25.0+1.1.1t" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ef9a9cc6ea7d9d5e7c4a913dc4b48d0e359eddf01af1dfec96ba7064b4aba10" +checksum = "3173cd3626c43e3854b1b727422a276e568d9ec5fe8cec197822cf52cfb743d6" dependencies = [ "cc", ] [[package]] name = "openssl-sys" -version = "0.9.81" +version = "0.9.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "176be2629957c157240f68f61f2d0053ad3a4ecfdd9ebf1e6521d18d9635cf67" +checksum = "23bbbf7854cd45b83958ebe919f0e8e516793727652e27fda10a8384cfc790b7" dependencies = [ "autocfg", "cc", @@ -5726,12 +5729,12 @@ dependencies = [ [[package]] name = "os_pipe" -version = "1.1.3" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a53dbb20faf34b16087a931834cba2d7a73cc74af2b7ef345a4c8324e2409a12" +checksum = "fb233f06c2307e1f5ce2ecad9f8121cffbbee2c95428f44ea85222e460d0d213" dependencies = [ "libc", - "windows-sys 0.45.0", + "winapi", ] [[package]] @@ -5785,15 +5788,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.7" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9069cbb9f99e3a5083476ccb29ceb1de18b9118cafa53e90c9551235de2b9521" +checksum = "4dc9e0dc2adc1c69d09143aff38d3d30c5c3f0df0dad82e6d25547af174ebec0" dependencies = [ "cfg-if", "libc", "redox_syscall", "smallvec", - "windows-sys 0.45.0", + "windows-sys 0.42.0", ] [[package]] @@ -5828,9 +5831,9 @@ checksum = "9e9ed2178b0575fff8e1b83b58ba6f75e727aafac2e1b6c795169ad3b17eb518" [[package]] name = "pem" -version = "1.1.1" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8835c273a76a90455d7344889b0964598e3316e2a79ede8e36f16bdcf2228b8" +checksum = "03c64931a1a212348ec4f3b4362585eca7159d0d09cbdf4a7f74f02173596fd4" dependencies = [ "base64 0.13.1", ] @@ -5905,9 +5908,9 @@ dependencies = [ [[package]] name = "petgraph" -version = "0.6.3" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dd7d28ee937e54fe3080c91faa1c3a46c06de6252988a7f4592ba2310ef22a4" +checksum = "e6d5014253a1331579ce62aa67443b4a658c5e7dd03d4bc6d302b94474888143" dependencies = [ "fixedbitset", "indexmap", @@ -6086,25 +6089,23 @@ dependencies = [ [[package]] name = "polling" -version = "2.6.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e1f879b2998099c2d69ab9605d145d5b661195627eccc680002c4918a7fb6fa" +checksum = "9f7d73f1eaed1ca1fb37b54dcc9b38e3b17d6c7b8ecb7abfffcac8d0351f17d4" dependencies = [ "autocfg", - "bitflags", "cfg-if", - "concurrent-queue", "libc", "log", - "pin-project-lite", - "windows-sys 0.45.0", + "wepoll-ffi", + "windows-sys 0.42.0", ] [[package]] name = "portable-atomic" -version = "0.3.19" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26f6a7b87c2e435a3241addceeeff740ff8b7e76b74c13bf9acb17fa454ea00b" +checksum = "15eb2c6e362923af47e13c23ca5afb859e83d54452c55b0b9ac763b8f7c1ac16" [[package]] name = "portpicker" @@ -6170,9 +6171,9 @@ checksum = "925383efa346730478fb4838dbe9137d2a47675ad789c546d150a6e1dd4ab31c" [[package]] name = "predicates" -version = "2.1.5" +version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59230a63c37f3e18569bdb90e4a89cbf5bf8b06fea0b84e65ea10cc4df47addd" +checksum = "ed6bd09a7f7e68f3f0bf710fb7ab9c4615a488b58b5f653382a687701e458c92" dependencies = [ "difflib", "itertools", @@ -6181,15 +6182,15 @@ dependencies = [ [[package]] name = "predicates-core" -version = "1.0.6" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b794032607612e7abeb4db69adb4e33590fa6cf1149e95fd7cb00e634b92f174" +checksum = "72f883590242d3c6fc5bf50299011695fa6590c2c70eac95ee1bdb9a733ad1a2" [[package]] name = "predicates-tree" -version = "1.0.8" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cfd78a1cd6926a74e7f263eb466b765c79409b176be05c3676fac27df378e96" +checksum = "54ff541861505aabf6ea722d2131ee980b8276e10a1297b94e896dd8b621850d" dependencies = [ "predicates-core", "termtree", @@ -6208,9 +6209,9 @@ dependencies = [ [[package]] name = "prettyplease" -version = "0.1.25" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8646e95016a7a6c4adea95bafa8a16baab64b583356217f2c85db4a39d9a86" +checksum = "c142c0e46b57171fe0c528bee8c5b7569e80f0c17e377cd0e30ea57dbc11bb51" dependencies = [ "proc-macro2 1.0.52", "syn 1.0.109", @@ -6241,12 +6242,13 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "1.3.1" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f4c021e1093a56626774e81216a4ce732a735e5bad4868a03f3ed65ca0c3919" +checksum = "eda0fc3b0fb7c975631757e14d9049da17374063edb6ebbcbc54d880d4fe94e9" dependencies = [ "once_cell", - "toml_edit", + "thiserror", + "toml 0.5.11", ] [[package]] @@ -6275,9 +6277,9 @@ dependencies = [ [[package]] name = "proc-macro-hack" -version = "0.5.20+deprecated" +version = "0.5.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" +checksum = "dbf0c48bc1d91375ae5c3cd81e3722dff1abcf81a30960240640d223f59fe0e5" [[package]] name = "proc-macro-nested" @@ -6368,7 +6370,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2c828f93f5ca4826f97fedcbd3f9a536c16b12cff3dbbb4a007f932bbad95b12" dependencies = [ "bytes 1.4.0", - "heck 0.4.1", + "heck 0.4.0", "itertools", "lazy_static", "log", @@ -6508,16 +6510,6 @@ dependencies = [ "serde", ] -[[package]] -name = "quick-xml" -version = "0.28.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a306703b4ad75d304a1bbc17d91d4399993caa163ad5028ffb044e5152ffcdd" -dependencies = [ - "memchr", - "serde", -] - [[package]] name = "quickcheck" version = "1.0.3" @@ -6564,12 +6556,6 @@ version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a24039f627d8285853cc90dcddf8c1ebfaa91f834566948872b225b9a28ed1b6" -[[package]] -name = "radium" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" - [[package]] name = "radix_trie" version = "0.2.1" @@ -6672,18 +6658,21 @@ dependencies = [ [[package]] name = "raw-cpuid" -version = "10.7.0" +version = "10.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c297679cb867470fa8c9f67dbba74a78d78e3e98d7cf2b08d6d71540f797332" +checksum = "a6823ea29436221176fe662da99998ad3b4db2c7f31e7b6f5fe43adccd6320bb" dependencies = [ "bitflags", ] [[package]] name = "raw-window-handle" -version = "0.5.1" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f851a03551ceefd30132e447f07f96cb7011d6b658374f3aed847333adb5559" +checksum = "ed7e3d950b66e19e0c372f3fa3fbbcf85b1746b571f74e0c2af6042a5c93420a" +dependencies = [ + "cty", +] [[package]] name = "rawpointer" @@ -6693,19 +6682,20 @@ checksum = "60a357793950651c4ed0f3f52338f53b2f809f32d83a07f72909fa13e4c6c1e3" [[package]] name = "rayon" -version = "1.7.0" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d2df5196e37bcc87abebc0053e20787d73847bb33134a69841207dd0a47f03b" +checksum = "1e060280438193c554f654141c9ea9417886713b7acd75974c85b18a69a88e0b" dependencies = [ + "crossbeam-deque", "either", "rayon-core", ] [[package]] name = "rayon-core" -version = "1.11.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b8f95bd6966f5c87776639160a66bd8ab9895d9d4ab01ddba9fc60661aebe8d" +checksum = "cac410af5d00ab6884528b4ab69d1e8e146e8d471201800fa1b4524126de6ad3" dependencies = [ "crossbeam-channel", "crossbeam-deque", @@ -6838,9 +6828,9 @@ dependencies = [ [[package]] name = "reqsign" -version = "0.8.5" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7db6d8d2cd7fa61403d14de670f98d7cedac38143681c124943d7bb69258b3a" +checksum = "ef4d5fefeaaa1e64f4aabb79da4ea68bf6d0e7935ad927728280d2a8e95735fc" dependencies = [ "anyhow", "backon", @@ -6855,7 +6845,7 @@ dependencies = [ "log", "once_cell", "percent-encoding", - "quick-xml 0.28.0", + "quick-xml", "rand 0.8.5", "rsa", "rust-ini", @@ -6893,8 +6883,8 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.20.8", - "rustls-pemfile 1.0.2", + "rustls 0.20.7", + "rustls-pemfile 1.0.1", "serde", "serde_json", "serde_urlencoded", @@ -7030,9 +7020,9 @@ dependencies = [ [[package]] name = "rsa" -version = "0.8.2" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55a77d189da1fee555ad95b7e50e7457d91c0e089ec68ca69ad2989413bbdab4" +checksum = "89b3896c9b7790b70a9aa314a30e4ae114200992a19c96cbe0ca6070edd32ab8" dependencies = [ "byteorder", "digest 0.10.6", @@ -7113,16 +7103,30 @@ dependencies = [ [[package]] name = "rustix" -version = "0.36.9" +version = "0.35.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd5c6ff11fecd55b40746d1995a02f2eb375bf8c00d192d521ee09f42bef37bc" +checksum = "727a1a6d65f786ec22df8a81ca3121107f235970dc1705ed681d3e6e8b9cd5f9" dependencies = [ "bitflags", "errno", - "io-lifetimes", + "io-lifetimes 0.7.5", "libc", - "linux-raw-sys", - "windows-sys 0.45.0", + "linux-raw-sys 0.0.46", + "windows-sys 0.42.0", +] + +[[package]] +name = "rustix" +version = "0.36.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb93e85278e08bb5788653183213d3a60fc242b10cb9be96586f5a73dcb67c23" +dependencies = [ + "bitflags", + "errno", + "io-lifetimes 1.0.3", + "libc", + "linux-raw-sys 0.1.3", + "windows-sys 0.42.0", ] [[package]] @@ -7140,9 +7144,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.8" +version = "0.20.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fff78fc74d175294f4e83b28343315ffcfb114b156f0185e9741cb5570f50e2f" +checksum = "539a2bfe908f471bfa933876bd1eb6a19cf2176d375f82ef7f99530a40e48c2c" dependencies = [ "log", "ring", @@ -7169,7 +7173,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0167bac7a9f490495f3c33013e7722b53cb087ecbe082fb0c6387c96f634ea50" dependencies = [ "openssl-probe", - "rustls-pemfile 1.0.2", + "rustls-pemfile 1.0.1", "schannel", "security-framework", ] @@ -7185,18 +7189,18 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "1.0.2" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d194b56d58803a43635bdc398cd17e383d6f71f9182b9a192c127ca42494a59b" +checksum = "0864aeff53f8c05aa08d86e5ef839d3dfcf07aeba2db32f12db0ef716e87bd55" dependencies = [ - "base64 0.21.0", + "base64 0.13.1", ] [[package]] name = "rustversion" -version = "1.0.12" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" +checksum = "97477e48b4cf8603ad5f7aaf897467cf42ab4218a38ef76fb14c2d6773a6d6a8" [[package]] name = "rusty-fork" @@ -7290,9 +7294,9 @@ checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" [[package]] name = "scratch" -version = "1.0.5" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" +checksum = "9c8132065adcfd6e02db789d9285a0deb2f3fcb04002865ab67d5fb103533898" [[package]] name = "sct" @@ -7473,27 +7477,18 @@ dependencies = [ [[package]] name = "serde_nanos" -version = "0.1.3" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ae801b7733ca8d6a2b580debe99f67f36826a0f5b8a36055dc6bc40f8d6bc71" +checksum = "e44969a61f5d316be20a42ff97816efb3b407a924d06824c3d8a49fa8450de0e" dependencies = [ "serde", ] [[package]] name = "serde_path_to_error" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db0969fff533976baadd92e08b1d102c5a3d8a8049eadfd69d4d1e3c5b2ed189" -dependencies = [ - "serde", -] - -[[package]] -name = "serde_plain" -version = "1.0.1" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6018081315db179d0ce57b1fe4b62a12a0028c9cf9bbef868c9cf477b3c34ae" +checksum = "184c643044780f7ceb59104cef98a5a6f12cb2288a7bc701ab93a362b49fd47d" dependencies = [ "serde", ] @@ -7511,9 +7506,9 @@ dependencies = [ [[package]] name = "serde_repr" -version = "0.1.11" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "395627de918015623b32e7669714206363a7fc00382bf477e72c1f7533e8eafc" +checksum = "1fe39d9fbb0ebf5eb2c7cb7e2a47e4f462fad1379f1166b8ae49ad9eae89a7ca" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -7585,7 +7580,7 @@ version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7cbcd6104f8a4ab6af7f6be2a0da6be86b9de3c401f6e86bb856ab2af739232f" dependencies = [ - "darling 0.14.4", + "darling 0.14.2", "proc-macro2 1.0.52", "quote 1.0.26", "syn 1.0.109", @@ -7683,9 +7678,9 @@ dependencies = [ [[package]] name = "shared_child" -version = "1.0.0" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0d94659ad3c2137fef23ae75b03d5241d633f8acded53d672decfa0e6e0caef" +checksum = "6be9f7d5565b1483af3e72975e2dee33879b3b86bd48c0929fccf6585d79e65a" dependencies = [ "libc", "winapi", @@ -7693,9 +7688,9 @@ dependencies = [ [[package]] name = "signal-hook" -version = "0.3.15" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "732768f1176d21d09e076c23a93123d40bba92d50c4058da34d45c8de8e682b9" +checksum = "a253b5e89e2698464fc26b545c9edceb338e18a89effeeecfea192c3025be29d" dependencies = [ "libc", "signal-hook-registry", @@ -7714,9 +7709,9 @@ dependencies = [ [[package]] name = "signal-hook-registry" -version = "1.4.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1" +checksum = "e51e73328dc4ac0c7ccbda3a494dfa03df1de2f46018127f60c693f2648455b0" dependencies = [ "libc", ] @@ -7758,12 +7753,6 @@ dependencies = [ "outref 0.1.0", ] -[[package]] -name = "simdutf8" -version = "0.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" - [[package]] name = "similar" version = "2.2.1" @@ -7816,9 +7805,9 @@ checksum = "ceb945e54128e09c43d8e4f1277851bd5044c6fc540bbaa2ad888f60b3da9ae7" [[package]] name = "slab" -version = "0.4.8" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6528351c9bc8ab22353f9d776db39a20288e8d6c37ef8cfe3317cf875eecfc2d" +checksum = "4614a76b2a8be0058caa9dbbaf66d988527d86d003c11a94fbd335d7661edcef" dependencies = [ "autocfg", ] @@ -7834,9 +7823,9 @@ dependencies = [ [[package]] name = "smol" -version = "1.3.0" +version = "1.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13f2b548cd8447f8de0fdf1c592929f70f4fc7039a05e47404b0d096ec6987a1" +checksum = "85cf3b5351f3e783c1d79ab5fc604eeed8b8ae9abd36b166e8b87a089efd85e4" dependencies = [ "async-channel", "async-executor", @@ -7847,6 +7836,7 @@ dependencies = [ "async-process", "blocking", "futures-lite", + "once_cell", ] [[package]] @@ -7883,7 +7873,7 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "475b3bbe5245c26f2d8a6f62d67c1f30eb9fffeccee721c45d162c3ebbdf81b2" dependencies = [ - "heck 0.4.1", + "heck 0.4.0", "proc-macro2 1.0.52", "quote 1.0.26", "syn 1.0.109", @@ -7897,9 +7887,9 @@ checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" [[package]] name = "socket2" -version = "0.4.9" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662" +checksum = "02e2d2db9033d13a1567121ddd7a095ee144db4e1ca1b1bda3419bc0da294ebd" dependencies = [ "libc", "winapi", @@ -7913,9 +7903,9 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "spin" -version = "0.9.6" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5d6e0250b93c8427a177b849d144a96d5acc57006149479403d7861ab721e34" +checksum = "7f6002a767bff9e83f8eeecf883ecb8011875a21ae8da43bffb817a57e78cc09" dependencies = [ "lock_api", ] @@ -7964,9 +7954,9 @@ dependencies = [ [[package]] name = "string_cache" -version = "0.8.7" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f91138e76242f575eb1d3b38b4f1362f10d3a43f47d182a5b359af488a02293b" +checksum = "213494b7a2b503146286049378ce02b482200519accc31872ee8be91fa820a08" dependencies = [ "new_debug_unreachable", "once_cell", @@ -8042,7 +8032,7 @@ version = "0.24.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" dependencies = [ - "heck 0.4.1", + "heck 0.4.0", "proc-macro2 1.0.52", "quote 1.0.26", "rustversion", @@ -8089,9 +8079,9 @@ dependencies = [ [[package]] name = "sync_wrapper" -version = "0.1.2" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" +checksum = "20518fe4a4c9acf048008599e464deb21beeae3d3578418951a189c235a7a9a8" [[package]] name = "synstructure" @@ -8134,12 +8124,6 @@ version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f764005d11ee5f36500a149ace24e00e3da98b0158b3e2d53a7495660d3f4d60" -[[package]] -name = "tap" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" - [[package]] name = "tcp-stream" version = "0.24.4" @@ -8148,7 +8132,7 @@ checksum = "09a4b0a70bac0a58ca6a7659d1328e34ee462339c70b0fa49f72bad1f278910a" dependencies = [ "cfg-if", "native-tls", - "rustls-pemfile 1.0.2", + "rustls-pemfile 1.0.1", ] [[package]] @@ -8166,7 +8150,7 @@ dependencies = [ "cfg-if", "fastrand", "redox_syscall", - "rustix", + "rustix 0.36.4", "windows-sys 0.42.0", ] @@ -8192,19 +8176,29 @@ dependencies = [ [[package]] name = "terminal_size" -version = "0.2.5" +version = "0.1.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c9afddd2cec1c0909f06b00ef33f94ab2cc0578c4a610aa208ddfec8aa2b43a" +checksum = "633c1a546cee861a1a6d0dc69ebeca693bf4296661ba7852b9d21d159e0506df" dependencies = [ - "rustix", - "windows-sys 0.45.0", + "libc", + "winapi", +] + +[[package]] +name = "terminal_size" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40ca90c434fd12083d1a6bdcbe9f92a14f96c8a1ba600ba451734ac334521f7a" +dependencies = [ + "rustix 0.35.13", + "windows-sys 0.42.0", ] [[package]] name = "termtree" -version = "0.4.1" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" +checksum = "95059e91184749cb66be6dc994f67f182b6d897cb3df74a5bf66b5e709295fd8" [[package]] name = "test-case" @@ -8290,21 +8284,21 @@ dependencies = [ [[package]] name = "thread_local" -version = "1.1.7" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdd6f064ccff2d6567adcb3873ca630700f00b5ad3f060c25b5dcfd9a4ce152" +checksum = "5516c27b78311c50bf42c071425c560ac799b11c30b31f87e3081965fe5e0180" dependencies = [ - "cfg-if", "once_cell", ] [[package]] name = "tikv-jemalloc-sys" -version = "0.5.3+5.3.0-patched" +version = "0.5.2+5.3.0-patched" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a678df20055b43e57ef8cddde41cdfda9a3c1a060b67f4c5836dfb1d78543ba8" +checksum = "ec45c14da997d0925c7835883e4d5c181f196fa142f8c19d7643d1e9af2592c3" dependencies = [ "cc", + "fs_extra", "libc", ] @@ -8377,9 +8371,9 @@ dependencies = [ [[package]] name = "tinyvec_macros" -version = "0.1.1" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" +checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" @@ -8425,9 +8419,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.8.2" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d266c00fde287f55d3f1c3e96c500c362a2b8c695076ec180f27918820bc6df8" +checksum = "9724f9a975fb987ef7a3cd9be0350edcbe130698af5b8f7a631e23d42d052484" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -8436,9 +8430,9 @@ dependencies = [ [[package]] name = "tokio-native-tls" -version = "0.3.1" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbae76ab933c85776efabc971569dd6119c580d8f5d448769dec1764bf796ef2" +checksum = "f7d995660bd2b7f8c1568414c1126076c13fbb725c40112dc0120b78eb9b717b" dependencies = [ "native-tls", "tokio", @@ -8486,7 +8480,7 @@ version = "0.23.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" dependencies = [ - "rustls 0.20.8", + "rustls 0.20.7", "tokio", "webpki 0.22.0", ] @@ -8536,7 +8530,7 @@ checksum = "54319c93411147bced34cb5609a80e0a8e44c5999c93903a81cd866630ec0bfd" dependencies = [ "futures-util", "log", - "rustls 0.20.8", + "rustls 0.20.7", "tokio", "tungstenite 0.18.0", ] @@ -8588,9 +8582,9 @@ dependencies = [ [[package]] name = "toml_edit" -version = "0.19.7" +version = "0.19.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc18466501acd8ac6a3f615dd29a3438f8ca6bb3b19537138b3106e575621274" +checksum = "08de71aa0d6e348f070457f85af8bd566e2bc452156a423ddf22861b3a953fae" dependencies = [ "indexmap", "serde", @@ -8623,7 +8617,7 @@ dependencies = [ "prost", "prost-derive", "rustls-native-certs 0.6.2", - "rustls-pemfile 1.0.2", + "rustls-pemfile 1.0.1", "tokio", "tokio-rustls", "tokio-stream", @@ -8683,6 +8677,7 @@ dependencies = [ "http-body", "http-range-header", "pin-project-lite", + "tower", "tower-layer", "tower-service", "tracing 0.1.37", @@ -8988,9 +8983,9 @@ dependencies = [ [[package]] name = "try-lock" -version = "0.2.4" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" +checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" [[package]] name = "tui" @@ -9066,9 +9061,9 @@ dependencies = [ [[package]] name = "typenum" -version = "1.16.0" +version = "1.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba" +checksum = "dcf81ac59edc17cc8697ff311e8f5ef2d99fcbd9817b34cec66f90b6c3dfd987" [[package]] name = "typetag" @@ -9131,15 +9126,15 @@ dependencies = [ [[package]] name = "unicode-bidi" -version = "0.3.11" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "524b68aca1d05e03fdf03fcdce2c6c94b6daf6d16861ddaa7e4f2b6638a9052c" +checksum = "099b7128301d285f79ddd55b9a83d5e6b9e97c92e0ea0daebee7263e932de992" [[package]] name = "unicode-ident" -version = "1.0.8" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5464a87b239f13a63a501f2701565754bae92d243d4bb7eb12f6d57d2269bf4" +checksum = "6ceab39d59e4c9499d4e5a8ee0e2735b891bb7308ac83dfb4e80cad195c9f6f3" [[package]] name = "unicode-normalization" @@ -9152,9 +9147,9 @@ dependencies = [ [[package]] name = "unicode-segmentation" -version = "1.10.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dd624098567895118886609431a7c3b8f516e41d30e0643f03d94592a147e36" +checksum = "0fdbf052a0783de01e944a6ce7a8cb939e295b1e7be835a1112c3b9a7f047a5a" [[package]] name = "unicode-width" @@ -9251,9 +9246,9 @@ checksum = "5190c9442dcdaf0ddd50f37420417d219ae5261bbf5db120d0f9bab996c9cba1" [[package]] name = "utf8parse" -version = "0.2.1" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" +checksum = "936e4b492acfd135421d8dca4b1aa80a7bfc26e702ef3af710e0752684df5372" [[package]] name = "uuid" @@ -10129,9 +10124,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.34" +version = "0.4.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f219e0d211ba40266969f6dbdd90636da12f75bee4fc9d6c23d1260dadb51454" +checksum = "23639446165ca5a5de86ae1d8896b737ae80319560fbaa4c2887b7da6e7ebd7d" dependencies = [ "cfg-if", "js-sys", @@ -10183,9 +10178,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.61" +version = "0.3.60" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e33b99f4b23ba3eec1a53ac264e35a755f00e966e0065077d6027c0f575b0b97" +checksum = "bcda906d8be16e728fd5adc5b729afad4e444e106ab28cd1c7256e54fa61510f" dependencies = [ "js-sys", "wasm-bindgen", @@ -10230,18 +10225,27 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.22.6" +version = "0.22.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6c71e40d7d2c34a5106301fb632274ca37242cd0c9d3e64dbece371a40a2d87" +checksum = "368bfe657969fb01238bb756d351dcade285e0f6fcbd36dcb23359a5169975be" dependencies = [ "webpki 0.22.0", ] +[[package]] +name = "wepoll-ffi" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d743fdedc5c64377b5fc2bc036b01c7fd642205a0d96356034ae3404d49eb7fb" +dependencies = [ + "cc", +] + [[package]] name = "which" -version = "4.4.0" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2441c784c52b289a054b7201fc93253e288f094e2f4be9058343127c4226a269" +checksum = "1c831fbbee9e129a8cf93e7747a82da9d95ba8e16621cae60ec2cdc849bacb7b" dependencies = [ "either", "libc", @@ -10334,9 +10338,9 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +checksum = "8e2522491fbfcd58cc84d47aeb2958948c4b8982e9a2d8a2a35bbaed431390e7" dependencies = [ "windows_aarch64_gnullvm", "windows_aarch64_msvc", @@ -10349,51 +10353,51 @@ dependencies = [ [[package]] name = "windows_aarch64_gnullvm" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" +checksum = "8c9864e83243fdec7fc9c5444389dcbbfd258f745e7853198f365e3c4968a608" [[package]] name = "windows_aarch64_msvc" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" +checksum = "4c8b1b673ffc16c47a9ff48570a9d85e25d265735c503681332589af6253c6c7" [[package]] name = "windows_i686_gnu" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" +checksum = "de3887528ad530ba7bdbb1faa8275ec7a1155a45ffa57c37993960277145d640" [[package]] name = "windows_i686_msvc" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" +checksum = "bf4d1122317eddd6ff351aa852118a2418ad4214e6613a50e0191f7004372605" [[package]] name = "windows_x86_64_gnu" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" +checksum = "c1040f221285e17ebccbc2591ffdc2d44ee1f9186324dd3e84e99ac68d699c45" [[package]] name = "windows_x86_64_gnullvm" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" +checksum = "628bfdf232daa22b0d64fdb62b09fcc36bb01f05a3939e20ab73aaf9470d0463" [[package]] name = "windows_x86_64_msvc" -version = "0.42.2" +version = "0.42.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" +checksum = "447660ad36a13288b1db4d4248e857b510e8c3a225c822ba4fb748c0aafecffd" [[package]] name = "winnow" -version = "0.3.6" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23d020b441f92996c80d94ae9166e8501e59c7bb56121189dc9eab3bd8216966" +checksum = "ee7b2c67f962bf5042bfd8b6a916178df33a26eec343ae064cb8e069f638fa6f" dependencies = [ "memchr", ] @@ -10439,15 +10443,6 @@ dependencies = [ "regex", ] -[[package]] -name = "wyz" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05f360fc0b24296329c78fda852a1e9ae82de9cf7b27dae4b7f62f118f77b9ed" -dependencies = [ - "tap", -] - [[package]] name = "xml-rs" version = "0.8.4" @@ -10501,9 +10496,9 @@ dependencies = [ [[package]] name = "zeroize_derive" -version = "1.3.3" +version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44bf07cb3e50ea2003396695d58bf46bc9887a1f362260446fad6bc4e79bd36c" +checksum = "3f8f187641dad4f680d25c4bfc4225b418165984179f26ca76ec4fb6441d3a17" dependencies = [ "proc-macro2 1.0.52", "quote 1.0.26", @@ -10526,7 +10521,7 @@ version = "0.12.3+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76eea132fb024e0e13fd9c2f5d5d595d8a967aa72382ac2f9d39fcc95afd0806" dependencies = [ - "zstd-safe 6.0.4+zstd.1.5.4", + "zstd-safe 6.0.3+zstd.1.5.2", ] [[package]] @@ -10541,9 +10536,9 @@ dependencies = [ [[package]] name = "zstd-safe" -version = "6.0.4+zstd.1.5.4" +version = "6.0.3+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7afb4b54b8910cf5447638cb54bf4e8a65cbedd783af98b98c62ffe91f185543" +checksum = "68e4a3f57d13d0ab7e478665c60f35e2a613dcd527851c2c7287ce5c787e134a" dependencies = [ "libc", "zstd-sys", @@ -10551,11 +10546,10 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.7+zstd.1.5.4" +version = "2.0.4+zstd.1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94509c3ba2fe55294d752b79842c530ccfab760192521df74a081a78d2b3c7f5" +checksum = "4fa202f2ef00074143e219d15b62ffc317d17cc33909feac471c044087cad7b0" dependencies = [ "cc", "libc", - "pkg-config", ] diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 1cf9582419de9..59205426f935c 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -16,6 +16,7 @@ use pulsar::{ TokioExecutor, }; use snafu::ResultExt; +use vector_common::sensitive_string::SensitiveString; use vector_config::configurable_component; use vector_core::config::DataType; @@ -32,7 +33,7 @@ pub struct PulsarSinkConfig { /// The Pulsar topic name to write events to. #[configurable(metadata(docs::examples = "topic-1234"))] - topic: String, + pub(crate) topic: String, /// The name of the producer. If not specified, the default name assigned by Pulsar will be used. #[configurable(metadata(docs::examples = "producer-name"))] @@ -67,7 +68,7 @@ pub struct PulsarSinkConfig { pub encoding: EncodingConfig, #[configurable(derived)] - pub auth: Option, + auth: Option, #[configurable(derived)] #[serde(default)] @@ -82,6 +83,16 @@ pub struct PulsarSinkConfig { pub acknowledgements: AcknowledgementsConfig, } +/// Event batching behavior. +#[configurable_component] +#[derive(Clone, Copy, Debug, Default)] +struct BatchConfig { + /// The maximum size of a batch before it is flushed. + #[configurable(metadata(docs::type_unit = "events"))] + #[configurable(metadata(docs::examples = 1000))] + pub max_events: Option, +} + /// Authentication configuration. #[configurable_component] #[derive(Clone, Debug)] @@ -168,7 +179,7 @@ impl PulsarSinkConfig { ) { (Some(name), Some(token), None) => builder.with_auth(Authentication { name: name.clone(), - data: token.as_bytes().to_vec(), + data: token.inner().as_bytes().to_vec(), }), (None, None, Some(oauth2)) => builder.with_auth_provider( OAuth2Authentication::client_credentials(OAuth2Params { @@ -222,7 +233,7 @@ impl PulsarSinkConfig { } } } - opts.batch_size = self.batch_size.to_owned(); + opts.batch_size = self.batch.max_events; if let SerializerConfig::Avro { avro } = self.encoding.config() { opts.schema = Some(proto::Schema { schema_data: avro.schema.as_bytes().into(), @@ -240,11 +251,13 @@ impl GenerateConfig for PulsarSinkConfig { endpoint: "pulsar://127.0.0.1:6650".to_string(), request: TowerRequestConfig::default(), topic: "topic-1234".to_string(), + producer_name: None, key_field: None, properties_key: None, - batch_size: None, + partition_key_field: None, + batch: Default::default(), compression: None, - encoding: TextSerializerConfig::new().into(), + encoding: TextSerializerConfig::default().into(), auth: None, acknowledgements: Default::default(), }) diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 9093bcf49db48..1bb5192be2c1e 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -37,8 +37,8 @@ impl DriverResponse for PulsarResponse { CountByteSize(1, self.event_byte_size) } - fn bytes_sent(&self) -> Option<(usize, &str)> { - Some((self.event_byte_size, "pulsar")) + fn bytes_sent(&self) -> Option { + Some(self.event_byte_size) } } diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index dace283e08d35..eaac9755f72d1 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -2,16 +2,6 @@ use async_trait::async_trait; use bytes::Bytes; use std::collections::HashMap; -use futures::{stream::BoxStream, StreamExt}; -use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; -use serde::Serialize; -use snafu::{ResultExt, Snafu}; -use tower::ServiceBuilder; -use vector_buffers::EventCount; -use vector_common::byte_size_of::ByteSizeOf; -use vector_core::event::LogEvent; -use vector_core::sink::StreamSink; - use crate::sinks::pulsar::config::PulsarSinkConfig; use crate::sinks::pulsar::encoder::PulsarEncoder; use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; @@ -25,6 +15,17 @@ use crate::{ codecs::{Encoder, Transformer}, event::Event, }; +use futures::{stream::BoxStream, StreamExt}; +use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; +use serde::Serialize; +use snafu::{ResultExt, Snafu}; +use tower::ServiceBuilder; +use vector_buffers::EventCount; +use vector_common::byte_size_of::ByteSizeOf; +use vector_core::{ + event::{EstimatedJsonEncodedSizeOf, LogEvent}, + sink::StreamSink, +}; #[derive(Debug, Snafu)] #[snafu(visibility(pub(crate)))] @@ -79,6 +80,12 @@ impl ByteSizeOf for PulsarEvent { } } +impl EstimatedJsonEncodedSizeOf for PulsarEvent { + fn estimated_json_encoded_size_of(&self) -> usize { + self.event.estimated_json_encoded_size_of() + } +} + pub(crate) async fn healthcheck(config: PulsarSinkConfig) -> crate::Result<()> { trace!("Healthcheck started."); let client = config.create_pulsar_client().await?; From f4ad1f9403de1951f20059847b8beb0d9e8ac420 Mon Sep 17 00:00:00 2001 From: neuronull Date: Wed, 15 Mar 2023 13:30:55 -0600 Subject: [PATCH 23/48] feedback bg --- src/sinks/pulsar/request_builder.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/sinks/pulsar/request_builder.rs b/src/sinks/pulsar/request_builder.rs index 2d32c03b2719b..ec104d0ebf508 100644 --- a/src/sinks/pulsar/request_builder.rs +++ b/src/sinks/pulsar/request_builder.rs @@ -36,8 +36,7 @@ impl RequestBuilder for PulsarRequestBuilder { type Error = io::Error; fn compression(&self) -> Compression { - // we don't use this compression from vector as Pulsar supports more - // compression formats + // Compression is handled by the pulsar crate through the producer settings. Compression::None } From 1dc2b7b7b9aaf748f08feec72dfc55df67a1bebc Mon Sep 17 00:00:00 2001 From: neuronull Date: Wed, 15 Mar 2023 13:41:20 -0600 Subject: [PATCH 24/48] more edits toward convergence --- src/internal_events/pulsar.rs | 2 +- src/sinks/pulsar/config.rs | 49 +++++++++++++++------------ src/sinks/pulsar/integration_tests.rs | 18 +++------- 3 files changed, 32 insertions(+), 37 deletions(-) diff --git a/src/internal_events/pulsar.rs b/src/internal_events/pulsar.rs index 21188ef5420be..35564b1f207ba 100644 --- a/src/internal_events/pulsar.rs +++ b/src/internal_events/pulsar.rs @@ -44,7 +44,7 @@ impl InternalEvent for PulsarPropertyExtractionError<'_> { message = "Failed to extract properties. Value should be a map of String -> Bytes.", error_code = "extracting_property", error_type = error_type::PARSER_FAILED, - stage = error_stage::RECEIVING, + stage = error_stage::PROCESSING, property_field = self.property_field, internal_log_rate_limit = true, ); diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 59205426f935c..7c599ba4e5f7a 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -29,7 +29,7 @@ pub struct PulsarSinkConfig { /// The endpoint should specify the pulsar protocol and port. #[serde(alias = "address")] #[configurable(metadata(docs::examples = "pulsar://127.0.0.1:6650"))] - endpoint: String, + pub(crate) endpoint: String, /// The Pulsar topic name to write events to. #[configurable(metadata(docs::examples = "topic-1234"))] @@ -37,7 +37,7 @@ pub struct PulsarSinkConfig { /// The name of the producer. If not specified, the default name assigned by Pulsar will be used. #[configurable(metadata(docs::examples = "producer-name"))] - producer_name: Option, + pub(crate) producer_name: Option, /// The log field name or tags key to use for the topic key. /// @@ -54,11 +54,11 @@ pub struct PulsarSinkConfig { /// Log field to use as Pulsar message key. #[configurable(metadata(docs::examples = "message"))] #[configurable(metadata(docs::examples = "my_field"))] - partition_key_field: Option, + pub(crate) partition_key_field: Option, #[configurable(derived)] #[serde(default)] - batch: BatchConfig, + pub(crate) batch: BatchConfig, #[configurable(derived)] #[serde(default)] @@ -68,7 +68,7 @@ pub struct PulsarSinkConfig { pub encoding: EncodingConfig, #[configurable(derived)] - auth: Option, + pub(crate) auth: Option, #[configurable(derived)] #[serde(default)] @@ -86,7 +86,7 @@ pub struct PulsarSinkConfig { /// Event batching behavior. #[configurable_component] #[derive(Clone, Copy, Debug, Default)] -struct BatchConfig { +pub(crate) struct BatchConfig { /// The maximum size of a batch before it is flushed. #[configurable(metadata(docs::type_unit = "events"))] #[configurable(metadata(docs::examples = 1000))] @@ -96,7 +96,7 @@ struct BatchConfig { /// Authentication configuration. #[configurable_component] #[derive(Clone, Debug)] -struct AuthConfig { +pub(crate) struct AuthConfig { /// Basic authentication name/username. /// /// This can be used either for basic authentication (username/password) or JWT authentication. @@ -168,6 +168,25 @@ pub enum PulsarCompression { Snappy, } +impl Default for PulsarSinkConfig { + fn default() -> Self { + Self { + endpoint: "pulsar://127.0.0.1:6650".to_string(), + topic: "topic-1234".to_string(), + producer_name: None, + key_field: None, + properties_key: None, + partition_key_field: None, + batch: Default::default(), + compression: None, + encoding: TextSerializerConfig::default().into(), + auth: None, + request: TowerRequestConfig::default(), + acknowledgements: Default::default(), + } + } +} + impl PulsarSinkConfig { pub(crate) async fn create_pulsar_client(&self) -> Result, PulsarError> { let mut builder = Pulsar::builder(&self.endpoint, TokioExecutor); @@ -247,21 +266,7 @@ impl PulsarSinkConfig { impl GenerateConfig for PulsarSinkConfig { fn generate_config() -> toml::Value { - toml::Value::try_from(Self { - endpoint: "pulsar://127.0.0.1:6650".to_string(), - request: TowerRequestConfig::default(), - topic: "topic-1234".to_string(), - producer_name: None, - key_field: None, - properties_key: None, - partition_key_field: None, - batch: Default::default(), - compression: None, - encoding: TextSerializerConfig::default().into(), - auth: None, - acknowledgements: Default::default(), - }) - .unwrap() + toml::Value::try_from(Self::default()).unwrap() } } diff --git a/src/sinks/pulsar/integration_tests.rs b/src/sinks/pulsar/integration_tests.rs index f35c4eae88e92..9de27aec8fd3e 100644 --- a/src/sinks/pulsar/integration_tests.rs +++ b/src/sinks/pulsar/integration_tests.rs @@ -1,16 +1,13 @@ -use crate::sinks::pulsar::config::PulsarSinkConfig; -use crate::sinks::pulsar::sink::PulsarSink; -use crate::sinks::util::TowerRequestConfig; -use codecs::TextSerializerConfig; +use crate::sinks::pulsar::{config::PulsarSinkConfig, sink::PulsarSink}; use futures::StreamExt; use pulsar::SubType; use std::collections::BTreeMap; use crate::event::Value; use crate::sinks::VectorSink; -use crate::test_util::components::assert_sink_compliance; use crate::test_util::{ - components::SINK_TAGS, random_lines_with_stream, random_string, trace_init, + components::{assert_sink_compliance, SINK_TAGS}, + random_lines_with_stream, random_string, trace_init, }; use bytes::Bytes; @@ -87,14 +84,7 @@ async fn pulsar_happy() { endpoint: pulsar_address(), // overriden by test topic: "".to_string(), - key_field: None, - properties_key: None, - batch_size: None, - compression: None, - encoding: TextSerializerConfig::new().into(), - auth: None, - acknowledgements: Default::default(), - request: TowerRequestConfig::default(), + ..Default::default() }; pulsar_happy_reuse(cnf).await From a5b2c5e6284c8c0aefafaf1e68b94a5ed950286c Mon Sep 17 00:00:00 2001 From: neuronull Date: Fri, 17 Mar 2023 14:57:58 -0600 Subject: [PATCH 25/48] fixing things --- scripts/integration/pulsar/test.yaml | 2 +- src/sinks/pulsar/config.rs | 61 +++++++++++++--------------- src/sinks/pulsar/service.rs | 17 ++++++-- src/sinks/pulsar/sink.rs | 2 +- src/sinks/pulsar/util.rs | 26 ++++++------ 5 files changed, 59 insertions(+), 49 deletions(-) diff --git a/scripts/integration/pulsar/test.yaml b/scripts/integration/pulsar/test.yaml index 9115ec5620eee..a63f7da772e4b 100644 --- a/scripts/integration/pulsar/test.yaml +++ b/scripts/integration/pulsar/test.yaml @@ -1,7 +1,7 @@ features: - pulsar-integration-tests -test_filter: '::pulsar::' +test_filter: '::pulsar::integration_tests::' env: PULSAR_ADDRESS: pulsar://pulsar:6650 diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 7c599ba4e5f7a..535e336bd1976 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -44,25 +44,22 @@ pub struct PulsarSinkConfig { /// If the field does not exist in the log or in tags, a blank value will be used. If unspecified, the key is not sent. /// /// Pulsar uses a hash of the key to choose the topic-partition or uses round-robin if the record has no key. - pub key_field: Option, + #[configurable(metadata(docs::examples = "message"))] + #[configurable(metadata(docs::examples = "my_field"))] + pub(crate) partition_key_field: Option, /// The log field name to use for the Pulsar properties. /// /// If omitted, no properties will be written. pub properties_key: Option, - /// Log field to use as Pulsar message key. - #[configurable(metadata(docs::examples = "message"))] - #[configurable(metadata(docs::examples = "my_field"))] - pub(crate) partition_key_field: Option, - #[configurable(derived)] #[serde(default)] pub(crate) batch: BatchConfig, #[configurable(derived)] #[serde(default)] - pub compression: Option, + pub compression: PulsarCompression, #[configurable(derived)] pub encoding: EncodingConfig, @@ -174,11 +171,10 @@ impl Default for PulsarSinkConfig { endpoint: "pulsar://127.0.0.1:6650".to_string(), topic: "topic-1234".to_string(), producer_name: None, - key_field: None, properties_key: None, partition_key_field: None, batch: Default::default(), - compression: None, + compression: Default::default(), encoding: TextSerializerConfig::default().into(), auth: None, request: TowerRequestConfig::default(), @@ -227,32 +223,33 @@ impl PulsarSinkConfig { batch_size: None, compression: None, }; - if let Some(config_compression) = &self.compression { - match config_compression { - PulsarCompression::None => opts.compression = Some(compression::Compression::None), - PulsarCompression::Lz4 => { - opts.compression = Some(compression::Compression::Lz4( - compression::CompressionLz4::default(), - )) - } - PulsarCompression::Zlib => { - opts.compression = Some(compression::Compression::Zlib( - compression::CompressionZlib::default(), - )) - } - PulsarCompression::Zstd => { - opts.compression = Some(compression::Compression::Zstd( - compression::CompressionZstd::default(), - )) - } - PulsarCompression::Snappy => { - opts.compression = Some(compression::Compression::Snappy( - compression::CompressionSnappy::default(), - )) - } + + match &self.compression { + PulsarCompression::None => opts.compression = Some(compression::Compression::None), + PulsarCompression::Lz4 => { + opts.compression = Some(compression::Compression::Lz4( + compression::CompressionLz4::default(), + )) + } + PulsarCompression::Zlib => { + opts.compression = Some(compression::Compression::Zlib( + compression::CompressionZlib::default(), + )) + } + PulsarCompression::Zstd => { + opts.compression = Some(compression::Compression::Zstd( + compression::CompressionZstd::default(), + )) + } + PulsarCompression::Snappy => { + opts.compression = Some(compression::Compression::Snappy( + compression::CompressionSnappy::default(), + )) } } + opts.batch_size = self.batch.max_events; + if let SerializerConfig::Avro { avro } = self.encoding.config() { opts.schema = Some(proto::Schema { schema_data: avro.schema.as_bytes().into(), diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 1bb5192be2c1e..fc68d6c1d0832 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -75,6 +75,7 @@ pub struct PulsarService { pulsar_client: Pulsar, producer_cache: SafeLru, producer_options: ProducerOptions, + producer_name: Option, } impl PulsarService { @@ -82,6 +83,7 @@ impl PulsarService { pulsar_client: Pulsar, producer_options: ProducerOptions, producer_cache_size: Option, + producer_name: Option, ) -> PulsarService { // Use a LRUCache to store a limited set of producers // Producers in Pulsar use a send buffer, so we want to limit the number of these @@ -92,6 +94,7 @@ impl PulsarService { pulsar_client, producer_cache, producer_options, + producer_name, } } @@ -103,12 +106,18 @@ impl PulsarService { client: Pulsar, producer_options: ProducerOptions, topic: &String, + name: Option, ) -> Result>>, PulsarError> { - let prod = client + let mut builder = client .producer() .with_topic(topic) .with_options(producer_options); - match prod.build().await { + + if let Some(name) = name { + builder = builder.with_name(name); + } + + match builder.build().await { Ok(p) => Ok(Arc::new(Mutex::new(p))), Err(e) => Err(e), } @@ -121,13 +130,14 @@ impl PulsarService { client: Pulsar, producer_options: ProducerOptions, topic: String, + name: Option, ) -> Arc>> { let mut pc = producer_cache.lock().await; match pc.contains(&topic) { false => { pc.put( topic.clone(), - PulsarService::build_producer(client, producer_options, &topic).await, + PulsarService::build_producer(client, producer_options, &topic, name).await, ); let f = pc.get(&topic).unwrap().as_ref().unwrap(); Arc::clone(f) @@ -155,6 +165,7 @@ impl Service for PulsarService { self.pulsar_client.clone(), self.producer_options.clone(), request.metadata.topic.clone(), + self.producer_name.clone(), ); let ts = request.metadata.timestamp_millis.to_owned(); Box::pin(async move { diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index eaac9755f72d1..396529c6ffde8 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -107,7 +107,7 @@ impl PulsarSink { let serializer = config.encoding.build()?; let request_settings = config.request.unwrap_with(&TowerRequestConfig::default()); let encoder = Encoder::<()>::new(serializer); - let service = PulsarService::new(client, producer_opts, None); + let service = PulsarService::new(client, producer_opts, None, config.producer_name.clone()); let topic = config.topic.clone(); Ok(PulsarSink { diff --git a/src/sinks/pulsar/util.rs b/src/sinks/pulsar/util.rs index ddc34ed0ae1de..40364c68abfa3 100644 --- a/src/sinks/pulsar/util.rs +++ b/src/sinks/pulsar/util.rs @@ -17,7 +17,7 @@ pub(super) fn make_pulsar_event( event: Event, ) -> Option { let topic = topic.render_string(&event).ok()?; - let key = get_key(&event, &config.key_field); + let key = get_key(&event, &config.partition_key_field); let timestamp_millis = get_timestamp_millis(&event, log_schema()); let properties = get_properties(&event, &config.properties_key); Some(PulsarEvent { @@ -29,17 +29,19 @@ pub(super) fn make_pulsar_event( }) } -fn get_key(event: &Event, key_field: &Option) -> Option { - key_field.as_ref().and_then(|key_field| match event { - Event::Log(log) => log - .get(key_field.as_str()) - .map(|value| value.coerce_to_bytes()), - Event::Metric(metric) => metric - .tags() - .and_then(|tags| tags.get(key_field)) - .map(|value| value.to_owned().into()), - _ => None, - }) +fn get_key(event: &Event, partition_key_field: &Option) -> Option { + partition_key_field + .as_ref() + .and_then(|partition_key_field| match event { + Event::Log(log) => log + .get(partition_key_field.as_str()) + .map(|value| value.coerce_to_bytes()), + Event::Metric(metric) => metric + .tags() + .and_then(|tags| tags.get(partition_key_field)) + .map(|value| value.to_owned().into()), + _ => None, + }) } fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option { From f366fd68f7f9f67d3171940d792074a8840a03c7 Mon Sep 17 00:00:00 2001 From: neuronull Date: Fri, 17 Mar 2023 15:12:48 -0600 Subject: [PATCH 26/48] found more things --- src/sinks/pulsar/config.rs | 6 ++++++ src/sinks/pulsar/util.rs | 11 +++-------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 535e336bd1976..0ddd0f603874b 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -2,6 +2,7 @@ use crate::sinks::util::TowerRequestConfig; use crate::{ codecs::EncodingConfig, config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, + schema, sinks::{ pulsar::sink::{healthcheck, PulsarSink}, Healthcheck, VectorSink, @@ -16,6 +17,7 @@ use pulsar::{ TokioExecutor, }; use snafu::ResultExt; +use value::Kind; use vector_common::sensitive_string::SensitiveString; use vector_config::configurable_component; use vector_core::config::DataType; @@ -283,7 +285,11 @@ impl SinkConfig for PulsarSinkConfig { } fn input(&self) -> Input { + let requirement = + schema::Requirement::empty().optional_meaning("timestamp", Kind::timestamp()); + Input::new(self.encoding.config().input_type() & (DataType::Log | DataType::Metric)) + .with_schema_requirement(requirement) } fn acknowledgements(&self) -> &AcknowledgementsConfig { diff --git a/src/sinks/pulsar/util.rs b/src/sinks/pulsar/util.rs index 40364c68abfa3..c6599d5915d81 100644 --- a/src/sinks/pulsar/util.rs +++ b/src/sinks/pulsar/util.rs @@ -5,8 +5,6 @@ use crate::template::Template; use bytes::Bytes; use std::collections::HashMap; use value::Value; -use vector_core::config::log_schema; -use vector_core::config::LogSchema; use vector_core::event::Event; /// Transforms an event into a Pulsar event by rendering the required template fields. @@ -18,7 +16,7 @@ pub(super) fn make_pulsar_event( ) -> Option { let topic = topic.render_string(&event).ok()?; let key = get_key(&event, &config.partition_key_field); - let timestamp_millis = get_timestamp_millis(&event, log_schema()); + let timestamp_millis = get_timestamp_millis(&event); let properties = get_properties(&event, &config.properties_key); Some(PulsarEvent { event, @@ -44,12 +42,9 @@ fn get_key(event: &Event, partition_key_field: &Option) -> Option }) } -fn get_timestamp_millis(event: &Event, log_schema: &'static LogSchema) -> Option { +fn get_timestamp_millis(event: &Event) -> Option { match &event { - Event::Log(log) => log - .get(log_schema.timestamp_key()) - .and_then(|v| v.as_timestamp()) - .copied(), + Event::Log(log) => log.get_timestamp().and_then(|v| v.as_timestamp()).copied(), Event::Metric(metric) => metric.timestamp(), _ => None, } From 1f33668b34bbc8e90008791586dd65002207937f Mon Sep 17 00:00:00 2001 From: neuronull Date: Fri, 17 Mar 2023 15:26:10 -0600 Subject: [PATCH 27/48] add protocol --- src/sinks/pulsar/sink.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index 396529c6ffde8..1b39a19bbe8fe 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -148,7 +148,9 @@ impl PulsarSink { Ok(req) => Some(req), } }) - .into_driver(service); + .into_driver(service) + .protocol("tcp"); + sink.run().await } } From b6b6e36436608acea96d0e3f18353c63e4b6cc12 Mon Sep 17 00:00:00 2001 From: neuronull Date: Fri, 17 Mar 2023 15:44:26 -0600 Subject: [PATCH 28/48] clippy --- src/sinks/pulsar/service.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index fc68d6c1d0832..b87d89a5ab236 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -174,7 +174,7 @@ impl Service for PulsarService { let body = request.body.clone(); let mut msg_builder = lp.create_message().with_content(body.as_ref()); if let Some(key) = request.metadata.key { - msg_builder = msg_builder.with_key(String::from_utf8_lossy(&*key)); + msg_builder = msg_builder.with_key(String::from_utf8_lossy(&key)); } if let Some(timestamp) = ts { msg_builder = msg_builder.event_time(timestamp as u64); @@ -182,7 +182,7 @@ impl Service for PulsarService { if let Some(properties) = request.metadata.properties { for (key, value) in properties { msg_builder = - msg_builder.with_property(key, String::from_utf8_lossy(&*value.clone())); + msg_builder.with_property(key, String::from_utf8_lossy(&value.clone())); } } From 85c1e2d421b817bf9ef90d09b9b7b800afba5052 Mon Sep 17 00:00:00 2001 From: neuronull Date: Fri, 17 Mar 2023 15:52:08 -0600 Subject: [PATCH 29/48] gen docs --- .../components/sinks/base/pulsar.cue | 140 +++++++----------- 1 file changed, 57 insertions(+), 83 deletions(-) diff --git a/website/cue/reference/components/sinks/base/pulsar.cue b/website/cue/reference/components/sinks/base/pulsar.cue index 9e48946282ad1..0ae45d98ecdf7 100644 --- a/website/cue/reference/components/sinks/base/pulsar.cue +++ b/website/cue/reference/components/sinks/base/pulsar.cue @@ -109,42 +109,6 @@ base: components: sinks: pulsar: configuration: { } } } - batch_size: { - description: """ - Determines the batch size. - - Defaults to 1000. - """ - required: false - type: uint: {} - } - compression: { - description: "Identifies the compression options that are available within Pulsar." - required: false - type: string: enum: { - Lz4: """ - [LZ4][lz4] compression. - - [lz4]: https://lz4.github.io/lz4/ - """ - None: "No compression." - Snappy: """ - [Snappy][snappy] compression. - - [snappy]: https://google.github.io/snappy/ - """ - Zlib: """ - [Zlib][zlib] compression. - - [zlib]: https://www.zlib.net - """ - Zstd: """ - [Zstd][zstd] compression. - - [zstd]: https://zstd.net - """ - } - } encoding: { description: "Configures how events are encoded into raw bytes." required: true @@ -293,8 +257,7 @@ base: components: sinks: pulsar: configuration: { required: true type: string: examples: ["pulsar://127.0.0.1:6650"] } -<<<<<<< HEAD - key_field: { + partition_key_field: { description: """ The log field name or tags key to use for the topic key. @@ -303,23 +266,12 @@ base: components: sinks: pulsar: configuration: { Pulsar uses a hash of the key to choose the topic-partition or uses round-robin if the record has no key. """ required: false - type: string: syntax: "literal" -||||||| 7e6727983 - partition_key_field: { - description: "Log field to use as Pulsar message key" - required: false - type: string: syntax: "literal" -======= - partition_key_field: { - description: "Log field to use as Pulsar message key." - required: false type: string: examples: ["message", "my_field"] } producer_name: { description: "The name of the producer. If not specified, the default name assigned by Pulsar will be used." required: false type: string: examples: ["producer-name"] ->>>>>>> master } properties_key: { description: """ @@ -328,7 +280,7 @@ base: components: sinks: pulsar: configuration: { If omitted, no properties will be written. """ required: false - type: string: syntax: "literal" + type: string: {} } request: { description: """ @@ -346,15 +298,9 @@ base: components: sinks: pulsar: configuration: { unstable performance and sink behavior. Proceed with caution. """ required: false - type: object: { - default: { - decrease_ratio: 0.9 - ewma_alpha: 0.4 - rtt_deviation_scale: 2.5 - } - options: { - decrease_ratio: { - description: """ + type: object: options: { + decrease_ratio: { + description: """ The fraction of the current value to set the new concurrency limit when decreasing the limit. Valid values are greater than `0` and less than `1`. Smaller values cause the algorithm to scale back rapidly @@ -362,11 +308,11 @@ base: components: sinks: pulsar: configuration: { Note that the new limit is rounded down after applying this ratio. """ - required: false - type: float: default: 0.9 - } - ewma_alpha: { - description: """ + required: false + type: float: default: 0.9 + } + ewma_alpha: { + description: """ The weighting of new measurements compared to older measurements. Valid values are greater than `0` and less than `1`. @@ -375,11 +321,11 @@ base: components: sinks: pulsar: configuration: { the current RTT. Smaller values cause this reference to adjust more slowly, which may be useful if a service has unusually high response variability. """ - required: false - type: float: default: 0.4 - } - rtt_deviation_scale: { - description: """ + required: false + type: float: default: 0.4 + } + rtt_deviation_scale: { + description: """ Scale of RTT deviations which are not considered anomalous. Valid values are greater than or equal to `0`, and we expect reasonable values to range from `1.0` to `3.0`. @@ -389,9 +335,8 @@ base: components: sinks: pulsar: configuration: { can ignore increases in RTT that are within an expected range. This factor is used to scale up the deviation to an appropriate range. Larger values cause the algorithm to ignore larger increases in the RTT. """ - required: false - type: float: default: 2.5 - } + required: false + type: float: default: 2.5 } } } @@ -399,22 +344,39 @@ base: components: sinks: pulsar: configuration: { description: "Configuration for outbound request concurrency." required: false type: { - number: {} string: { - const: "adaptive" default: "none" + enum: { + adaptive: """ + Concurrency will be managed by Vector's [Adaptive Request Concurrency][arc] feature. + + [arc]: https://vector.dev/docs/about/under-the-hood/networking/arc/ + """ + none: """ + A fixed concurrency of 1. + + Only one request can be outstanding at any given time. + """ + } } + uint: {} } } rate_limit_duration_secs: { - description: "The time window, in seconds, used for the `rate_limit_num` option." + description: "The time window used for the `rate_limit_num` option." required: false - type: uint: default: 1 + type: uint: { + default: 1 + unit: "seconds" + } } rate_limit_num: { description: "The maximum number of requests allowed within the `rate_limit_duration_secs` time window." required: false - type: uint: default: 9223372036854775807 + type: uint: { + default: 9223372036854775807 + unit: "requests" + } } retry_attempts: { description: """ @@ -423,7 +385,10 @@ base: components: sinks: pulsar: configuration: { The default, for all intents and purposes, represents an infinite number of retries. """ required: false - type: uint: default: 9223372036854775807 + type: uint: { + default: 9223372036854775807 + unit: "retries" + } } retry_initial_backoff_secs: { description: """ @@ -432,22 +397,31 @@ base: components: sinks: pulsar: configuration: { After the first retry has failed, the fibonacci sequence will be used to select future backoffs. """ required: false - type: uint: default: 1 + type: uint: { + default: 1 + unit: "seconds" + } } retry_max_duration_secs: { - description: "The maximum amount of time, in seconds, to wait between retries." + description: "The maximum amount of time to wait between retries." required: false - type: uint: default: 3600 + type: uint: { + default: 3600 + unit: "seconds" + } } timeout_secs: { description: """ - The maximum time a request can take before being aborted. + The time a request can take before being aborted. It is highly recommended that you do not lower this value below the service’s internal timeout, as this could create orphaned requests, pile on retries, and result in duplicate data downstream. """ required: false - type: uint: default: 60 + type: uint: { + default: 60 + unit: "seconds" + } } } } From e4aedce46e8e5408e06a3ced48bb80f134918102 Mon Sep 17 00:00:00 2001 From: neuronull Date: Fri, 17 Mar 2023 15:56:55 -0600 Subject: [PATCH 30/48] spell check --- src/internal_events/pulsar.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/internal_events/pulsar.rs b/src/internal_events/pulsar.rs index 35564b1f207ba..1ce64909b8c90 100644 --- a/src/internal_events/pulsar.rs +++ b/src/internal_events/pulsar.rs @@ -50,7 +50,7 @@ impl InternalEvent for PulsarPropertyExtractionError<'_> { ); counter!( "component_errors_total", 1, - "error_code" => "extracing_property", + "error_code" => "extracting_property", "error_type" => error_type::PARSER_FAILED, "stage" => error_stage::PROCESSING, ); From 77a9c08b542c79c2c805a41f85c6973bdfa64b78 Mon Sep 17 00:00:00 2001 From: neuronull Date: Mon, 20 Mar 2023 16:31:32 -0600 Subject: [PATCH 31/48] feedback sg --- src/sinks/pulsar/sink.rs | 18 ++++++------------ src/validate.rs | 2 ++ 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index 1b39a19bbe8fe..59c0c27424d25 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -45,9 +45,9 @@ pub(crate) struct PulsarSink { topic_template: Template, } -/// Stores the event together with the extracted keys, topics, etc +/// Stores the event together with the extracted keys, topics, etc. /// This is passed into the `RequestBuilder` which then splits it out into the event -/// and metadata containing the keys, and metadata +/// and metadata containing the keys, and metadata. /// This event needs to be created prior to building the request so we can filter out /// any events that error whilst rendering the templates. #[derive(Serialize)] @@ -70,7 +70,7 @@ impl ByteSizeOf for PulsarEvent { fn allocated_bytes(&self) -> usize { self.event.size_of() + self.topic.size_of() - + self.key.as_ref().map_or(0, |bytes| bytes.clone().size_of()) + + self.key.as_ref().map_or(0, |bytes| bytes.size_of()) + self.properties.as_ref().map_or(0, |props| { props .iter() @@ -87,13 +87,11 @@ impl EstimatedJsonEncodedSizeOf for PulsarEvent { } pub(crate) async fn healthcheck(config: PulsarSinkConfig) -> crate::Result<()> { - trace!("Healthcheck started."); let client = config.create_pulsar_client().await?; let topic = Template::try_from(config.topic) .context(TopicTemplateSnafu)? .render_string(&LogEvent::from_str_legacy(""))?; client.lookup_topic(topic).await?; - trace!("Healthcheck completed."); Ok(()) } @@ -140,13 +138,9 @@ impl PulsarSink { }) .request_builder(None, request_builder) .filter_map(|request| async move { - match request { - Err(e) => { - error!("Failed to build Pulsar request: {:?}.", e); - None - } - Ok(req) => Some(req), - } + request + .map_err(|e| error!("Failed to build Pulsar request: {:?}.", e)) + .ok() }) .into_driver(service) .protocol("tcp"); diff --git a/src/validate.rs b/src/validate.rs index 50c7005698edb..0803e11423a2a 100644 --- a/src/validate.rs +++ b/src/validate.rs @@ -220,6 +220,7 @@ async fn validate_healthchecks( fmt.error(error); }; + trace!("Healthcheck for {id} starting."); match tokio::spawn(healthcheck).await { Ok(Ok(_)) => { if config @@ -240,6 +241,7 @@ async fn validate_healthchecks( } Err(_) => failed(format!("Health check for \"{}\" panicked", id)), } + trace!("Healthcheck for {id} done."); } validated From e05f8ff3e4760ef4c77c7a4aa01cb1853d9de236 Mon Sep 17 00:00:00 2001 From: neuronull Date: Tue, 21 Mar 2023 11:07:02 -0600 Subject: [PATCH 32/48] feedback sg --- src/sinks/pulsar/config.rs | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 0ddd0f603874b..f9fb71a31e4c0 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -37,7 +37,7 @@ pub struct PulsarSinkConfig { #[configurable(metadata(docs::examples = "topic-1234"))] pub(crate) topic: String, - /// The name of the producer. If not specified, the default name assigned by Pulsar will be used. + /// The name of the producer. If not specified, Pulsar will generate a unique name. #[configurable(metadata(docs::examples = "producer-name"))] pub(crate) producer_name: Option, @@ -57,7 +57,7 @@ pub struct PulsarSinkConfig { #[configurable(derived)] #[serde(default)] - pub(crate) batch: BatchConfig, + pub(crate) batch: PulsarBatchConfig, #[configurable(derived)] #[serde(default)] @@ -67,11 +67,7 @@ pub struct PulsarSinkConfig { pub encoding: EncodingConfig, #[configurable(derived)] - pub(crate) auth: Option, - - #[configurable(derived)] - #[serde(default)] - pub request: TowerRequestConfig, + pub(crate) auth: Option, #[configurable(derived)] #[serde( @@ -85,7 +81,7 @@ pub struct PulsarSinkConfig { /// Event batching behavior. #[configurable_component] #[derive(Clone, Copy, Debug, Default)] -pub(crate) struct BatchConfig { +pub(crate) struct PulsarBatchConfig { /// The maximum size of a batch before it is flushed. #[configurable(metadata(docs::type_unit = "events"))] #[configurable(metadata(docs::examples = 1000))] @@ -95,7 +91,7 @@ pub(crate) struct BatchConfig { /// Authentication configuration. #[configurable_component] #[derive(Clone, Debug)] -pub(crate) struct AuthConfig { +pub(crate) struct PulsarAuthConfig { /// Basic authentication name/username. /// /// This can be used either for basic authentication (username/password) or JWT authentication. @@ -179,7 +175,6 @@ impl Default for PulsarSinkConfig { compression: Default::default(), encoding: TextSerializerConfig::default().into(), auth: None, - request: TowerRequestConfig::default(), acknowledgements: Default::default(), } } From 19f948700d83ac2966a86a521c01ff076af9c353 Mon Sep 17 00:00:00 2001 From: neuronull Date: Tue, 21 Mar 2023 12:43:16 -0600 Subject: [PATCH 33/48] feedback sg- simplify the producer in the Service --- src/sinks/pulsar/config.rs | 30 ++++----- src/sinks/pulsar/integration_tests.rs | 2 +- src/sinks/pulsar/request_builder.rs | 2 - src/sinks/pulsar/service.rs | 90 ++++----------------------- src/sinks/pulsar/sink.rs | 42 +++++++------ 5 files changed, 54 insertions(+), 112 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index f9fb71a31e4c0..ac13c794bab59 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -1,27 +1,29 @@ -use crate::sinks::util::TowerRequestConfig; -use crate::{ - codecs::EncodingConfig, - config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, - schema, - sinks::{ - pulsar::sink::{healthcheck, PulsarSink}, - Healthcheck, VectorSink, - }, -}; -use codecs::{encoding::SerializerConfig, TextSerializerConfig}; use futures_util::FutureExt; +use snafu::ResultExt; +use value::Kind; + use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; use pulsar::error::AuthenticationError; use pulsar::{ compression, message::proto, Authentication, Error as PulsarError, ProducerOptions, Pulsar, TokioExecutor, }; -use snafu::ResultExt; -use value::Kind; + +use codecs::{encoding::SerializerConfig, TextSerializerConfig}; use vector_common::sensitive_string::SensitiveString; use vector_config::configurable_component; use vector_core::config::DataType; +use crate::{ + codecs::EncodingConfig, + config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, + schema, + sinks::{ + pulsar::sink::{healthcheck, PulsarSink}, + Healthcheck, VectorSink, + }, +}; + /// Configuration for the `pulsar` sink. #[configurable_component(sink("pulsar"))] #[derive(Clone, Debug)] @@ -272,7 +274,7 @@ impl SinkConfig for PulsarSinkConfig { .await .context(super::sink::CreatePulsarSinkSnafu)?; - let sink = PulsarSink::new(client, self.clone())?; + let sink = PulsarSink::new(client, self.clone()).await?; let hc = healthcheck(self.clone()).boxed(); diff --git a/src/sinks/pulsar/integration_tests.rs b/src/sinks/pulsar/integration_tests.rs index 9de27aec8fd3e..f05f1e2f9fbdf 100644 --- a/src/sinks/pulsar/integration_tests.rs +++ b/src/sinks/pulsar/integration_tests.rs @@ -61,7 +61,7 @@ async fn pulsar_happy_reuse(mut cnf: PulsarSinkConfig) { .unwrap(); assert_sink_compliance(&SINK_TAGS, async move { - let sink = PulsarSink::new(pulsar, cnf).unwrap(); + let sink = PulsarSink::new(pulsar, cnf).await.unwrap(); let sink = VectorSink::from_event_streamsink(sink); sink.run(input_events).await }) diff --git a/src/sinks/pulsar/request_builder.rs b/src/sinks/pulsar/request_builder.rs index ec104d0ebf508..46de3966300f7 100644 --- a/src/sinks/pulsar/request_builder.rs +++ b/src/sinks/pulsar/request_builder.rs @@ -20,7 +20,6 @@ pub(super) struct PulsarMetadata { pub key: Option, pub properties: Option>, pub timestamp_millis: Option, - pub topic: String, } pub(super) struct PulsarRequestBuilder { @@ -54,7 +53,6 @@ impl RequestBuilder for PulsarRequestBuilder { key: input.key, timestamp_millis: input.timestamp_millis, properties: input.properties, - topic: input.topic, }; (metadata, builder, input.event) } diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index b87d89a5ab236..1a19374ea8577 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -3,7 +3,6 @@ use std::task::{Context, Poll}; use bytes::Bytes; use futures::future::BoxFuture; -use lru::LruCache; use pulsar::{Error as PulsarError, Executor, Producer, ProducerOptions, Pulsar}; use tokio::sync::Mutex; use tower::Service; @@ -14,7 +13,6 @@ use crate::event::{EventFinalizers, EventStatus, Finalizable}; use crate::internal_events::PulsarSendingError; use crate::sinks::pulsar::request_builder::PulsarMetadata; use crate::sinks::util::retries::RetryLogic; -use std::num::NonZeroUsize; use vector_common::request_metadata::{MetaDescriptive, RequestMetadata}; #[derive(Clone)] @@ -63,90 +61,35 @@ impl RetryLogic for PulsarRetryLogic { type Response = PulsarResponse; fn is_retriable_error(&self, _error: &Self::Error) -> bool { - // TODO improve retry logic + // TODO expand retry logic on different pulsar error types true } } -type SafeLru = Arc>>, PulsarError>>>>; - #[derive(Clone)] pub struct PulsarService { - pulsar_client: Pulsar, - producer_cache: SafeLru, - producer_options: ProducerOptions, - producer_name: Option, + // pulsar::Producer does not implement Clone + producer: Arc>>, } impl PulsarService { - pub(crate) fn new( + pub(crate) async fn new( pulsar_client: Pulsar, producer_options: ProducerOptions, - producer_cache_size: Option, producer_name: Option, - ) -> PulsarService { - // Use a LRUCache to store a limited set of producers - // Producers in Pulsar use a send buffer, so we want to limit the number of these - let producer_cache = Arc::new(Mutex::new(LruCache::new( - producer_cache_size.unwrap_or(NonZeroUsize::new(100).unwrap()), - ))); - PulsarService { - pulsar_client, - producer_cache, - producer_options, - producer_name, - } - } - - /// Build a producer that is wrapped in an Arc to allow for the producer - /// to control access. - /// - /// NOTE: Pulsar client library should likely be improved to simplify this - async fn build_producer( - client: Pulsar, - producer_options: ProducerOptions, topic: &String, - name: Option, - ) -> Result>>, PulsarError> { - let mut builder = client + ) -> Result, pulsar::Error> { + let mut builder = pulsar_client .producer() .with_topic(topic) .with_options(producer_options); - if let Some(name) = name { + if let Some(name) = producer_name { builder = builder.with_name(name); } - - match builder.build().await { - Ok(p) => Ok(Arc::new(Mutex::new(p))), - Err(e) => Err(e), - } - } - - /// Pulsar requires a producer object be created per topic - /// This method will build a producer if it hasn't been created or caches it otherwise - async fn get_or_build_producer( - producer_cache: SafeLru, - client: Pulsar, - producer_options: ProducerOptions, - topic: String, - name: Option, - ) -> Arc>> { - let mut pc = producer_cache.lock().await; - match pc.contains(&topic) { - false => { - pc.put( - topic.clone(), - PulsarService::build_producer(client, producer_options, &topic, name).await, - ); - let f = pc.get(&topic).unwrap().as_ref().unwrap(); - Arc::clone(f) - } - true => { - let f = pc.get(&topic).unwrap().as_ref().unwrap(); - Arc::clone(f) - } - } + builder.build().await.map(|producer| PulsarService { + producer: Arc::new(Mutex::new(producer)), + }) } } @@ -160,19 +103,12 @@ impl Service for PulsarService { } fn call(&mut self, request: PulsarRequest) -> Self::Future { - let prod_future = PulsarService::get_or_build_producer( - Arc::clone(&self.producer_cache), - self.pulsar_client.clone(), - self.producer_options.clone(), - request.metadata.topic.clone(), - self.producer_name.clone(), - ); + let arc_producer = Arc::clone(&self.producer); let ts = request.metadata.timestamp_millis.to_owned(); Box::pin(async move { - let p = prod_future.await; - let mut lp = p.lock().await; + let mut producer = arc_producer.lock().await; let body = request.body.clone(); - let mut msg_builder = lp.create_message().with_content(body.as_ref()); + let mut msg_builder = producer.create_message().with_content(body.as_ref()); if let Some(key) = request.metadata.key { msg_builder = msg_builder.with_key(String::from_utf8_lossy(&key)); } diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index 59c0c27424d25..09688c666f3f0 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -1,25 +1,26 @@ use async_trait::async_trait; use bytes::Bytes; +use futures::{stream::BoxStream, StreamExt}; +use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; +use serde::Serialize; +use snafu::{ResultExt, Snafu}; use std::collections::HashMap; +use tower::ServiceBuilder; -use crate::sinks::pulsar::config::PulsarSinkConfig; -use crate::sinks::pulsar::encoder::PulsarEncoder; -use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; -use crate::sinks::pulsar::service::{PulsarRetryLogic, PulsarService}; -use crate::sinks::pulsar::util; -use crate::sinks::util::{ - ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig, TowerRequestSettings, +use super::{ + config::PulsarSinkConfig, + encoder::PulsarEncoder, + request_builder::PulsarRequestBuilder, + service::{PulsarRetryLogic, PulsarService}, + util, }; + use crate::template::{Template, TemplateParseError}; use crate::{ codecs::{Encoder, Transformer}, event::Event, + sinks::util::{ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig}, }; -use futures::{stream::BoxStream, StreamExt}; -use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; -use serde::Serialize; -use snafu::{ResultExt, Snafu}; -use tower::ServiceBuilder; use vector_buffers::EventCount; use vector_common::byte_size_of::ByteSizeOf; use vector_core::{ @@ -40,7 +41,6 @@ pub(crate) struct PulsarSink { transformer: Transformer, encoder: Encoder<()>, service: PulsarService, - request_settings: TowerRequestSettings, config: PulsarSinkConfig, topic_template: Template, } @@ -96,22 +96,26 @@ pub(crate) async fn healthcheck(config: PulsarSinkConfig) -> crate::Result<()> { } impl PulsarSink { - pub(crate) fn new( + pub(crate) async fn new( client: Pulsar, config: PulsarSinkConfig, ) -> crate::Result { let producer_opts = config.build_producer_options(); let transformer = config.encoding.transformer(); let serializer = config.encoding.build()?; - let request_settings = config.request.unwrap_with(&TowerRequestConfig::default()); let encoder = Encoder::<()>::new(serializer); - let service = PulsarService::new(client, producer_opts, None, config.producer_name.clone()); + let service = PulsarService::new( + client, + producer_opts, + config.producer_name.clone(), + &config.topic, + ) + .await?; let topic = config.topic.clone(); Ok(PulsarSink { config, transformer, - request_settings, encoder, service, topic_template: Template::try_from(topic).context(TopicTemplateSnafu)?, @@ -119,8 +123,10 @@ impl PulsarSink { } async fn run_inner(self: Box, input: BoxStream<'_, Event>) -> Result<(), ()> { + let request_config = TowerRequestConfig::default(); + let request_settings = request_config.unwrap_with(&request_config); let service = ServiceBuilder::new() - .settings(self.request_settings, PulsarRetryLogic) + .settings(request_settings, PulsarRetryLogic) .service(self.service); let request_builder = PulsarRequestBuilder { encoder: PulsarEncoder { From 5da63218ac019736763f596e1066472b97da6a0b Mon Sep 17 00:00:00 2001 From: neuronull Date: Tue, 21 Mar 2023 13:34:27 -0600 Subject: [PATCH 34/48] auto gen docs --- .../components/sinks/base/pulsar.cue | 145 +----------------- 1 file changed, 1 insertion(+), 144 deletions(-) diff --git a/website/cue/reference/components/sinks/base/pulsar.cue b/website/cue/reference/components/sinks/base/pulsar.cue index 0ae45d98ecdf7..870586412bfa4 100644 --- a/website/cue/reference/components/sinks/base/pulsar.cue +++ b/website/cue/reference/components/sinks/base/pulsar.cue @@ -269,7 +269,7 @@ base: components: sinks: pulsar: configuration: { type: string: examples: ["message", "my_field"] } producer_name: { - description: "The name of the producer. If not specified, the default name assigned by Pulsar will be used." + description: "The name of the producer. If not specified, Pulsar will generate a unique name." required: false type: string: examples: ["producer-name"] } @@ -282,149 +282,6 @@ base: components: sinks: pulsar: configuration: { required: false type: string: {} } - request: { - description: """ - Middleware settings for outbound requests. - - Various settings can be configured, such as concurrency and rate limits, timeouts, etc. - """ - required: false - type: object: options: { - adaptive_concurrency: { - description: """ - Configuration of adaptive concurrency parameters. - - These parameters typically do not require changes from the default, and incorrect values can lead to meta-stable or - unstable performance and sink behavior. Proceed with caution. - """ - required: false - type: object: options: { - decrease_ratio: { - description: """ - The fraction of the current value to set the new concurrency limit when decreasing the limit. - - Valid values are greater than `0` and less than `1`. Smaller values cause the algorithm to scale back rapidly - when latency increases. - - Note that the new limit is rounded down after applying this ratio. - """ - required: false - type: float: default: 0.9 - } - ewma_alpha: { - description: """ - The weighting of new measurements compared to older measurements. - - Valid values are greater than `0` and less than `1`. - - ARC uses an exponentially weighted moving average (EWMA) of past RTT measurements as a reference to compare with - the current RTT. Smaller values cause this reference to adjust more slowly, which may be useful if a service has - unusually high response variability. - """ - required: false - type: float: default: 0.4 - } - rtt_deviation_scale: { - description: """ - Scale of RTT deviations which are not considered anomalous. - - Valid values are greater than or equal to `0`, and we expect reasonable values to range from `1.0` to `3.0`. - - When calculating the past RTT average, we also compute a secondary “deviation” value that indicates how variable - those values are. We use that deviation when comparing the past RTT average to the current measurements, so we - can ignore increases in RTT that are within an expected range. This factor is used to scale up the deviation to - an appropriate range. Larger values cause the algorithm to ignore larger increases in the RTT. - """ - required: false - type: float: default: 2.5 - } - } - } - concurrency: { - description: "Configuration for outbound request concurrency." - required: false - type: { - string: { - default: "none" - enum: { - adaptive: """ - Concurrency will be managed by Vector's [Adaptive Request Concurrency][arc] feature. - - [arc]: https://vector.dev/docs/about/under-the-hood/networking/arc/ - """ - none: """ - A fixed concurrency of 1. - - Only one request can be outstanding at any given time. - """ - } - } - uint: {} - } - } - rate_limit_duration_secs: { - description: "The time window used for the `rate_limit_num` option." - required: false - type: uint: { - default: 1 - unit: "seconds" - } - } - rate_limit_num: { - description: "The maximum number of requests allowed within the `rate_limit_duration_secs` time window." - required: false - type: uint: { - default: 9223372036854775807 - unit: "requests" - } - } - retry_attempts: { - description: """ - The maximum number of retries to make for failed requests. - - The default, for all intents and purposes, represents an infinite number of retries. - """ - required: false - type: uint: { - default: 9223372036854775807 - unit: "retries" - } - } - retry_initial_backoff_secs: { - description: """ - The amount of time to wait before attempting the first retry for a failed request. - - After the first retry has failed, the fibonacci sequence will be used to select future backoffs. - """ - required: false - type: uint: { - default: 1 - unit: "seconds" - } - } - retry_max_duration_secs: { - description: "The maximum amount of time to wait between retries." - required: false - type: uint: { - default: 3600 - unit: "seconds" - } - } - timeout_secs: { - description: """ - The time a request can take before being aborted. - - It is highly recommended that you do not lower this value below the service’s internal timeout, as this could - create orphaned requests, pile on retries, and result in duplicate data downstream. - """ - required: false - type: uint: { - default: 60 - unit: "seconds" - } - } - } - } topic: { description: "The Pulsar topic name to write events to." required: true From ea1e13368b2308fb5b4bd69ff9a2cae333b342e9 Mon Sep 17 00:00:00 2001 From: neuronull Date: Wed, 22 Mar 2023 15:40:17 -0600 Subject: [PATCH 35/48] Revert "feedback sg- simplify the producer in the Service" This reverts commit 19f948700d83ac2966a86a521c01ff076af9c353. --- src/sinks/pulsar/config.rs | 30 +++++---- src/sinks/pulsar/integration_tests.rs | 2 +- src/sinks/pulsar/request_builder.rs | 2 + src/sinks/pulsar/service.rs | 90 +++++++++++++++++++++++---- src/sinks/pulsar/sink.rs | 42 ++++++------- 5 files changed, 112 insertions(+), 54 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index ac13c794bab59..f9fb71a31e4c0 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -1,29 +1,27 @@ +use crate::sinks::util::TowerRequestConfig; +use crate::{ + codecs::EncodingConfig, + config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, + schema, + sinks::{ + pulsar::sink::{healthcheck, PulsarSink}, + Healthcheck, VectorSink, + }, +}; +use codecs::{encoding::SerializerConfig, TextSerializerConfig}; use futures_util::FutureExt; -use snafu::ResultExt; -use value::Kind; - use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; use pulsar::error::AuthenticationError; use pulsar::{ compression, message::proto, Authentication, Error as PulsarError, ProducerOptions, Pulsar, TokioExecutor, }; - -use codecs::{encoding::SerializerConfig, TextSerializerConfig}; +use snafu::ResultExt; +use value::Kind; use vector_common::sensitive_string::SensitiveString; use vector_config::configurable_component; use vector_core::config::DataType; -use crate::{ - codecs::EncodingConfig, - config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, - schema, - sinks::{ - pulsar::sink::{healthcheck, PulsarSink}, - Healthcheck, VectorSink, - }, -}; - /// Configuration for the `pulsar` sink. #[configurable_component(sink("pulsar"))] #[derive(Clone, Debug)] @@ -274,7 +272,7 @@ impl SinkConfig for PulsarSinkConfig { .await .context(super::sink::CreatePulsarSinkSnafu)?; - let sink = PulsarSink::new(client, self.clone()).await?; + let sink = PulsarSink::new(client, self.clone())?; let hc = healthcheck(self.clone()).boxed(); diff --git a/src/sinks/pulsar/integration_tests.rs b/src/sinks/pulsar/integration_tests.rs index f05f1e2f9fbdf..9de27aec8fd3e 100644 --- a/src/sinks/pulsar/integration_tests.rs +++ b/src/sinks/pulsar/integration_tests.rs @@ -61,7 +61,7 @@ async fn pulsar_happy_reuse(mut cnf: PulsarSinkConfig) { .unwrap(); assert_sink_compliance(&SINK_TAGS, async move { - let sink = PulsarSink::new(pulsar, cnf).await.unwrap(); + let sink = PulsarSink::new(pulsar, cnf).unwrap(); let sink = VectorSink::from_event_streamsink(sink); sink.run(input_events).await }) diff --git a/src/sinks/pulsar/request_builder.rs b/src/sinks/pulsar/request_builder.rs index 46de3966300f7..ec104d0ebf508 100644 --- a/src/sinks/pulsar/request_builder.rs +++ b/src/sinks/pulsar/request_builder.rs @@ -20,6 +20,7 @@ pub(super) struct PulsarMetadata { pub key: Option, pub properties: Option>, pub timestamp_millis: Option, + pub topic: String, } pub(super) struct PulsarRequestBuilder { @@ -53,6 +54,7 @@ impl RequestBuilder for PulsarRequestBuilder { key: input.key, timestamp_millis: input.timestamp_millis, properties: input.properties, + topic: input.topic, }; (metadata, builder, input.event) } diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 1a19374ea8577..b87d89a5ab236 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -3,6 +3,7 @@ use std::task::{Context, Poll}; use bytes::Bytes; use futures::future::BoxFuture; +use lru::LruCache; use pulsar::{Error as PulsarError, Executor, Producer, ProducerOptions, Pulsar}; use tokio::sync::Mutex; use tower::Service; @@ -13,6 +14,7 @@ use crate::event::{EventFinalizers, EventStatus, Finalizable}; use crate::internal_events::PulsarSendingError; use crate::sinks::pulsar::request_builder::PulsarMetadata; use crate::sinks::util::retries::RetryLogic; +use std::num::NonZeroUsize; use vector_common::request_metadata::{MetaDescriptive, RequestMetadata}; #[derive(Clone)] @@ -61,35 +63,90 @@ impl RetryLogic for PulsarRetryLogic { type Response = PulsarResponse; fn is_retriable_error(&self, _error: &Self::Error) -> bool { - // TODO expand retry logic on different pulsar error types + // TODO improve retry logic true } } +type SafeLru = Arc>>, PulsarError>>>>; + #[derive(Clone)] pub struct PulsarService { - // pulsar::Producer does not implement Clone - producer: Arc>>, + pulsar_client: Pulsar, + producer_cache: SafeLru, + producer_options: ProducerOptions, + producer_name: Option, } impl PulsarService { - pub(crate) async fn new( + pub(crate) fn new( pulsar_client: Pulsar, producer_options: ProducerOptions, + producer_cache_size: Option, producer_name: Option, + ) -> PulsarService { + // Use a LRUCache to store a limited set of producers + // Producers in Pulsar use a send buffer, so we want to limit the number of these + let producer_cache = Arc::new(Mutex::new(LruCache::new( + producer_cache_size.unwrap_or(NonZeroUsize::new(100).unwrap()), + ))); + PulsarService { + pulsar_client, + producer_cache, + producer_options, + producer_name, + } + } + + /// Build a producer that is wrapped in an Arc to allow for the producer + /// to control access. + /// + /// NOTE: Pulsar client library should likely be improved to simplify this + async fn build_producer( + client: Pulsar, + producer_options: ProducerOptions, topic: &String, - ) -> Result, pulsar::Error> { - let mut builder = pulsar_client + name: Option, + ) -> Result>>, PulsarError> { + let mut builder = client .producer() .with_topic(topic) .with_options(producer_options); - if let Some(name) = producer_name { + if let Some(name) = name { builder = builder.with_name(name); } - builder.build().await.map(|producer| PulsarService { - producer: Arc::new(Mutex::new(producer)), - }) + + match builder.build().await { + Ok(p) => Ok(Arc::new(Mutex::new(p))), + Err(e) => Err(e), + } + } + + /// Pulsar requires a producer object be created per topic + /// This method will build a producer if it hasn't been created or caches it otherwise + async fn get_or_build_producer( + producer_cache: SafeLru, + client: Pulsar, + producer_options: ProducerOptions, + topic: String, + name: Option, + ) -> Arc>> { + let mut pc = producer_cache.lock().await; + match pc.contains(&topic) { + false => { + pc.put( + topic.clone(), + PulsarService::build_producer(client, producer_options, &topic, name).await, + ); + let f = pc.get(&topic).unwrap().as_ref().unwrap(); + Arc::clone(f) + } + true => { + let f = pc.get(&topic).unwrap().as_ref().unwrap(); + Arc::clone(f) + } + } } } @@ -103,12 +160,19 @@ impl Service for PulsarService { } fn call(&mut self, request: PulsarRequest) -> Self::Future { - let arc_producer = Arc::clone(&self.producer); + let prod_future = PulsarService::get_or_build_producer( + Arc::clone(&self.producer_cache), + self.pulsar_client.clone(), + self.producer_options.clone(), + request.metadata.topic.clone(), + self.producer_name.clone(), + ); let ts = request.metadata.timestamp_millis.to_owned(); Box::pin(async move { - let mut producer = arc_producer.lock().await; + let p = prod_future.await; + let mut lp = p.lock().await; let body = request.body.clone(); - let mut msg_builder = producer.create_message().with_content(body.as_ref()); + let mut msg_builder = lp.create_message().with_content(body.as_ref()); if let Some(key) = request.metadata.key { msg_builder = msg_builder.with_key(String::from_utf8_lossy(&key)); } diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index 09688c666f3f0..59c0c27424d25 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -1,26 +1,25 @@ use async_trait::async_trait; use bytes::Bytes; -use futures::{stream::BoxStream, StreamExt}; -use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; -use serde::Serialize; -use snafu::{ResultExt, Snafu}; use std::collections::HashMap; -use tower::ServiceBuilder; -use super::{ - config::PulsarSinkConfig, - encoder::PulsarEncoder, - request_builder::PulsarRequestBuilder, - service::{PulsarRetryLogic, PulsarService}, - util, +use crate::sinks::pulsar::config::PulsarSinkConfig; +use crate::sinks::pulsar::encoder::PulsarEncoder; +use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; +use crate::sinks::pulsar::service::{PulsarRetryLogic, PulsarService}; +use crate::sinks::pulsar::util; +use crate::sinks::util::{ + ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig, TowerRequestSettings, }; - use crate::template::{Template, TemplateParseError}; use crate::{ codecs::{Encoder, Transformer}, event::Event, - sinks::util::{ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig}, }; +use futures::{stream::BoxStream, StreamExt}; +use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; +use serde::Serialize; +use snafu::{ResultExt, Snafu}; +use tower::ServiceBuilder; use vector_buffers::EventCount; use vector_common::byte_size_of::ByteSizeOf; use vector_core::{ @@ -41,6 +40,7 @@ pub(crate) struct PulsarSink { transformer: Transformer, encoder: Encoder<()>, service: PulsarService, + request_settings: TowerRequestSettings, config: PulsarSinkConfig, topic_template: Template, } @@ -96,26 +96,22 @@ pub(crate) async fn healthcheck(config: PulsarSinkConfig) -> crate::Result<()> { } impl PulsarSink { - pub(crate) async fn new( + pub(crate) fn new( client: Pulsar, config: PulsarSinkConfig, ) -> crate::Result { let producer_opts = config.build_producer_options(); let transformer = config.encoding.transformer(); let serializer = config.encoding.build()?; + let request_settings = config.request.unwrap_with(&TowerRequestConfig::default()); let encoder = Encoder::<()>::new(serializer); - let service = PulsarService::new( - client, - producer_opts, - config.producer_name.clone(), - &config.topic, - ) - .await?; + let service = PulsarService::new(client, producer_opts, None, config.producer_name.clone()); let topic = config.topic.clone(); Ok(PulsarSink { config, transformer, + request_settings, encoder, service, topic_template: Template::try_from(topic).context(TopicTemplateSnafu)?, @@ -123,10 +119,8 @@ impl PulsarSink { } async fn run_inner(self: Box, input: BoxStream<'_, Event>) -> Result<(), ()> { - let request_config = TowerRequestConfig::default(); - let request_settings = request_config.unwrap_with(&request_config); let service = ServiceBuilder::new() - .settings(request_settings, PulsarRetryLogic) + .settings(self.request_settings, PulsarRetryLogic) .service(self.service); let request_builder = PulsarRequestBuilder { encoder: PulsarEncoder { From f0eaa9a8d4567ac89628fde1df643866695fd8bb Mon Sep 17 00:00:00 2001 From: neuronull Date: Wed, 22 Mar 2023 17:09:33 -0600 Subject: [PATCH 36/48] multi producer magic? --- src/sinks/pulsar/config.rs | 1 - src/sinks/pulsar/service.rs | 142 ++++++++++-------------------------- src/sinks/pulsar/sink.rs | 15 +--- 3 files changed, 43 insertions(+), 115 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index f9fb71a31e4c0..d64a51743c929 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -1,4 +1,3 @@ -use crate::sinks::util::TowerRequestConfig; use crate::{ codecs::EncodingConfig, config::{AcknowledgementsConfig, GenerateConfig, Input, SinkConfig, SinkContext}, diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index b87d89a5ab236..903a2a4390a95 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -1,10 +1,11 @@ +use std::collections::HashMap; use std::sync::Arc; use std::task::{Context, Poll}; use bytes::Bytes; use futures::future::BoxFuture; -use lru::LruCache; -use pulsar::{Error as PulsarError, Executor, Producer, ProducerOptions, Pulsar}; +use pulsar::producer::Message; +use pulsar::{Error as PulsarError, Executor, MultiTopicProducer, ProducerOptions, Pulsar}; use tokio::sync::Mutex; use tower::Service; use vector_common::internal_event::CountByteSize; @@ -13,8 +14,6 @@ use vector_core::stream::DriverResponse; use crate::event::{EventFinalizers, EventStatus, Finalizable}; use crate::internal_events::PulsarSendingError; use crate::sinks::pulsar::request_builder::PulsarMetadata; -use crate::sinks::util::retries::RetryLogic; -use std::num::NonZeroUsize; use vector_common::request_metadata::{MetaDescriptive, RequestMetadata}; #[derive(Clone)] @@ -54,98 +53,29 @@ impl MetaDescriptive for PulsarRequest { } } -/// Pulsar retry logic. -#[derive(Debug, Default, Clone)] -pub struct PulsarRetryLogic; - -impl RetryLogic for PulsarRetryLogic { - type Error = PulsarError; - type Response = PulsarResponse; - - fn is_retriable_error(&self, _error: &Self::Error) -> bool { - // TODO improve retry logic - true - } -} - -type SafeLru = Arc>>, PulsarError>>>>; - #[derive(Clone)] pub struct PulsarService { - pulsar_client: Pulsar, - producer_cache: SafeLru, - producer_options: ProducerOptions, - producer_name: Option, + producer: Arc>>, } impl PulsarService { pub(crate) fn new( pulsar_client: Pulsar, producer_options: ProducerOptions, - producer_cache_size: Option, producer_name: Option, ) -> PulsarService { - // Use a LRUCache to store a limited set of producers - // Producers in Pulsar use a send buffer, so we want to limit the number of these - let producer_cache = Arc::new(Mutex::new(LruCache::new( - producer_cache_size.unwrap_or(NonZeroUsize::new(100).unwrap()), - ))); - PulsarService { - pulsar_client, - producer_cache, - producer_options, - producer_name, - } - } + // TODO put retry logic into building the pulsar client (with_connection_retry_options() / + // with_operation_retry_options) + let mut builder = pulsar_client.producer().with_options(producer_options); - /// Build a producer that is wrapped in an Arc to allow for the producer - /// to control access. - /// - /// NOTE: Pulsar client library should likely be improved to simplify this - async fn build_producer( - client: Pulsar, - producer_options: ProducerOptions, - topic: &String, - name: Option, - ) -> Result>>, PulsarError> { - let mut builder = client - .producer() - .with_topic(topic) - .with_options(producer_options); - - if let Some(name) = name { + if let Some(name) = producer_name { builder = builder.with_name(name); } - match builder.build().await { - Ok(p) => Ok(Arc::new(Mutex::new(p))), - Err(e) => Err(e), - } - } + let producer = builder.build_multi_topic(); - /// Pulsar requires a producer object be created per topic - /// This method will build a producer if it hasn't been created or caches it otherwise - async fn get_or_build_producer( - producer_cache: SafeLru, - client: Pulsar, - producer_options: ProducerOptions, - topic: String, - name: Option, - ) -> Arc>> { - let mut pc = producer_cache.lock().await; - match pc.contains(&topic) { - false => { - pc.put( - topic.clone(), - PulsarService::build_producer(client, producer_options, &topic, name).await, - ); - let f = pc.get(&topic).unwrap().as_ref().unwrap(); - Arc::clone(f) - } - true => { - let f = pc.get(&topic).unwrap().as_ref().unwrap(); - Arc::clone(f) - } + PulsarService { + producer: Arc::new(Mutex::new(producer)), } } } @@ -160,33 +90,39 @@ impl Service for PulsarService { } fn call(&mut self, request: PulsarRequest) -> Self::Future { - let prod_future = PulsarService::get_or_build_producer( - Arc::clone(&self.producer_cache), - self.pulsar_client.clone(), - self.producer_options.clone(), - request.metadata.topic.clone(), - self.producer_name.clone(), - ); - let ts = request.metadata.timestamp_millis.to_owned(); + let producer = Arc::clone(&self.producer); + let topic = request.metadata.topic.clone(); + let event_time = request + .metadata + .timestamp_millis + .to_owned() + .map(|t| t as u64); + Box::pin(async move { - let p = prod_future.await; - let mut lp = p.lock().await; + let mut producer = producer.lock().await; let body = request.body.clone(); - let mut msg_builder = lp.create_message().with_content(body.as_ref()); - if let Some(key) = request.metadata.key { - msg_builder = msg_builder.with_key(String::from_utf8_lossy(&key)); - } - if let Some(timestamp) = ts { - msg_builder = msg_builder.event_time(timestamp as u64); - } - if let Some(properties) = request.metadata.properties { - for (key, value) in properties { - msg_builder = - msg_builder.with_property(key, String::from_utf8_lossy(&value.clone())); + + let mut properties = HashMap::new(); + if let Some(props) = request.metadata.properties { + for (key, value) in props { + properties.insert(key, String::from_utf8_lossy(&value).to_string()); } } - match msg_builder.send().await { + let partition_key = request + .metadata + .key + .map(|key| String::from_utf8_lossy(&key).to_string()); + + let message = Message { + payload: body.as_ref().to_vec(), + properties, + partition_key, + event_time, + ....Default::default() + }; + + match producer.send(topic, message).await { Ok(resp) => match resp.await { Ok(_) => Ok(PulsarResponse { event_byte_size: request.request_metadata.events_byte_size(), diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index 59c0c27424d25..ba48b06253b53 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -5,11 +5,9 @@ use std::collections::HashMap; use crate::sinks::pulsar::config::PulsarSinkConfig; use crate::sinks::pulsar::encoder::PulsarEncoder; use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; -use crate::sinks::pulsar::service::{PulsarRetryLogic, PulsarService}; +use crate::sinks::pulsar::service::PulsarService; use crate::sinks::pulsar::util; -use crate::sinks::util::{ - ServiceBuilderExt, SinkBuilderExt, TowerRequestConfig, TowerRequestSettings, -}; +use crate::sinks::util::SinkBuilderExt; use crate::template::{Template, TemplateParseError}; use crate::{ codecs::{Encoder, Transformer}, @@ -40,7 +38,6 @@ pub(crate) struct PulsarSink { transformer: Transformer, encoder: Encoder<()>, service: PulsarService, - request_settings: TowerRequestSettings, config: PulsarSinkConfig, topic_template: Template, } @@ -103,15 +100,13 @@ impl PulsarSink { let producer_opts = config.build_producer_options(); let transformer = config.encoding.transformer(); let serializer = config.encoding.build()?; - let request_settings = config.request.unwrap_with(&TowerRequestConfig::default()); let encoder = Encoder::<()>::new(serializer); - let service = PulsarService::new(client, producer_opts, None, config.producer_name.clone()); + let service = PulsarService::new(client, producer_opts, config.producer_name.clone()); let topic = config.topic.clone(); Ok(PulsarSink { config, transformer, - request_settings, encoder, service, topic_template: Template::try_from(topic).context(TopicTemplateSnafu)?, @@ -119,9 +114,7 @@ impl PulsarSink { } async fn run_inner(self: Box, input: BoxStream<'_, Event>) -> Result<(), ()> { - let service = ServiceBuilder::new() - .settings(self.request_settings, PulsarRetryLogic) - .service(self.service); + let service = ServiceBuilder::new().service(self.service); let request_builder = PulsarRequestBuilder { encoder: PulsarEncoder { transformer: self.transformer.clone(), From ec48acb2ba501211e81f06c04612a3b80b0aff94 Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 08:48:46 -0600 Subject: [PATCH 37/48] two too many --- src/sinks/pulsar/service.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 903a2a4390a95..331fa032b3774 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -119,7 +119,7 @@ impl Service for PulsarService { properties, partition_key, event_time, - ....Default::default() + ..Default::default() }; match producer.send(topic, message).await { From 3de6bfd590008db1b26412612cacb10e49965e2a Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 11:10:09 -0600 Subject: [PATCH 38/48] make topic a Template in config --- src/sinks/pulsar/config.rs | 6 ++++-- src/sinks/pulsar/sink.rs | 34 +++++++++++++++------------------- 2 files changed, 19 insertions(+), 21 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index d64a51743c929..460aa21964e93 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -6,6 +6,7 @@ use crate::{ pulsar::sink::{healthcheck, PulsarSink}, Healthcheck, VectorSink, }, + template::Template, }; use codecs::{encoding::SerializerConfig, TextSerializerConfig}; use futures_util::FutureExt; @@ -34,7 +35,7 @@ pub struct PulsarSinkConfig { /// The Pulsar topic name to write events to. #[configurable(metadata(docs::examples = "topic-1234"))] - pub(crate) topic: String, + pub(crate) topic: Template, /// The name of the producer. If not specified, Pulsar will generate a unique name. #[configurable(metadata(docs::examples = "producer-name"))] @@ -166,7 +167,8 @@ impl Default for PulsarSinkConfig { fn default() -> Self { Self { endpoint: "pulsar://127.0.0.1:6650".to_string(), - topic: "topic-1234".to_string(), + topic: Template::try_from("topic-1234") + .expect("Unable to parse default template topic"), producer_name: None, properties_key: None, partition_key_field: None, diff --git a/src/sinks/pulsar/sink.rs b/src/sinks/pulsar/sink.rs index ba48b06253b53..0d7e13ec6c84d 100644 --- a/src/sinks/pulsar/sink.rs +++ b/src/sinks/pulsar/sink.rs @@ -1,23 +1,18 @@ use async_trait::async_trait; use bytes::Bytes; +use futures::{stream::BoxStream, StreamExt}; +use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; +use serde::Serialize; +use snafu::Snafu; use std::collections::HashMap; +use tower::ServiceBuilder; -use crate::sinks::pulsar::config::PulsarSinkConfig; -use crate::sinks::pulsar::encoder::PulsarEncoder; -use crate::sinks::pulsar::request_builder::PulsarRequestBuilder; -use crate::sinks::pulsar::service::PulsarService; -use crate::sinks::pulsar::util; -use crate::sinks::util::SinkBuilderExt; -use crate::template::{Template, TemplateParseError}; use crate::{ codecs::{Encoder, Transformer}, event::Event, + sinks::util::SinkBuilderExt, + template::Template, }; -use futures::{stream::BoxStream, StreamExt}; -use pulsar::{Error as PulsarError, Pulsar, TokioExecutor}; -use serde::Serialize; -use snafu::{ResultExt, Snafu}; -use tower::ServiceBuilder; use vector_buffers::EventCount; use vector_common::byte_size_of::ByteSizeOf; use vector_core::{ @@ -25,13 +20,16 @@ use vector_core::{ sink::StreamSink, }; +use super::{ + config::PulsarSinkConfig, encoder::PulsarEncoder, request_builder::PulsarRequestBuilder, + service::PulsarService, util, +}; + #[derive(Debug, Snafu)] #[snafu(visibility(pub(crate)))] pub(crate) enum BuildError { #[snafu(display("creating pulsar producer failed: {}", source))] CreatePulsarSink { source: PulsarError }, - #[snafu(display("invalid topic template: {}", source))] - TopicTemplate { source: TemplateParseError }, } pub(crate) struct PulsarSink { @@ -85,9 +83,7 @@ impl EstimatedJsonEncodedSizeOf for PulsarEvent { pub(crate) async fn healthcheck(config: PulsarSinkConfig) -> crate::Result<()> { let client = config.create_pulsar_client().await?; - let topic = Template::try_from(config.topic) - .context(TopicTemplateSnafu)? - .render_string(&LogEvent::from_str_legacy(""))?; + let topic = config.topic.render_string(&LogEvent::from_str_legacy(""))?; client.lookup_topic(topic).await?; Ok(()) } @@ -102,14 +98,14 @@ impl PulsarSink { let serializer = config.encoding.build()?; let encoder = Encoder::<()>::new(serializer); let service = PulsarService::new(client, producer_opts, config.producer_name.clone()); - let topic = config.topic.clone(); + let topic_template = config.topic.clone(); Ok(PulsarSink { config, transformer, encoder, service, - topic_template: Template::try_from(topic).context(TopicTemplateSnafu)?, + topic_template, }) } From bad92815bd2e9e0cfd7a431ed45dbe30b6cd15f3 Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 11:31:08 -0600 Subject: [PATCH 39/48] batch settings one line --- src/sinks/pulsar/config.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 460aa21964e93..213b92b5533c2 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -218,7 +218,7 @@ impl PulsarSinkConfig { access_mode: Some(0), metadata: Default::default(), schema: None, - batch_size: None, + batch_size: self.batch.max_events, compression: None, }; @@ -246,8 +246,6 @@ impl PulsarSinkConfig { } } - opts.batch_size = self.batch.max_events; - if let SerializerConfig::Avro { avro } = self.encoding.config() { opts.schema = Some(proto::Schema { schema_data: avro.schema.as_bytes().into(), From b0841b5873495ffb7f97bb903f03de24125abc18 Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 11:37:09 -0600 Subject: [PATCH 40/48] batch comment --- src/sinks/pulsar/config.rs | 3 +++ .../cue/reference/components/sinks/base/pulsar.cue | 14 +++++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 213b92b5533c2..77e3998d4d2a4 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -83,6 +83,9 @@ pub struct PulsarSinkConfig { #[derive(Clone, Copy, Debug, Default)] pub(crate) struct PulsarBatchConfig { /// The maximum size of a batch before it is flushed. + /// + /// Note this is an unsigned 32 bit integer which is a smaller capacity than + /// many of the other sink batch settings. #[configurable(metadata(docs::type_unit = "events"))] #[configurable(metadata(docs::examples = 1000))] pub max_events: Option, diff --git a/website/cue/reference/components/sinks/base/pulsar.cue b/website/cue/reference/components/sinks/base/pulsar.cue index 870586412bfa4..2e4e6c7bb6271 100644 --- a/website/cue/reference/components/sinks/base/pulsar.cue +++ b/website/cue/reference/components/sinks/base/pulsar.cue @@ -87,8 +87,13 @@ base: components: sinks: pulsar: configuration: { description: "Event batching behavior." required: false type: object: options: max_events: { - description: "The maximum size of a batch before it is flushed." - required: false + description: """ + The maximum size of a batch before it is flushed. + + Note this is an unsigned 32 bit integer which is a smaller capacity than + many of the other sink batch settings. + """ + required: false type: uint: { examples: [1000] unit: "events" @@ -285,6 +290,9 @@ base: components: sinks: pulsar: configuration: { topic: { description: "The Pulsar topic name to write events to." required: true - type: string: examples: ["topic-1234"] + type: string: { + examples: ["topic-1234"] + syntax: "template" + } } } From da06b92b6c8b3efbb9af6120fee55e56eaf67e1a Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 11:50:43 -0600 Subject: [PATCH 41/48] add retry logic to client builder --- src/sinks/pulsar/config.rs | 16 +++++++++++++--- src/sinks/pulsar/service.rs | 2 -- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index 77e3998d4d2a4..eb15892b01fb6 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -10,12 +10,14 @@ use crate::{ }; use codecs::{encoding::SerializerConfig, TextSerializerConfig}; use futures_util::FutureExt; -use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; -use pulsar::error::AuthenticationError; use pulsar::{ - compression, message::proto, Authentication, Error as PulsarError, ProducerOptions, Pulsar, + authentication::oauth2::{OAuth2Authentication, OAuth2Params}, + compression, + message::proto, + Authentication, ConnectionRetryOptions, Error as PulsarError, ProducerOptions, Pulsar, TokioExecutor, }; +use pulsar::{error::AuthenticationError, OperationRetryOptions}; use snafu::ResultExt; use value::Kind; use vector_common::sensitive_string::SensitiveString; @@ -212,6 +214,14 @@ impl PulsarSinkConfig { }; } + // Apply configuration for reconnection exponential backoff. + let retry_opts = ConnectionRetryOptions::default(); + builder = builder.with_connection_retry_options(retry_opts); + + // Apply configuration for retrying Pulsar operations. + let operation_retry_opts = OperationRetryOptions::default(); + builder = builder.with_operation_retry_options(operation_retry_opts); + builder.build().await } diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 331fa032b3774..b6c7d74f85ded 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -64,8 +64,6 @@ impl PulsarService { producer_options: ProducerOptions, producer_name: Option, ) -> PulsarService { - // TODO put retry logic into building the pulsar client (with_connection_retry_options() / - // with_operation_retry_options) let mut builder = pulsar_client.producer().with_options(producer_options); if let Some(name) = producer_name { From 52880293532907cea973f1be79fcd2340590d008 Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 14:50:37 -0600 Subject: [PATCH 42/48] use optionaltargetpath --- src/internal_events/pulsar.rs | 8 +++---- src/sinks/pulsar/config.rs | 13 +++++++----- src/sinks/pulsar/tests.rs | 11 ++++++---- src/sinks/pulsar/util.rs | 39 +++++++++++++++++++---------------- 4 files changed, 40 insertions(+), 31 deletions(-) diff --git a/src/internal_events/pulsar.rs b/src/internal_events/pulsar.rs index 1ce64909b8c90..ffa5f51883d8a 100644 --- a/src/internal_events/pulsar.rs +++ b/src/internal_events/pulsar.rs @@ -34,18 +34,18 @@ impl InternalEvent for PulsarSendingError { } } -pub struct PulsarPropertyExtractionError<'a> { - pub property_field: &'a str, +pub struct PulsarPropertyExtractionError { + pub property_field: F, } -impl InternalEvent for PulsarPropertyExtractionError<'_> { +impl InternalEvent for PulsarPropertyExtractionError { fn emit(self) { error!( message = "Failed to extract properties. Value should be a map of String -> Bytes.", error_code = "extracting_property", error_type = error_type::PARSER_FAILED, stage = error_stage::PROCESSING, - property_field = self.property_field, + property_field = %self.property_field, internal_log_rate_limit = true, ); counter!( diff --git a/src/sinks/pulsar/config.rs b/src/sinks/pulsar/config.rs index eb15892b01fb6..5fd61f8cb5489 100644 --- a/src/sinks/pulsar/config.rs +++ b/src/sinks/pulsar/config.rs @@ -10,6 +10,7 @@ use crate::{ }; use codecs::{encoding::SerializerConfig, TextSerializerConfig}; use futures_util::FutureExt; +use lookup::lookup_v2::OptionalTargetPath; use pulsar::{ authentication::oauth2::{OAuth2Authentication, OAuth2Params}, compression, @@ -43,19 +44,21 @@ pub struct PulsarSinkConfig { #[configurable(metadata(docs::examples = "producer-name"))] pub(crate) producer_name: Option, - /// The log field name or tags key to use for the topic key. + /// The log field name or tags key to use for the partition key. /// - /// If the field does not exist in the log or in tags, a blank value will be used. If unspecified, the key is not sent. + /// If the field does not exist in the log event or metric tags, a blank value will be used. + /// + /// If omitted, the key is not sent. /// /// Pulsar uses a hash of the key to choose the topic-partition or uses round-robin if the record has no key. #[configurable(metadata(docs::examples = "message"))] #[configurable(metadata(docs::examples = "my_field"))] - pub(crate) partition_key_field: Option, + pub(crate) partition_key_field: Option, - /// The log field name to use for the Pulsar properties. + /// The log field name to use for the Pulsar properties key. /// /// If omitted, no properties will be written. - pub properties_key: Option, + pub properties_key: Option, #[configurable(derived)] #[serde(default)] diff --git a/src/sinks/pulsar/tests.rs b/src/sinks/pulsar/tests.rs index c78ce04f50f77..3464f194c89bb 100644 --- a/src/sinks/pulsar/tests.rs +++ b/src/sinks/pulsar/tests.rs @@ -1,5 +1,6 @@ use crate::event::Event; use crate::sinks::pulsar::config::PulsarSinkConfig; +use lookup::lookup_v2::OptionalTargetPath; use std::collections::BTreeMap; use value::Value; use vector_config::component::GenerateConfig; @@ -15,16 +16,18 @@ fn generate_config() { #[test] fn pulsar_get_headers() { - let properties_key = "properties"; + let properties_key = OptionalTargetPath::try_from("properties".to_string()) + .expect("unable to parse OptionalTargetPath"); let mut property_values = BTreeMap::new(); property_values.insert("a-key".to_string(), Value::Bytes(Bytes::from("a-value"))); property_values.insert("b-key".to_string(), Value::Bytes(Bytes::from("b-value"))); let mut event = Event::Log(LogEvent::from("hello")); - event.as_mut_log().insert(properties_key, property_values); + event + .as_mut_log() + .insert(properties_key.path.as_ref().unwrap(), property_values); - let properties = - super::util::get_properties(&event, &Some(properties_key.to_string())).unwrap(); + let properties = super::util::get_properties(&event, &Some(properties_key)).unwrap(); assert_eq!(properties.get("a-key").unwrap(), "a-value".as_bytes()); assert_eq!(properties.get("b-key").unwrap(), "b-value".as_bytes()); } diff --git a/src/sinks/pulsar/util.rs b/src/sinks/pulsar/util.rs index c6599d5915d81..f658bd8e394dc 100644 --- a/src/sinks/pulsar/util.rs +++ b/src/sinks/pulsar/util.rs @@ -3,6 +3,7 @@ use crate::sinks::pulsar::config::PulsarSinkConfig; use crate::sinks::pulsar::sink::PulsarEvent; use crate::template::Template; use bytes::Bytes; +use lookup::lookup_v2::OptionalTargetPath; use std::collections::HashMap; use value::Value; use vector_core::event::Event; @@ -27,16 +28,18 @@ pub(super) fn make_pulsar_event( }) } -fn get_key(event: &Event, partition_key_field: &Option) -> Option { +fn get_key(event: &Event, partition_key_field: &Option) -> Option { partition_key_field .as_ref() .and_then(|partition_key_field| match event { - Event::Log(log) => log - .get(partition_key_field.as_str()) - .map(|value| value.coerce_to_bytes()), - Event::Metric(metric) => metric - .tags() - .and_then(|tags| tags.get(partition_key_field)) + Event::Log(log) => partition_key_field + .path + .as_ref() + .and_then(|path| log.get(path).map(|value| value.coerce_to_bytes())), + Event::Metric(metric) => partition_key_field + .path + .as_ref() + .and_then(|path| metric.tags().and_then(|tags| tags.get(&path.to_string()))) .map(|value| value.to_owned().into()), _ => None, }) @@ -53,12 +56,12 @@ fn get_timestamp_millis(event: &Event) -> Option { pub(super) fn get_properties( event: &Event, - properties_key: &Option, + properties_key: &Option, ) -> Option> { properties_key.as_ref().and_then(|properties_key| { - if let Event::Log(log) = event { - if let Some(properties) = log.get(properties_key.as_str()) { - match properties { + properties_key.path.as_ref().and_then(|path| { + event.maybe_as_log().and_then(|log| { + log.get(path).and_then(|properties| match properties { Value::Object(headers_map) => { let mut property_map = HashMap::new(); for (key, value) in headers_map { @@ -66,20 +69,20 @@ pub(super) fn get_properties( property_map.insert(key.clone(), value_bytes.clone()); } else { emit!(PulsarPropertyExtractionError { - property_field: properties_key + property_field: path }); } } - return Some(property_map); + Some(property_map) } _ => { emit!(PulsarPropertyExtractionError { - property_field: properties_key + property_field: path }); + None } - } - } - } - None + }) + }) + }) }) } From 6d4368b8807f1da8f45f8fde3f1e3f4a389c08a9 Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 15:56:58 -0600 Subject: [PATCH 43/48] fix int tests --- src/sinks/pulsar/integration_tests.rs | 34 ++++++++++++++------------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/src/sinks/pulsar/integration_tests.rs b/src/sinks/pulsar/integration_tests.rs index 9de27aec8fd3e..ac11fa940cb0b 100644 --- a/src/sinks/pulsar/integration_tests.rs +++ b/src/sinks/pulsar/integration_tests.rs @@ -5,6 +5,7 @@ use std::collections::BTreeMap; use crate::event::Value; use crate::sinks::VectorSink; +use crate::template::Template; use crate::test_util::{ components::{assert_sink_compliance, SINK_TAGS}, random_lines_with_stream, random_string, trace_init, @@ -22,33 +23,35 @@ async fn pulsar_happy_reuse(mut cnf: PulsarSinkConfig) { let prop_1_value = "prop-1-value"; let num_events = 1_000; let (input, events) = random_lines_with_stream(100, num_events, None); + let prop_key_opt = cnf.properties_key.clone(); let input_events = events.map(move |mut events| { // if a property_key is defined, add some properties! - if let Some(prop_key) = &prop_key_opt { - let properties_key = prop_key; - let mut property_values = BTreeMap::new(); - property_values.insert( - prop_1_key.to_owned(), - Value::Bytes(Bytes::from(prop_1_value)), - ); - events.iter_logs_mut().for_each(move |log| { - log.insert(properties_key.as_str(), property_values.clone()); - }); - events - } else { - events + if let Some(properties_key) = &prop_key_opt { + if let Some(properties_key) = &properties_key.path { + let mut property_values = BTreeMap::new(); + property_values.insert( + prop_1_key.to_owned(), + Value::Bytes(Bytes::from(prop_1_value)), + ); + events.iter_logs_mut().for_each(move |log| { + log.insert(properties_key, property_values.clone()); + }); + return events; + } } + events }); - let topic = format!("test-{}", random_string(10)); + let topic_str = format!("test-{}", random_string(10)); + let topic = Template::try_from(topic_str.clone()).expect("Unable to parse template"); cnf.topic = topic.clone(); let pulsar = cnf.create_pulsar_client().await.unwrap(); let mut consumer = pulsar .consumer() - .with_topic(&topic) + .with_topic(&topic_str) .with_consumer_name("VectorTestConsumer") .with_subscription_type(SubType::Shared) .with_subscription("VectorTestSub") @@ -83,7 +86,6 @@ async fn pulsar_happy() { let cnf = PulsarSinkConfig { endpoint: pulsar_address(), // overriden by test - topic: "".to_string(), ..Default::default() }; From 906f6a1f1f3d1252c71e06cc6bc9432b05c5f76a Mon Sep 17 00:00:00 2001 From: neuronull Date: Thu, 23 Mar 2023 16:27:16 -0600 Subject: [PATCH 44/48] autogen docs --- website/cue/reference/components/sinks/base/pulsar.cue | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/website/cue/reference/components/sinks/base/pulsar.cue b/website/cue/reference/components/sinks/base/pulsar.cue index 2e4e6c7bb6271..2034a607bd4a0 100644 --- a/website/cue/reference/components/sinks/base/pulsar.cue +++ b/website/cue/reference/components/sinks/base/pulsar.cue @@ -264,9 +264,11 @@ base: components: sinks: pulsar: configuration: { } partition_key_field: { description: """ - The log field name or tags key to use for the topic key. + The log field name or tags key to use for the partition key. - If the field does not exist in the log or in tags, a blank value will be used. If unspecified, the key is not sent. + If the field does not exist in the log event or metric tags, a blank value will be used. + + If omitted, the key is not sent. Pulsar uses a hash of the key to choose the topic-partition or uses round-robin if the record has no key. """ @@ -280,7 +282,7 @@ base: components: sinks: pulsar: configuration: { } properties_key: { description: """ - The log field name to use for the Pulsar properties. + The log field name to use for the Pulsar properties key. If omitted, no properties will be written. """ From c3ed18b0c584922d869b7967becff7f7f8e0a80a Mon Sep 17 00:00:00 2001 From: neuronull Date: Mon, 10 Apr 2023 17:17:00 -0600 Subject: [PATCH 45/48] reduce scope of the lock being held --- src/sinks/pulsar/service.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index b6c7d74f85ded..e5fbab59a6a2b 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -53,7 +53,7 @@ impl MetaDescriptive for PulsarRequest { } } -#[derive(Clone)] +//#[derive(Clone)] pub struct PulsarService { producer: Arc>>, } @@ -97,7 +97,6 @@ impl Service for PulsarService { .map(|t| t as u64); Box::pin(async move { - let mut producer = producer.lock().await; let body = request.body.clone(); let mut properties = HashMap::new(); @@ -120,7 +119,9 @@ impl Service for PulsarService { ..Default::default() }; - match producer.send(topic, message).await { + let fut = producer.lock().await.send(topic, message).await; + + match fut { Ok(resp) => match resp.await { Ok(_) => Ok(PulsarResponse { event_byte_size: request.request_metadata.events_byte_size(), From 9e85fe9b5cc9c19a22e43836a2d4ae449091733e Mon Sep 17 00:00:00 2001 From: neuronull Date: Mon, 10 Apr 2023 17:24:32 -0600 Subject: [PATCH 46/48] remove unecessary clone --- src/sinks/pulsar/service.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index e5fbab59a6a2b..0a69338c2b234 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -53,7 +53,6 @@ impl MetaDescriptive for PulsarRequest { } } -//#[derive(Clone)] pub struct PulsarService { producer: Arc>>, } From 12111d25166cf661d3ec089e961a037c1f1c99bf Mon Sep 17 00:00:00 2001 From: neuronull Date: Mon, 10 Apr 2023 17:32:08 -0600 Subject: [PATCH 47/48] check lock readiness in poll_ready() --- src/sinks/pulsar/service.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 0a69338c2b234..6d1ff8a2b90ab 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -83,7 +83,10 @@ impl Service for PulsarService { type Future = BoxFuture<'static, Result>; fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll> { - Poll::Ready(Ok(())) + match self.producer.try_lock() { + Ok(_) => Poll::Ready(Ok(())), + Err(_) => Poll::Pending, + } } fn call(&mut self, request: PulsarRequest) -> Self::Future { From f1a45e89ddcbfdd79e645364297fc8290a0ef461 Mon Sep 17 00:00:00 2001 From: neuronull Date: Tue, 11 Apr 2023 13:18:39 -0600 Subject: [PATCH 48/48] add comments about the mutex --- src/sinks/pulsar/service.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/sinks/pulsar/service.rs b/src/sinks/pulsar/service.rs index 6d1ff8a2b90ab..d723748fd1c7a 100644 --- a/src/sinks/pulsar/service.rs +++ b/src/sinks/pulsar/service.rs @@ -54,6 +54,8 @@ impl MetaDescriptive for PulsarRequest { } pub struct PulsarService { + // NOTE: the reason for the Mutex here is because the `Producer` from the pulsar crate + // needs to be `mut`, and the `Service::call()` returns a Future. producer: Arc>>, } @@ -121,6 +123,12 @@ impl Service for PulsarService { ..Default::default() }; + // The locking if this mutex is not normal in `Service::call()` implementations, but we + // at least can limit the scope of the lock by placing it here, and reduce the + // possibility of performance impact by checking the `try_lock()` result in + // `poll_ready()`. This sink is already limited to sequential request handling due to + // the pulsar API, so this shouldn't impact performance from a concurrent requests + // standpoint. let fut = producer.lock().await.send(topic, message).await; match fut {