From 16856f116f579046e2779f806e665c71b6bba177 Mon Sep 17 00:00:00 2001 From: Matthias Wahl Date: Thu, 19 Jan 2023 15:47:56 +0100 Subject: [PATCH] Add app id and flow id to common ids and logs Signed-off-by: Matthias Wahl --- src/codec.rs | 2 +- src/config.rs | 8 +- src/connectors.rs | 59 +++--- src/connectors/impls/elastic.rs | 8 +- src/connectors/impls/gbq/writer.rs | 20 +- src/connectors/impls/gbq/writer/sink.rs | 38 ++-- src/connectors/impls/gcl/writer/sink.rs | 18 +- src/connectors/impls/gcs/streamer.rs | 67 ++++--- src/connectors/impls/http/meta.rs | 4 +- src/connectors/impls/http/server.rs | 2 +- src/connectors/impls/kafka.rs | 4 +- src/connectors/impls/metronome.rs | 4 +- src/connectors/impls/oneshot.rs | 2 +- src/connectors/impls/otel/client.rs | 4 +- src/connectors/impls/otel/server.rs | 6 +- src/connectors/prelude.rs | 1 - src/connectors/sink.rs | 19 +- src/connectors/sink/channel_sink.rs | 2 +- src/connectors/source.rs | 21 +- src/connectors/tests.rs | 18 +- src/connectors/tests/bench.rs | 3 +- src/connectors/tests/http/client.rs | 2 +- src/connectors/tests/kafka/consumer.rs | 6 +- src/connectors/tests/wal.rs | 4 +- src/connectors/utils/reconnect.rs | 7 +- src/ids.rs | 122 ++++++++++++ src/lib.rs | 12 +- src/pipeline.rs | 70 ++++--- src/preprocessor.rs | 15 +- src/raft.rs | 24 ++- src/raft/api.rs | 104 ++++++---- src/raft/api/apps.rs | 93 +++++---- src/raft/api/client.rs | 80 ++++---- src/raft/api/cluster.rs | 67 ++++--- src/raft/api/kv.rs | 24 ++- src/raft/api/worker.rs | 14 +- src/raft/archive.rs | 15 +- src/raft/network/raft_network_impl.rs | 21 +- src/raft/node.rs | 66 ++++--- src/raft/store.rs | 115 +++++------ src/raft/store/statemachine/apps.rs | 186 ++++++++---------- src/raft/store/statemachine/nodes.rs | 26 ++- src/raft/test/learner.rs | 31 +-- src/system.rs | 84 ++++---- src/system/flow.rs | 102 ++++------ src/system/flow_supervisor.rs | 80 +++++--- static/openapi.yaml | 26 ++- tests/flows.rs | 8 +- tests/query.rs | 4 +- tests/query_error.rs | 4 +- tests/query_runtime_error.rs | 4 +- tremor-api/src/api.rs | 43 ++-- tremor-api/src/api/flow.rs | 15 +- tremor-api/src/api/model.rs | 5 +- tremor-cli/src/cluster.rs | 54 ++--- tremor-cli/src/debug.rs | 4 +- tremor-cli/src/doc.rs | 9 +- tremor-cli/src/run.rs | 7 +- tremor-cli/src/server.rs | 5 +- tremor-cli/src/test/unit.rs | 3 +- .../integration/system-metrics/config.troy | 11 +- tremor-common/src/lib.rs | 4 +- tremor-common/src/{ids.rs => uids.rs} | 71 +++---- tremor-erl/src/tremor_binding.erl | 77 -------- tremor-erl/src/tremor_flow.erl | 15 +- tremor-pipeline/src/event.rs | 12 +- tremor-pipeline/src/executable_graph.rs | 44 ++--- tremor-pipeline/src/lib.rs | 38 ++-- tremor-pipeline/src/macros.rs | 2 +- tremor-pipeline/src/op.rs | 10 +- tremor-pipeline/src/op/debug/history.rs | 8 +- tremor-pipeline/src/op/generic/batch.rs | 14 +- tremor-pipeline/src/op/generic/counter.rs | 2 +- tremor-pipeline/src/op/grouper/bucket.rs | 6 +- .../src/op/identity/passthrough.rs | 2 +- tremor-pipeline/src/op/qos/backpressure.rs | 16 +- tremor-pipeline/src/op/qos/percentile.rs | 12 +- tremor-pipeline/src/op/qos/roundrobin.rs | 10 +- tremor-pipeline/src/op/trickle/operator.rs | 8 +- tremor-pipeline/src/op/trickle/script.rs | 2 +- tremor-pipeline/src/op/trickle/select.rs | 6 +- tremor-pipeline/src/op/trickle/select/test.rs | 8 +- .../src/op/trickle/simple_select.rs | 2 +- tremor-pipeline/src/op/trickle/window.rs | 4 +- tremor-pipeline/src/query.rs | 16 +- tremor-script-nif/build.rs | 2 +- tremor-script/src/arena.rs | 4 +- tremor-script/src/ast/helper.rs | 5 +- tremor-script/src/ast/module.rs | 2 +- tremor-script/src/ast/optimizer.rs | 11 ++ tremor-script/src/ast/query/raw.rs | 2 +- tremor-script/src/ast/raw.rs | 2 +- tremor-script/src/deploy.rs | 8 +- tremor-script/src/lib.rs | 14 +- tremor-script/src/query.rs | 4 +- tremor-script/src/script.rs | 4 +- tremor-value/Cargo.toml | 2 +- 97 files changed, 1278 insertions(+), 1053 deletions(-) create mode 100644 src/ids.rs rename tremor-common/src/{ids.rs => uids.rs} (74%) delete mode 100644 tremor-erl/src/tremor_binding.erl diff --git a/src/codec.rs b/src/codec.rs index 8a722335b1..d4966b83a9 100644 --- a/src/codec.rs +++ b/src/codec.rs @@ -101,9 +101,9 @@ pub fn resolve(config: &config::Codec) -> Result> { "binflux" => Ok(Box::::default()), "csv" => Ok(Box::new(csv::Csv {})), "dogstatsd" => Ok(Box::::default()), + "influx" => Ok(Box::new(influx::Influx {})), "json" => json::from_config(config.config.as_ref()), "msgpack" => Ok(Box::new(msgpack::MsgPack {})), - "influx" => Ok(Box::new(influx::Influx {})), "null" => Ok(Box::new(null::Null {})), "statsd" => Ok(Box::::default()), "string" => Ok(Box::new(string::String {})), diff --git a/src/config.rs b/src/config.rs index 86c4892505..7bd7bebd1a 100644 --- a/src/config.rs +++ b/src/config.rs @@ -322,7 +322,7 @@ mod tests { use serde::Deserialize; use super::*; - use crate::{errors::Result, system::flow}; + use crate::{errors::Result, ids::FlowInstanceId}; #[test] fn test_reconnect_serde() -> Result<()> { @@ -351,7 +351,7 @@ mod tests { #[test] fn test_config_builtin_preproc_with_config() -> Result<()> { let c = Connector::from_config( - &Alias::new("flow", "my_otel_client"), + &Alias::new(FlowInstanceId::new("app", "flow"), "my_otel_client"), ConnectorType::from("otel_client".to_string()), &literal!({ "preprocessors": [ {"name": "snot", "config": { "separator": "\n" }}], @@ -382,10 +382,10 @@ mod tests { "reconnect": {}, "metrics_interval_s": "wrong_type" }); - let id = Alias::new(flow::Alias::new("flow"), "my_id"); + let id = Alias::new(FlowInstanceId::new("app", "flow"), "my_id"); let res = Connector::from_config(&id, "fancy_schmancy".into(), &config); assert!(res.is_err()); - assert_eq!(String::from("Invalid Definition for connector \"flow::my_id\": Expected type I64 for key metrics_interval_s but got String"), res.err().map(|e| e.to_string()).unwrap_or_default()); + assert_eq!(String::from("Invalid Definition for connector \"app/flow::my_id\": Expected type I64 for key metrics_interval_s but got String"), res.err().map(|e| e.to_string()).unwrap_or_default()); } #[test] diff --git a/src/connectors.rs b/src/connectors.rs index de67a8b777..f9bae453eb 100644 --- a/src/connectors.rs +++ b/src/connectors.rs @@ -29,35 +29,38 @@ mod google; #[cfg(test)] pub(crate) mod tests; -use self::metrics::{SinkReporter, SourceReporter}; -use self::sink::{SinkAddr, SinkContext, SinkMsg}; -use self::source::{SourceAddr, SourceContext, SourceMsg}; use self::utils::quiescence::QuiescenceBeacon; +use self::{prelude::Attempt, utils::reconnect}; +use self::{ + sink::{SinkAddr, SinkContext, SinkMsg}, + utils::{metrics::SourceReporter, reconnect::ConnectionLostNotifier}, +}; +use self::{ + source::{SourceAddr, SourceContext, SourceMsg}, + utils::{metrics::SinkReporter, reconnect::ReconnectRuntime}, +}; pub(crate) use crate::config::Connector as ConnectorConfig; use crate::{ channel::{bounded, Sender}, errors::{connector_send_err, Error, Kind as ErrorKind, Result}, + ids::FlowInstanceId, instance::State, log_error, pipeline, qsize, - system::{flow, KillSwitch, Runtime}, + system::{KillSwitch, Runtime}, }; use beef::Cow; use futures::Future; use halfbrown::HashMap; +use simd_json::{Builder, Mutable, ValueAccess}; use std::{fmt::Display, time::Duration}; use tokio::task::{self, JoinHandle}; use tremor_common::{ - ids::{ConnectorId, ConnectorIdGen, SourceId}, ports::{Port, ERR, IN, OUT}, + uids::{ConnectorUId, ConnectorUIdGen, SourceUId}, }; use tremor_pipeline::METRICS_CHANNEL; use tremor_script::ast::DeployEndpoint; use tremor_value::Value; -use utils::reconnect::{Attempt, ConnectionLostNotifier, ReconnectRuntime}; -use value_trait::{Builder, Mutable, ValueAccess}; - -/// quiescence stuff -pub(crate) use utils::{metrics, reconnect}; /// Accept timeout pub(crate) const ACCEPT_TIMEOUT: Duration = Duration::from_millis(100); @@ -314,6 +317,7 @@ pub(crate) trait Context: Display + Clone { /// connector context #[derive(Clone)] pub(crate) struct ConnectorContext { + node_id: openraft::NodeId, /// alias of the connector instance pub(crate) alias: Alias, /// type of the connector @@ -326,7 +330,7 @@ pub(crate) struct ConnectorContext { impl Display for ConnectorContext { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "[Connector::{}]", &self.alias) + write!(f, "[Node::{}][Connector::{}]", self.node_id, &self.alias) } } @@ -421,15 +425,23 @@ pub(crate) type Known = /// # Errors /// if the connector can not be built or the config is invalid pub(crate) async fn spawn( + node_id: openraft::NodeId, alias: &Alias, - connector_id_gen: &mut ConnectorIdGen, + connector_id_gen: &mut ConnectorUIdGen, builder: &dyn ConnectorBuilder, config: ConnectorConfig, kill_switch: &KillSwitch, ) -> Result { // instantiate connector let connector = builder.build(alias, &config, kill_switch).await?; - let r = connector_task(alias.clone(), connector, config, connector_id_gen.next_id()).await?; + let r = connector_task( + node_id, + alias.clone(), + connector, + config, + connector_id_gen.next_id(), + ) + .await?; Ok(r) } @@ -437,10 +449,11 @@ pub(crate) async fn spawn( #[allow(clippy::too_many_lines)] // instantiates the connector and starts listening for control plane messages async fn connector_task( + node_id: openraft::NodeId, alias: Alias, mut connector: Box, config: ConnectorConfig, - uid: ConnectorId, + uid: ConnectorUId, ) -> Result { let qsize = qsize(); // channel for connector-level control plane communication @@ -465,12 +478,13 @@ async fn connector_task( .into()); } let source_builder = source::builder( - SourceId::from(uid), + SourceUId::from(uid), &config, codec_requirement, source_metrics_reporter, )?; let source_ctx = SourceContext { + node_id, alias: alias.clone(), uid: uid.into(), connector_type: config.connector_type.clone(), @@ -485,6 +499,7 @@ async fn connector_task( ); let sink_builder = sink::builder(&config, codec_requirement, &alias, sink_metrics_reporter)?; let sink_ctx = SinkContext::new( + node_id, uid.into(), alias.clone(), config.connector_type.clone(), @@ -504,11 +519,11 @@ async fn connector_task( sink: sink_addr, }; - let mut reconnect: ReconnectRuntime = - ReconnectRuntime::new(&connector_addr, notifier.clone(), &config.reconnect); + let mut reconnect = ReconnectRuntime::new(&connector_addr, notifier.clone(), &config.reconnect); let notifier = reconnect.notifier(); let ctx = ConnectorContext { + node_id, alias: alias.clone(), connector_type: config.connector_type.clone(), quiescence_beacon: quiescence_beacon.clone(), @@ -1128,13 +1143,13 @@ where /// unique instance alias/id of a connector within a deployment #[derive(Debug, PartialEq, PartialOrd, Eq, Hash, Clone, Serialize, Deserialize)] pub struct Alias { - flow_alias: flow::Alias, + flow_alias: FlowInstanceId, connector_alias: String, } impl Alias { /// construct a new `ConnectorId` from the id of the containing flow and the connector instance id - pub fn new(flow_alias: impl Into, connector_alias: impl Into) -> Self { + pub fn new(flow_alias: impl Into, connector_alias: impl Into) -> Self { Self { flow_alias: flow_alias.into(), connector_alias: connector_alias.into(), @@ -1143,7 +1158,7 @@ impl Alias { /// get a reference to the flow alias #[must_use] - pub fn flow_alias(&self) -> &flow::Alias { + pub fn flow_alias(&self) -> &FlowInstanceId { &self.flow_alias } @@ -1296,7 +1311,7 @@ where #[cfg(test)] pub(crate) mod unit_tests { - use crate::system::flow; + use crate::ids::FlowInstanceId; use super::*; @@ -1312,7 +1327,7 @@ pub(crate) mod unit_tests { pub(crate) fn new(tx: Sender) -> Self { Self { t: ConnectorType::from("snot"), - alias: Alias::new(flow::Alias::new("fake"), "fake"), + alias: Alias::new(FlowInstanceId::new("app", "fake"), "fake"), notifier: reconnect::ConnectionLostNotifier::new(tx), beacon: QuiescenceBeacon::default(), } diff --git a/src/connectors/impls/elastic.rs b/src/connectors/impls/elastic.rs index f1d9dfbc6c..845af7b7a0 100644 --- a/src/connectors/impls/elastic.rs +++ b/src/connectors/impls/elastic.rs @@ -960,7 +960,7 @@ mod tests { use elasticsearch::http::request::Body; use super::*; - use crate::config::Connector as ConnectorConfig; + use crate::{config::Connector as ConnectorConfig, ids::FlowInstanceId}; #[tokio::test(flavor = "multi_thread")] async fn connector_builder_empty_nodes() -> Result<()> { @@ -969,14 +969,14 @@ mod tests { "nodes": [] } }); - let alias = Alias::new("flow", "my_elastic"); + let alias = Alias::new(FlowInstanceId::new("app", "flow"), "my_elastic"); let builder = super::Builder::default(); let connector_config = ConnectorConfig::from_config(&alias, builder.connector_type(), &config)?; let kill_switch = KillSwitch::dummy(); assert_eq!( String::from( - "Invalid Definition for connector \"flow::my_elastic\": empty nodes provided" + "Invalid Definition for connector \"app/flow::my_elastic\": empty nodes provided" ), builder .build(&alias, &connector_config, &kill_switch) @@ -998,7 +998,7 @@ mod tests { ] } }); - let alias = Alias::new("snot", "my_elastic"); + let alias = Alias::new(FlowInstanceId::new("app", "snot"), "my_elastic"); let builder = super::Builder::default(); let connector_config = ConnectorConfig::from_config(&alias, builder.connector_type(), &config)?; diff --git a/src/connectors/impls/gbq/writer.rs b/src/connectors/impls/gbq/writer.rs index 0730d74b45..f3050f7745 100644 --- a/src/connectors/impls/gbq/writer.rs +++ b/src/connectors/impls/gbq/writer.rs @@ -84,12 +84,13 @@ impl ConnectorBuilder for Builder { #[cfg(test)] mod tests { use tokio::sync::broadcast; - use tremor_common::ids::SinkId; + use tremor_common::uids::SinkUId; use super::*; - use crate::connectors::reconnect::ConnectionLostNotifier; use crate::connectors::sink::builder; - use crate::connectors::{metrics::SinkReporter, utils::quiescence::QuiescenceBeacon}; + use crate::connectors::utils::quiescence::QuiescenceBeacon; + use crate::connectors::{reconnect::ConnectionLostNotifier, utils::metrics::SinkReporter}; + use crate::ids::FlowInstanceId; #[tokio::test(flavor = "multi_thread")] pub async fn can_spawn_sink() -> Result<()> { @@ -105,8 +106,9 @@ mod tests { let sink_address = connector .create_sink( SinkContext::new( - SinkId::default(), - Alias::new("a", "b"), + openraft::NodeId::default(), + SinkUId::default(), + Alias::new(FlowInstanceId::new("app", "a"), "b"), ConnectorType::default(), QuiescenceBeacon::default(), ConnectionLostNotifier::new(crate::channel::bounded(128).0), @@ -114,8 +116,12 @@ mod tests { builder( &ConnectorConfig::default(), CodecReq::Structured, - &Alias::new("a", "b"), - SinkReporter::new(Alias::new("a", "b"), broadcast::channel(1).0, None), + &Alias::new(FlowInstanceId::new("app", "a"), "b"), + SinkReporter::new( + Alias::new(FlowInstanceId::new("app", "a"), "b"), + broadcast::channel(1).0, + None, + ), )?, ) .await?; diff --git a/src/connectors/impls/gbq/writer/sink.rs b/src/connectors/impls/gbq/writer/sink.rs index 81e19ebab7..648bf0d5fb 100644 --- a/src/connectors/impls/gbq/writer/sink.rs +++ b/src/connectors/impls/gbq/writer/sink.rs @@ -676,6 +676,7 @@ mod test { use crate::connectors::{ google::tests::TestTokenProvider, utils::quiescence::QuiescenceBeacon, }; + use crate::ids::{AppId, FlowInstanceId}; use bytes::Bytes; use futures::future::Ready; use googapis::google::cloud::bigquery::storage::v1::table_field_schema::Mode; @@ -691,7 +692,7 @@ mod test { use std::task::Poll; use tonic::body::BoxBody; use tonic::codegen::Service; - use tremor_common::ids::SinkId; + use tremor_common::uids::SinkUId; use value_trait::StaticNode; struct HardcodedChannelFactory { @@ -795,7 +796,8 @@ mod test { scale: 0, }], &SinkContext::new( - SinkId::default(), + openraft::NodeId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -824,7 +826,7 @@ mod test { scale: 0, }], &SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -862,7 +864,7 @@ mod test { scale: 0, }], &SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -902,7 +904,7 @@ mod test { scale: 0, }], &SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1127,8 +1129,8 @@ mod test { let (rx, _tx) = bounded(1024); let ctx = SinkContext::new( - SinkId::default(), - Alias::new("flow", "connector"), + SinkUId::default(), + Alias::new(FlowInstanceId::new(AppId::default(), "flow"), "connector"), ConnectorType::default(), QuiescenceBeacon::default(), ConnectionLostNotifier::new(rx), @@ -1176,7 +1178,7 @@ mod test { let (rx, _tx) = bounded(1024); let ctx = SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1221,7 +1223,7 @@ mod test { let (rx, _tx) = bounded(1024); let ctx = SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1267,7 +1269,7 @@ mod test { let (rx, _tx) = bounded(1024); let ctx = SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1311,7 +1313,7 @@ mod test { let (rx, _tx) = bounded(1024); let ctx = SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1346,7 +1348,7 @@ mod test { let (rx, _tx) = bounded(1024); let ctx = SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1406,7 +1408,7 @@ mod test { "", Event::signal_tick(), &SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1448,7 +1450,7 @@ mod test { "", Event::signal_tick(), &SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1521,7 +1523,7 @@ mod test { ); let ctx = SinkContext::new( - SinkId::default(), + SinkUId::default(), Alias::new("flow", "connector"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -1603,8 +1605,8 @@ mod test { ); let ctx = SinkContext::new( - SinkId::default(), - Alias::new("flow", "connector"), + SinkUId::default(), + AAlias::new(FlowInstanceId::new(AppId::default(), "flow"), "connector"), ConnectorType::default(), QuiescenceBeacon::default(), ConnectionLostNotifier::new(crate::channel::bounded(1024).0), @@ -1647,7 +1649,7 @@ mod test { CodecReq::Structured, vec![], &ConnectorType::from(""), - &Alias::new("flow", "connector"), + &Alias::new(FlowInstanceId::new(AppId::default(), "flow"), "connector"), )?, 0, ) diff --git a/src/connectors/impls/gcl/writer/sink.rs b/src/connectors/impls/gcl/writer/sink.rs index 108a8491cc..e367e9bfe2 100644 --- a/src/connectors/impls/gcl/writer/sink.rs +++ b/src/connectors/impls/gcl/writer/sink.rs @@ -253,7 +253,7 @@ mod test { use crate::connectors::{ google::tests::TestTokenProvider, utils::quiescence::QuiescenceBeacon, }; - use bytes::Bytes; + use crate::ids::{AppId, FlowInstanceId}; use futures::future::Ready; use googapis::google::logging::r#type::LogSeverity; use googapis::google::logging::v2::WriteLogEntriesResponse; @@ -267,7 +267,7 @@ mod test { }; use tonic::body::BoxBody; use tonic::codegen::Service; - use tremor_common::ids::SinkId; + use tremor_common::uids::SinkUId; use tremor_pipeline::CbAction::Trigger; use tremor_pipeline::EventId; use tremor_value::{literal, structurize}; @@ -362,8 +362,9 @@ mod test { tx, MockChannelFactory, ); - let ctx = SinkContext::new( - SinkId::default(), + let sink_context = SinkContext::new( + openraft::NodeId::default(), + SinkUId::default(), Alias::new("a", "b"), ConnectorType::default(), QuiescenceBeacon::default(), @@ -393,7 +394,7 @@ mod test { CodecReq::Structured, vec![], &"a".into(), - &Alias::new("a", "b"), + &Alias::new(FlowInstanceId::new(AppId::default(), "a"), "b"), )?, 0, ) @@ -459,8 +460,9 @@ mod test { "", Event::signal_tick(), &SinkContext::new( - SinkId::default(), - Alias::new("", ""), + // NodeId::default() + SinkUId::default(), + Alias::new(FlowInstanceId::new(AppId::default(), ""), ""), ConnectorType::default(), QuiescenceBeacon::default(), ConnectionLostNotifier::new(rx), @@ -470,7 +472,7 @@ mod test { CodecReq::Structured, vec![], &ConnectorType::from(""), - &Alias::new("", ""), + &Alias::new(FlowInstanceId::new(AppId::default(), ""), ""), )?, 0, ) diff --git a/src/connectors/impls/gcs/streamer.rs b/src/connectors/impls/gcs/streamer.rs index a4234f39aa..bb4b6d4b55 100644 --- a/src/connectors/impls/gcs/streamer.rs +++ b/src/connectors/impls/gcs/streamer.rs @@ -409,9 +409,10 @@ pub(crate) mod tests { impls::gcs::streamer::Mode, reconnect::ConnectionLostNotifier, utils::quiescence::QuiescenceBeacon, }, + ids::FlowInstanceId, }; use halfbrown::HashMap; - use tremor_common::ids::{ConnectorIdGen, SinkId}; + use tremor_common::uids::{ConnectorUIdGen, SinkUId}; use tremor_pipeline::EventId; use tremor_value::literal; @@ -527,7 +528,7 @@ pub(crate) mod tests { }); let mut config = Config::new(&raw_config).expect("config should be valid"); - let alias = Alias::new("flow", "conn"); + let alias = Alias::new(FlowInstanceId::new("app", "flow"), "conn"); config.normalize(&alias); assert_eq!(256 * 1024, config.buffer_size); } @@ -567,9 +568,10 @@ pub(crate) mod tests { let (connection_lost_tx, _) = bounded(10); - let alias = Alias::new("a", "b"); + let alias = Alias::new(FlowInstanceId::new("app", "a"), "b"); let context = SinkContext::new( - SinkId::default(), + openraft::NodeId::default(), + SinkUId::default(), alias.clone(), "gcs_streamer".into(), QuiescenceBeacon::default(), @@ -752,9 +754,10 @@ pub(crate) mod tests { let (connection_lost_tx, _) = bounded(10); - let alias = Alias::new("a", "b"); + let alias = Alias::new(FlowInstanceId::new("app", "a"), "b"); let context = SinkContext::new( - SinkId::default(), + openraft::NodeId::default(), + SinkUId::default(), alias.clone(), "gcs_streamer".into(), QuiescenceBeacon::default(), @@ -925,9 +928,10 @@ pub(crate) mod tests { let (connection_lost_tx, _) = bounded(10); - let alias = Alias::new("a", "b"); + let alias = Alias::new(FlowInstanceId::new("app", "a"), "b"); let context = SinkContext::new( - SinkId::default(), + openraft::NodeId::default(), + SinkUId::default(), alias.clone(), "gcs_streamer".into(), QuiescenceBeacon::default(), @@ -1019,9 +1023,10 @@ pub(crate) mod tests { let (connection_lost_tx, _) = bounded(10); - let alias = Alias::new("a", "b"); + let alias = Alias::new(FlowInstanceId::new("app", "a"), "b"); let context = SinkContext::new( - SinkId::default(), + openraft::NodeId::default(), + SinkUId::default(), alias.clone(), "gcs_streamer".into(), QuiescenceBeacon::default(), @@ -1072,9 +1077,10 @@ pub(crate) mod tests { let (connection_lost_tx, _) = bounded(10); - let alias = Alias::new("a", "b"); + let alias = Alias::new(FlowInstanceId::new("app", "a"), "b"); let context = SinkContext::new( - SinkId::default(), + openraft::NodeId::default(), + SinkUId::default(), alias.clone(), "gcs_streamer".into(), QuiescenceBeacon::default(), @@ -1296,9 +1302,10 @@ pub(crate) mod tests { let (connection_lost_tx, _) = bounded(10); - let alias = Alias::new("a", "b"); + let alias = Alias::new(FlowInstanceId::new("app", "a"), "b"); let context = SinkContext::new( - SinkId::default(), + openraft::NodeId::default(), + SinkUId::default(), alias.clone(), "gcs_streamer".into(), QuiescenceBeacon::default(), @@ -1444,13 +1451,19 @@ pub(crate) mod tests { metrics_interval_s: None, }; let kill_switch = KillSwitch::dummy(); - let alias = Alias::new("snot", "badger"); - let mut connector_id_gen = ConnectorIdGen::default(); + let alias = Alias::new(FlowInstanceId::new("app", "snot"), "badger"); + let mut connector_id_gen = ConnectorUIdGen::default(); // lets cover create-sink here - let addr = - crate::connectors::spawn(&alias, &mut connector_id_gen, &builder, cfg, &kill_switch) - .await?; + let addr = crate::connectors::spawn( + openraft::NodeId::default(), + &alias, + &mut connector_id_gen, + &builder, + cfg, + &kill_switch, + ) + .await?; let (tx, mut rx) = bounded(1); addr.stop(tx).await?; assert!(rx.recv().await.expect("rx empty").res.is_ok()); @@ -1474,13 +1487,19 @@ pub(crate) mod tests { metrics_interval_s: None, }; let kill_switch = KillSwitch::dummy(); - let alias = Alias::new("snot", "badger"); - let mut connector_id_gen = ConnectorIdGen::default(); + let alias = Alias::new(FlowInstanceId::new("app", "snot"), "badger"); + let mut connector_id_gen = ConnectorUIdGen::default(); // lets cover create-sink here - let addr = - crate::connectors::spawn(&alias, &mut connector_id_gen, &builder, cfg, &kill_switch) - .await?; + let addr = crate::connectors::spawn( + openraft::NodeId::default(), + &alias, + &mut connector_id_gen, + &builder, + cfg, + &kill_switch, + ) + .await?; let (tx, mut rx) = bounded(1); addr.stop(tx).await?; assert!(rx.recv().await.expect("rx empty").res.is_ok()); diff --git a/src/connectors/impls/http/meta.rs b/src/connectors/impls/http/meta.rs index 037584cec9..7189255879 100644 --- a/src/connectors/impls/http/meta.rs +++ b/src/connectors/impls/http/meta.rs @@ -300,6 +300,8 @@ pub(super) fn extract_response_meta(response: &Response) -> Result Result<()> { @@ -315,7 +317,7 @@ mod test { CodecReq::Optional("json"), vec![], &ConnectorType("http".into()), - &Alias::new("flow", "http"), + &Alias::new(FlowInstanceId::new("app", "flow"), "http"), )?; let config = client::Config::new(&c)?; diff --git a/src/connectors/impls/http/server.rs b/src/connectors/impls/http/server.rs index 2b5af9f4b8..4496128a59 100644 --- a/src/connectors/impls/http/server.rs +++ b/src/connectors/impls/http/server.rs @@ -47,7 +47,7 @@ use std::{ sync::Arc, }; use tokio::{sync::oneshot, task::JoinHandle}; -use tremor_common::ids::Id; +use tremor_common::uids::UId; #[derive(Deserialize, Debug, Clone)] #[serde(deny_unknown_fields)] diff --git a/src/connectors/impls/kafka.rs b/src/connectors/impls/kafka.rs index 3bfcac75bf..8b4a6be26c 100644 --- a/src/connectors/impls/kafka.rs +++ b/src/connectors/impls/kafka.rs @@ -14,7 +14,7 @@ pub(crate) mod consumer; pub(crate) mod producer; -use crate::connectors::prelude::*; +use crate::connectors::{prelude::*, utils::metrics::make_metrics_payload}; use beef::Cow; use halfbrown::HashMap; use rdkafka::{error::KafkaError, ClientContext, Statistics}; @@ -362,7 +362,7 @@ mod tests { &literal!({ "measurement": "kafka_consumer_stats", "tags": { - "connector": "fake::fake" + "connector": "app/fake::fake" }, "fields": { "rx_msgs": 42, diff --git a/src/connectors/impls/metronome.rs b/src/connectors/impls/metronome.rs index 4a7fd2db42..f37298bba0 100644 --- a/src/connectors/impls/metronome.rs +++ b/src/connectors/impls/metronome.rs @@ -136,11 +136,11 @@ impl Source for MetronomeSource { #[cfg(test)] mod tests { - use crate::{config::Reconnect, connectors::prelude::*}; + use crate::{config::Reconnect, connectors::prelude::*, ids::FlowInstanceId}; #[tokio::test(flavor = "multi_thread")] async fn missing_config() -> Result<()> { - let alias = Alias::new("flow", "connector"); + let alias = Alias::new(FlowInstanceId::new("app", "flow"), "connector"); let builder = super::Builder::default(); let connector_config = super::ConnectorConfig { connector_type: builder.connector_type(), diff --git a/src/connectors/impls/oneshot.rs b/src/connectors/impls/oneshot.rs index 0aa06e0c1f..80e2259ba9 100644 --- a/src/connectors/impls/oneshot.rs +++ b/src/connectors/impls/oneshot.rs @@ -58,7 +58,7 @@ impl Connector for Oneshot { let source = OneshotSource { value: self.value.take(), }; - builder.spawn(source, source_context).map(Some) + Ok(Some(builder.spawn(source, source_context))) } fn codec_requirements(&self) -> CodecReq { diff --git a/src/connectors/impls/otel/client.rs b/src/connectors/impls/otel/client.rs index b0f52cf8c0..ff28efbf0d 100644 --- a/src/connectors/impls/otel/client.rs +++ b/src/connectors/impls/otel/client.rs @@ -208,11 +208,13 @@ impl Sink for OtelSink { #[cfg(test)] mod tests { + use crate::ids::FlowInstanceId; + use super::*; #[tokio::test(flavor = "multi_thread")] async fn otel_client_builder() -> Result<()> { - let alias = Alias::new("flow", "my_otel_client"); + let alias = Alias::new(FlowInstanceId::new("app", "flow"), "my_otel_client"); let with_processors = literal!({ "config": { "url": "localhost:4317", diff --git a/src/connectors/impls/otel/server.rs b/src/connectors/impls/otel/server.rs index 2bc7209882..f63290769b 100644 --- a/src/connectors/impls/otel/server.rs +++ b/src/connectors/impls/otel/server.rs @@ -194,13 +194,15 @@ impl Source for OtelSource { #[cfg(test)] mod tests { + use crate::ids::FlowInstanceId; + use super::*; // use env_logger; // use http_types::Method; #[tokio::test(flavor = "multi_thread")] async fn otel_client_builder() -> Result<()> { - let alias = Alias::new("test", "my_otel_server"); + let alias = Alias::new(FlowInstanceId::new("app", "test"), "my_otel_server"); let with_processors = literal!({ "config": { "url": "localhost:4317", @@ -211,7 +213,7 @@ mod tests { ConnectorType("otel_server".into()), &with_processors, )?; - let alias = Alias::new("flow", "my_otel_server"); + let alias = Alias::new(FlowInstanceId::new("app", "flow"), "my_otel_server"); let builder = super::Builder::default(); let kill_switch = KillSwitch::dummy(); diff --git a/src/connectors/prelude.rs b/src/connectors/prelude.rs index 245a864659..efec364862 100644 --- a/src/connectors/prelude.rs +++ b/src/connectors/prelude.rs @@ -15,7 +15,6 @@ pub(crate) use crate::{ channel::{bounded, Receiver, Sender}, connectors::{ - metrics::make_metrics_payload, sink::{ channel_sink::{ChannelSink, ChannelSinkRuntime}, AsyncSinkReply, ContraflowData, EventSerializer, ReplySender, Sink, SinkAck, SinkAddr, diff --git a/src/connectors/sink.rs b/src/connectors/sink.rs index 1d86849ec9..c35fbc114a 100644 --- a/src/connectors/sink.rs +++ b/src/connectors/sink.rs @@ -19,11 +19,11 @@ pub(crate) mod channel_sink; /// Utility for limiting concurrency (by sending `CB::Close` messages when a maximum concurrency value is reached) pub(crate) mod concurrency_cap; -use super::{utils::metrics::SinkReporter, CodecReq}; use crate::config::{ Codec as CodecConfig, Connector as ConnectorConfig, Postprocessor as PostprocessorConfig, }; use crate::connectors::utils::reconnect::{Attempt, ConnectionLostNotifier}; +use crate::connectors::{utils::metrics::SinkReporter, CodecReq}; use crate::connectors::{Alias, ConnectorType, Context, Msg, QuiescenceBeacon, StreamDone}; use crate::errors::Result; use crate::pipeline; @@ -45,8 +45,8 @@ use tokio::task; use tokio_stream::wrappers::{ReceiverStream, UnboundedReceiverStream}; use tremor_common::time::nanotime; use tremor_common::{ - ids::{SinkId, SourceId}, ports::Port, + uids::{SinkUId, SourceUId}, }; use tremor_pipeline::{CbAction, Event, EventId, OpMeta, SignalKind, DEFAULT_STREAM_ID}; use tremor_script::{ast::DeployEndpoint, EventPayload}; @@ -255,8 +255,9 @@ pub(crate) trait SinkRuntime: Send + Sync { /// context for the connector sink #[derive(Clone)] pub(crate) struct SinkContextInner { + pub(crate) node_id: openraft::NodeId, /// the connector unique identifier - pub(crate) uid: SinkId, + pub(crate) uid: SinkUId, /// the connector alias pub(crate) alias: Alias, /// the connector type @@ -271,20 +272,22 @@ pub(crate) struct SinkContextInner { #[derive(Clone)] pub(crate) struct SinkContext(Arc); impl SinkContext { - pub(crate) fn uid(&self) -> SinkId { + pub(crate) fn uid(&self) -> SinkUId { self.0.uid } pub(crate) fn notifier(&self) -> &ConnectionLostNotifier { &self.0.notifier } pub(crate) fn new( - uid: SinkId, + node_id: openraft::NodeId, + uid: SinkUId, alias: Alias, connector_type: ConnectorType, quiescence_beacon: QuiescenceBeacon, notifier: ConnectionLostNotifier, ) -> SinkContext { Self(Arc::new(SinkContextInner { + node_id, uid, alias, connector_type, @@ -296,7 +299,7 @@ impl SinkContext { impl Display for SinkContext { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "[Sink::{}]", &self.0.alias) + write!(f, "[Node::{}][Sink::{}]", self.0.node_id, &self.0.alias) } } @@ -605,9 +608,9 @@ where // pipelines connected to IN port pipelines: Vec<(DeployEndpoint, pipeline::Addr)>, // set of source ids we received start signals from - starts_received: HashSet, + starts_received: HashSet, // set of connector ids we received drain signals from - drains_received: HashSet, // TODO: use a bitset for both? + drains_received: HashSet, // TODO: use a bitset for both? drain_channel: Option>, state: SinkState, } diff --git a/src/connectors/sink/channel_sink.rs b/src/connectors/sink/channel_sink.rs index 031a7c1a14..3c4e7c15d9 100644 --- a/src/connectors/sink/channel_sink.rs +++ b/src/connectors/sink/channel_sink.rs @@ -35,7 +35,7 @@ use tokio::{ task::{self, JoinHandle}, time::timeout, }; -use tremor_common::{ids::Id, time::nanotime}; +use tremor_common::{time::nanotime, uids::UId}; use tremor_pipeline::{CbAction, Event, SignalKind}; use tremor_value::Value; use value_trait::ValueAccess; diff --git a/src/connectors/source.rs b/src/connectors/source.rs index 7aa20a5516..768f2ccb86 100644 --- a/src/connectors/source.rs +++ b/src/connectors/source.rs @@ -17,9 +17,9 @@ /// A simple source that is fed with `SourceReply` via a channel. pub mod channel_source; +use super::{utils::metrics::SourceReporter, CodecReq, Connectivity}; use crate::channel::{unbounded, Sender, UnboundedReceiver, UnboundedSender}; use crate::connectors::{ - metrics::SourceReporter, utils::reconnect::{Attempt, ConnectionLostNotifier}, Alias, ConnectorType, Context, Msg, QuiescenceBeacon, StreamDone, }; @@ -44,9 +44,9 @@ use std::collections::{btree_map::Entry, BTreeMap}; use std::fmt::Display; use tokio::task; use tremor_common::{ - ids::{Id, SinkId, SourceId}, ports::{Port, ERR, OUT}, time::nanotime, + uids::{SinkUId, SourceUId, UId}, }; use tremor_pipeline::{ CbAction, Event, EventId, EventIdGenerator, EventOriginUri, DEFAULT_STREAM_ID, @@ -55,8 +55,6 @@ use tremor_script::{ast::DeployEndpoint, prelude::BaseExpr, EventPayload, ValueA use tremor_value::{literal, Value}; use value_trait::Builder; -use super::{CodecReq, Connectivity}; - #[derive(Debug)] /// Messages a Source can receive pub(crate) enum SourceMsg { @@ -280,8 +278,9 @@ pub(crate) trait StreamReader: Send { /// context for a source #[derive(Clone)] pub(crate) struct SourceContext { + pub(crate) node_id: openraft::NodeId, /// connector uid - pub uid: SourceId, + pub(crate) uid: SourceUId, /// connector alias pub(crate) alias: Alias, @@ -296,7 +295,7 @@ pub(crate) struct SourceContext { impl Display for SourceContext { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "[Source::{}]", &self.alias) + write!(f, "[Node::{}][Source::{}]", self.node_id, &self.alias) } } @@ -388,7 +387,7 @@ impl SourceManagerBuilder { /// # Errors /// - on invalid connector configuration pub(crate) fn builder( - source_uid: SourceId, + source_uid: SourceUId, config: &ConnectorConfig, connector_default_codec: CodecReq, source_metrics_reporter: SourceReporter, @@ -425,7 +424,7 @@ pub(crate) fn builder( /// maintaining stream state // TODO: there is optimization potential here for reusing codec and preprocessors after a stream got ended struct Streams { - uid: SourceId, + uid: SourceUId, codec_config: CodecConfig, preprocessor_configs: Vec, states: BTreeMap, @@ -437,7 +436,7 @@ impl Streams { } /// constructor fn new( - uid: SourceId, + uid: SourceUId, codec_config: config::Codec, preprocessor_configs: Vec, ) -> Self { @@ -494,7 +493,7 @@ impl Streams { /// build a stream fn build_stream( - source_uid: SourceId, + source_uid: SourceUId, stream_id: u64, codec_config: &CodecConfig, codec_overwrite: Option, @@ -565,7 +564,7 @@ where /// Gather all the sinks that reported being started. /// This will give us some knowledge on the topology and most importantly /// on how many `Drained` messages to wait. Assumes a static topology. - started_sinks: HashSet, + started_sinks: HashSet, num_started_sinks: u64, /// is counted up for each call to `pull_data` in order to identify the pull call /// an event is originating from. We can only ack or fail pulls. diff --git a/src/connectors/tests.rs b/src/connectors/tests.rs index 77026f244b..53cbd1349e 100644 --- a/src/connectors/tests.rs +++ b/src/connectors/tests.rs @@ -68,18 +68,17 @@ use crate::{ StatusReport, }, errors::Result, + ids::FlowInstanceId, instance::State, - pipeline, qsize, - system::flow::Alias as FlowAlias, - Event, + pipeline, qsize, Event, }; use log::{debug, info}; use std::time::Duration; use std::{collections::HashMap, time::Instant}; use tokio::{sync::oneshot, task, time::timeout}; use tremor_common::{ - ids::{ConnectorIdGen, Id, SourceId}, ports::{Port, ERR, IN, OUT}, + uids::{ConnectorUIdGen, SourceUId, UId}, }; use tremor_pipeline::{CbAction, EventId}; use tremor_script::{ast::DeployEndpoint, lexer::Location, NodeMeta}; @@ -99,8 +98,8 @@ impl ConnectorHarness { input_ports: Vec>, output_ports: Vec>, ) -> Result { - let alias = ConnectorAlias::new("test", alias); - let mut connector_id_gen = ConnectorIdGen::new(); + let alias = ConnectorAlias::new(FlowInstanceId::new("app", "test"), alias); + let mut connector_id_gen = ConnectorUIdGen::new(); let mut known_connectors = HashMap::new(); for builder in builtin_connector_types() { @@ -108,6 +107,7 @@ impl ConnectorHarness { } let raw_config = config::Connector::from_config(&alias, builder.connector_type(), defn)?; let connector_addr = connectors::spawn( + openraft::NodeId::default(), &alias, &mut connector_id_gen, builder, @@ -200,7 +200,7 @@ impl ConnectorHarness { // ensure we notify the connector that its sink part is connected self.addr .send_sink(SinkMsg::Signal { - signal: Event::signal_start(SourceId::new(1)), + signal: Event::signal_start(SourceUId::new(1)), }) .await?; @@ -223,7 +223,7 @@ impl ConnectorHarness { let cr = rx.recv().await.ok_or_else(empty_error)?; debug!("Stop result received."); cr.res?; - //self.handle.cancel().await; + //self.handle.abort(); let out_events = self .pipes .get_mut(&OUT) @@ -377,7 +377,7 @@ impl TestPipeline { self.addr.send_mgmt(pipeline::MgmtMsg::Stop).await } pub(crate) fn new(alias: String) -> Self { - let flow_id = FlowAlias::new("test"); + let flow_id = FlowInstanceId::new("TEST", "test"); let qsize = qsize(); let (tx, rx) = bounded(qsize); let (tx_cf, rx_cf) = unbounded(); diff --git a/src/connectors/tests/bench.rs b/src/connectors/tests/bench.rs index 2c4bfc5b35..9a27525d89 100644 --- a/src/connectors/tests/bench.rs +++ b/src/connectors/tests/bench.rs @@ -14,6 +14,7 @@ use super::ConnectorHarness; use crate::{ connectors::{impls::bench, prelude::KillSwitch, sink::SinkMsg}, errors::Result, + ids::BOOTSTRAP_NODE_ID, system::{flow_supervisor, Runtime, WorldConfig}, }; use std::{io::Write, time::Duration}; @@ -40,7 +41,7 @@ async fn stop_after_events() -> Result<()> { "iters": 2 } }); - let (world, world_handle) = Runtime::start(WorldConfig::default()).await?; + let (world, world_handle) = Runtime::start(BOOTSTRAP_NODE_ID, WorldConfig::default()).await?; let mut harness = ConnectorHarness::new_with_kill_switch( function_name!(), &bench::Builder::default(), diff --git a/src/connectors/tests/http/client.rs b/src/connectors/tests/http/client.rs index 539164b665..1a5fe3ef42 100644 --- a/src/connectors/tests/http/client.rs +++ b/src/connectors/tests/http/client.rs @@ -684,7 +684,7 @@ async fn missing_tls_config_https() -> Result<()> { .map(|e| e.to_string()) .unwrap_or_default(); - assert_eq!("Invalid Definition for connector \"test::missing_tls_config_https\": missing tls config with 'https' url. Set 'tls' to 'true' or provide a full tls config.", res); + assert_eq!("Invalid Definition for connector \"app/test::missing_tls_config_https\": missing tls config with 'https' url. Set 'tls' to 'true' or provide a full tls config.", res); Ok(()) } diff --git a/src/connectors/tests/kafka/consumer.rs b/src/connectors/tests/kafka/consumer.rs index 974067fcb0..baae798a33 100644 --- a/src/connectors/tests/kafka/consumer.rs +++ b/src/connectors/tests/kafka/consumer.rs @@ -223,7 +223,7 @@ async fn transactional_retry() -> Result<()> { assert_eq!( &literal!({ "error": "SIMD JSON error: InternalError at character 0 ('}')", - "source": "test::transactional_retry", + "source": "app/test::transactional_retry", "stream_id": 8_589_934_592_u64, "pull_id": 1u64 }), @@ -437,7 +437,7 @@ async fn custom_no_retry() -> Result<()> { assert_eq!( &literal!({ "error": "SIMD JSON error: InternalError at character 0 ('}')", - "source": "test::custom_no_retry", + "source": "app/test::custom_no_retry", "stream_id": 8_589_934_592_u64, "pull_id": 1u64 }), @@ -640,7 +640,7 @@ async fn performance() -> Result<()> { assert_eq!( &literal!({ "error": "SIMD JSON error: InternalError at character 0 ('}')", - "source": "test::performance", + "source": "app/test::performance", "stream_id": 8_589_934_592_u64, "pull_id": 1u64 }), diff --git a/src/connectors/tests/wal.rs b/src/connectors/tests/wal.rs index 1e418f085c..609cce8f3d 100644 --- a/src/connectors/tests/wal.rs +++ b/src/connectors/tests/wal.rs @@ -15,8 +15,8 @@ use super::ConnectorHarness; use crate::{connectors::impls::wal, errors::Result}; use std::time::Duration; use tremor_common::{ - ids::{Id, SourceId}, ports::IN, + uids::{SourceUId, UId}, }; use tremor_pipeline::{CbAction, Event, EventIdGenerator}; use tremor_value::{literal, prelude::*, Value}; @@ -40,7 +40,7 @@ async fn wal() -> Result<()> { harness.wait_for_connected().await?; harness.consume_initial_sink_contraflow().await?; - let source_id = SourceId::new(1); + let source_id = SourceUId::new(1); let mut id_gen = EventIdGenerator::new(source_id); let value = Value::from(42_u64); let meta = Value::object(); diff --git a/src/connectors/utils/reconnect.rs b/src/connectors/utils/reconnect.rs index 16ede15a13..290e2db94e 100644 --- a/src/connectors/utils/reconnect.rs +++ b/src/connectors/utils/reconnect.rs @@ -386,6 +386,7 @@ mod tests { use super::*; use crate::{ connectors::{utils::quiescence::QuiescenceBeacon, CodecReq}, + ids::FlowInstanceId, qsize, }; @@ -446,7 +447,7 @@ mod tests { async fn failfast_runtime() -> Result<()> { let (tx, _rx) = bounded(qsize()); let notifier = ConnectionLostNotifier::new(tx.clone()); - let alias = Alias::new("flow", "test"); + let alias = Alias::new(FlowInstanceId::new("app", "flow"), "test"); let addr = Addr { alias: alias.clone(), source: None, @@ -460,6 +461,7 @@ mod tests { }; let qb = QuiescenceBeacon::default(); let ctx = ConnectorContext { + node_id: openraft::NodeId::default(), alias, connector_type: "fake".into(), quiescence_beacon: qb, @@ -478,7 +480,7 @@ mod tests { async fn backoff_runtime() -> Result<()> { let (tx, mut rx) = bounded(qsize()); let notifier = ConnectionLostNotifier::new(tx.clone()); - let alias = Alias::new("flow", "test"); + let alias = Alias::new(FlowInstanceId::new("app", "flow"), "test"); let addr = Addr { alias: alias.clone(), source: None, @@ -497,6 +499,7 @@ mod tests { }; let qb = QuiescenceBeacon::default(); let ctx = ConnectorContext { + node_id: openraft::NodeId::default(), alias, connector_type: "fake".into(), quiescence_beacon: qb, diff --git a/src/ids.rs b/src/ids.rs new file mode 100644 index 0000000000..dceecb4908 --- /dev/null +++ b/src/ids.rs @@ -0,0 +1,122 @@ +// Copyright 2022, The Tremor Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::{Display, Formatter}; + +use tremor_script::ast::DeployFlow; + +/// An `App` is an isolated container that is defined by +/// a troy file with possibly multiple flow definitions. +/// An `App` needs to have a unique name inside a tremor cluster. +/// Flow instances (and thus connector and pipeline instances) are spawned in the context +/// of an app and thus can have similar aliases/ids +#[derive(Serialize, Deserialize, Debug, Clone, Hash, Eq, PartialEq, PartialOrd)] +pub struct AppId(pub String); +impl Display for AppId { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +impl From for AppId { + fn from(value: String) -> Self { + Self(value) + } +} + +impl From<&str> for AppId { + fn from(value: &str) -> Self { + Self(value.to_string()) + } +} + +/// This default implementation should not be used in the clustered context +impl Default for AppId { + fn default() -> Self { + Self("default".to_string()) + } +} + +/// Identifier of a Flow definition +#[derive(Serialize, Deserialize, Debug, Clone, Hash, Eq, PartialEq)] +pub struct FlowDefinitionId(pub String); +impl Display for FlowDefinitionId { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +impl From for FlowDefinitionId { + fn from(value: String) -> Self { + Self(value) + } +} + +impl From<&str> for FlowDefinitionId { + fn from(value: &str) -> Self { + Self(value.to_string()) + } +} + +/// Unique identifier of a `Flow` instance within a tremor cluster +/// A flow instance is always part of an `App` and thus always needs an `AppId` to be fully qualified. +/// The `Flow` id needs to be unique within the App, regardless of the flow definition this instance is based upon. +/// An actual running instance of a flow +#[derive(Debug, PartialEq, PartialOrd, Eq, Hash, Clone, Serialize, Deserialize)] +pub struct FlowInstanceId { + app_id: AppId, + alias: String, +} + +impl FlowInstanceId { + /// construct a new flow if from some stringy thingy + pub fn new(app_id: impl Into, alias: impl Into) -> Self { + Self { + app_id: app_id.into(), + alias: alias.into(), + } + } + + pub fn from_deploy(app_id: impl Into, deploy: &DeployFlow) -> Self { + Self { + app_id: app_id.into(), + alias: deploy.instance_alias.clone(), + } + } + + #[must_use] + pub fn app_id(&self) -> &AppId { + &self.app_id + } + + #[must_use] + pub fn alias(&self) -> &String { + &self.alias + } +} + +impl std::fmt::Display for FlowInstanceId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}/{}", &self.app_id, &self.alias) + } +} + +impl From<&str> for FlowInstanceId { + fn from(value: &str) -> Self { + Self::new(AppId::default(), value) + } +} + +/// fixed node id used for root cluster nodes that have been bootstrapping the cluster +pub const BOOTSTRAP_NODE_ID: openraft::NodeId = 0; diff --git a/src/lib.rs b/src/lib.rs index 66e086067b..7ca94bb03c 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -77,6 +77,9 @@ pub mod instance; pub mod raft; +/// Common Identifiers for entities within the Runtime +pub mod ids; + /// Metrics instance name pub static mut INSTANCE: &str = "tremor"; @@ -95,7 +98,7 @@ pub type OpConfig = tremor_value::Value<'static>; pub(crate) mod channel; /// Default Q Size -const QSIZE: AtomicUsize = AtomicUsize::new(128); +static QSIZE: AtomicUsize = AtomicUsize::new(128); pub(crate) fn qsize() -> usize { QSIZE.load(std::sync::atomic::Ordering::Relaxed) @@ -134,12 +137,15 @@ macro_rules! log_error { #[cfg(test)] mod tests { use super::*; - use crate::system::{ShutdownMode, WorldConfig}; + use crate::{ + ids::BOOTSTRAP_NODE_ID, + system::{ShutdownMode, WorldConfig}, + }; use std::io::Write; #[tokio::test(flavor = "multi_thread")] async fn test_load_troy_file() -> Result<()> { - let (world, handle) = Runtime::start(WorldConfig::default()).await?; + let (world, handle) = Runtime::start(BOOTSTRAP_NODE_ID, WorldConfig::default()).await?; let troy_file = tempfile::NamedTempFile::new()?; troy_file.as_file().write_all( r#" diff --git a/src/pipeline.rs b/src/pipeline.rs index df93439c91..77bac21017 100644 --- a/src/pipeline.rs +++ b/src/pipeline.rs @@ -18,15 +18,15 @@ use crate::{ use crate::{ connectors::{self, sink::SinkMsg, source::SourceMsg}, errors::{pipe_send_e, Result}, + ids::FlowInstanceId, instance::State, primerge::PriorityMerge, - system::flow, }; use futures::StreamExt; use std::{fmt, time::Duration}; use tokio::task::{self, JoinHandle}; use tokio_stream::wrappers::{ReceiverStream, UnboundedReceiverStream}; -use tremor_common::{ids::OperatorIdGen, ports::Port, time::nanotime}; +use tremor_common::{ports::Port, time::nanotime, uids::OperatorUIdGen}; use tremor_pipeline::{ errors::ErrorKind as PipelineErrorKind, CbAction, Event, ExecutableGraph, SignalKind, }; @@ -39,13 +39,13 @@ type EventSet = Vec<(Port<'static>, Event)>; #[derive(Debug, PartialEq, PartialOrd, Eq, Hash, Clone, Serialize, Deserialize)] pub(crate) struct Alias { - flow_alias: flow::Alias, + flow_alias: FlowInstanceId, pipeline_alias: String, } impl Alias { pub(crate) fn new( - flow_alias: impl Into, + flow_alias: impl Into, pipeline_alias: impl Into, ) -> Self { Self { @@ -54,6 +54,10 @@ impl Alias { } } + pub(crate) fn flow_alias(&self) -> &FlowInstanceId { + &self.flow_alias + } + pub(crate) fn pipeline_alias(&self) -> &str { self.pipeline_alias.as_str() } @@ -61,7 +65,7 @@ impl Alias { impl std::fmt::Display for Alias { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}::{}", self.flow_alias, self.pipeline_alias) + write!(f, "{}::{}", self.flow_alias(), self.pipeline_alias()) } } @@ -175,9 +179,10 @@ impl TryFrom for OutputTarget { } pub(crate) fn spawn( + node_id: openraft::NodeId, pipeline_alias: Alias, config: &tremor_pipeline::query::Query, - operator_id_gen: &mut OperatorIdGen, + operator_id_gen: &mut OperatorUIdGen, ) -> Result { let qsize = qsize(); let mut pipeline = config.to_executable_graph(operator_id_gen)?; @@ -208,6 +213,7 @@ pub(crate) fn spawn( let addr = Addr::new(tx, cf_tx, mgmt_tx, pipeline_alias.clone()); task::spawn(pipeline_task( + node_id, pipeline_alias, pipeline, rx, @@ -503,31 +509,25 @@ fn maybe_send(r: Result<()>) { /// /// currently only used for printing struct PipelineContext { + node_id: openraft::NodeId, alias: Alias, } -impl std::fmt::Display for PipelineContext { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "[Pipeline::{}]", &self.alias) +impl PipelineContext { + fn new(node_id: openraft::NodeId, alias: Alias) -> Self { + Self { node_id, alias } } } -impl From<&Alias> for PipelineContext { - fn from(alias: &Alias) -> Self { - Self { - alias: alias.clone(), - } - } -} - -impl From for PipelineContext { - fn from(alias: Alias) -> Self { - Self { alias } +impl std::fmt::Display for PipelineContext { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "[Node:{}][Pipeline::{}]", self.node_id, &self.alias) } } #[allow(clippy::too_many_lines)] pub(crate) async fn pipeline_task( + node_id: openraft::NodeId, id: Alias, mut pipeline: ExecutableGraph, rx: Receiver>, @@ -537,7 +537,7 @@ pub(crate) async fn pipeline_task( ) -> Result<()> { pipeline.id = id.to_string(); - let ctx = PipelineContext::from(&id); + let ctx = PipelineContext::new(node_id, id.clone()); let mut dests: Dests = halfbrown::HashMap::new(); let mut inputs: Inputs = halfbrown::HashMap::new(); @@ -717,9 +717,9 @@ mod tests { }; use std::time::Instant; use tremor_common::{ - ids::Id as _, - ids::SourceId, ports::{IN, OUT}, + uids::SourceUId, + uids::UId as _, }; use tremor_pipeline::{EventId, OpMeta}; use tremor_script::{aggr_registry, lexer::Location, NodeMeta, FN_REGISTRY}; @@ -728,23 +728,26 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn report() -> Result<()> { let _ = env_logger::try_init(); - let mut operator_id_gen = OperatorIdGen::new(); + let mut operator_id_gen = OperatorUIdGen::new(); let trickle = r#"select event from in into out;"#; let aggr_reg = aggr_registry(); let query = tremor_pipeline::query::Query::parse(&trickle, &*FN_REGISTRY.read()?, &aggr_reg)?; let addr = spawn( - Alias::new("report", "test-pipe1"), + openraft::NodeId::default(), + Alias::new(FlowInstanceId::new("app", "report"), "test-pipe1"), &query, &mut operator_id_gen, )?; let addr2 = spawn( - Alias::new("report", "test-pipe2"), + openraft::NodeId::default(), + Alias::new(FlowInstanceId::new("app", "report"), "test-pipe2"), &query, &mut operator_id_gen, )?; let addr3 = spawn( - Alias::new("report", "test-pipe3"), + openraft::NodeId::default(), + Alias::new(FlowInstanceId::new("app", "report"), "test-pipe3"), &query, &mut operator_id_gen, )?; @@ -834,13 +837,18 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn pipeline_spawn() -> Result<()> { let _ = env_logger::try_init(); - let mut operator_id_gen = OperatorIdGen::new(); + let mut operator_id_gen = OperatorUIdGen::new(); let trickle = r#"select event from in into out;"#; let aggr_reg = aggr_registry(); - let pipeline_id = Alias::new("flow", "test-pipe"); + let pipeline_id = Alias::new(FlowInstanceId::new("app", "flow"), "test-pipe"); let query = tremor_pipeline::query::Query::parse(&trickle, &*FN_REGISTRY.read()?, &aggr_reg)?; - let addr = spawn(pipeline_id, &query, &mut operator_id_gen)?; + let addr = spawn( + openraft::NodeId::default(), + pipeline_id, + &query, + &mut operator_id_gen, + )?; let (tx, mut rx) = bounded(1); addr.send_mgmt(MgmtMsg::Inspect(tx.clone())).await?; @@ -924,7 +932,7 @@ mod tests { } // send a signal - addr.send(Box::new(Msg::Signal(Event::signal_drain(SourceId::new( + addr.send(Box::new(Msg::Signal(Event::signal_drain(SourceUId::new( 42, ))))) .await?; diff --git a/src/preprocessor.rs b/src/preprocessor.rs index 1c873b9af4..711907d7c5 100644 --- a/src/preprocessor.rs +++ b/src/preprocessor.rs @@ -168,6 +168,7 @@ pub fn finish(preprocessors: &mut [Box], alias: &Alias) -> Res #[cfg(test)] mod test { use super::*; + use crate::ids::FlowInstanceId; use crate::postprocessor::{self as post, separate::Separate as SeparatePost, Postprocessor}; use crate::Result; @@ -334,7 +335,7 @@ mod test { let data = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]; let wire = post_p.process(0, 0, &data)?; let (start, end) = wire[0].split_at(7); - let alias = Alias::new("test", "test"); + let alias = Alias::new(FlowInstanceId::new("app", "test"), "test"); let mut pps: Vec> = vec![Box::new(pre_p)]; let recv = preprocess(pps.as_mut_slice(), &mut it, start.to_vec(), &alias)?; assert!(recv.is_empty()); @@ -599,7 +600,7 @@ mod test { fn single_pre_process_head_ok() { let pre = Box::new(BadPreprocessor {}); let alias = crate::connectors::Alias::new( - crate::system::flow::Alias::new("chucky"), + crate::ids::FlowInstanceId::new("app", "chucky"), "chucky".to_string(), ); let mut ingest_ns = 0_u64; @@ -613,7 +614,7 @@ mod test { assert_eq!("nily", noop.name()); let pre = Box::new(BadPreprocessor {}); let alias = crate::connectors::Alias::new( - crate::system::flow::Alias::new("chucky"), + crate::ids::FlowInstanceId::new("app", "chucky"), "chucky".to_string(), ); let mut ingest_ns = 0_u64; @@ -625,7 +626,7 @@ mod test { fn single_pre_finish_ok() { let pre = Box::new(BadPreprocessor {}); let alias = crate::connectors::Alias::new( - crate::system::flow::Alias::new("chucky"), + crate::ids::FlowInstanceId::new("app", "chucky"), "chucky".to_string(), ); let r = finish(&mut [pre], &alias); @@ -642,7 +643,7 @@ mod test { #[test] fn preprocess_finish_head_fail() { let alias = crate::connectors::Alias::new( - crate::system::flow::Alias::new("chucky"), + crate::ids::FlowInstanceId::new("app", "chucky"), "chucky".to_string(), ); let pre = Box::new(BadFinisher {}); @@ -653,7 +654,7 @@ mod test { #[test] fn preprocess_finish_tail_fail() { let alias = crate::connectors::Alias::new( - crate::system::flow::Alias::new("chucky"), + crate::ids::FlowInstanceId::new("app", "chucky"), "chucky".to_string(), ); let noop = Box::new(NoOp {}); @@ -665,7 +666,7 @@ mod test { #[test] fn preprocess_finish_multi_ok() { let alias = crate::connectors::Alias::new( - crate::system::flow::Alias::new("xyz"), + crate::ids::FlowInstanceId::new("test", "xyz"), "xyz".to_string(), ); let noop1 = Box::new(NoOp {}); diff --git a/src/raft.rs b/src/raft.rs index 7e51da672f..d353620805 100644 --- a/src/raft.rs +++ b/src/raft.rs @@ -16,6 +16,7 @@ use std::{ sync::Mutex, }; use store::{TremorRequest, TremorResponse}; +use tokio::task::JoinError; /// load a default raft config /// # Errors @@ -41,6 +42,7 @@ pub enum ClusterError { Serde(serde_json::Error), Config(ConfigError), Client(Error), + JoinError(JoinError), // FIXME: this is a horrible hack Runtime(Mutex), } @@ -104,18 +106,24 @@ impl From for ClusterError { } } +impl From for ClusterError { + fn from(e: JoinError) -> Self { + ClusterError::JoinError(e) + } +} impl Display for ClusterError { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - ClusterError::Other(s) => write!(f, "{}", s), - ClusterError::Rocks(s) => write!(f, "{}", s), - ClusterError::Io(s) => write!(f, "{}", s), - ClusterError::Store(s) => write!(f, "{}", s), - ClusterError::Initialize(e) => write!(f, "{}", e), - ClusterError::Config(e) => write!(f, "{}", e), + ClusterError::Other(e) => e.fmt(f), + ClusterError::Rocks(e) => e.fmt(f), + ClusterError::Io(e) => e.fmt(f), + ClusterError::Store(e) => e.fmt(f), + ClusterError::Initialize(e) => e.fmt(f), + ClusterError::Config(e) => e.fmt(f), ClusterError::Runtime(e) => write!(f, "{:?}", e.lock()), - ClusterError::Serde(e) => write!(f, "{e}"), + ClusterError::Serde(e) => e.fmt(f), ClusterError::Client(e) => e.fmt(f), + ClusterError::JoinError(e) => e.fmt(f), } } } @@ -127,7 +135,7 @@ type ClusterResult = Result; /// # Errors /// When the node can't be removed pub async fn remove_node( - node_id: NodeId, + node_id: openraft::NodeId, api_addr: &T, ) -> Result<(), crate::errors::Error> { let client = api::client::Tremor::new(api_addr)?; diff --git a/src/raft/api.rs b/src/raft/api.rs index 0ac30e2eec..ad8d465676 100644 --- a/src/raft/api.rs +++ b/src/raft/api.rs @@ -21,19 +21,16 @@ mod cluster; mod kv; pub(crate) mod worker; -use crate::raft::{ - node::Addr, - store::{self, AppId, FlowId, InstanceId, StateApp, Store, TremorResponse}, - TremorRaftImpl, -}; -use async_std::{ - channel::{bounded, Sender}, - future::TimeoutError, - prelude::FutureExt, - task::JoinHandle, +use crate::channel::{bounded, Sender}; +use crate::{ + ids::{AppId, FlowDefinitionId, FlowInstanceId}, + raft::{ + node::Addr, + store::{self, StateApp, Store, TremorResponse}, + TremorRaftImpl, + }, }; use futures::Future; -use halfbrown::HashMap; use openraft::{ error::{ AddLearnerError, ChangeMembershipError, ClientReadError, ClientWriteError, Fatal, @@ -42,8 +39,10 @@ use openraft::{ raft::{AddLearnerResponse, ClientWriteResponse}, LogId, NodeId, StorageError, }; +use std::collections::HashMap; use std::{collections::BTreeSet, num::ParseIntError, sync::Arc, time::Duration}; use tide::{http::headers::LOCATION, Body, Endpoint, Request, Response, StatusCode}; +use tokio::{task::JoinHandle, time::timeout}; use self::apps::AppState; @@ -86,7 +85,7 @@ pub(crate) fn initialize( store: Arc, ) -> (JoinHandle<()>, Arc) { let (store_tx, store_rx) = bounded(64); // arbitrary choice to block upon too much concurrent incoming requests - let handle = async_std::task::spawn(worker::api_worker(store, store_rx)); + let handle = tokio::task::spawn(worker::api_worker(store, store_rx)); let state = Arc::new(ServerState { id, addr, @@ -164,16 +163,15 @@ pub enum AppError { /// app not found AppNotFound(AppId), - FlowNotFound(AppId, FlowId), - InstanceNotFound(AppId, InstanceId), + FlowNotFound(AppId, FlowDefinitionId), + InstanceNotFound(FlowInstanceId), /// App has at least 1 running instances (and thus cannot be uninstalled) - HasInstances(AppId, Vec), - InstanceAlreadyExists(AppId, InstanceId), + HasInstances(AppId, Vec), + InstanceAlreadyExists(FlowInstanceId), /// provided flow args are invalid InvalidArgs { - app: AppId, - flow: FlowId, - instance: InstanceId, + flow: FlowDefinitionId, + instance: FlowInstanceId, errors: Vec, }, } @@ -194,10 +192,34 @@ impl std::fmt::Display for AppError { .collect::>() .join(", ") ), - Self::InstanceAlreadyExists(app_id, instance_id) => write!(f, "App \"{app_id}\" already has an instance \"{instance_id}\""), - Self::InstanceNotFound(app_id, instance_id) => write!(f, "No instance \"{instance_id}\" in App \"{app_id}\" found"), - Self::FlowNotFound(app_id, flow_id) => write!(f, "Flow \"{flow_id}\" not found in App \"{app_id}\"."), - Self::InvalidArgs { app, flow, instance, errors } => write!(f, "Invalid Arguments provided for instance \"{instance}\" of Flow \"{flow}\" in App \"{app}\": {}", errors.iter().map(ToString::to_string).collect::>().join(", ")), + Self::InstanceAlreadyExists(instance_id) => write!( + f, + "App \"{}\" already has an instance \"{}\"", + instance_id.app_id(), + instance_id.alias() + ), + Self::InstanceNotFound(instance_id) => write!( + f, + "No instance \"{}\" in App \"{}\" found", + instance_id.alias(), + instance_id.app_id() + ), + Self::FlowNotFound(app_id, flow_id) => { + write!(f, "Flow \"{flow_id}\" not found in App \"{app_id}\".") + } + Self::InvalidArgs { + flow, + instance, + errors, + } => write!( + f, + "Invalid Arguments provided for instance \"{instance}\" of Flow \"{flow}\": {}", + errors + .iter() + .map(ToString::to_string) + .collect::>() + .join(", ") + ), } } } @@ -211,7 +233,10 @@ pub enum APIError { leader_url: String, }, /// HTTP related error - HTTP { status: StatusCode, message: String }, + HTTP { + status: StatusCode, + message: String, + }, /// raft fatal error, includes StorageError Fatal(Fatal), /// We don't have a quorum to read @@ -226,7 +251,10 @@ pub enum APIError { App(AppError), /// Error in the runtime Runtime(String), - + // Timeout + Timeout, + // recv error + Recv, /// fallback error type Other(String), } @@ -298,13 +326,13 @@ where T: serde::Serialize + serde::Deserialize<'static>, { Err(if let Some(leader_id) = e.leader_id { - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(APIStoreReq::GetNode(leader_id, tx)) .await?; // we can only forward to the leader if we have the node in our state machine - if let Some(leader_addr) = rx.recv().timeout(API_WORKER_TIMEOUT).await?? { + if let Some(leader_addr) = timeout(API_WORKER_TIMEOUT, rx.recv()).await?.flatten() { let mut leader_url = url::Url::parse(&format!("{}://{}", req.url().scheme(), leader_addr.api()))?; leader_url.set_path(req.url().path()); @@ -335,13 +363,15 @@ impl From for Response { StatusCode::BadRequest } APIError::Other(_) - | &APIError::ChangeMembership(_) + | APIError::ChangeMembership(_) | APIError::Store(_) | APIError::Storage(_) | APIError::Fatal(_) | APIError::NoQuorum(_) | APIError::Runtime(_) + | APIError::Recv | APIError::App(_) => StatusCode::InternalServerError, + APIError::Timeout => StatusCode::GatewayTimeout, APIError::HTTP { status, .. } => *status, }; let mut builder = Response::builder(status); @@ -383,6 +413,8 @@ impl std::fmt::Display for APIError { Self::Storage(e) => write!(f, "Storage: {e}"), Self::Runtime(e) => write!(f, "Runtime: {e}"), Self::App(e) => write!(f, "App: {e}"), + Self::Timeout => write!(f, "Timeout"), + Self::Recv => write!(f, "Recv"), } } } @@ -433,20 +465,14 @@ impl From for APIError { } } -impl From> for APIError { - fn from(e: async_std::channel::SendError) -> Self { +impl From> for APIError { + fn from(e: crate::channel::SendError) -> Self { Self::Other(e.to_string()) } } -impl From for APIError { - fn from(e: async_std::channel::RecvError) -> Self { - Self::Other(e.to_string()) - } -} - -impl From for APIError { - fn from(e: TimeoutError) -> Self { - Self::Other(e.to_string()) +impl From for APIError { + fn from(_: tokio::time::error::Elapsed) -> Self { + Self::Timeout } } diff --git a/src/raft/api/apps.rs b/src/raft/api/apps.rs index 1ce9a66bed..74b435e817 100644 --- a/src/raft/api/apps.rs +++ b/src/raft/api/apps.rs @@ -11,7 +11,10 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. +use super::APIError; use crate::{ + channel::bounded, + ids::{AppId, FlowDefinitionId, FlowInstanceId}, instance::IntendedState, raft::{ api::{ @@ -20,15 +23,15 @@ use crate::{ }, archive::{get_app, TremorAppDef}, store::{ - AppId, AppsRequest as AppsCmd, FlowId, FlowInstance, InstanceId, Instances, StateApp, - TremorInstanceState, TremorRequest, TremorResponse, TremorStart, + AppsRequest as AppsCmd, FlowInstance, Instances, StateApp, TremorInstanceState, + TremorRequest, TremorResponse, TremorStart, }, }, }; -use async_std::{channel::bounded, prelude::FutureExt}; use std::collections::HashMap; use std::{fmt::Display, sync::Arc}; use tide::Route; +use tokio::time::timeout; pub(crate) fn install_rest_endpoints(parent: &mut Route>) { let mut apps_endpoint = parent.at("/apps"); @@ -46,12 +49,16 @@ async fn install_app(mut req: APIRequest) -> APIResult { let app = get_app(&file)?; let app_id = app.name().clone(); - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(APIStoreReq::GetApp(app_id.clone(), tx)) .await?; - if rx.recv().timeout(API_WORKER_TIMEOUT).await??.is_some() { + if timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .flatten() + .is_some() + { return Err(AppError::AlreadyInstalled(app.name).into()); } let request = TremorRequest::Apps(AppsCmd::InstallApp { @@ -69,17 +76,22 @@ async fn install_app(mut req: APIRequest) -> APIResult { async fn uninstall_app(req: APIRequest) -> APIResult { let app_id = AppId(req.param("app")?.to_string()); - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(APIStoreReq::GetApp(app_id.clone(), tx)) .await?; - let app = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let app = timeout(API_WORKER_TIMEOUT, rx.recv()).await?.flatten(); if let Some(app) = app { if !app.instances.is_empty() { - return Err( - AppError::HasInstances(app_id, app.instances.keys().cloned().collect()).into(), - ); + return Err(AppError::HasInstances( + app_id.clone(), + app.instances + .keys() + .map(|alias| FlowInstanceId::new(app_id.clone(), alias)) + .collect(), + ) + .into()); } } else { return Err(AppError::AppNotFound(app_id.clone()).into()); @@ -136,14 +148,15 @@ impl Display for AppState { for ( name, FlowInstance { - definition: id, + definition, config, state, + .. }, ) in &self.instances { writeln!(f, " - {name}")?; - writeln!(f, " Flow: {id}")?; + writeln!(f, " Flow definition: {definition}")?; writeln!(f, " Config:")?; if config.is_empty() { writeln!(f, " -")?; @@ -160,34 +173,37 @@ impl Display for AppState { } async fn list(req: APIRequest) -> APIResult> { - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state().store_tx.send(APIStoreReq::GetApps(tx)).await?; - let apps = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let apps = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; Ok(apps) } -async fn start(mut req: APIRequest) -> APIResult { +async fn start(mut req: APIRequest) -> APIResult { let body: TremorStart = req.body_json().await?; let instance_id = body.instance.clone(); let app_name = AppId(req.param("app")?.to_string()); - let flow_name = FlowId(req.param("flow")?.to_string()); + let flow_name = FlowDefinitionId(req.param("flow")?.to_string()); - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(APIStoreReq::GetApp(app_name.clone(), tx)) .await?; - let app = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let app = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; if let Some(app) = app { - if app.instances.contains_key(&body.instance) { - return Err(AppError::InstanceAlreadyExists(app_name, body.instance).into()); + if app.instances.contains_key(instance_id.alias()) { + return Err(AppError::InstanceAlreadyExists(body.instance).into()); } if let Some(flow) = app.app.flows.get(&flow_name) { if let Some(errors) = config_errors(&flow.args, &body.config) { return Err(AppError::InvalidArgs { - app: app_name, flow: flow_name, instance: body.instance, errors, @@ -217,11 +233,11 @@ async fn start(mut req: APIRequest) -> APIResult { Ok(instance_id) } -async fn manage_instance(mut req: APIRequest) -> APIResult { +async fn manage_instance(mut req: APIRequest) -> APIResult { let body: TremorInstanceState = req.body_json().await?; let app_id = AppId(req.param("app")?.to_string()); - let instance_id = InstanceId(req.param("instance")?.to_string()); + let instance_id = FlowInstanceId::new(app_id.clone(), req.param("instance")?.to_string()); // FIXME: this is not only for this but all the API functions as we're running in a potentially // problematic situation here. // @@ -246,15 +262,17 @@ async fn manage_instance(mut req: APIRequest) -> APIResult { // Solution? We might need to put a single process inbetween the API and the raft algorithm that // serializes all commands to ensure no command is executed before the previous one has been fully // handled - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(APIStoreReq::GetApp(app_id.clone(), tx)) .await?; - let app = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let app = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; if let Some(app) = app { - if !app.instances.contains_key(&instance_id) { - return Err(AppError::InstanceNotFound(app_id, instance_id).into()); + if !app.instances.contains_key(instance_id.alias()) { + return Err(AppError::InstanceNotFound(instance_id).into()); } } else { return Err(AppError::AppNotFound(app_id).into()); @@ -265,7 +283,6 @@ async fn manage_instance(mut req: APIRequest) -> APIResult { }; let request = TremorRequest::Apps(AppsCmd::InstanceStateChange { - app: app_id.clone(), instance: instance_id.clone(), state, }); @@ -278,25 +295,25 @@ async fn manage_instance(mut req: APIRequest) -> APIResult { Ok(instance_id) } -async fn stop_instance(req: APIRequest) -> APIResult { +async fn stop_instance(req: APIRequest) -> APIResult { let app_id = AppId(req.param("app")?.to_string()); - let instance_id = InstanceId(req.param("instance")?.to_string()); - let (tx, rx) = bounded(1); + let instance_id = FlowInstanceId::new(app_id.clone(), req.param("instance")?.to_string()); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(APIStoreReq::GetApp(app_id.clone(), tx)) .await?; - if let Some(app) = rx.recv().timeout(API_WORKER_TIMEOUT).await?? { - if !app.instances.contains_key(&instance_id) { - return Err(AppError::InstanceNotFound(app_id, instance_id).into()); + if let Some(app) = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)? + { + if !app.instances.contains_key(instance_id.alias()) { + return Err(AppError::InstanceNotFound(instance_id).into()); } } else { return Err(AppError::AppNotFound(app_id).into()); } - let request = TremorRequest::Apps(AppsCmd::Undeploy { - app: app_id.clone(), - instance: instance_id.clone(), - }); + let request = TremorRequest::Apps(AppsCmd::Undeploy(instance_id.clone())); req.state() .raft .client_write(request) diff --git a/src/raft/api/client.rs b/src/raft/api/client.rs index d40cf1208b..85c4649931 100644 --- a/src/raft/api/client.rs +++ b/src/raft/api/client.rs @@ -14,13 +14,11 @@ //! Tremor Rest API Client use crate::errors::Result; +use crate::ids::{AppId, FlowDefinitionId, FlowInstanceId}; use crate::raft::{ api::apps::AppState, node::Addr, - store::{ - AppId, FlowId, InstanceId, TremorInstanceState, TremorResponse, TremorSet, TremorStart, - }, - NodeId, + store::{TremorInstanceState, TremorResponse, TremorSet, TremorStart}, }; use halfbrown::HashMap; use openraft::{LogId, RaftMetrics}; @@ -73,7 +71,7 @@ impl Tremor { { let target_url = { let target_addr = &self.endpoint; - format!("http://{}/v1/{}", target_addr, uri) + format!("http://{target_addr}/v1/{uri}") }; debug!(">>> client send {method} request to {target_url}"); let mut request = self.inner.request(method, &target_url); @@ -185,19 +183,18 @@ impl Tremor { /// if the api call fails pub async fn start( &self, - app: &AppId, - flow: &FlowId, - instance: &InstanceId, + flow: &FlowDefinitionId, + instance: &FlowInstanceId, config: std::collections::HashMap, running: bool, - ) -> ClientResult { + ) -> ClientResult { let req = TremorStart { instance: instance.clone(), config, running, }; - self.api_req::( - &format!("api/apps/{app}/flows/{flow}"), + self.api_req::( + &format!("api/apps/{}/flows/{flow}", instance.app_id()), Method::POST, Some(&req), ) @@ -215,12 +212,15 @@ impl Tremor { /// if the api call fails pub async fn change_instance_state( &self, - app: &AppId, - instance: &InstanceId, + instance: &FlowInstanceId, state: TremorInstanceState, - ) -> ClientResult { + ) -> ClientResult { self.api_req( - &format!("api/apps/{app}/instances/{instance}"), + &format!( + "api/apps/{}/instances/{}", + instance.app_id(), + instance.alias() + ), Method::POST, Some(&state), ) @@ -236,13 +236,13 @@ impl Tremor { /// /// # Errors /// if the api call fails - pub async fn stop_instance( - &self, - app: &AppId, - instance: &InstanceId, - ) -> ClientResult { + pub async fn stop_instance(&self, instance: &FlowInstanceId) -> ClientResult { self.api_req( - &format!("api/apps/{app}/instances/{instance}"), + &format!( + "api/apps/{}/instances/{}", + instance.app_id(), + instance.alias() + ), Method::DELETE, None::<&()>, ) @@ -268,7 +268,7 @@ impl Tremor { /// Make the given node known to the cluster and assign it a unique `node_id` /// # Errors /// If the api call fails - pub async fn add_node(&self, addr: &Addr) -> ClientResult { + pub async fn add_node(&self, addr: &Addr) -> ClientResult { self.api_req("cluster/nodes", Method::POST, Some(addr)) .await } @@ -278,7 +278,7 @@ impl Tremor { /// After this call a node is not reachable anymore for all nodes still participating in the cluster /// # Errors /// if the api call fails - pub async fn remove_node(&self, node_id: &NodeId) -> ClientResult<()> { + pub async fn remove_node(&self, node_id: &openraft::NodeId) -> ClientResult<()> { self.api_req( &format!("cluster/nodes/{node_id}"), Method::DELETE, @@ -291,7 +291,7 @@ impl Tremor { /// /// # Errors /// if the api call fails - pub async fn get_nodes(&self) -> ClientResult> { + pub async fn get_nodes(&self) -> ClientResult> { self.api_req("cluster/nodes", Method::GET, None::<&()>) .await } @@ -303,7 +303,7 @@ impl Tremor { /// /// # Errors /// if the api call fails e.g. because the node is already a learner - pub async fn add_learner(&self, node_id: &NodeId) -> ClientResult> { + pub async fn add_learner(&self, node_id: &openraft::NodeId) -> ClientResult> { self.api_req( &format!("cluster/learners/{node_id}"), Method::PUT, @@ -318,8 +318,8 @@ impl Tremor { /// /// # Errors /// if the api call fails - pub async fn remove_learner(&self, id: &NodeId) -> ClientResult { - self.api_req::<(), NodeId>(&format!("cluster/learners/{id}"), Method::DELETE, None) + pub async fn remove_learner(&self, id: &openraft::NodeId) -> ClientResult<()> { + self.api_req::<(), ()>(&format!("cluster/learners/{id}"), Method::DELETE, None) .await } @@ -330,9 +330,16 @@ impl Tremor { /// /// # Errors /// if the api call fails - pub async fn promote_voter(&self, id: &NodeId) -> ClientResult> { - self.api_req::<(), Option>(&format!("cluster/voters/{id}"), Method::PUT, None) - .await + pub async fn promote_voter( + &self, + id: &openraft::NodeId, + ) -> ClientResult> { + self.api_req::<(), Option>( + &format!("cluster/voters/{id}"), + Method::PUT, + None, + ) + .await } /// Demote node with `node_id` from voter back to learner. @@ -342,9 +349,16 @@ impl Tremor { /// /// # Errors /// if the api call fails - pub async fn demote_voter(&self, id: &NodeId) -> ClientResult> { - self.api_req::<(), Option>(&format!("cluster/voters/{id}"), Method::DELETE, None) - .await + pub async fn demote_voter( + &self, + id: &openraft::NodeId, + ) -> ClientResult> { + self.api_req::<(), Option>( + &format!("cluster/voters/{id}"), + Method::DELETE, + None, + ) + .await } /// Get the metrics about the cluster. diff --git a/src/raft/api/cluster.rs b/src/raft/api/cluster.rs index 6c3f803a8f..3181b7ae7e 100644 --- a/src/raft/api/cluster.rs +++ b/src/raft/api/cluster.rs @@ -12,17 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use async_std::{channel::bounded, prelude::FutureExt}; -use halfbrown::HashMap; -use tide::{http::StatusCode, Route}; - -use crate::raft::{ - api::{wrapp, APIError, APIResult, ServerState, ToAPIResult}, - node::Addr, - store::{NodesRequest, TremorRequest}, +use crate::{ + channel::bounded, + raft::{ + api::{wrapp, APIError, APIResult, ServerState, ToAPIResult}, + node::Addr, + store::{NodesRequest, TremorRequest}, + }, }; use openraft::{LogId, NodeId, RaftMetrics}; +use std::collections::HashMap; use std::sync::Arc; +use tide::{http::StatusCode, Route}; +use tokio::time::timeout; use super::{APIRequest, APIStoreReq, API_WORKER_TIMEOUT}; @@ -50,9 +52,11 @@ pub(crate) fn install_rest_endpoints(app: &mut Route>) { async fn get_nodes(req: APIRequest) -> APIResult> { let state = req.state(); state.raft.client_read().await.to_api_result(&req).await?; - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); state.store_tx.send(APIStoreReq::GetNodes(tx)).await?; - let nodes = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let nodes = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; Ok(nodes) } @@ -71,13 +75,15 @@ async fn add_node(mut req: APIRequest) -> APIResult { // 2. ensure we don't add the node twice if it is already there // we need to make sure we don't hold on to the state machine lock any further here - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); state .store_tx .send(APIStoreReq::GetNodeId(addr.clone(), tx)) .await?; - let maybe_existing_node_id = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let maybe_existing_node_id = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; if let Some(existing_node_id) = maybe_existing_node_id { Ok(existing_node_id) } else { @@ -110,12 +116,14 @@ async fn add_node(mut req: APIRequest) -> APIResult { async fn remove_node(req: APIRequest) -> APIResult<()> { let node_id = req.param("node_id")?.parse::()?; // make sure the node is not a learner or a voter - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(APIStoreReq::GetLastMembership(tx)) .await?; - let membership = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let membership = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; if membership.contains(&node_id) { return Err(APIError::HTTP { status: StatusCode::Conflict, @@ -147,12 +155,14 @@ async fn add_learner(req: APIRequest) -> APIResult> { // 2. check that the node has already been added // we need to make sure we don't hold on to the state machine lock any further here - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); state .store_tx .send(APIStoreReq::GetNode(node_id, tx)) .await?; - let node_addr = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let node_addr = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; if node_addr.is_none() { return Err(APIError::HTTP { status: StatusCode::NotFound, @@ -174,14 +184,15 @@ async fn add_learner(req: APIRequest) -> APIResult> { /// async fn remove_learner(req: APIRequest) -> APIResult<()> { let node_id = req.param("node_id")?.parse::()?; + debug!( + "[API {} {}] Removing learner {node_id}", + req.method(), + req.url().path() + ); let state = req.state(); // remove the node as learner - state - .raft - .remove_learner(node_id) - .await - .to_api_result(&req) - .await + let result = state.raft.remove_learner(node_id).await; + result.to_api_result(&req).await } /// Changes specified learners to members, or remove members. @@ -190,12 +201,14 @@ async fn promote_voter(req: APIRequest) -> APIResult> { let state = req.state(); // we introduce a new scope here to release the lock on the state machine // not releasing it can lead to dead-locks, if executed on the leader (as the store is shared between the API and the raft engine) - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); state .store_tx .send(APIStoreReq::GetLastMembership(tx)) .await?; - let mut membership = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let mut membership = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; let value = if membership.insert(node_id) { // only update state if not already in the membership config @@ -218,12 +231,14 @@ async fn demote_voter(req: APIRequest) -> APIResult> { let node_id: NodeId = req.param("node_id")?.parse::()?; let state = req.state(); // scoping here to not hold the state machine locked for too long - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); state .store_tx .send(APIStoreReq::GetLastMembership(tx)) .await?; - let mut membership = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let mut membership = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; let value = if membership.remove(&node_id) { req.state() .raft diff --git a/src/raft/api/kv.rs b/src/raft/api/kv.rs index f047eadf35..3c9b3ff6ae 100644 --- a/src/raft/api/kv.rs +++ b/src/raft/api/kv.rs @@ -12,12 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::raft::api::{wrapp, APIError, APIRequest, APIResult, ServerState, ToAPIResult}; -use crate::raft::store::{TremorResponse, TremorSet}; -use async_std::channel::bounded; -use async_std::prelude::FutureExt; +use crate::{ + channel::bounded, + raft::{ + api::{wrapp, APIError, APIRequest, APIResult, ServerState, ToAPIResult}, + store::{TremorResponse, TremorSet}, + }, +}; use std::sync::Arc; use tide::Route; +use tokio::time::timeout; use super::API_WORKER_TIMEOUT; @@ -53,12 +57,14 @@ async fn write(mut req: APIRequest) -> APIResult { /// read a value from the current node, not necessarily the leader, thus this value can be stale async fn read(mut req: APIRequest) -> APIResult { let key: String = req.body_json().await?; - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(super::APIStoreReq::KVGet(key, tx)) .await?; - let value = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let value = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; Ok(TremorResponse { value }) } @@ -69,11 +75,13 @@ async fn consistent_read(mut req: APIRequest) -> APIResult { // this will fail if we are not a leader state.raft.client_read().await.to_api_result(&req).await?; // here we are safe to read - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); req.state() .store_tx .send(super::APIStoreReq::KVGet(key, tx)) .await?; - let value = rx.recv().timeout(API_WORKER_TIMEOUT).await??; + let value = timeout(API_WORKER_TIMEOUT, rx.recv()) + .await? + .ok_or(APIError::Recv)?; Ok(TremorResponse { value }) } diff --git a/src/raft/api/worker.rs b/src/raft/api/worker.rs index 5fc248eee5..892e4c0581 100644 --- a/src/raft/api/worker.rs +++ b/src/raft/api/worker.rs @@ -12,13 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use async_std::channel::{Receiver, Sender}; -use halfbrown::HashMap; +use crate::channel::{Receiver, Sender}; +use std::collections::HashMap; use std::sync::Arc; -use crate::raft::{ - api::APIStoreReq, - store::{AppId, Store}, +use crate::{ + ids::AppId, + raft::{api::APIStoreReq, store::Store}, }; use super::apps::AppState; @@ -29,8 +29,8 @@ async fn send(tx: Sender, t: T) { } } -pub(super) async fn api_worker(store: Arc, store_rx: Receiver) { - while let Ok(msg) = store_rx.recv().await { +pub(super) async fn api_worker(store: Arc, mut store_rx: Receiver) { + while let Some(msg) = store_rx.recv().await { match msg { APIStoreReq::GetApp(app_id, tx) => { let sm = store.state_machine.read().await; diff --git a/src/raft/archive.rs b/src/raft/archive.rs index 258d8bc8eb..d6062e368c 100644 --- a/src/raft/archive.rs +++ b/src/raft/archive.rs @@ -12,7 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::errors::Result; +use crate::{ + errors::Result, + ids::{AppId, FlowDefinitionId}, +}; use async_std::path::PathBuf; use sha2::{Digest, Sha256}; use simd_json::OwnedValue; @@ -37,8 +40,6 @@ use tremor_script::{ FN_REGISTRY, }; -use super::store::{AppId, FlowId}; - #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct AppFlow { /// arguments with possible default values @@ -52,7 +53,7 @@ pub struct TremorAppDef { /// hash of all the included files /// starting with the main.troy and then all `use`d files in order pub sha256: String, - pub flows: HashMap, + pub flows: HashMap, } impl TremorAppDef { @@ -130,7 +131,7 @@ pub(crate) fn build_archive_from_source(name: &str, src: &str) -> Result .iter() .map(|(k, v)| { Ok(( - FlowId(k.to_string()), + FlowDefinitionId(k.to_string()), AppFlow { args: v .clone() @@ -152,7 +153,7 @@ pub(crate) fn build_archive_from_source(name: &str, src: &str) -> Result )) }) .collect::>()?; - if !flows.contains_key(&FlowId("main".to_string())) { + if !flows.contains_key(&FlowDefinitionId("main".to_string())) { let w = Warning { class: Class::Behaviour, outer: deploy.extent(), @@ -269,7 +270,7 @@ pub fn extract(src: &[u8]) -> Result<(TremorAppDef, Deploy, Vec)> .map(|p| p.to_string_lossy().to_string()) .collect(); let id = module.pop().ok_or("No module name")?; - let module = NodeId::new(id.clone(), module.clone(), NodeMeta::dummy().to_owned()); + let module = NodeId::new(id.clone(), module.clone(), NodeMeta::dummy()); info!("included library: {}", entry.path()?.to_string_lossy()); let mut contents = String::new(); diff --git a/src/raft/network/raft_network_impl.rs b/src/raft/network/raft_network_impl.rs index f6e383a2b7..f5914bf3f7 100644 --- a/src/raft/network/raft_network_impl.rs +++ b/src/raft/network/raft_network_impl.rs @@ -15,7 +15,6 @@ use crate::raft::{ network::RaftClient, store::{Store, TremorRequest}, - NodeId, }; use async_trait::async_trait; use dashmap::{mapref::entry::Entry, DashMap}; @@ -32,7 +31,7 @@ use tarpc::{client::RpcError, context}; #[derive(Clone, Debug)] pub struct Network { store: Arc, - pool: DashMap, + pool: DashMap, } impl Network { @@ -46,7 +45,7 @@ impl Network { /// Create a new TCP client for the given `target` node /// /// This requires the `target` to be known to the cluster state. - async fn new_client(&self, target: NodeId) -> anyhow::Result { + async fn new_client(&self, target: openraft::NodeId) -> anyhow::Result { let sm = self.store.state_machine.read().await; let addr = sm .nodes @@ -64,7 +63,7 @@ impl Network { /// Ensure we have a client to the node identified by `target` /// /// Pick it from the pool first, if that fails, create a new one - async fn ensure_client(&self, target: NodeId) -> anyhow::Result { + async fn ensure_client(&self, target: openraft::NodeId) -> anyhow::Result { // TODO: get along without the cloning let client = match self.pool.entry(target) { Entry::Occupied(oe) => oe.get().clone(), @@ -77,13 +76,13 @@ impl Network { Ok(client) } - fn drop_client(&self, target: NodeId) { + fn drop_client(&self, target: openraft::NodeId) { self.pool.remove(&target); } fn handle_response( &self, - target: NodeId, + target: openraft::NodeId, response: Result, RpcError>, ) -> anyhow::Result { match response { @@ -116,7 +115,7 @@ impl RaftNetwork for Network { // and might lead to some nasty timeouts async fn send_append_entries( &self, - target: NodeId, + target: openraft::NodeId, rpc: AppendEntriesRequest, ) -> anyhow::Result { let client = self.ensure_client(target).await?; @@ -125,14 +124,18 @@ impl RaftNetwork for Network { async fn send_install_snapshot( &self, - target: NodeId, + target: openraft::NodeId, rpc: InstallSnapshotRequest, ) -> anyhow::Result { let client = self.ensure_client(target).await?; self.handle_response(target, client.snapshot(context::current(), rpc).await) } - async fn send_vote(&self, target: NodeId, rpc: VoteRequest) -> anyhow::Result { + async fn send_vote( + &self, + target: openraft::NodeId, + rpc: VoteRequest, + ) -> anyhow::Result { let client = self.ensure_client(target).await?; self.handle_response(target, client.vote(context::current(), rpc).await) } diff --git a/src/raft/node.rs b/src/raft/node.rs index 9df1382983..4d218755db 100644 --- a/src/raft/node.rs +++ b/src/raft/node.rs @@ -14,19 +14,16 @@ //! The entirety of a cluster node as a struct use crate::{ + channel::{bounded, Sender}, errors::Result, raft::{ api::{self, ServerState}, network::{raft, Raft as TarPCRaftService}, store::{NodesRequest, Store, TremorRequest, TremorResponse}, - ClusterError, ClusterResult, Network, NodeId, + ClusterError, ClusterResult, Network, }, system::{Runtime, ShutdownMode, WorldConfig}, }; -use async_std::{ - channel::{bounded, Sender}, - task::{self, JoinHandle}, -}; use futures::{future, prelude::*}; use openraft::{Config, Raft}; use std::{ @@ -40,6 +37,8 @@ use tarpc::{ tokio_serde::formats::Json, }; +use tokio::task::{self, JoinHandle}; + #[derive(Clone, Debug)] pub struct ClusterNodeKillSwitch { sender: Sender, @@ -65,7 +64,7 @@ pub struct Running { impl Running { #[must_use] - pub fn node_data(&self) -> (NodeId, Addr) { + pub fn node_data(&self) -> (openraft::NodeId, Addr) { (self.server_state.id(), self.server_state.addr().clone()) } @@ -82,7 +81,8 @@ impl Running { runtime: Runtime, runtime_handle: JoinHandle>, ) -> ClusterResult { - let (kill_switch_tx, kill_switch_rx) = bounded(1); + let node_id = server_state.id(); + let (kill_switch_tx, mut kill_switch_rx) = bounded(1); let tcp_server_state = Arc::new(raft.clone()); let mut listener = @@ -117,48 +117,48 @@ impl Running { let mut kill_switch_future = Box::pin(kill_switch_rx.recv().fuse()); futures::select! { _ = tcp_future => { - warn!("TCP cluster API shutdown."); + warn!("[Node {node_id}] TCP cluster API shutdown."); // Important: this will free and drop the store and thus the rocksdb - api_worker_handle.cancel().await; + api_worker_handle.abort(); raft.shutdown().await.map_err(|_| ClusterError::from("Error shutting down local raft node"))?; runtime.stop(ShutdownMode::Graceful).await?; - runtime_future.await?; + runtime_future.await??; } http_res = http_future => { if let Err(e) = http_res { - error!("HTTP cluster API failed: {e}"); + error!("[Node {node_id}] HTTP cluster API failed: {e}"); } // Important: this will free and drop the store and thus the rocksdb - api_worker_handle.cancel().await; + api_worker_handle.abort(); raft.shutdown().await.map_err(|_| ClusterError::from("Error shutting down local raft node"))?; runtime.stop(ShutdownMode::Graceful).await?; - runtime_future.await?; + runtime_future.await??; } runtime_res = runtime_future => { if let Err(e) = runtime_res { - error!("Local runtime failed: {e}"); + error!("[Node {node_id}] Local runtime failed: {e}"); } // Important: this will free and drop the store and thus the rocksdb - api_worker_handle.cancel().await; + api_worker_handle.abort(); // runtime is already down, we only need to stop local raft raft.shutdown().await.map_err(|_| ClusterError::from("Error shutting down local raft node"))?; } shutdown_mode = kill_switch_future => { let shutdown_mode = shutdown_mode.unwrap_or(ShutdownMode::Forceful); - info!("Tremor cluster node stopping in {shutdown_mode:?} mode"); + info!("[Node {node_id}] Node stopping in {shutdown_mode:?} mode"); // Important: this will free and drop the store and thus the rocksdb - api_worker_handle.cancel().await; + api_worker_handle.abort(); // tcp and http api stopped listening as we don't poll them no more raft.shutdown().await.map_err(|_| ClusterError::from("Error shutting down local raft node"))?; - info!("Raft engine did stop."); - info!("Stopping the Tremor runtime..."); + info!("[Node {node_id}] Raft engine did stop."); + info!("[Node {node_id}] Stopping the Tremor runtime..."); runtime.stop(shutdown_mode).await?; - runtime_future.await?; - info!("Tremor runtime stopped."); + runtime_future.await??; + info!("[Node {node_id}] Tremor runtime stopped."); } } - info!("Tremor cluster node stopped"); + info!("[Node {node_id}] Tremor cluster node stopped"); Ok::<(), ClusterError>(()) }); @@ -184,7 +184,7 @@ impl Running { /// # Errors /// if the node failed to run pub async fn join(self) -> ClusterResult<()> { - self.run_handle.await + self.run_handle.await? } } @@ -251,12 +251,14 @@ impl Node { db_dir: impl AsRef, raft_config: Config, ) -> ClusterResult { + let db = Store::init_db(&db_dir)?; + // ensure we have working node data + let (node_id, addr) = Store::get_self(&db)?; + let world_config = WorldConfig::default(); // TODO: make configurable - let (runtime, runtime_handle) = Runtime::start(world_config).await?; + let (runtime, runtime_handle) = Runtime::start(node_id, world_config).await?; - let store: Arc = Store::load(&db_dir, runtime.clone()).await?; - // ensure we have working node data - let (node_id, addr) = store.get_self()?; + let store: Arc = Store::load(Arc::new(db), runtime.clone()).await?; let node = Self::new(db_dir, raft_config.clone()); let network = Network::new(store.clone()); @@ -312,11 +314,11 @@ impl Node { "Waiting for {}s before retrying to join...", join_wait.as_secs() ); - task::sleep(join_wait).await; + tokio::time::sleep(join_wait).await; }; let world_config = WorldConfig::default(); // TODO: make configurable - let (runtime, runtime_handle) = Runtime::start(world_config).await?; + let (runtime, runtime_handle) = Runtime::start(node_id, world_config).await?; let store = Store::bootstrap(node_id, &addr, &self.db_dir, runtime.clone()).await?; let network = Network::new(store.clone()); let raft = Raft::new(node_id, self.raft_config.clone(), network, store.clone()); @@ -355,9 +357,9 @@ impl Node { /// # Errors /// if bootstrapping a a leader fails pub async fn bootstrap_as_single_node_cluster(&mut self, addr: Addr) -> ClusterResult { + let node_id = openraft::NodeId::default(); let world_config = WorldConfig::default(); // TODO: make configurable - let (runtime, runtime_handle) = Runtime::start(world_config).await?; - let node_id = NodeId::default(); + let (runtime, runtime_handle) = Runtime::start(node_id, world_config).await?; let store = Store::bootstrap(node_id, &addr, &self.db_dir, runtime.clone()).await?; let network = Network::new(store.clone()); @@ -383,7 +385,7 @@ impl Node { .ok_or_else(|| { ClusterError::Other("Invalid Response from raft for AddNode".to_string()) })? - .parse::() + .parse::() .map_err(|e| { ClusterError::Other(format!("Invalid node_id returned from AddNode: {e}")) })?; diff --git a/src/raft/store.rs b/src/raft/store.rs index e9534b9821..99ad11d46d 100644 --- a/src/raft/store.rs +++ b/src/raft/store.rs @@ -14,13 +14,13 @@ mod statemachine; -pub use self::statemachine::apps::{AppId, FlowId, InstanceId}; pub(crate) use self::statemachine::apps::{FlowInstance, Instances, StateApp}; use self::statemachine::{SerializableTremorStateMachine, TremorStateMachine}; use crate::{ errors::Error as RuntimeError, + ids::{AppId, FlowDefinitionId, FlowInstanceId}, instance::IntendedState, - raft::{archive::TremorAppDef, ClusterError, NodeId}, + raft::{archive::TremorAppDef, ClusterError}, system::Runtime, }; use async_std::sync::RwLock; @@ -62,7 +62,7 @@ pub enum NodesRequest { AddNode { addr: Addr }, /// Remove Node with the given `node_id` /// This command should be committed after removing a learner from the cluster. - RemoveNode { node_id: NodeId }, + RemoveNode { node_id: openraft::NodeId }, } /// Operations on apps and their instances @@ -79,19 +79,18 @@ pub enum AppsRequest { /// Deploy and Start a flow of an installed app Deploy { app: AppId, - flow: FlowId, - instance: InstanceId, + flow: FlowDefinitionId, + instance: FlowInstanceId, config: std::collections::HashMap, state: IntendedState, }, /// Stopps and Undeploys an instance of a app - Undeploy { app: AppId, instance: InstanceId }, + Undeploy(FlowInstanceId), /// Requests a instance state change InstanceStateChange { - app: AppId, - instance: InstanceId, + instance: FlowInstanceId, state: IntendedState, }, } @@ -114,7 +113,7 @@ impl AppData for TremorRequest {} #[derive(Debug, Clone, Serialize, Deserialize)] pub(crate) struct TremorStart { - pub(crate) instance: InstanceId, + pub(crate) instance: FlowInstanceId, pub(crate) config: std::collections::HashMap, pub(crate) running: bool, } @@ -209,10 +208,10 @@ pub enum Error { Tremor(Mutex), TremorScript(Mutex), MissingApp(AppId), - MissingFlow(AppId, FlowId), - MissingInstance(AppId, InstanceId), + MissingFlow(AppId, FlowDefinitionId), + MissingInstance(FlowInstanceId), RunningInstances(AppId), - NodeAlreadyAdded(NodeId), + NodeAlreadyAdded(openraft::NodeId), Other(Box), } @@ -273,19 +272,19 @@ impl StdError for Error {} impl Display for Error { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - Error::MissingCf(cf) => write!(f, "missing column family: `{}`", cf), - Error::Utf8(e) => write!(f, "invalid utf8: {}", e), - Error::StrUtf8(e) => write!(f, "invalid utf8: {}", e), - Error::JSON(e) => write!(f, "invalid json: {}", e), - Error::RocksDB(e) => write!(f, "rocksdb error: {}", e), - Error::Io(e) => write!(f, "io error: {}", e), + Error::MissingCf(cf) => write!(f, "missing column family: `{cf}`"), + Error::Utf8(e) => write!(f, "invalid utf8: {e}"), + Error::StrUtf8(e) => write!(f, "invalid utf8: {e}"), + Error::JSON(e) => write!(f, "invalid json: {e}"), + Error::RocksDB(e) => write!(f, "rocksdb error: {e}"), + Error::Io(e) => write!(f, "io error: {e}"), Error::Tremor(e) => write!(f, "tremor error: {:?}", e.lock()), Error::TremorScript(e) => write!(f, "tremor script error: {:?}", e.lock()), - Error::Other(e) => write!(f, "other error: {}", e), - Error::MissingApp(app) => write!(f, "missing app: {}", app), - Error::MissingFlow(app, flow) => write!(f, "missing flow: {}::{}", app, flow), - Error::MissingInstance(app, instance) => { - write!(f, "missing instance: {}::{}", app, instance) + Error::Other(e) => write!(f, "other error: {e}"), + Error::MissingApp(app) => write!(f, "missing app: {app}"), + Error::MissingFlow(app, flow) => write!(f, "missing flow: {app}::{flow}"), + Error::MissingInstance(instance) => { + write!(f, "missing instance: {instance}") } Error::Storage(e) => write!(f, "Storage: {e}"), Error::NodeAlreadyAdded(node_id) => write!(f, "Node {node_id} already added"), @@ -516,7 +515,7 @@ impl RaftStorage for Store { self.put( self.db.cf_logs()?, &id, - &serde_json::to_vec(entry).map_err(logs_w_err)?, + serde_json::to_vec(entry).map_err(logs_w_err)?, ) .map_err(logs_w_err)?; } @@ -718,7 +717,7 @@ impl Store { /// bootstrapping constructor - storing the given node data in the db pub(crate) async fn bootstrap>( - node_id: NodeId, + node_id: openraft::NodeId, addr: &Addr, db_path: P, world: Runtime, @@ -735,35 +734,6 @@ impl Store { Ok(Arc::new(Self { state_machine, db })) } - pub(crate) fn get_self(&self) -> Result<(NodeId, Addr), ClusterError> { - let id = self - .get_self_node_id()? - .ok_or("invalid cluster store, node_id missing")?; - let addr = self - .get_self_addr()? - .ok_or("invalid cluster store, node_addr missing")?; - - Ok((id, addr)) - } - - /// # Errors - /// if the store fails to read the RPC address - pub fn get_self_addr(&self) -> Result, Error> { - Ok(self - .db - .get_cf(self.db.cf_self()?, Store::NODE_ADDR)? - .map(|v| serde_json::from_slice(&v)) - .transpose()?) - } - - /// # Errors - /// if the store fails to read the node id - pub fn get_self_node_id(&self) -> Result, Error> { - self.db - .get_cf(self.db.cf_self()?, Store::NODE_ID)? - .map(|v| bin_to_id(&v)) - .transpose() - } /// Initialize the database /// /// This function is safe and never cleans up or resets the current state, @@ -779,14 +749,7 @@ impl Store { } /// loading constructor - loading the given database - /// - /// verifying that we have some node-data stored - pub(crate) async fn load>( - db_path: P, - world: Runtime, - ) -> Result, ClusterError> { - let db = Self::init_db(db_path)?; - let db = Arc::new(db); + pub(crate) async fn load(db: Arc, world: Runtime) -> Result, ClusterError> { let state_machine = RwLock::new( TremorStateMachine::new(db.clone(), world.clone()) .await @@ -797,14 +760,38 @@ impl Store { } /// Store the information about the current node itself in the `db` - fn set_self(db: &DB, node_id: NodeId, addr: &Addr) -> Result<(), ClusterError> { + fn set_self(db: &DB, node_id: openraft::NodeId, addr: &Addr) -> Result<(), ClusterError> { let node_id_bytes = id_to_bin(node_id)?; let addr_bytes = serde_json::to_vec(addr)?; let cf = db.cf_self()?; db.put_cf(cf, Store::NODE_ID, node_id_bytes)?; - db.put_cf(cf, Store::NODE_ADDR, &addr_bytes)?; + db.put_cf(cf, Store::NODE_ADDR, addr_bytes)?; Ok(()) } + + pub(crate) fn get_self(db: &DB) -> Result<(openraft::NodeId, Addr), ClusterError> { + let id = Self::get_self_node_id(db)?.ok_or("invalid cluster store, node_id missing")?; + let addr = Self::get_self_addr(db)?.ok_or("invalid cluster store, node_addr missing")?; + + Ok((id, addr)) + } + + /// # Errors + /// if the store fails to read the RPC address + pub fn get_self_addr(db: &DB) -> Result, Error> { + Ok(db + .get_cf(db.cf_self()?, Store::NODE_ADDR)? + .map(|v| serde_json::from_slice(&v)) + .transpose()?) + } + + /// # Errors + /// if the store fails to read the node id + pub fn get_self_node_id(db: &DB) -> Result, Error> { + db.get_cf(db.cf_self()?, Store::NODE_ID)? + .map(|v| bin_to_id(&v)) + .transpose() + } } #[cfg(test)] diff --git a/src/raft/store/statemachine/apps.rs b/src/raft/store/statemachine/apps.rs index ac5d30ba4e..25cbbcb160 100644 --- a/src/raft/store/statemachine/apps.rs +++ b/src/raft/store/statemachine/apps.rs @@ -12,14 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::{ - collections::HashSet, - fmt::{Display, Formatter}, - sync::Arc, -}; +use std::{collections::HashSet, sync::Arc}; -use halfbrown::HashMap; +use crate::{ + ids::{AppId, FlowDefinitionId, FlowInstanceId}, + instance::IntendedState, + raft::{ + archive::{extract, get_app, TremorAppDef}, + store::{ + self, + statemachine::{sm_d_err, sm_r_err, sm_w_err, RaftStateMachine}, + store_w_err, AppsRequest, StorageResult, TremorResponse, + }, + }, + system::Runtime, +}; use rocksdb::ColumnFamily; +use std::collections::HashMap; use tremor_script::{ arena::{self, Arena}, ast::{ @@ -32,63 +41,16 @@ use tremor_script::{ AggrRegistry, FN_REGISTRY, }; -use crate::{ - instance::IntendedState, - raft::{ - archive::{extract, get_app, TremorAppDef}, - store::{ - self, - statemachine::{sm_d_err, sm_r_err, sm_w_err, RaftStateMachine}, - store_w_err, AppsRequest, StorageResult, TremorResponse, - }, - }, - system::{flow::Alias as FlowAlias, Runtime}, -}; - -#[derive(Serialize, Deserialize, Debug, Clone, Hash, Eq, PartialEq)] -pub struct InstanceId(pub String); -impl Display for InstanceId { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.0) - } -} - -// FIXME: deduplicate those ids -impl From for FlowAlias { - fn from(instance_id: InstanceId) -> FlowAlias { - FlowAlias::new(instance_id.0) - } -} - -impl From<&InstanceId> for FlowAlias { - fn from(instance_id: &InstanceId) -> FlowAlias { - FlowAlias::new(&instance_id.0) - } -} - -#[derive(Serialize, Deserialize, Debug, Clone, Hash, Eq, PartialEq)] -pub struct FlowId(pub String); -impl Display for FlowId { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.0) - } -} -#[derive(Serialize, Deserialize, Debug, Clone, Hash, Eq, PartialEq)] -pub struct AppId(pub String); -impl Display for AppId { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.0) - } -} - #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] pub struct FlowInstance { + /// Identifier of the instance + pub id: FlowInstanceId, /// the id of the flow definition this instance is based upon - pub definition: FlowId, + pub definition: FlowDefinitionId, pub config: HashMap, pub state: IntendedState, } -pub type Instances = HashMap; +pub type Instances = HashMap; #[derive(Debug, Clone)] pub(crate) struct StateApp { @@ -145,17 +107,18 @@ impl RaftStateMachine for AppsStateMachine { .collect::, store::Error>>()?; // start instances and put them into state machine state - for (app_id, instances) in instances { + for (app_id, app_instances) in instances { for ( - instance, + _, FlowInstance { - definition: id, + id, + definition, config, state, }, - ) in instances + ) in app_instances { - me.deploy_flow(&app_id, id, instance, config, state) + me.deploy_flow(&app_id, definition, id, config, state) .await .map_err(store::Error::Storage)?; } @@ -200,11 +163,11 @@ impl RaftStateMachine for AppsStateMachine { // redeploy existing instance with different config if s_flow.config != flow.config { info!("Flow instance {app_id}/{instance_id} with parameters differ, redeploying..."); - self.stop_and_remove_flow(app_id, instance_id).await?; + self.stop_and_remove_flow(&s_flow.id).await?; self.deploy_flow( app_id, s_flow.definition.clone(), - instance_id.clone(), + s_flow.id.clone(), s_flow.config.clone(), // important: this is the new config s_flow.state, ) @@ -212,22 +175,30 @@ impl RaftStateMachine for AppsStateMachine { } else if s_flow.state != flow.state { // same flow, same config, different state - just change state - self.change_flow_state(app_id, instance_id, s_flow.state) - .await - .map_err(sm_w_err)?; + self.change_flow_state( + &FlowInstanceId::new(app_id.clone(), instance_id), + s_flow.state, + ) + .await + .map_err(sm_w_err)?; } } else { // stop and remove instances that are not in the snapshot - self.stop_and_remove_flow(app_id, instance_id).await?; + self.stop_and_remove_flow(&FlowInstanceId::new( + app_id.clone(), + instance_id, + )) + .await?; } } // deploy instances that are not in self for (s_instance_id, s_flow) in snapshot_instances { - if !instances.contains_key(s_instance_id) { + let flow_id = FlowInstanceId::new(app_id.clone(), s_instance_id); + if !instances.contains_key(flow_id.alias()) { self.deploy_flow( app_id, s_flow.definition.clone(), - s_instance_id.clone(), + flow_id, s_flow.config.clone(), s_flow.state, ) @@ -287,18 +258,14 @@ impl RaftStateMachine for AppsStateMachine { value: Some(instance.to_string()), }) } - AppsRequest::Undeploy { app, instance } => { - self.stop_and_remove_flow(app, instance).await?; + AppsRequest::Undeploy(instance) => { + self.stop_and_remove_flow(instance).await?; Ok(TremorResponse { value: Some(instance.to_string()), }) } - AppsRequest::InstanceStateChange { - app, - instance, - state, - } => { - self.change_flow_state(app, instance, *state).await?; + AppsRequest::InstanceStateChange { instance, state } => { + self.change_flow_state(instance, *state).await?; Ok(TremorResponse { value: Some(instance.to_string()), }) @@ -357,8 +324,8 @@ impl AppsStateMachine { async fn deploy_flow( &mut self, app_id: &AppId, - flow: FlowId, - instance: InstanceId, + flow: FlowDefinitionId, + instance: FlowInstanceId, config: HashMap, intended_state: IntendedState, ) -> StorageResult<()> { @@ -399,7 +366,6 @@ impl AppsStateMachine { let reg = &*FN_REGISTRY.read().map_err(store_w_err)?; let mut helper = Helper::new(reg, &fake_aggr_reg); Optimizer::new(&helper) - .visitor .walk_flow_definition(&mut defn) .map_err(store::Error::from)?; @@ -409,21 +375,26 @@ impl AppsStateMachine { .walk_flow_definition(&mut defn) .map_err(store::Error::from)?; Optimizer::new(&helper) - .visitor .walk_flow_definition(&mut defn) .map_err(store::Error::from)?; } + let mid = Box::new(defn.meta().clone()); let deploy = DeployFlow { - mid: Box::new(defn.meta().clone()), - from_target: tremor_script::ast::NodeId::new(&flow.0, &[app_id.0.clone()]), - instance_alias: instance.0.clone(), + mid: mid.clone(), + from_target: tremor_script::ast::NodeId::new( + flow.0.clone(), + vec![app_id.0.clone()], + mid, + ), + instance_alias: instance.alias().to_string(), defn, docs: None, }; app.instances.insert( - instance.clone(), + instance.alias().to_string(), FlowInstance { + id: instance.clone(), definition: flow, config, state: intended_state, // we are about to apply this state further below @@ -440,28 +411,29 @@ impl AppsStateMachine { .map_err(store_w_err)?; // deploy the flow but don't start it yet - self.world.deploy_flow(&deploy).await.map_err(sm_w_err)?; + dbg!("DEPLOY"); + self.world + .deploy_flow(app_id.clone(), &deploy) + .await + .map_err(sm_w_err)?; // change the flow state to the intended state + dbg!("START"); self.world - .change_flow_state(instance.into(), intended_state) + .change_flow_state(instance, intended_state) .await .map_err(sm_w_err)?; Ok(()) } - async fn stop_and_remove_flow( - &mut self, - app_id: &AppId, - instance_id: &InstanceId, - ) -> StorageResult<()> { - info!("Stop and remove flow {app_id}/{instance_id}"); - if let Some(app) = self.apps.get_mut(app_id) { - if app.instances.get(instance_id).is_some() { + async fn stop_and_remove_flow(&mut self, instance_id: &FlowInstanceId) -> StorageResult<()> { + info!("Stop and remove flow {instance_id}"); + if let Some(app) = self.apps.get_mut(instance_id.app_id()) { + if app.instances.get(instance_id.alias()).is_some() { self.world - .stop_flow(instance_id.into()) + .stop_flow(instance_id.clone()) .await .map_err(sm_d_err)?; - app.instances.remove(instance_id); + app.instances.remove(instance_id.alias()); } } Ok(()) @@ -476,8 +448,9 @@ impl AppsStateMachine { } // stop instances then delete the app for (instance_id, _instance) in app.instances { + let flow_instance_id = FlowInstanceId::new(app.app.name().clone(), instance_id); self.world - .stop_flow(instance_id.into()) + .stop_flow(flow_instance_id) .await .map_err(sm_d_err)?; } @@ -497,24 +470,23 @@ impl AppsStateMachine { async fn change_flow_state( &mut self, - app_id: &AppId, - instance_id: &InstanceId, + instance_id: &FlowInstanceId, intended_state: IntendedState, ) -> StorageResult<()> { - info!("Change flow state {app_id}/{instance_id} to {intended_state}"); + info!("Change flow state {instance_id} to {intended_state}"); let app = self .apps - .get_mut(app_id) - .ok_or_else(|| store::Error::MissingApp(app_id.clone()))?; + .get_mut(instance_id.app_id()) + .ok_or_else(|| store::Error::MissingApp(instance_id.app_id().clone()))?; let instance = app .instances - .get_mut(instance_id) - .ok_or_else(|| store::Error::MissingInstance(app_id.clone(), instance_id.clone()))?; + .get_mut(instance_id.alias()) + .ok_or_else(|| store::Error::MissingInstance(instance_id.clone()))?; // set the intended state in our state machine instance.state = intended_state; // ... and attempt to bring the flow instance in the runtime in the desired state self.world - .change_flow_state(instance_id.into(), intended_state) + .change_flow_state(instance_id.clone(), intended_state) .await .map_err(sm_w_err)?; Ok(()) diff --git a/src/raft/store/statemachine/nodes.rs b/src/raft/store/statemachine/nodes.rs index da40bb33e4..0c5250c441 100644 --- a/src/raft/store/statemachine/nodes.rs +++ b/src/raft/store/statemachine/nodes.rs @@ -15,15 +15,13 @@ //! nodes raft sub-statemachine //! handling all the nodes that are known to the cluster and is responsible for assigning node ids +use rocksdb::ColumnFamily; +use std::collections::HashMap; use std::sync::{ atomic::{AtomicU64, Ordering}, Arc, }; -use halfbrown::HashMap; -use openraft::NodeId; -use rocksdb::ColumnFamily; - use crate::{ raft::{ node::Addr, @@ -42,7 +40,7 @@ use crate::{ pub(crate) struct NodesStateMachine { db: Arc, next_node_id: Arc, - known_nodes: HashMap, + known_nodes: HashMap, } impl NodesStateMachine { @@ -86,7 +84,7 @@ impl RaftStateMachine for NodesStateMachine { let addr: Addr = serde_json::from_slice(&value_raw)?; Ok((node_id, addr)) }) - .collect::, StoreError>>()?; + .collect::, StoreError>>()?; Ok(Self { db: db.clone(), @@ -133,7 +131,7 @@ impl RaftStateMachine for NodesStateMachine { } impl NodesStateMachine { - fn next_node_id(&self) -> StorageResult { + fn next_node_id(&self) -> StorageResult { let s = self.next_node_id.fetch_add(1, Ordering::SeqCst); let s_bytes = id_to_bin(s)?; self.db @@ -152,7 +150,7 @@ impl NodesStateMachine { } /// find a `NodeId` for the given `addr` if it is already stored - pub(crate) fn find_node_id(&self, addr: &Addr) -> Option<&NodeId> { + pub(crate) fn find_node_id(&self, addr: &Addr) -> Option<&openraft::NodeId> { self.known_nodes .iter() .find(|(_node_id, existing_addr)| *existing_addr == addr) @@ -160,17 +158,17 @@ impl NodesStateMachine { } /// get the `Addr` of the node identified by `node_id` if it is stored - pub(crate) fn get_node(&self, node_id: NodeId) -> Option<&Addr> { + pub(crate) fn get_node(&self, node_id: openraft::NodeId) -> Option<&Addr> { self.known_nodes.get(&node_id) } /// get all known nodes with their `NodeId` and `Addr` - pub(crate) fn get_nodes(&self) -> &HashMap { + pub(crate) fn get_nodes(&self) -> &HashMap { &self.known_nodes } /// add the node identified by `addr` if not already there and assign and return a new `node_id` - fn add_node(&mut self, addr: &Addr) -> StorageResult { + fn add_node(&mut self, addr: &Addr) -> StorageResult { if let Some(node_id) = self.find_node_id(addr) { Err(store_w_err(StoreError::NodeAlreadyAdded(*node_id))) } else { @@ -182,7 +180,7 @@ impl NodesStateMachine { } /// store the given addr under the given `node_id`, without creating a new one - fn store_node(&mut self, node_id: NodeId, addr: &Addr) -> StorageResult<()> { + fn store_node(&mut self, node_id: openraft::NodeId, addr: &Addr) -> StorageResult<()> { let node_id_bytes = id_to_bin(node_id)?; self.db .put_cf( @@ -195,7 +193,7 @@ impl NodesStateMachine { Ok(()) } - fn remove_node(&mut self, node_id: NodeId) -> StorageResult<()> { + fn remove_node(&mut self, node_id: openraft::NodeId) -> StorageResult<()> { self.known_nodes.remove(&node_id); let node_id_bytes = id_to_bin(node_id)?; self.db @@ -208,5 +206,5 @@ impl NodesStateMachine { #[derive(Serialize, Deserialize, Clone, Debug)] pub(crate) struct NodesSnapshot { next_node_id: u64, - nodes: HashMap, + nodes: HashMap, } diff --git a/src/raft/test/learner.rs b/src/raft/test/learner.rs index ea6c2c22fc..0b80f0f008 100644 --- a/src/raft/test/learner.rs +++ b/src/raft/test/learner.rs @@ -1,12 +1,3 @@ -use std::time::Duration; - -use halfbrown::HashMap; - -use crate::raft::{ - archive::build_archive_from_source, - store::{FlowId, InstanceId}, -}; - // Copyright 2022, The Tremor Team // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -21,6 +12,12 @@ use crate::raft::{ // See the License for the specific language governing permissions and // limitations under the License. use super::prelude::*; +use crate::{ + ids::{FlowDefinitionId, FlowInstanceId}, + raft::archive::build_archive_from_source, +}; +use std::collections::HashMap; +use std::time::Duration; #[tokio::test(flavor = "multi_thread")] async fn add_learner_test() -> ClusterResult<()> { @@ -60,10 +57,14 @@ async fn add_learner_test() -> ClusterResult<()> { "learner not to be part of cluster voters" ); // remove the learner again + dbg!("remove learner", &learner_node_id); client0.remove_learner(&learner_node_id).await?; + dbg!("stop learner"); learner_node.stop().await?; + dbg!("remove learner node"); client0.remove_node(&learner_node_id).await?; + dbg!("removed"); // TODO: deploy an app and see if the learner also runs it // TODO: verify the whole lifecycle shenanigans of app instances with and without learner @@ -136,12 +137,10 @@ end; let archive = build_archive_from_source("main", app_entrypoint.as_str())?; let app_id = client0.install(&archive).await?; - let flow_id = FlowId("main".to_string()); - let instance = InstanceId("01".to_string()); + let flow_id = FlowDefinitionId("main".to_string()); + let instance = FlowInstanceId::new(app_id, "01".to_string()); let config = HashMap::new(); - let instance_id = client0 - .start(&app_id, &flow_id, &instance, config, true) - .await?; + let instance_id = client0.start(&flow_id, &instance, config, true).await?; // wait for the app to be actually started // wait for the file to exist @@ -151,7 +150,9 @@ end; // wait another short while for all nodes to finish writing tokio::time::sleep(Duration::from_millis(500)).await; // stop the flow instance - client0.stop_instance(&app_id, &instance_id).await?; + dbg!("Stopping instance"); + client0.stop_instance(&instance_id).await?; + dbg!("Instance stopped"); // shut the nodes down learner_node.stop().await?; node2.stop().await?; diff --git a/src/system.rs b/src/system.rs index 6f6f19cd96..96f35ec233 100644 --- a/src/system.rs +++ b/src/system.rs @@ -17,15 +17,18 @@ pub mod flow; /// contains the runtime actor starting and maintaining flows pub mod flow_supervisor; +use std::time::Duration; + use self::flow::Flow; use crate::{ channel::{bounded, Sender}, connectors, errors::{empty_error, Error, Kind as ErrorKind, Result}, + ids::{AppId, FlowInstanceId}, instance::IntendedState as IntendedInstanceState, log_error, }; -use std::time::Duration; +use openraft::NodeId; use tokio::{sync::oneshot, task::JoinHandle, time::timeout}; use tremor_script::{ ast, @@ -138,18 +141,18 @@ impl Runtime { Err(e) => { log_error!(h.format_error(&e), "Error: {e}"); - return Err(format!("failed to load troy file: {}", src).into()); + return Err(format!("failed to load troy file: {src}").into()); } }; let mut count = 0; // first deploy them for flow in deployable.iter_flows() { - self.deploy_flow(flow).await?; + self.deploy_flow(AppId::default(), flow).await?; } // start flows in a second step for flow in deployable.iter_flows() { - self.start_flow(flow::Alias::new(&flow.instance_alias)) + self.start_flow(FlowInstanceId::new(AppId::default(), &flow.instance_alias)) .await?; count += 1; } @@ -161,35 +164,42 @@ impl Runtime { /// This flow instance is not started yet. /// # Errors /// If the flow can't be deployed - pub async fn deploy_flow(&self, flow: &ast::DeployFlow<'static>) -> Result<()> { + pub async fn deploy_flow( + &self, + app_id: AppId, + flow: &ast::DeployFlow<'static>, + ) -> Result { + // FIXME: return a FlowInstanceId here let (tx, rx) = oneshot::channel(); self.flows .send(flow_supervisor::Msg::DeployFlow { + app: app_id, flow: Box::new(flow.clone()), sender: tx, }) .await?; - if let Err(e) = rx.await? { - let err_str = match e { - Error( - ErrorKind::Script(e) - | ErrorKind::Pipeline(tremor_pipeline::errors::ErrorKind::Script(e)), - _, - ) => { - let mut h = crate::ToStringHighlighter::new(); - h.format_error(&tremor_script::errors::Error::from(e))?; - h.finalize()?; - h.to_string() - } - err => err.to_string(), - }; - error!( - "Error starting deployment of flow {}: {}", - flow.instance_alias, &err_str - ); - Err(ErrorKind::DeployFlowError(flow.instance_alias.clone(), err_str).into()) - } else { - Ok(()) + match rx.await? { + Err(e) => { + let err_str = match e { + Error( + ErrorKind::Script(e) + | ErrorKind::Pipeline(tremor_pipeline::errors::ErrorKind::Script(e)), + _, + ) => { + let mut h = crate::ToStringHighlighter::new(); + h.format_error(&tremor_script::errors::Error::from(e))?; + h.finalize()?; + h.to_string() + } + err => err.to_string(), + }; + error!( + "Error starting deployment of flow {}: {err_str}", + flow.instance_alias + ); + Err(ErrorKind::DeployFlowError(flow.instance_alias.clone(), err_str).into()) + } + Ok(flow_id) => Ok(flow_id), } } @@ -197,7 +207,7 @@ impl Runtime { /// if the flow state change fails pub async fn change_flow_state( &self, - id: flow::Alias, + id: FlowInstanceId, intended_state: IntendedInstanceState, ) -> Result<()> { let (reply_tx, mut reply_rx) = bounded(1); @@ -215,7 +225,7 @@ impl Runtime { /// /// # Errors /// if the flow can't be started - pub async fn start_flow(&self, id: flow::Alias) -> Result<()> { + pub async fn start_flow(&self, id: FlowInstanceId) -> Result<()> { self.change_flow_state(id, IntendedInstanceState::Running) .await } @@ -224,7 +234,7 @@ impl Runtime { /// /// # Errors /// if the flow can't be stopped - pub async fn stop_flow(&self, id: flow::Alias) -> Result<()> { + pub async fn stop_flow(&self, id: FlowInstanceId) -> Result<()> { self.change_flow_state(id, IntendedInstanceState::Stopped) .await } @@ -233,7 +243,7 @@ impl Runtime { /// /// # Errors /// if the flow can't be paused - pub async fn pause_flow(&self, id: flow::Alias) -> Result<()> { + pub async fn pause_flow(&self, id: FlowInstanceId) -> Result<()> { self.change_flow_state(id, IntendedInstanceState::Paused) .await } @@ -241,7 +251,7 @@ impl Runtime { /// /// # Errors /// if the flow can't be resumed - pub async fn resume_flow(&self, id: flow::Alias) -> Result<()> { + pub async fn resume_flow(&self, id: FlowInstanceId) -> Result<()> { self.start_flow(id).await // equivalent } @@ -268,11 +278,10 @@ impl Runtime { /// /// # Errors /// * if we fail to send the request or fail to receive it - pub async fn get_flow(&self, flow_id: String) -> Result { + pub async fn get_flow(&self, flow_id: FlowInstanceId) -> Result { let (flow_tx, flow_rx) = oneshot::channel(); - let flow_id = flow::Alias::new(flow_id); self.flows - .send(flow_supervisor::Msg::GetFlow(flow_id.clone(), flow_tx)) + .send(flow_supervisor::Msg::GetFlow(flow_id, flow_tx)) .await?; flow_rx.await? } @@ -293,8 +302,11 @@ impl Runtime { /// /// # Errors /// * if the world manager can't be started - pub async fn start(config: WorldConfig) -> Result<(Self, JoinHandle>)> { - let (system_h, system, kill_switch) = flow_supervisor::FlowSupervisor::new().start(); + pub async fn start( + node_id: NodeId, + config: WorldConfig, + ) -> Result<(Self, JoinHandle>)> { + let (system_h, system, kill_switch) = flow_supervisor::FlowSupervisor::new(node_id).start(); let world = Self { flows: system, diff --git a/src/system/flow.rs b/src/system/flow.rs index 5b36ad79c2..0ccd9d0017 100644 --- a/src/system/flow.rs +++ b/src/system/flow.rs @@ -19,6 +19,7 @@ use crate::{ use crate::{ connectors::{self, ConnectorResult, Known}, errors::{Error, Kind as ErrorKind, Result}, + ids::FlowInstanceId, instance::{IntendedState, State}, log_error, pipeline::{self, InputTarget}, @@ -30,55 +31,14 @@ use hashbrown::HashMap; use std::{collections::HashSet, ops::ControlFlow, pin::Pin, time::Duration}; use tokio::{task, time::timeout}; use tokio_stream::wrappers::ReceiverStream; -use tremor_common::ids::{ConnectorIdGen, OperatorIdGen}; +use tremor_common::uids::{ConnectorUIdGen, OperatorUIdGen}; use tremor_script::{ - ast::{self, ConnectStmt, DeployFlow, Helper}, + ast::{self, ConnectStmt, Helper}, errors::{error_generic, not_defined_err}, }; use super::ShutdownMode; -/// unique identifier of a flow instance within a tremor instance -#[derive(Debug, PartialEq, PartialOrd, Eq, Hash, Clone, Serialize, Deserialize)] -pub struct Alias(String); // FIXME: we need to add the app_id to this - -impl Alias { - /// construct a new flow if from some stringy thingy - pub fn new(alias: impl Into) -> Self { - Self(alias.into()) - } - - /// reference this id as a stringy thing again - #[must_use] - pub fn as_str(&self) -> &str { - self.0.as_str() - } -} - -impl From<&DeployFlow<'_>> for Alias { - fn from(f: &DeployFlow) -> Self { - Self(f.instance_alias.to_string()) - } -} - -impl From<&str> for Alias { - fn from(e: &str) -> Self { - Self(e.to_string()) - } -} - -impl From for Alias { - fn from(alias: String) -> Self { - Self(alias) - } -} - -impl std::fmt::Display for Alias { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - self.0.fmt(f) - } -} - #[derive(Debug)] /// Control Plane message accepted by each binding control plane handler pub(crate) enum Msg { @@ -103,7 +63,7 @@ type Addr = Sender; /// A deployed Flow instance #[derive(Debug, Clone)] pub struct Flow { - alias: Alias, + alias: FlowInstanceId, addr: Addr, } @@ -111,7 +71,7 @@ pub struct Flow { #[derive(Serialize, Deserialize, Debug)] pub struct StatusReport { /// the id of the instance this report describes - pub alias: Alias, + pub alias: FlowInstanceId, /// the current state pub status: State, /// the created connectors @@ -119,7 +79,7 @@ pub struct StatusReport { } impl Flow { - pub(crate) fn id(&self) -> &Alias { + pub(crate) fn id(&self) -> &FlowInstanceId { &self.alias } @@ -207,16 +167,17 @@ impl Flow { /// If any of the operations of spawning connectors, linking pipelines and connectors or spawning the flow instance /// fails. pub(crate) async fn deploy( + node_id: openraft::NodeId, + flow_id: FlowInstanceId, flow: ast::DeployFlow<'static>, - operator_id_gen: &mut OperatorIdGen, - connector_id_gen: &mut ConnectorIdGen, + operator_id_gen: &mut OperatorUIdGen, + connector_id_gen: &mut ConnectorUIdGen, known_connectors: &Known, kill_switch: &KillSwitch, // FIXME: add AppContext ) -> Result { let mut pipelines = HashMap::new(); let mut connectors = HashMap::new(); - let flow_alias = Alias::from(&flow); for create in &flow.defn.creates { let alias: &str = &create.instance_alias; @@ -224,7 +185,7 @@ impl Flow { ast::CreateTargetDefinition::Connector(defn) => { let mut defn = defn.clone(); defn.params.ingest_creational_with(&create.with)?; - let connector_alias = connectors::Alias::new(flow_alias.clone(), alias); + let connector_alias = connectors::Alias::new(flow_id.clone(), alias); let config = crate::Connector::from_defn(&connector_alias, &defn)?; let builder = known_connectors @@ -235,6 +196,7 @@ impl Flow { connectors.insert( alias.to_string(), connectors::spawn( + node_id, &connector_alias, connector_id_gen, builder.as_ref(), @@ -252,11 +214,12 @@ impl Flow { defn.to_query(&create.with, &mut helper)? }; - let pipeline_alias = pipeline::Alias::new(flow_alias.clone(), alias); + let pipeline_alias = pipeline::Alias::new(flow_id.clone(), alias); let pipeline = tremor_pipeline::query::Query( tremor_script::query::Query::from_query(query), ); - let addr = pipeline::spawn(pipeline_alias, &pipeline, operator_id_gen)?; + let addr = + pipeline::spawn(node_id, pipeline_alias, &pipeline, operator_id_gen)?; pipelines.insert(alias.to_string(), addr); } } @@ -268,14 +231,14 @@ impl Flow { } let addr = spawn_task( - flow_alias.clone(), + flow_id.clone(), pipelines, &connectors, &flow.defn.connections, ); let this = Flow { - alias: flow_alias.clone(), + alias: flow_id.clone(), addr, }; @@ -422,7 +385,7 @@ enum MsgWrapper { } struct RunningFlow { - id: Alias, + id: FlowInstanceId, state: State, expected_starts: usize, expected_drains: usize, @@ -441,7 +404,7 @@ struct RunningFlow { impl RunningFlow { fn new( - id: Alias, + id: FlowInstanceId, pipelines: HashMap, connectors: &HashMap, links: &[ConnectStmt], @@ -852,6 +815,7 @@ impl RunningFlow { error!("{prefix} Error stopping pipeline {pipeline:?}: {e}"); } } + self.change_state(State::Stopped).await; // nothing to do, we can break right away Ok(ControlFlow::Break(())) } @@ -939,7 +903,7 @@ impl RunningFlow { /// task handling flow instance control plane #[allow(clippy::too_many_lines)] fn spawn_task( - id: Alias, + id: FlowInstanceId, pipelines: HashMap, connectors: &HashMap, links: &[ConnectStmt], @@ -953,9 +917,12 @@ fn spawn_task( #[cfg(test)] mod tests { use super::*; - use crate::{connectors::ConnectorBuilder, instance, qsize}; - - use tremor_common::ids::{ConnectorIdGen, OperatorIdGen}; + use crate::{ + connectors::ConnectorBuilder, + ids::{FlowInstanceId, BOOTSTRAP_NODE_ID}, + instance, qsize, + }; + use tremor_common::uids::{ConnectorUIdGen, OperatorUIdGen}; use tremor_script::{ast::DeployStmt, deploy::Deploy, FN_REGISTRY}; use tremor_value::literal; @@ -1072,8 +1039,8 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn flow_spawn() -> Result<()> { - let mut operator_id_gen = OperatorIdGen::default(); - let mut connector_id_gen = ConnectorIdGen::default(); + let mut operator_id_gen = OperatorUIdGen::default(); + let mut connector_id_gen = ConnectorUIdGen::default(); let aggr_reg = tremor_script::aggr_registry(); let src = r#" define flow test @@ -1114,6 +1081,8 @@ mod tests { let builder = connector::FakeBuilder { tx: connector_tx }; known_connectors.insert(builder.connector_type(), Box::new(builder)); let flow = Flow::deploy( + BOOTSTRAP_NODE_ID, + FlowInstanceId::new("app", "test"), deploy, &mut operator_id_gen, &mut connector_id_gen, @@ -1127,11 +1096,14 @@ mod tests { start_rx.recv().await.ok_or_else(empty_error)??; let connector = flow.get_connector("foo".to_string()).await?; - assert_eq!(String::from("test::foo"), connector.alias.to_string()); + assert_eq!(String::from("app/test::foo"), connector.alias.to_string()); let connectors = flow.get_connectors().await?; assert_eq!(1, connectors.len()); - assert_eq!(String::from("test::foo"), connectors[0].alias.to_string()); + assert_eq!( + String::from("app/test::foo"), + connectors[0].alias.to_string() + ); // assert the flow has started and events are flowing let event = connector_rx.recv().await.ok_or("empty")?; @@ -1149,7 +1121,7 @@ mod tests { assert_eq!(instance::State::Running, report.status); assert_eq!(1, report.connectors.len()); - let (tx, rx) = bounded(1); + let (tx, mut rx) = bounded(1); flow.pause(tx.clone()).await?; rx.recv().await.ok_or_else(empty_error)??; let report = flow.report_status().await?; diff --git a/src/system/flow_supervisor.rs b/src/system/flow_supervisor.rs index 44c5ef2d1f..0bf5487f5d 100644 --- a/src/system/flow_supervisor.rs +++ b/src/system/flow_supervisor.rs @@ -16,20 +16,18 @@ use crate::{ channel::{bounded, Sender}, connectors::{self, ConnectorBuilder, ConnectorType}, errors::{empty_error, Kind as ErrorKind, Result}, + ids::{AppId, FlowInstanceId}, instance::IntendedState, log_error, qsize, - system::{ - flow::{Alias, Flow}, - KillSwitch, DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT, - }, + system::{flow::Flow, KillSwitch, DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT}, }; -use hashbrown::{hash_map::Entry, HashMap}; +use std::collections::{hash_map::Entry, HashMap}; use tokio::{ sync::oneshot, task::{self, JoinHandle}, time::timeout, }; -use tremor_common::ids::{ConnectorIdGen, OperatorIdGen}; +use tremor_common::uids::{ConnectorUIdGen, OperatorUIdGen}; use tremor_script::ast::DeployFlow; pub(crate) type Channel = Sender; @@ -39,15 +37,17 @@ pub(crate) type Channel = Sender; pub(crate) enum Msg { /// Deploy a flow, instantiate it, but does not start it or any child instance (connector, pipeline) DeployFlow { + /// the App this flow shall be a part of + app: AppId, /// the `deploy flow` AST flow: Box>, /// result sender - sender: oneshot::Sender>, + sender: oneshot::Sender>, }, /// change instance state ChangeInstanceState { /// unique ID for the `Flow` instance to start - id: Alias, + id: FlowInstanceId, /// The state the instance should be changed to intended_state: IntendedState, /// result sender @@ -60,7 +60,7 @@ pub(crate) enum Msg { builder: Box, }, GetFlows(oneshot::Sender>>), - GetFlow(Alias, oneshot::Sender>), + GetFlow(FlowInstanceId, oneshot::Sender>), /// Initiate the Quiescence process Drain(oneshot::Sender>), /// stop this manager @@ -69,19 +69,21 @@ pub(crate) enum Msg { #[derive(Debug)] pub(crate) struct FlowSupervisor { - flows: HashMap, - operator_id_gen: OperatorIdGen, - connector_id_gen: ConnectorIdGen, + node_id: openraft::NodeId, + flows: HashMap, + operator_id_gen: OperatorUIdGen, + connector_id_gen: ConnectorUIdGen, known_connectors: connectors::Known, } impl FlowSupervisor { - pub fn new() -> Self { + pub fn new(node_id: openraft::NodeId) -> Self { Self { + node_id, flows: HashMap::new(), known_connectors: connectors::Known::new(), - operator_id_gen: OperatorIdGen::new(), - connector_id_gen: ConnectorIdGen::new(), + operator_id_gen: OperatorUIdGen::new(), + connector_id_gen: ConnectorUIdGen::new(), } } @@ -97,14 +99,17 @@ impl FlowSupervisor { async fn handle_deploy( &mut self, + app_id: AppId, flow: DeployFlow<'static>, - sender: oneshot::Sender>, + sender: oneshot::Sender>, kill_switch: &KillSwitch, ) { - let id = Alias::from(&flow); + let id = FlowInstanceId::from_deploy(app_id, &flow); let res = match self.flows.entry(id.clone()) { Entry::Occupied(_occupied) => Err(ErrorKind::DuplicateFlow(id.to_string()).into()), Entry::Vacant(vacant) => Flow::deploy( + self.node_id, + id.clone(), flow, &mut self.operator_id_gen, &mut self.connector_id_gen, @@ -114,6 +119,7 @@ impl FlowSupervisor { .await .map(|deploy| { vacant.insert(deploy); + id }), }; log_error!( @@ -128,7 +134,7 @@ impl FlowSupervisor { "Error sending ListFlows response: {e}" ); } - fn handle_get_flow(&self, id: &Alias, reply_tx: oneshot::Sender>) { + fn handle_get_flow(&self, id: &FlowInstanceId, reply_tx: oneshot::Sender>) { log_error!( reply_tx .send( @@ -142,19 +148,21 @@ impl FlowSupervisor { ); } - async fn handle_terminate(&self) -> Result<()> { + async fn handle_terminate(&mut self) -> Result<()> { info!("Stopping Manager ..."); if !self.flows.is_empty() { // send stop to each deployment let (tx, mut rx) = bounded(self.flows.len()); let mut expected_stops: usize = 0; - for flow in self.flows.values() { - log_error!( + // drain the flows, we are stopping anyways, this is the last interaction with them + for (_, flow) in self.flows.drain() { + if !log_error!( flow.stop(tx.clone()).await, "Failed to stop Deployment \"{alias}\": {e}", alias = flow.id() - ); - expected_stops += 1; + ) { + expected_stops += 1; + } } timeout( @@ -170,11 +178,11 @@ impl FlowSupervisor { Result::Ok(()) }), ) - .await??; + .await???; } Ok(()) } - async fn handle_drain(&self, sender: oneshot::Sender>) { + async fn handle_drain(&mut self, sender: oneshot::Sender>) { if self.flows.is_empty() { log_error!( sender.send(Ok(())).map_err(|_| "send error"), @@ -185,7 +193,7 @@ impl FlowSupervisor { info!("Draining all {num_flows} Flows ..."); let mut alive_flows = 0_usize; let (tx, mut rx) = bounded(num_flows); - for (_, flow) in &self.flows { + for (_, flow) in self.flows.drain() { if !log_error!( flow.stop(tx.clone()).await, "Failed to drain Deployment \"{alias}\": {e}", @@ -212,12 +220,21 @@ impl FlowSupervisor { } async fn handle_change_state( - &self, - id: Alias, + &mut self, + id: FlowInstanceId, intended_state: IntendedState, reply_tx: Sender>, ) -> Result<()> { - if let Some(flow) = self.flows.get(&id) { + if let IntendedState::Stopped = intended_state { + // we remove the flow as it won't be reachable anymore, once it is stopped + // keeping it around will lead to errors upon stopping + if let Some(flow) = self.flows.remove(&id) { + flow.stop(reply_tx).await?; + Ok(()) + } else { + Err(ErrorKind::FlowNotFound(id.to_string()).into()) + } + } else if let Some(flow) = self.flows.get(&id) { flow.change_state(intended_state, reply_tx).await?; Ok(()) } else { @@ -238,8 +255,9 @@ impl FlowSupervisor { builder, .. } => self.handle_register_connector_type(connector_type, builder), - Msg::DeployFlow { flow, sender } => { - self.handle_deploy(*flow, sender, &task_kill_switch).await; + Msg::DeployFlow { app, flow, sender } => { + self.handle_deploy(app, *flow, sender, &task_kill_switch) + .await; } Msg::GetFlows(reply_tx) => self.handle_get_flows(reply_tx), Msg::GetFlow(id, reply_tx) => self.handle_get_flow(&id, reply_tx), diff --git a/static/openapi.yaml b/static/openapi.yaml index 81a46ceaba..e06b37cd9f 100644 --- a/static/openapi.yaml +++ b/static/openapi.yaml @@ -81,12 +81,18 @@ paths: application/yaml: schema: $ref: '#/components/schemas/flows' - /v1/flows/{flow-id}: + /v1/flows/{app-id}/{flow-id}: parameters: + - name: app-id + in: path + required: true + description: The unique id of an app inside the runtime + schema: + type: string - name: flow-id in: path required: true - description: The unique id of the flow in the runtime + description: The unique id of the flow in the the app identified by `app_id` schema: type: string get: @@ -141,8 +147,14 @@ paths: '404': description: The flow 'flow-id' wasnt found. It is thus not deployed in the runtime. - /v1/flows/{flow-id}/connectors: + /v1/flows/{app-id}/{flow-id}/connectors: parameters: + - name: app-id + in: path + required: true + description: The unique id of an app inside the runtime + schema: + type: string - name: flow-id in: path required: true @@ -167,8 +179,14 @@ paths: $ref: '#/components/schemas/connectors' '404': description: The flow 'flow-id' wasnt found. It is thus not deployed in the runtime. - /v1/flows/{flow-id}/connectors/{connector-id}: + /v1/flows/{app-id}/{flow-id}/connectors/{connector-id}: parameters: + - name: app-id + in: path + required: true + description: The unique id of an app inside the runtime + schema: + type: string - name: flow-id in: path required: true diff --git a/tests/flows.rs b/tests/flows.rs index df18dc5ebb..db962fd614 100644 --- a/tests/flows.rs +++ b/tests/flows.rs @@ -17,6 +17,7 @@ use std::time::Duration; use tremor_common::file; use tremor_runtime::{ errors::*, + ids::{AppId, FlowInstanceId}, system::{Runtime, ShutdownMode, WorldConfig}, }; use tremor_script::{deploy::Deploy, module::Manager}; @@ -52,10 +53,11 @@ macro_rules! test_cases { let config = WorldConfig{ debug_connectors: true, }; - let (runtime, h) = Runtime::start(config).await?; + let (runtime, h) = Runtime::start(openraft::NodeId::default(), config).await?; + let app_id = AppId::default(); for flow in deployable.iter_flows() { - let flow_alias = flow.instance_alias.to_string().into(); - runtime.deploy_flow(flow).await?; + let flow_alias = FlowInstanceId::new(app_id.clone(), &flow.instance_alias); + runtime.deploy_flow(app_id.clone(), flow).await?; runtime.start_flow(flow_alias).await?; } runtime.stop(ShutdownMode::Forceful).await?; diff --git a/tests/query.rs b/tests/query.rs index 59e8b859fc..8bda6ef9d2 100644 --- a/tests/query.rs +++ b/tests/query.rs @@ -13,7 +13,7 @@ // limitations under the License. use pretty_assertions::assert_eq; use std::io::prelude::*; -use tremor_common::{file, ids::OperatorIdGen, ports::IN}; +use tremor_common::{file, ports::IN, uids::OperatorUIdGen}; use tremor_pipeline::ExecutableGraph; use tremor_pipeline::{query::Query, EventOriginUri}; use tremor_pipeline::{Event, EventId}; @@ -26,7 +26,7 @@ use tremor_script::utils::*; fn to_pipe(query: String) -> Result { let aggr_reg = tremor_script::aggr_registry(); - let mut idgen = OperatorIdGen::new(); + let mut idgen = OperatorUIdGen::new(); let q = Query::parse(&query, &*FN_REGISTRY.read()?, &aggr_reg)?; Ok(q.to_executable_graph(&mut idgen)?) } diff --git a/tests/query_error.rs b/tests/query_error.rs index 41e381ff17..985c10cabd 100644 --- a/tests/query_error.rs +++ b/tests/query_error.rs @@ -16,7 +16,7 @@ use regex::Regex; use serial_test::serial; use std::io::prelude::*; use std::path::Path; -use tremor_common::{file, ids::OperatorIdGen}; +use tremor_common::{file, uids::OperatorUIdGen}; use tremor_pipeline::query::Query; use tremor_pipeline::ExecutableGraph; use tremor_runtime::errors::*; @@ -26,7 +26,7 @@ use tremor_script::FN_REGISTRY; fn to_executable_graph(query: &str) -> Result { let aggr_reg = tremor_script::aggr_registry(); - let mut idgen = OperatorIdGen::new(); + let mut idgen = OperatorUIdGen::new(); let q = Query::parse(&query, &*FN_REGISTRY.read()?, &aggr_reg)?; Ok(q.to_executable_graph(&mut idgen)?) } diff --git a/tests/query_runtime_error.rs b/tests/query_runtime_error.rs index 21f8871d88..90778574c1 100644 --- a/tests/query_runtime_error.rs +++ b/tests/query_runtime_error.rs @@ -14,7 +14,7 @@ use pretty_assertions::assert_eq; use std::io::prelude::*; use std::path::PathBuf; -use tremor_common::{file, ids::OperatorIdGen, ports::IN}; +use tremor_common::{file, ports::IN, uids::OperatorUIdGen}; use tremor_pipeline::query::Query; use tremor_pipeline::ExecutableGraph; @@ -30,7 +30,7 @@ use tremor_script::utils::*; fn to_pipe(query: &str) -> Result { let aggr_reg = tremor_script::aggr_registry(); - let mut idgen = OperatorIdGen::new(); + let mut idgen = OperatorUIdGen::new(); let q = Query::parse(&query, &*FN_REGISTRY.read()?, &aggr_reg)?; Ok(q.to_executable_graph(&mut idgen)?) } diff --git a/tremor-api/src/api.rs b/tremor-api/src/api.rs index 90a260963b..8b160e5e49 100644 --- a/tremor-api/src/api.rs +++ b/tremor-api/src/api.rs @@ -181,14 +181,14 @@ pub fn serve(host: String, world: &Runtime) -> JoinHandle> { .at("/flows") .get(|r| handle_api_request(r, flow::list_flows)); v1_app - .at("/flows/:id") + .at("/flows/:app_id/:id") .get(|r| handle_api_request(r, flow::get_flow)) .patch(|r| handle_api_request(r, flow::patch_flow_status)); v1_app - .at("/flows/:id/connectors") + .at("/flows/:app_id/:id/connectors") .get(|r| handle_api_request(r, flow::get_flow_connectors)); v1_app - .at("/flows/:id/connectors/:connector") + .at("/flows/:app_id/:id/connectors/:connector") .get(|r| handle_api_request(r, flow::get_flow_connector_status)) .patch(|r| handle_api_request(r, flow::patch_flow_connector_status)); @@ -216,6 +216,7 @@ mod tests { use tokio::net::TcpListener; use tremor_runtime::{ errors::Result as RuntimeResult, + ids::AppId, instance::State as InstanceState, system::{ShutdownMode, WorldConfig}, }; @@ -233,7 +234,8 @@ mod tests { let config = WorldConfig { debug_connectors: true, }; - let (world, world_handle) = Runtime::start(config).await?; + let (world, world_handle) = + Runtime::start(tremor_runtime::raft::NodeId::default(), config).await?; let free_port = { let listener = TcpListener::bind("127.0.0.1:0").await?; @@ -273,8 +275,9 @@ mod tests { _other => None, }) .expect("No deploy in the given troy file"); - world.deploy_flow(&deploy).await?; - world.start_flow("api_test".into()).await?; + let app_id = AppId::default(); + let flow_id = world.deploy_flow(app_id, &deploy).await?; + world.start_flow(flow_id).await?; // check the status endpoint let start = Instant::now(); @@ -329,7 +332,7 @@ mod tests { .body_json::>() .await?; assert_eq!(1, body.len()); - assert_eq!("api_test", body[0].alias.as_str()); + assert_eq!("default/api_test", body[0].alias.to_string().as_str()); assert_eq!(InstanceState::Running, body[0].status); assert_eq!(1, body[0].connectors.len()); assert_eq!(String::from("my_null"), body[0].connectors[0]); @@ -339,19 +342,19 @@ mod tests { assert_eq!(StatusCode::NotFound, res.status()); let body = client - .get("/v1/flows/api_test") + .get("/v1/flows/default/api_test") .await? .body_json::() .await?; - assert_eq!("api_test", body.alias.as_str()); + assert_eq!("default/api_test", body.alias.to_string().as_str()); assert_eq!(InstanceState::Running, body.status); assert_eq!(1, body.connectors.len()); assert_eq!(String::from("my_null"), body.connectors[0]); // patch flow status let body = client - .patch("/v1/flows/api_test") + .patch("/v1/flows/default/api_test") .body_json(&PatchStatus { status: InstanceState::Paused, })? @@ -359,14 +362,14 @@ mod tests { .body_json::() .await?; - assert_eq!("api_test", body.alias.as_str()); + assert_eq!("default/api_test", body.alias.to_string().as_str()); assert_eq!(InstanceState::Paused, body.status); assert_eq!(1, body.connectors.len()); assert_eq!(String::from("my_null"), body.connectors[0]); // invalid patch let mut res = client - .patch("/v1/flows/api_test") + .patch("/v1/flows/default/api_test") .body_json(&PatchStatus { status: InstanceState::Failed, })? @@ -376,7 +379,7 @@ mod tests { // resume let body = client - .patch("/v1/flows/api_test") + .patch("/v1/flows/default/api_test") .body_json(&PatchStatus { status: InstanceState::Running, })? @@ -384,14 +387,14 @@ mod tests { .body_json::() .await?; - assert_eq!("api_test", body.alias.as_str()); + assert_eq!("default/api_test", body.alias.to_string().as_str()); assert_eq!(InstanceState::Running, body.status); assert_eq!(1, body.connectors.len()); assert_eq!(String::from("my_null"), body.connectors[0]); // list flow connectors let body = client - .get("/v1/flows/api_test/connectors") + .get("/v1/flows/default/api_test/connectors") .await? .body_json::() .await? @@ -423,13 +426,13 @@ mod tests { // get flow connector let mut res = client - .get("/v1/flows/api_test/connectors/i_do_not_exist") + .get("/v1/flows/default/api_test/connectors/i_do_not_exist") .await?; assert_eq!(StatusCode::NotFound, res.status()); res.body_bytes().await?; //consume body let body = client - .get("/v1/flows/api_test/connectors/my_null") + .get("/v1/flows/default/api_test/connectors/my_null") .await? .body_json::() .await? @@ -459,7 +462,7 @@ mod tests { // patch flow connector status let body = client - .patch("/v1/flows/api_test/connectors/my_null") + .patch("/v1/flows/default/api_test/connectors/my_null") .body_json(&PatchStatus { status: InstanceState::Paused, })? @@ -493,7 +496,7 @@ mod tests { // invalid patch let mut res = client - .patch("/v1/flows/api_test/connectors/my_null") + .patch("/v1/flows/default/api_test/connectors/my_null") .body_json(&PatchStatus { status: InstanceState::Failed, })? @@ -503,7 +506,7 @@ mod tests { // resume let body = client - .patch("/v1/flows/api_test/connectors/my_null") + .patch("/v1/flows/default/api_test/connectors/my_null") .body_json(&PatchStatus { status: InstanceState::Running, })? diff --git a/tremor-api/src/api/flow.rs b/tremor-api/src/api/flow.rs index e9ad9f0537..90436b3f4e 100644 --- a/tremor-api/src/api/flow.rs +++ b/tremor-api/src/api/flow.rs @@ -18,6 +18,7 @@ use crate::{ api::prelude::*, model::{ApiConnectorStatusReport, ApiFlowStatusReport, PatchStatus}, }; +use tremor_runtime::ids::FlowInstanceId; pub(crate) async fn list_flows(req: Request) -> Result { let world = &req.state().world; @@ -32,7 +33,9 @@ pub(crate) async fn list_flows(req: Request) -> Result { pub(crate) async fn get_flow(req: Request) -> Result { let world = &req.state().world; + let app_id = req.param("app_id")?.to_string(); let flow_id = req.param("id")?.to_string(); + let flow_id = FlowInstanceId::new(app_id, flow_id); let flow = world.get_flow(flow_id).await?; let report = flow.report_status().await?; reply(&req, ApiFlowStatusReport::from(report), StatusCode::Ok) @@ -41,7 +44,9 @@ pub(crate) async fn get_flow(req: Request) -> Result { pub(crate) async fn patch_flow_status(mut req: Request) -> Result { let patch_status_payload: PatchStatus = req.body_json().await?; let world = &req.state().world; + let app_id = req.param("app_id")?.to_string(); let flow_id = req.param("id")?.to_string(); + let flow_id = FlowInstanceId::new(app_id, flow_id); let flow = world.get_flow(flow_id.clone()).await?; let current_status = flow.report_status().await?; let report = match (current_status.status, patch_status_payload.status) { @@ -50,12 +55,12 @@ pub(crate) async fn patch_flow_status(mut req: Request) -> Result { current_status } (InstanceState::Running, InstanceState::Paused) => { - world.pause_flow(flow_id.into()).await?; + world.pause_flow(flow_id).await?; flow.report_status().await? } (InstanceState::Paused, InstanceState::Running) => { - world.resume_flow(flow_id.into()).await?; + world.resume_flow(flow_id).await?; flow.report_status().await? } // TODO: we could stop a flow by patching its status to `Stopped` @@ -71,7 +76,9 @@ pub(crate) async fn patch_flow_status(mut req: Request) -> Result { pub(crate) async fn get_flow_connectors(req: Request) -> Result { let world = &req.state().world; + let app_id = req.param("app_id")?.to_string(); let flow_id = req.param("id")?.to_string(); + let flow_id = FlowInstanceId::new(app_id, flow_id); let flow = world.get_flow(flow_id).await?; let connectors = flow.get_connectors().await?; let mut result: Vec = Vec::with_capacity(connectors.len()); @@ -84,7 +91,9 @@ pub(crate) async fn get_flow_connectors(req: Request) -> Result { pub(crate) async fn get_flow_connector_status(req: Request) -> Result { let world = &req.state().world; + let app_id = req.param("app_id")?.to_string(); let flow_id = req.param("id")?.to_string(); + let flow_id = FlowInstanceId::new(app_id, flow_id); let connector_id = req.param("connector")?.to_string(); let flow = world.get_flow(flow_id).await?; @@ -95,7 +104,9 @@ pub(crate) async fn get_flow_connector_status(req: Request) -> Result pub(crate) async fn patch_flow_connector_status(mut req: Request) -> Result { let patch_status_payload: PatchStatus = req.body_json().await?; + let app_id = req.param("app_id")?.to_string(); let flow_id = req.param("id")?.to_string(); + let flow_id = FlowInstanceId::new(app_id, flow_id); let connector_id = req.param("connector")?.to_string(); let world = &req.state().world; diff --git a/tremor-api/src/api/model.rs b/tremor-api/src/api/model.rs index ddb3a37e7f..f72392bcb1 100644 --- a/tremor-api/src/api/model.rs +++ b/tremor-api/src/api/model.rs @@ -16,14 +16,15 @@ use crate::api::prelude::*; use halfbrown::HashMap; use tremor_runtime::{ connectors::{Connectivity, StatusReport as ConnectorStatusReport}, + ids::FlowInstanceId, instance::State, - system::flow::{Alias as FlowAlias, StatusReport as FlowStatusReport}, + system::flow::StatusReport as FlowStatusReport, }; use tremor_script::ast::DeployEndpoint; #[derive(Clone, Debug, Deserialize, Serialize)] pub(crate) struct ApiFlowStatusReport { - pub(crate) alias: FlowAlias, + pub(crate) alias: FlowInstanceId, pub(crate) status: State, pub(crate) connectors: Vec, } diff --git a/tremor-cli/src/cluster.rs b/tremor-cli/src/cluster.rs index 1fb2653476..4e3319ada8 100644 --- a/tremor-cli/src/cluster.rs +++ b/tremor-cli/src/cluster.rs @@ -16,22 +16,23 @@ use crate::{ cli::{AppsCommands, Cluster, ClusterCommand}, errors::{Error, Result}, }; -use async_std::{io::ReadExt, stream::StreamExt, task}; - -use halfbrown::HashMap; +use futures::StreamExt; use signal_hook::consts::signal::{SIGINT, SIGQUIT, SIGTERM}; use signal_hook::low_level::signal_name; -use signal_hook_async_std::Signals; +use signal_hook_tokio::Signals; use simd_json::OwnedValue; +use std::collections::HashMap; use std::path::Path; +use tokio::{io::AsyncReadExt, task}; use tremor_common::asy::file; use tremor_runtime::{ + ids::{AppId, FlowDefinitionId, FlowInstanceId}, raft::{ api::client::{print_metrics, Tremor as Client}, archive, node::{Addr, ClusterNodeKillSwitch, Node}, remove_node, - store::{AppId, FlowId, InstanceId, TremorInstanceState}, + store::TremorInstanceState, ClusterError, NodeId, }, system::ShutdownMode, @@ -41,7 +42,7 @@ use tremor_runtime::{ /// variable `TREMOR_API_ADDRESS` fn get_api(input: Option) -> Result { input.map_or_else( - || Ok(std::env::var("TREMOR_API_ADDRESS").map_err(|e| format!("{}", e))?), + || Ok(std::env::var("TREMOR_API_ADDRESS").map_err(|e| e.to_string())?), Ok, ) } @@ -144,7 +145,7 @@ impl Cluster { } info!("Tremor stopped."); signal_handle.close(); - signal_handler_task.cancel().await; + signal_handler_task.abort(); } // target/debug/tremor cluster start --db-dir temp/test-db2 --api 127.0.0.1:8002 --rpc 127.0.0.1:9002 --join 127.0.0.1:8001 // target/debug/tremor cluster start --db-dir temp/test-db3 --api 127.0.0.1:8003 --rpc 127.0.0.1:9003 --join 127.0.0.1:8001 @@ -198,7 +199,7 @@ impl Cluster { error!("Error: {e}"); } signal_handle.close(); - signal_handler_task.cancel().await; + signal_handler_task.abort(); } ClusterCommand::Remove { node, api } => { let api_addr = get_api(api)?; @@ -288,65 +289,64 @@ impl AppsCommands { } else { HashMap::default() }; - let flow = flow.map_or_else(|| FlowId("main".to_string()), FlowId); + let flow = + flow.map_or_else(|| FlowDefinitionId("main".to_string()), FlowDefinitionId); let app_id = AppId(app); - let instance_id = InstanceId(instance); + let instance_id = FlowInstanceId::new(app_id, instance); let running = !paused; let res = client - .start(&app_id, &flow, &instance_id, config, running) + .start(&flow, &instance_id, config, running) .await .map_err(|e| format!("error: {e}")); if let Err(e) = res { - eprintln!("Instance `{app_id}/{instance_id}` failed to start: {e:?}"); + eprintln!("Instance `{instance_id}` failed to start: {e:?}"); } else { - println!("Instance `{app_id}/{instance_id}` successfully started",); + println!("Instance `{instance_id}` successfully started",); } } AppsCommands::Stop { app, instance } => { let client = Client::new(api)?; - let app_id = AppId(app); - let instance_id = InstanceId(instance); + let instance_id = FlowInstanceId::new(app, instance); let res = client - .stop_instance(&app_id, &instance_id) + .stop_instance(&instance_id) .await .map_err(|e| format!("error: {e}")); if let Err(e) = res { - eprintln!("Instance `{app_id}/{instance_id}` failed to stop: {e:?}"); + eprintln!("Instance `{instance_id}` failed to stop: {e:?}"); } else { - println!("Instance `{app_id}/{instance_id}` stopped",); + println!("Instance `{instance_id}` stopped",); } } AppsCommands::Pause { app, instance } => { let client = Client::new(api)?; let app_id = AppId(app); - let instance_id = InstanceId(instance); + let instance_id = FlowInstanceId::new(app_id, instance); let res = client - .change_instance_state(&app_id, &instance_id, TremorInstanceState::Pause) + .change_instance_state(&instance_id, TremorInstanceState::Pause) .await .map_err(|e| format!("error: {e}")); if let Err(e) = res { - eprintln!("Instance `{app_id}/{instance_id}` failed to pause: {e:?}"); + eprintln!("Instance `{instance_id}` failed to pause: {e:?}"); } else { - println!("Instance `{app_id}/{instance_id}` successfully paused",); + println!("Instance `{instance_id}` successfully paused",); } } AppsCommands::Resume { app, instance } => { let client = Client::new(api)?; - let app_id = AppId(app); - let instance_id = InstanceId(instance); + let instance_id = FlowInstanceId::new(app, instance); let res = client - .change_instance_state(&app_id, &instance_id, TremorInstanceState::Resume) + .change_instance_state(&instance_id, TremorInstanceState::Resume) .await .map_err(|e| format!("error: {e}")); if let Err(e) = res { - eprintln!("Instance `{app_id}/{instance_id}` failed to resume: {e:?}"); + eprintln!("Instance `{instance_id}` failed to resume: {e:?}"); } else { - println!("Instance `{app_id}/{instance_id}` successfully resumed",); + println!("Instance `{instance_id}` successfully resumed",); } } } diff --git a/tremor-cli/src/debug.rs b/tremor-cli/src/debug.rs index 887c3f2375..7cbf7c4d9a 100644 --- a/tremor-cli/src/debug.rs +++ b/tremor-cli/src/debug.rs @@ -23,7 +23,7 @@ use lexer::Lexer; use std::io::Write; use std::io::{self, Read}; use termcolor::{Color, ColorSpec}; -use tremor_common::ids::OperatorIdGen; +use tremor_common::uids::OperatorUIdGen; use tremor_script::highlighter::{Dumb as TermNoHighlighter, Highlighter, Term as TermHighlighter}; use tremor_script::lexer::{self, Token}; use tremor_script::pos::{Span, Spanned}; @@ -266,7 +266,7 @@ impl DbgDot { let env = env::setup()?; match Query::parse(&data.raw, &env.fun, &env.aggr) { Ok(runnable) => { - let mut idgen = OperatorIdGen::new(); + let mut idgen = OperatorUIdGen::new(); let g = tremor_pipeline::query::Query(runnable).to_executable_graph(&mut idgen)?; diff --git a/tremor-cli/src/doc.rs b/tremor-cli/src/doc.rs index 544818c802..3e545f5ba7 100644 --- a/tremor-cli/src/doc.rs +++ b/tremor-cli/src/doc.rs @@ -15,12 +15,15 @@ use crate::cli::Doc; use crate::errors::{Error, Result}; use crate::util::visit_path_str; -use halfbrown::HashMap; +use std::collections::HashMap; use std::ffi::OsStr; use std::io::Read; use std::path::{Path, PathBuf}; -use tremor_script::module::{Id, Module}; use tremor_script::{arena::Arena, ast::NodeId}; +use tremor_script::{ + module::{Id, Module}, + NodeMeta, +}; fn push_line(line: &str, buf: &mut String) { buf.push_str(line); @@ -168,7 +171,7 @@ fn gen_doc( let mut ids = Vec::new(); let (aid, raw) = Arena::insert(&raw)?; Module::load( - NodeId::new(&"doc", &[]), + NodeId::new("doc".to_string(), Vec::new(), NodeMeta::dummy()), module_id, &mut ids, aid, diff --git a/tremor-cli/src/run.rs b/tremor-cli/src/run.rs index e3bee83def..e0cad3e574 100644 --- a/tremor-cli/src/run.rs +++ b/tremor-cli/src/run.rs @@ -20,11 +20,12 @@ use std::io::prelude::*; use std::io::{self, BufReader, BufWriter, Read, Write}; use tremor_common::{ file, - ids::OperatorIdGen, ports::{Port, IN}, time::nanotime, + uids::OperatorUIdGen, }; use tremor_pipeline::{Event, EventId}; +use tremor_runtime::raft::NodeId; use tremor_runtime::{ codec::Codec, config, @@ -339,7 +340,7 @@ impl Run { let mut egress = Egress::from_args(self)?; let runnable = tremor_pipeline::query::Query(runnable); - let mut idgen = OperatorIdGen::new(); + let mut idgen = OperatorUIdGen::new(); let mut pipeline = runnable.to_executable_graph(&mut idgen)?; let id = 0_u64; @@ -422,7 +423,7 @@ impl Run { let config = WorldConfig { debug_connectors: true, }; - let (world, handle) = Runtime::start(config).await?; + let (world, handle) = Runtime::start(NodeId::default(), config).await?; tremor_runtime::load_troy_file(&world, &self.script).await?; handle.await??; Ok(()) diff --git a/tremor-cli/src/server.rs b/tremor-cli/src/server.rs index 20d6c662a0..653d8254db 100644 --- a/tremor-cli/src/server.rs +++ b/tremor-cli/src/server.rs @@ -25,6 +25,7 @@ use std::io::Write; use std::sync::atomic::Ordering; use tremor_api as api; use tremor_common::file; +use tremor_runtime::raft::NodeId; use tremor_runtime::system::{Runtime, ShutdownMode}; use tremor_runtime::{self, version}; @@ -35,7 +36,7 @@ macro_rules! log_and_print_error { }; } -async fn handle_signals(mut signals: Signals, world: World) { +async fn handle_signals(mut signals: Signals, world: Runtime) { while let Some(signal) = signals.next().await { info!( "Received SIGNAL: {}", @@ -98,7 +99,7 @@ impl ServerRun { debug_connectors: self.debug_connectors, }; - let (world, handle) = Runtime::start(config).await?; + let (world, handle) = Runtime::start(NodeId::default(), config).await?; // signal handling let signals = Signals::new([SIGTERM, SIGINT, SIGQUIT])?; diff --git a/tremor-cli/src/test/unit.rs b/tremor-cli/src/test/unit.rs index 5f47a9b086..2dd39d1a88 100644 --- a/tremor-cli/src/test/unit.rs +++ b/tremor-cli/src/test/unit.rs @@ -29,8 +29,9 @@ use tremor_script::{ highlighter::{Dumb as DumbHighlighter, Highlighter, Term as TermHighlighter}, interpreter::{AggrType, Env, ExecOpts, LocalStack}, prelude::*, - Value, NO_AGGRS, NO_CONSTS, + NO_AGGRS, NO_CONSTS, }; +use tremor_value::Value; const EXEC_OPTS: ExecOpts = ExecOpts { result_needed: true, diff --git a/tremor-cli/tests/integration/system-metrics/config.troy b/tremor-cli/tests/integration/system-metrics/config.troy index fa60bfedee..4845bfcef5 100644 --- a/tremor-cli/tests/integration/system-metrics/config.troy +++ b/tremor-cli/tests/integration/system-metrics/config.troy @@ -9,6 +9,7 @@ flow use integration; use std::time::nanos; use tremor::connectors; + use tremor::pipelines; # connector definitions define connector in from metronome @@ -78,15 +79,15 @@ flow end; # update state counters match event of - case matched = %{ tags ~= %{`pipeline` == "main1::main1", port == "in" }, fields ~= %{ present count } } => + case matched = %{ tags ~= %{`pipeline` == "default/main1::main1", port == "in" }, fields ~= %{ present count } } => let state.pipeline_in = matched.fields.count - case matched = %{ tags ~= %{ `pipeline` == "main1::main1", port == "out"}, fields ~= %{ present count } } => + case matched = %{ tags ~= %{ `pipeline` == "default/main1::main1", port == "out"}, fields ~= %{ present count } } => let state.pipeline_out = matched.fields.count - case matched = %{ tags ~= %{ `connector` == "main1::in_here", port == "out" }, fields ~= %{ present count } } => + case matched = %{ tags ~= %{ `connector` == "default/main1::in_here", port == "out" }, fields ~= %{ present count } } => let state.source_out = matched.fields.count - case matched = %{ tags ~= %{ `connector` == "main1::in_here", port == "err" }, fields ~= %{ present count } } => + case matched = %{ tags ~= %{ `connector` == "default/main1::in_here", port == "err" }, fields ~= %{ present count } } => let state.source_err = matched.fields.count - case matched = %{ tags ~= %{ `connector` == "main1::out_here", port == "in" }, fields ~= %{ present count } } => + case matched = %{ tags ~= %{ `connector` == "default/main1::out_here", port == "in" }, fields ~= %{ present count } } => let state.sink_in = matched.fields.count case _ => null end; diff --git a/tremor-common/src/lib.rs b/tremor-common/src/lib.rs index d8568dece1..bd92c266ef 100644 --- a/tremor-common/src/lib.rs +++ b/tremor-common/src/lib.rs @@ -31,8 +31,8 @@ pub mod file; /// Time related functions pub mod time; -/// common id handling -pub mod ids; +/// common uid handling +pub mod uids; /// Random numbers pub mod rand; diff --git a/tremor-common/src/ids.rs b/tremor-common/src/uids.rs similarity index 74% rename from tremor-common/src/ids.rs rename to tremor-common/src/uids.rs index 75b510b663..4faafc8829 100644 --- a/tremor-common/src/ids.rs +++ b/tremor-common/src/uids.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -// common id handling +//! Collection of common unique numeric identifiers for internal use. +//! Those are more efficient than stupid strings. use std::{marker::PhantomData, ops::Deref}; -/// operator id +/// operator uid #[derive( Debug, PartialEq, @@ -30,9 +31,9 @@ use std::{marker::PhantomData, ops::Deref}; simd_json_derive::Serialize, simd_json_derive::Deserialize, )] -pub struct OperatorId(u64); +pub struct OperatorUId(u64); -impl Id for OperatorId { +impl UId for OperatorUId { fn new(id: u64) -> Self { Self(id) } @@ -42,13 +43,13 @@ impl Id for OperatorId { } } -impl std::fmt::Display for OperatorId { +impl std::fmt::Display for OperatorUId { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "{}", self.0) } } -impl std::str::FromStr for OperatorId { +impl std::str::FromStr for OperatorUId { type Err = std::num::ParseIntError; fn from_str(s: &str) -> Result { @@ -57,7 +58,7 @@ impl std::str::FromStr for OperatorId { } } -/// connector id +/// connector uid #[derive( Debug, PartialEq, @@ -71,8 +72,8 @@ impl std::str::FromStr for OperatorId { simd_json_derive::Serialize, simd_json_derive::Deserialize, )] -pub struct ConnectorId(u64); -impl Id for ConnectorId { +pub struct ConnectorUId(u64); +impl UId for ConnectorUId { fn new(id: u64) -> Self { Self(id) } @@ -82,13 +83,13 @@ impl Id for ConnectorId { } } -impl std::fmt::Display for ConnectorId { +impl std::fmt::Display for ConnectorUId { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "{}", self.0) } } -impl Deref for ConnectorId { +impl Deref for ConnectorUId { type Target = u64; fn deref(&self) -> &Self::Target { @@ -96,7 +97,7 @@ impl Deref for ConnectorId { } } -impl AsRef for ConnectorId { +impl AsRef for ConnectorUId { fn as_ref(&self) -> &u64 { &self.0 } @@ -116,16 +117,16 @@ impl AsRef for ConnectorId { simd_json_derive::Serialize, simd_json_derive::Deserialize, )] -pub struct SinkId(ConnectorId); -impl From for SinkId { - fn from(cid: ConnectorId) -> Self { +pub struct SinkUId(ConnectorUId); +impl From for SinkUId { + fn from(cid: ConnectorUId) -> Self { Self(cid) } } -impl Id for SinkId { +impl UId for SinkUId { fn new(id: u64) -> Self { - Self(ConnectorId::new(id)) + Self(ConnectorUId::new(id)) } fn id(&self) -> u64 { @@ -133,7 +134,7 @@ impl Id for SinkId { } } -impl std::fmt::Display for SinkId { +impl std::fmt::Display for SinkUId { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "Sink({})", self.0) } @@ -153,11 +154,11 @@ impl std::fmt::Display for SinkId { simd_json_derive::Serialize, simd_json_derive::Deserialize, )] -pub struct SourceId(ConnectorId); +pub struct SourceUId(ConnectorUId); -impl Id for SourceId { +impl UId for SourceUId { fn new(id: u64) -> Self { - Self(ConnectorId::new(id)) + Self(ConnectorUId::new(id)) } fn id(&self) -> u64 { @@ -165,19 +166,19 @@ impl Id for SourceId { } } -impl From for SourceId { - fn from(cid: ConnectorId) -> Self { +impl From for SourceUId { + fn from(cid: ConnectorUId) -> Self { Self(cid) } } -impl std::fmt::Display for SourceId { +impl std::fmt::Display for SourceUId { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "Source({})", self.0) } } -/// Identifier trait used everywhere within tremor -pub trait Id { +/// Unique numeric Identifier trait used everywhere within tremor +pub trait UId { /// constructor from a unique integer fn new(id: u64) -> Self; @@ -187,12 +188,12 @@ pub trait Id { #[derive(Debug)] /// id generator -pub struct IdGen { +pub struct UIdGen { current: u64, _marker: PhantomData, } -impl IdGen { +impl UIdGen { #[must_use] /// constructor pub fn new() -> Self { @@ -208,7 +209,7 @@ impl IdGen { } } -impl Default for IdGen { +impl Default for UIdGen { fn default() -> Self { Self::new() } @@ -216,9 +217,9 @@ impl Default for IdGen { /// operator id generator - generates consecutive u64 values /// this one will be shared by all pipelines - so we ensure unique operator ids -pub type OperatorIdGen = IdGen; +pub type OperatorUIdGen = UIdGen; /// connector id generator - generates consecutive u64 values -pub type ConnectorIdGen = IdGen; +pub type ConnectorUIdGen = UIdGen; #[cfg(test)] mod tests { @@ -227,8 +228,8 @@ mod tests { #[test] fn id_gen() { - let mut idgen = IdGen::::default(); - let ids: Vec = std::iter::repeat_with(|| idgen.next_id()) + let mut idgen = UIdGen::::default(); + let ids: Vec = std::iter::repeat_with(|| idgen.next_id()) .take(100) .collect(); @@ -244,8 +245,8 @@ mod tests { #[test] fn id_can_be_created_from_string() { - let id = OperatorId::from_str("1024"); + let id = OperatorUId::from_str("1024"); - assert_eq!(Ok(OperatorId(1024)), id); + assert_eq!(Ok(OperatorUId(1024)), id); } } diff --git a/tremor-erl/src/tremor_binding.erl b/tremor-erl/src/tremor_binding.erl deleted file mode 100644 index bdf4e3da57..0000000000 --- a/tremor-erl/src/tremor_binding.erl +++ /dev/null @@ -1,77 +0,0 @@ --module(tremor_binding). - --export([ - example/0, - list/1, - find/2, - publish/2, - unpublish/2 - ]). - --xref_ignore([ - example/0, - list/3, - find/2, - publish/2, - unpublish/2 - ]). - --define(ENDPOINT, "binding"). - - - -%%-------------------------------------------------------------------- -%% @doc -%% @end -%%-------------------------------------------------------------------- --spec list(tremor_api:connection()) -> - {ok, JSON :: binary()}. - -list(C) -> - tremor_http:get(?ENDPOINT, C). - - -%%-------------------------------------------------------------------- -%% @doc -%% @end -%%-------------------------------------------------------------------- --spec find(ID :: binary(), tremor_api:connection()) -> - {ok, JSON :: binary()}. - -find(UUID, C) -> - tremor_http:get([?ENDPOINT, $/, UUID], C). - - - -%%-------------------------------------------------------------------- -%% @doc -%% @end -%%-------------------------------------------------------------------- --spec publish(Spec :: map(), tremor_api:connection()) -> - {error, duplicate} | - {error, bad_content} | - ok. - -publish(Spec, C) -> - tremor_http:post([?ENDPOINT], Spec, C). - -%%-------------------------------------------------------------------- -%% @doc -%% @end -%%-------------------------------------------------------------------- --spec unpublish(ID :: map(), tremor_api:connection()) -> - {error, duplicate} | - {error, bad_content} | - ok. - -unpublish(Id, C) -> - tremor_http:delete([?ENDPOINT, $/, Id], [], C). - -example() -> - #{ - id => <<"test">>, - type => <<"file">>, - config => #{ - source => <<"file.txt">> - } - }. diff --git a/tremor-erl/src/tremor_flow.erl b/tremor-erl/src/tremor_flow.erl index 77e160a1e8..83783ebc05 100644 --- a/tremor-erl/src/tremor_flow.erl +++ b/tremor-erl/src/tremor_flow.erl @@ -16,6 +16,7 @@ -define(API_VERSION, "v1"). -define(ENDPOINT, "flows"). +-define(APP, "default"). -export([ list/1, get/2, @@ -43,7 +44,7 @@ list(Conn) -> %%------------------------------------------------------------------------ -spec get(Alias :: binary(), tremor_api:connection()) -> {ok, JSON::binary()}. get(Alias, Conn) -> - tremor_http:get([?API_VERSION, $/, ?ENDPOINT, $/, Alias], Conn). + tremor_http:get([?API_VERSION, $/, ?ENDPOINT, $/, ?APP, $/, Alias], Conn). %%------------------------------------------------------------------------ @@ -53,7 +54,7 @@ get(Alias, Conn) -> %%------------------------------------------------------------------------ -spec pause(Alias :: binary(), tremor_api:connection()) -> {ok, JSON::binary()}. pause(Alias, Conn) -> - tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, Alias], #{ status => <<"paused">> }, Conn). + tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, ?APP, $/, Alias], #{ status => <<"paused">> }, Conn). %%------------------------------------------------------------------------ %% @doc @@ -62,7 +63,7 @@ pause(Alias, Conn) -> %%------------------------------------------------------------------------ -spec resume(Alias :: binary(), tremor_api:connection()) -> {ok, JSON::binary()}. resume(Alias, Conn) -> - tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, Alias], #{ status => <<"running">> }, Conn). + tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, ?APP, $/, Alias], #{ status => <<"running">> }, Conn). %%------------------------------------------------------------------------ @@ -72,7 +73,7 @@ resume(Alias, Conn) -> %%------------------------------------------------------------------------ -spec list_connectors(FlowAlias :: binary(), tremor_api:connection()) -> {ok, JSON::binary()}. list_connectors(FlowAlias, Conn) -> - tremor_http:get([?API_VERSION, $/, ?ENDPOINT, $/, FlowAlias , $/, <<"connectors">>], Conn). + tremor_http:get([?API_VERSION, $/, ?ENDPOINT, $/, ?APP, $/, FlowAlias , $/, <<"connectors">>], Conn). %%------------------------------------------------------------------------ %% @doc @@ -82,7 +83,7 @@ list_connectors(FlowAlias, Conn) -> %%------------------------------------------------------------------------ -spec get_connector(FlowAlias :: binary(), ConnectorAlias :: binary(), tremor_api:connection()) -> {ok, JSON::binary()}. get_connector(FlowAlias, ConnectorAlias, Conn) -> - tremor_http:get([?API_VERSION, $/, ?ENDPOINT, $/, FlowAlias, $/, <<"connectors">>, $/, ConnectorAlias], Conn). + tremor_http:get([?API_VERSION, $/, ?ENDPOINT, $/, ?APP, $/, FlowAlias, $/, <<"connectors">>, $/, ConnectorAlias], Conn). %%------------------------------------------------------------------------ %% @doc @@ -92,7 +93,7 @@ get_connector(FlowAlias, ConnectorAlias, Conn) -> %%------------------------------------------------------------------------ -spec pause_connector(FlowAlias :: binary(), ConnectorAlias :: binary(), tremor_api:connection()) -> {ok, JSON::binary()}. pause_connector(FlowAlias, ConnectorAlias, Conn) -> - tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, FlowAlias, $/, <<"connectors">>, $/, ConnectorAlias], #{status => <<"paused">>}, Conn). + tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, ?APP, $/, FlowAlias, $/, <<"connectors">>, $/, ConnectorAlias], #{status => <<"paused">>}, Conn). %%------------------------------------------------------------------------ %% @doc @@ -102,4 +103,4 @@ pause_connector(FlowAlias, ConnectorAlias, Conn) -> %%------------------------------------------------------------------------ -spec resume_connector(FlowAlias :: binary(), ConnectorAlias :: binary(), tremor_api:connection()) -> {ok, JSON::binary()}. resume_connector(FlowAlias, ConnectorAlias, Conn) -> - tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, FlowAlias, $/, <<"connectors">>, $/, ConnectorAlias], #{status => <<"running">>}, Conn). + tremor_http:patch([?API_VERSION, $/, ?ENDPOINT, $/, ?APP, $/, FlowAlias, $/, <<"connectors">>, $/, ConnectorAlias], #{status => <<"running">>}, Conn). diff --git a/tremor-pipeline/src/event.rs b/tremor-pipeline/src/event.rs index 2fa22270fe..69685b1206 100644 --- a/tremor-pipeline/src/event.rs +++ b/tremor-pipeline/src/event.rs @@ -14,8 +14,8 @@ use crate::{CbAction, EventId, OpMeta, SignalKind}; use std::mem::swap; -use tremor_common::ids::SourceId; use tremor_common::time::nanotime; +use tremor_common::uids::SourceUId; use tremor_script::prelude::*; use tremor_script::{EventOriginUri, EventPayload}; @@ -58,7 +58,7 @@ impl Event { /// create a drain signal event originating at the connector with the given `source_id` #[must_use] - pub fn signal_drain(source_id: SourceId) -> Self { + pub fn signal_drain(source_id: SourceUId) -> Self { Self { ingest_ns: nanotime(), kind: Some(SignalKind::Drain(source_id)), @@ -68,7 +68,7 @@ impl Event { /// create start signal for the given `SourceId` #[must_use] - pub fn signal_start(uid: SourceId) -> Self { + pub fn signal_start(uid: SourceUId) -> Self { Self { ingest_ns: nanotime(), kind: Some(SignalKind::Start(uid)), @@ -464,7 +464,7 @@ mod test { use super::*; use crate::Result; use simd_json::OwnedValue; - use tremor_common::ids::{Id, OperatorId}; + use tremor_common::uids::{OperatorUId, UId}; use tremor_script::ValueAndMeta; use tremor_value::Object; @@ -609,7 +609,7 @@ mod test { assert_eq!(e.insight_fail().cb, CbAction::Fail); let mut clone = e.clone(); - let op_id = OperatorId::new(1); + let op_id = OperatorUId::new(1); clone.op_meta.insert(op_id, OwnedValue::null()); let ack_with_timing = clone.insight_ack_with_timing(100); assert_eq!(ack_with_timing.cb, CbAction::Ack); @@ -617,7 +617,7 @@ mod test { let (_, m) = ack_with_timing.data.parts(); assert_eq!(Some(100), m.get_u64("time")); - e.op_meta.insert(OperatorId::new(42), OwnedValue::null()); + e.op_meta.insert(OperatorUId::new(42), OwnedValue::null()); } #[test] diff --git a/tremor-pipeline/src/executable_graph.rs b/tremor-pipeline/src/executable_graph.rs index 3dcdc87ece..a41a829825 100644 --- a/tremor-pipeline/src/executable_graph.rs +++ b/tremor-pipeline/src/executable_graph.rs @@ -23,7 +23,7 @@ use crate::{op::EventAndInsights, Event, NodeKind, Operator}; use beef::Cow; use halfbrown::HashMap; use std::{fmt, fmt::Display}; -use tremor_common::{ids::OperatorId, ports::Port, stry}; +use tremor_common::{ports::Port, stry, uids::OperatorUId}; use tremor_script::{ast::Helper, ast::Stmt}; use tremor_value::Value; @@ -81,7 +81,7 @@ impl PartialEq for NodeConfig { impl NodeConfig { pub(crate) fn to_op( &self, - uid: OperatorId, + uid: OperatorUId, resolver: NodeLookupFn, helper: &mut Helper<'static, '_>, ) -> Result { @@ -114,7 +114,7 @@ pub struct OperatorNode { /// The executable operator pub op: Box, /// Tremor unique identifyer - pub uid: OperatorId, + pub uid: OperatorUId, /// The original config pub config: NodeConfig, } @@ -125,7 +125,7 @@ impl Operator for OperatorNode { } fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, port: &Port<'static>, state: &mut Value<'static>, event: Event, @@ -138,7 +138,7 @@ impl Operator for OperatorNode { } fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, state: &mut Value<'static>, signal: &mut Event, ) -> Result { @@ -148,7 +148,7 @@ impl Operator for OperatorNode { fn handles_contraflow(&self) -> bool { self.op.handles_contraflow() } - fn on_contraflow(&mut self, _uid: OperatorId, contraevent: &mut Event) { + fn on_contraflow(&mut self, _uid: OperatorUId, contraevent: &mut Event) { self.op.on_contraflow(self.uid, contraevent); } @@ -621,9 +621,9 @@ mod test { }, Result, METRICS_CHANNEL, }; - use tremor_common::ids::Id; + use tremor_common::uids::UId; use tremor_script::prelude::*; - fn pass(uid: OperatorId, id: &'static str) -> Result { + fn pass(uid: OperatorUId, id: &'static str) -> Result { let config = NodeConfig::from_config(&"passthrough", None); Ok(OperatorNode { id: id.into(), @@ -636,7 +636,7 @@ mod test { } #[test] fn operator_node() -> Result<()> { - let op_id = OperatorId::new(0); + let op_id = OperatorUId::new(0); let mut n = pass(op_id, "passthrough")?; assert!(!n.handles_contraflow()); assert!(!n.handles_signal()); @@ -687,7 +687,7 @@ mod test { impl Operator for AllOperator { fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, event: Event, @@ -700,7 +700,7 @@ mod test { fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _state: &mut Value<'static>, _signal: &mut Event, ) -> Result { @@ -712,7 +712,7 @@ mod test { true } - fn on_contraflow(&mut self, _uid: OperatorId, _insight: &mut Event) {} + fn on_contraflow(&mut self, _uid: OperatorUId, _insight: &mut Event) {} fn metrics( &self, @@ -733,7 +733,7 @@ mod test { kind: NodeKind::Operator, op_type: "test".into(), op: Box::new(AllOperator {}), - uid: OperatorId::default(), + uid: OperatorUId::default(), config: NodeConfig::default(), } } @@ -786,9 +786,9 @@ mod test { #[tokio::test(flavor = "multi_thread")] async fn eg_metrics() -> Result<()> { - let mut in_n = pass(OperatorId::new(1), "in")?; + let mut in_n = pass(OperatorUId::new(1), "in")?; in_n.kind = NodeKind::Input; - let mut out_n = pass(OperatorId::new(2), "out")?; + let mut out_n = pass(OperatorUId::new(2), "out")?; out_n.kind = NodeKind::Output(OUT); // The graph is in -> 1 -> 2 -> out, we pre-stack the edges since we do not @@ -881,18 +881,18 @@ mod test { #[tokio::test(flavor = "multi_thread")] async fn eg_optimize() -> Result<()> { - let mut in_n = pass(OperatorId::new(0), "in")?; + let mut in_n = pass(OperatorUId::new(0), "in")?; in_n.kind = NodeKind::Input; - let mut out_n = pass(OperatorId::new(1), "out")?; + let mut out_n = pass(OperatorUId::new(1), "out")?; out_n.kind = NodeKind::Output(OUT); // The graph is in -> 1 -> 2 -> out, we pre-stack the edges since we do not // need to have order in here. let graph = vec![ - in_n, // 0 - all_op("all-1"), // 1 - pass(OperatorId::new(2), "nop")?, // 2 - all_op("all-2"), // 3 - out_n, // 4 + in_n, // 0 + all_op("all-1"), // 1 + pass(OperatorUId::new(2), "nop")?, // 2 + all_op("all-2"), // 3 + out_n, // 4 ]; let mut inputs = HashMap::new(); diff --git a/tremor-pipeline/src/lib.rs b/tremor-pipeline/src/lib.rs index 94a9f1b19b..9b4ad72495 100644 --- a/tremor-pipeline/src/lib.rs +++ b/tremor-pipeline/src/lib.rs @@ -46,8 +46,8 @@ use std::str::FromStr; use std::{borrow::Borrow, cmp::Ordering}; use tokio::sync::broadcast::{self, Receiver, Sender}; use tremor_common::{ - ids::{Id, OperatorId, SinkId, SourceId}, ports::Port, + uids::{OperatorUId, SinkUId, SourceUId, UId}, }; use tremor_script::{ ast::{self, Helper}, @@ -82,7 +82,7 @@ pub type ConfigMap = Option>; /// A lookup function to used to look up operators pub type NodeLookupFn = fn( config: &NodeConfig, - uid: OperatorId, + uid: OperatorUId, node: Option<&ast::Stmt<'static>>, helper: &mut Helper<'static, '_>, ) -> Result; @@ -197,23 +197,23 @@ where // TODO: optimization: - use two Vecs, one for operator ids, one for operator metadata (Values) // - make it possible to trace operators with and without metadata // - insert with bisect (numbers of operators tracked will be low single digit numbers most of the time) -pub struct OpMeta(BTreeMap, OwnedValue>); +pub struct OpMeta(BTreeMap, OwnedValue>); impl OpMeta { /// inserts a value - pub fn insert(&mut self, key: OperatorId, value: V) -> Option + pub fn insert(&mut self, key: OperatorUId, value: V) -> Option where OwnedValue: From, { self.0.insert(PrimStr(key), OwnedValue::from(value)) } /// reads a value - pub fn get(&mut self, key: OperatorId) -> Option<&OwnedValue> { + pub fn get(&mut self, key: OperatorUId) -> Option<&OwnedValue> { self.0.get(&PrimStr(key)) } /// checks existance of a key #[must_use] - pub fn contains_key(&self, key: OperatorId) -> bool { + pub fn contains_key(&self, key: OperatorUId) -> bool { self.0.contains_key(&PrimStr(key)) } @@ -288,9 +288,9 @@ pub enum CbAction { Fail, /// Notify all upstream sources that this sink has started, notifying them of its existence. /// Will be used for tracking for which sinks to wait during Drain. - SinkStart(SinkId), + SinkStart(SinkUId), /// answer to a `SignalKind::Drain(uid)` signal from a connector with the same uid - Drained(SourceId, SinkId), + Drained(SourceUId, SinkUId), } impl Default for CbAction { fn default() -> Self { @@ -817,30 +817,30 @@ impl EventIdGenerator { #[must_use] /// create a new generator for the `Source` identified by `source_id` using the default stream id - pub fn new(source_id: SourceId) -> Self { + pub fn new(source_id: SourceUId) -> Self { Self(source_id.id(), DEFAULT_STREAM_ID, 0) } #[must_use] /// create a new generator for the `Operator` identified by `operator_id` using the default stream id - pub fn for_operator(operator_id: OperatorId) -> Self { + pub fn for_operator(operator_id: OperatorUId) -> Self { Self(operator_id.id(), DEFAULT_STREAM_ID, 0) } #[must_use] /// create a new generator for the `Operator` identified by `operator_id` with `stream_id` - pub fn for_operator_with_stream(operator_id: OperatorId, stream_id: u64) -> Self { + pub fn for_operator_with_stream(operator_id: OperatorUId, stream_id: u64) -> Self { Self(operator_id.id(), stream_id, 0) } #[must_use] /// create a new generator using the given source and stream id - pub fn new_with_stream(source_id: SourceId, stream_id: u64) -> Self { + pub fn new_with_stream(source_id: SourceUId, stream_id: u64) -> Self { Self(source_id.id(), stream_id, 0) } /// set the source id - pub fn set_source(&mut self, source_id: SourceId) { + pub fn set_source(&mut self, source_id: SourceUId) { self.0 = source_id.id(); } @@ -862,7 +862,7 @@ impl EventIdGenerator { pub enum SignalKind { // Lifecycle /// Start signal, containing the source uid which just started - Start(SourceId), + Start(SourceUId), /// Shutdown Signal Shutdown, // Pause, TODO debug trace @@ -876,7 +876,7 @@ pub enum SignalKind { /// This signal must be answered with a Drain contraflow event containing the same uid (u64) /// this way a contraflow event will not be interpreted by connectors for which it isn't meant /// reception of such Drain contraflow event notifies the signal sender that the intermittent pipeline is drained and can be safely disconnected - Drain(SourceId), + Drain(SourceUId), } // We ignore this since it's a simple lookup table @@ -915,7 +915,7 @@ fn factory(node: &NodeConfig) -> Result> { Ok(factory) } -fn operator(uid: OperatorId, node: &NodeConfig) -> Result> { +fn operator(uid: OperatorUId, node: &NodeConfig) -> Result> { factory(node)?.node_to_operator(uid, node) } @@ -959,9 +959,9 @@ mod test { #[test] fn op_meta_merge() { - let op_id1 = OperatorId::new(1); - let op_id2 = OperatorId::new(2); - let op_id3 = OperatorId::new(3); + let op_id1 = OperatorUId::new(1); + let op_id2 = OperatorUId::new(2); + let op_id3 = OperatorUId::new(3); let mut m1 = OpMeta::default(); let mut m2 = OpMeta::default(); m1.insert(op_id1, 1); diff --git a/tremor-pipeline/src/macros.rs b/tremor-pipeline/src/macros.rs index f688040592..91cae21965 100644 --- a/tremor-pipeline/src/macros.rs +++ b/tremor-pipeline/src/macros.rs @@ -26,7 +26,7 @@ macro_rules! op { impl $crate::op::InitializableOperator for $factory { fn node_to_operator( &self, - $uid: tremor_common::ids::OperatorId, + $uid: tremor_common::uids::OperatorUId, $node: &$crate::NodeConfig, ) -> $crate::errors::Result> { $constructor diff --git a/tremor-pipeline/src/op.rs b/tremor-pipeline/src/op.rs index 08b5e86c3b..96beb3830e 100644 --- a/tremor-pipeline/src/op.rs +++ b/tremor-pipeline/src/op.rs @@ -28,7 +28,7 @@ use crate::errors::Result; use beef::Cow; use halfbrown::HashMap; use regex::Regex; -use tremor_common::{ids::OperatorId, ports::Port}; +use tremor_common::{ports::Port, uids::OperatorUId}; use tremor_value::Value; lazy_static::lazy_static! { @@ -81,7 +81,7 @@ pub trait Operator: std::fmt::Debug + Send + Sync { /// if the event can not be processed fn on_event( &mut self, - uid: OperatorId, + uid: OperatorUId, port: &Port<'static>, state: &mut Value<'static>, event: Event, @@ -100,7 +100,7 @@ pub trait Operator: std::fmt::Debug + Send + Sync { /// if the singal can not be processed fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _state: &mut Value<'static>, _signal: &mut Event, ) -> Result { @@ -119,7 +119,7 @@ pub trait Operator: std::fmt::Debug + Send + Sync { /// /// # Errors /// if the insight can not be processed - fn on_contraflow(&mut self, _uid: OperatorId, _insight: &mut Event) { + fn on_contraflow(&mut self, _uid: OperatorUId, _insight: &mut Event) { // Make the trait signature nicer } @@ -154,7 +154,7 @@ pub trait InitializableOperator { /// /// # Errors //// if no operator con be instanciated from the provided NodeConfig - fn node_to_operator(&self, uid: OperatorId, node: &NodeConfig) -> Result>; + fn node_to_operator(&self, uid: OperatorUId, node: &NodeConfig) -> Result>; } /// Trait for detecting errors in config and the key names are included in errors diff --git a/tremor-pipeline/src/op/debug/history.rs b/tremor-pipeline/src/op/debug/history.rs index 8c0b3ad3ba..edd762b207 100644 --- a/tremor-pipeline/src/op/debug/history.rs +++ b/tremor-pipeline/src/op/debug/history.rs @@ -77,7 +77,7 @@ struct History { impl Operator for History { fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, mut event: Event, @@ -119,7 +119,7 @@ impl Operator for History { } fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _state: &mut Value<'static>, signal: &mut Event, ) -> Result { @@ -157,7 +157,7 @@ impl Operator for History { #[cfg(test)] mod test { - use tremor_common::ids::Id; + use tremor_common::uids::UId; use super::*; use crate::EventId; @@ -170,7 +170,7 @@ mod test { name: "snot".to_string(), }, }; - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let event = Event { id: EventId::from_id(0, 0, 1), ingest_ns: 1, diff --git a/tremor-pipeline/src/op/generic/batch.rs b/tremor-pipeline/src/op/generic/batch.rs index 89069888b8..30862eb54c 100644 --- a/tremor-pipeline/src/op/generic/batch.rs +++ b/tremor-pipeline/src/op/generic/batch.rs @@ -99,7 +99,7 @@ impl Operator for Batch { /// with a new event id tracking all events within that batch fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, event: Event, @@ -170,7 +170,7 @@ impl Operator for Batch { fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _state: &mut Value<'static>, signal: &mut Event, ) -> Result { @@ -211,12 +211,12 @@ impl Operator for Batch { mod test { use super::*; use simd_json_derive::Serialize; - use tremor_common::ids::Id; + use tremor_common::uids::UId; use tremor_value::Value; #[test] fn size() { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut idgen = EventIdGenerator::for_operator(operator_id); let mut op = Batch { config: Config { @@ -279,7 +279,7 @@ mod test { #[test] fn time() -> Result<()> { - let operator_id = OperatorId::new(42); + let operator_id = OperatorUId::new(42); let node_config = NodeConfig::from_config( &"badger", Some(literal!({ @@ -356,7 +356,7 @@ mod test { #[test] fn signal() { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut idgen = EventIdGenerator::for_operator(operator_id); let mut op = Batch { config: Config { @@ -432,7 +432,7 @@ mod test { #[test] fn forbid_empty_batches() -> Result<()> { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut idgen = EventIdGenerator::for_operator(operator_id); let mut op = Batch { config: Config { diff --git a/tremor-pipeline/src/op/generic/counter.rs b/tremor-pipeline/src/op/generic/counter.rs index 99e2dee609..b9b1c98151 100644 --- a/tremor-pipeline/src/op/generic/counter.rs +++ b/tremor-pipeline/src/op/generic/counter.rs @@ -42,7 +42,7 @@ op!(CounterFactory(_uid, _node) { impl Operator for Counter { fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, state: &mut Value<'static>, mut event: Event, diff --git a/tremor-pipeline/src/op/grouper/bucket.rs b/tremor-pipeline/src/op/grouper/bucket.rs index c7de52ee96..1aa3ef7a58 100644 --- a/tremor-pipeline/src/op/grouper/bucket.rs +++ b/tremor-pipeline/src/op/grouper/bucket.rs @@ -157,7 +157,7 @@ impl std::fmt::Debug for Grouper { impl Operator for Grouper { fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, event: Event, @@ -236,12 +236,12 @@ impl Operator for Grouper { #[cfg(test)] mod test { use super::*; - use tremor_common::ids::Id; + use tremor_common::uids::UId; use tremor_value::Value; #[test] fn bucket() -> Result<()> { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut op = Grouper { buckets: HashMap::new(), }; diff --git a/tremor-pipeline/src/op/identity/passthrough.rs b/tremor-pipeline/src/op/identity/passthrough.rs index 88e34dcca1..26d32fe70b 100644 --- a/tremor-pipeline/src/op/identity/passthrough.rs +++ b/tremor-pipeline/src/op/identity/passthrough.rs @@ -24,7 +24,7 @@ op!(PassthroughFactory (_uid, _node) { impl Operator for Passthrough { fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, event: Event, diff --git a/tremor-pipeline/src/op/qos/backpressure.rs b/tremor-pipeline/src/op/qos/backpressure.rs index d4cb167533..29c0817ef4 100644 --- a/tremor-pipeline/src/op/qos/backpressure.rs +++ b/tremor-pipeline/src/op/qos/backpressure.rs @@ -152,7 +152,7 @@ op!(BackpressureFactory(_uid, node) { impl Operator for Backpressure { fn on_event( &mut self, - uid: OperatorId, + uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, mut event: Event, @@ -183,7 +183,7 @@ impl Operator for Backpressure { fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _state: &mut Value<'static>, signal: &mut Event, ) -> Result { @@ -203,7 +203,7 @@ impl Operator for Backpressure { }) } - fn on_contraflow(&mut self, uid: OperatorId, insight: &mut Event) { + fn on_contraflow(&mut self, uid: OperatorUId, insight: &mut Event) { // If the related event never touched this operator we don't take // action if !insight.op_meta.contains_key(uid) { @@ -237,12 +237,12 @@ impl Operator for Backpressure { mod test { use super::*; use crate::SignalKind; - use tremor_common::ids::Id; + use tremor_common::uids::UId; use tremor_value::Object; #[test] fn pass_wo_error() { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut op: Backpressure = Config { timeout: 100_000_000, steps: vec![1, 10, 100], @@ -287,7 +287,7 @@ mod test { #[test] fn halt_on_error() { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut op: Backpressure = Config { timeout: 100_000_000, steps: vec![1, 10, 100], @@ -384,7 +384,7 @@ mod test { #[test] fn halt_on_error_cb() -> Result<()> { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut op: Backpressure = Config { timeout: 100_000_000, steps: vec![1, 10, 100], @@ -493,7 +493,7 @@ mod test { #[test] fn walk_backoff() { - let operator_id = OperatorId::new(0); + let operator_id = OperatorUId::new(0); let mut op: Backpressure = Config { timeout: 100_000_000, steps: vec![1, 10, 100], diff --git a/tremor-pipeline/src/op/qos/percentile.rs b/tremor-pipeline/src/op/qos/percentile.rs index 6c29620031..8cf6fe2a3b 100644 --- a/tremor-pipeline/src/op/qos/percentile.rs +++ b/tremor-pipeline/src/op/qos/percentile.rs @@ -100,7 +100,7 @@ op!(PercentileFactory(_uid, node) { impl Operator for Percentile { fn on_event( &mut self, - uid: OperatorId, + uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, mut event: Event, @@ -127,7 +127,7 @@ impl Operator for Percentile { true } - fn on_contraflow(&mut self, uid: OperatorId, insight: &mut Event) { + fn on_contraflow(&mut self, uid: OperatorUId, insight: &mut Event) { // If the related event never touched this operator we don't take // action if !insight.op_meta.contains_key(uid) { @@ -151,12 +151,12 @@ impl Operator for Percentile { mod test { #![allow(clippy::float_cmp)] use super::*; - use tremor_common::ids::Id; + use tremor_common::uids::UId; use tremor_value::Object; #[test] fn pass_wo_error() { - let uid = OperatorId::new(0); + let uid = OperatorUId::new(0); let mut op: Percentile = Config { timeout: 100_000_000, step_up: default_step_up(), @@ -207,7 +207,7 @@ mod test { step_up: default_step_up(), } .into(); - let uid = OperatorId::new(42); + let uid = OperatorUId::new(42); let mut state = Value::null(); @@ -270,7 +270,7 @@ mod test { step_up: 0.1, } .into(); - let uid = OperatorId::new(123); + let uid = OperatorUId::new(123); // An contraflow that fails the timeout let mut m = Object::new(); m.insert("time".into(), 200_000_000.into()); diff --git a/tremor-pipeline/src/op/qos/roundrobin.rs b/tremor-pipeline/src/op/qos/roundrobin.rs index ff690d97e1..3e1126fcd1 100644 --- a/tremor-pipeline/src/op/qos/roundrobin.rs +++ b/tremor-pipeline/src/op/qos/roundrobin.rs @@ -96,7 +96,7 @@ if let Some(map) = &node.config { impl Operator for RoundRobin { fn on_event( &mut self, - uid: OperatorId, + uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, mut event: Event, @@ -126,7 +126,7 @@ impl Operator for RoundRobin { } fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _state: &mut Value<'static>, signal: &mut Event, ) -> Result { @@ -147,7 +147,7 @@ impl Operator for RoundRobin { true } - fn on_contraflow(&mut self, uid: OperatorId, insight: &mut Event) { + fn on_contraflow(&mut self, uid: OperatorUId, insight: &mut Event) { let RoundRobin { ref mut outputs, .. } = *self; @@ -182,13 +182,13 @@ impl Operator for RoundRobin { #[cfg(test)] mod test { - use tremor_common::ids::Id; + use tremor_common::uids::UId; use super::*; #[test] fn multi_output_block() { - let uid = OperatorId::new(0); + let uid = OperatorUId::new(0); let mut op: RoundRobin = Config { outputs: vec!["out".into(), "out2".into()], } diff --git a/tremor-pipeline/src/op/trickle/operator.rs b/tremor-pipeline/src/op/trickle/operator.rs index fb84f9d716..873b366ee7 100644 --- a/tremor-pipeline/src/op/trickle/operator.rs +++ b/tremor-pipeline/src/op/trickle/operator.rs @@ -43,7 +43,7 @@ fn mk_node_config(id: String, op_type: String, config: Value) -> NodeConfig { impl TrickleOperator { pub fn with_stmt( - operator_uid: OperatorId, + operator_uid: OperatorUId, defn: &ast::OperatorDefinition<'static>, helper: &mut Helper, ) -> Result { @@ -65,7 +65,7 @@ impl TrickleOperator { impl Operator for TrickleOperator { fn on_event( &mut self, - uid: OperatorId, + uid: OperatorUId, port: &Port<'static>, state: &mut Value<'static>, event: Event, @@ -78,7 +78,7 @@ impl Operator for TrickleOperator { } fn on_signal( &mut self, - uid: OperatorId, + uid: OperatorUId, state: &mut Value<'static>, signal: &mut Event, ) -> Result { @@ -88,7 +88,7 @@ impl Operator for TrickleOperator { fn handles_contraflow(&self) -> bool { self.op.handles_contraflow() } - fn on_contraflow(&mut self, uid: OperatorId, contraevent: &mut Event) { + fn on_contraflow(&mut self, uid: OperatorUId, contraevent: &mut Event) { self.op.on_contraflow(uid, contraevent); } diff --git a/tremor-pipeline/src/op/trickle/script.rs b/tremor-pipeline/src/op/trickle/script.rs index c64b841670..ba5bcbde00 100644 --- a/tremor-pipeline/src/op/trickle/script.rs +++ b/tremor-pipeline/src/op/trickle/script.rs @@ -39,7 +39,7 @@ impl Operator for Script { } fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, port: &Port<'static>, state: &mut Value<'static>, mut event: Event, diff --git a/tremor-pipeline/src/op/trickle/select.rs b/tremor-pipeline/src/op/trickle/select.rs index ad6ec2bf1c..972817d22d 100644 --- a/tremor-pipeline/src/op/trickle/select.rs +++ b/tremor-pipeline/src/op/trickle/select.rs @@ -48,7 +48,7 @@ pub(crate) struct Select { impl Select { pub fn from_stmt( - operator_uid: OperatorId, + operator_uid: OperatorUId, windows: Vec<(String, window::Impl)>, select: &ast::SelectStmt<'static>, ) -> Self { @@ -172,7 +172,7 @@ impl Operator for Select { // so the state can never be changed. fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, _state: &mut Value<'static>, mut event: Event, @@ -312,7 +312,7 @@ impl Operator for Select { fn on_signal( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _state: &mut Value<'static>, signal: &mut Event, ) -> Result { diff --git a/tremor-pipeline/src/op/trickle/select/test.rs b/tremor-pipeline/src/op/trickle/select/test.rs index d9abf916f6..a5c1902ed5 100644 --- a/tremor-pipeline/src/op/trickle/select/test.rs +++ b/tremor-pipeline/src/op/trickle/select/test.rs @@ -21,7 +21,7 @@ use crate::EventId; use super::*; -use tremor_common::ids::Id; +use tremor_common::uids::UId; use tremor_script::ast::{self, Helper, Ident, Literal}; use tremor_script::{ast::Consts, NodeMeta}; use tremor_script::{ @@ -30,8 +30,8 @@ use tremor_script::{ }; use tremor_value::{literal, Value}; -fn test_uid() -> OperatorId { - OperatorId::new(42) +fn test_uid() -> OperatorUId { + OperatorUId::new(42) } fn test_select_stmt(stmt: tremor_script::ast::Select) -> SelectStmt { @@ -100,7 +100,7 @@ fn test_event_tx(s: u64, transactional: bool, group: u64) -> Event { } } -fn test_select(uid: OperatorId, stmt: &SelectStmt<'static>) -> Select { +fn test_select(uid: OperatorUId, stmt: &SelectStmt<'static>) -> Select { let windows = vec![ ( "w15s".into(), diff --git a/tremor-pipeline/src/op/trickle/simple_select.rs b/tremor-pipeline/src/op/trickle/simple_select.rs index a59f50b949..1e9002c3f2 100644 --- a/tremor-pipeline/src/op/trickle/simple_select.rs +++ b/tremor-pipeline/src/op/trickle/simple_select.rs @@ -51,7 +51,7 @@ impl SimpleSelect { impl Operator for SimpleSelect { fn on_event( &mut self, - _uid: OperatorId, + _uid: OperatorUId, _port: &Port<'static>, state: &mut Value<'static>, event: Event, diff --git a/tremor-pipeline/src/op/trickle/window.rs b/tremor-pipeline/src/op/trickle/window.rs index f30f3c97e7..7c2458d471 100644 --- a/tremor-pipeline/src/op/trickle/window.rs +++ b/tremor-pipeline/src/op/trickle/window.rs @@ -15,7 +15,7 @@ use super::select::execute_select_and_having; use crate::{Event, EventId, EventIdGenerator, OpMeta}; use std::{borrow::Cow as SCow, convert::TryFrom}; -use tremor_common::{ids::OperatorId, ports::Port, stry}; +use tremor_common::{ports::Port, stry, uids::OperatorUId}; use tremor_script::{ self, ast::{AggrSlice, Aggregates, Consts, RunConsts, Script, Select, WindowDefinition}, @@ -69,7 +69,7 @@ impl GroupWindow { /// Crate chain of tilt frames from a iterator of windows pub(crate) fn from_windows<'i, I>( aggrs: &AggrSlice<'static>, - operator_id: OperatorId, + operator_id: OperatorUId, mut iter: I, ) -> Option> where diff --git a/tremor-pipeline/src/query.rs b/tremor-pipeline/src/query.rs index be6ca0a283..718e18631a 100644 --- a/tremor-pipeline/src/query.rs +++ b/tremor-pipeline/src/query.rs @@ -40,8 +40,8 @@ use std::{ collections::{BTreeSet, HashSet}, }; use tremor_common::{ - ids::{OperatorId, OperatorIdGen}, ports::Port, + uids::{OperatorUId, OperatorUIdGen}, }; use tremor_script::ast::optimizer::Optimizer; use tremor_script::{ @@ -158,7 +158,7 @@ impl Query { /// if the trickle script can not be parsed pub fn parse(script: &S, reg: &Registry, aggr_reg: &AggrRegistry) -> Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { Ok(Self(tremor_script::query::Query::parse( script, reg, aggr_reg, @@ -170,7 +170,7 @@ impl Query { /// # Errors /// if the graph can not be turned into a pipeline #[allow(clippy::too_many_lines)] - pub fn to_executable_graph(&self, idgen: &mut OperatorIdGen) -> Result { + pub fn to_executable_graph(&self, idgen: &mut OperatorUIdGen) -> Result { let aggr_reg = tremor_script::aggr_registry(); let reg = tremor_script::FN_REGISTRY.read()?; let mut helper = Helper::new(®, &aggr_reg); @@ -759,7 +759,7 @@ fn into_name(prefix: &str, port: &str) -> String { } fn select( - operator_uid: OperatorId, + operator_uid: OperatorUId, config: &NodeConfig, node: &ast::SelectStmt<'static>, helper: &Helper<'static, '_>, @@ -798,7 +798,7 @@ fn select( } fn operator( - operator_uid: OperatorId, + operator_uid: OperatorUId, node: &ast::OperatorDefinition<'static>, helper: &mut Helper, ) -> Result> { @@ -811,7 +811,7 @@ fn operator( pub(crate) fn supported_operators( config: &NodeConfig, - uid: OperatorId, + uid: OperatorUId, node: Option<&ast::Stmt<'static>>, helper: &mut Helper<'static, '_>, ) -> Result { @@ -843,7 +843,7 @@ pub(crate) fn supported_operators( mod test { use super::*; use crate::Result; - use tremor_common::ids::Id; + use tremor_common::uids::UId; #[test] fn query() -> Result<()> { @@ -867,7 +867,7 @@ mod test { let src = "select event from in/test_in into out/test_out;"; let q = Query::parse(&src, &*tremor_script::FN_REGISTRY.read()?, &aggr_reg)?; - let mut idgen = OperatorIdGen::new(); + let mut idgen = OperatorUIdGen::new(); let first = idgen.next_id(); let g = q.to_executable_graph(&mut idgen)?; assert!(g.inputs.contains_key("in/test_in")); diff --git a/tremor-script-nif/build.rs b/tremor-script-nif/build.rs index f309cdc9ed..0d9364af52 100644 --- a/tremor-script-nif/build.rs +++ b/tremor-script-nif/build.rs @@ -38,7 +38,7 @@ fn main() { if host_triple != target_triple { libpath = libpath.join(&target_triple); } - libpath = libpath.join(&profile).join(file_name); + libpath = libpath.join(profile).join(file_name); // Create file in `here` and write the path to the directory of // where to find libtremor diff --git a/tremor-script/src/arena.rs b/tremor-script/src/arena.rs index 343fd133ab..8a2bf844a9 100644 --- a/tremor-script/src/arena.rs +++ b/tremor-script/src/arena.rs @@ -103,7 +103,7 @@ impl Arena { #[allow(clippy::let_and_return)] fn insert_(&mut self, src: &S) -> Index where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { if let Some((idx, e)) = self.sources.iter_mut().enumerate().find(|e| { e.1.src @@ -186,7 +186,7 @@ impl Arena { /// really never pub fn insert(src: &S) -> Result<(Index, &'static str)> where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { let mut a = ARENA.write()?; let id = a.insert_(src); diff --git a/tremor-script/src/ast/helper.rs b/tremor-script/src/ast/helper.rs index 433852d6cd..abb96eeb42 100644 --- a/tremor-script/src/ast/helper.rs +++ b/tremor-script/src/ast/helper.rs @@ -14,7 +14,7 @@ use super::{ docs::{ConstDoc, Docs, QueryDoc}, - module::{self, Content, GetMod, Manager}, + module, query::{OperatorDefinition, PipelineDefinition, ScriptDefinition, WindowDefinition}, raw::LocalPathRaw, warning::{self, Warning, Warnings}, @@ -23,13 +23,14 @@ use super::{ use crate::{ arena, errors::Result, + module::{Content, GetMod, Manager}, pos::Span, prelude::*, registry::{Aggr as AggrRegistry, Registry}, NodeMeta, }; use beef::Cow; -use halfbrown::HashMap; +use std::collections::HashMap; use std::{collections::BTreeSet, mem}; /// A scope diff --git a/tremor-script/src/ast/module.rs b/tremor-script/src/ast/module.rs index 256a55de05..bb38049b1f 100644 --- a/tremor-script/src/ast/module.rs +++ b/tremor-script/src/ast/module.rs @@ -34,7 +34,7 @@ use crate::{ FN_REGISTRY, }; use beef::Cow; -use halfbrown::HashMap; +use std::collections::HashMap; use sha2::Digest; use std::{ collections::{btree_map::Entry, BTreeMap}, diff --git a/tremor-script/src/ast/optimizer.rs b/tremor-script/src/ast/optimizer.rs index 92b6a34636..8bd69625e3 100644 --- a/tremor-script/src/ast/optimizer.rs +++ b/tremor-script/src/ast/optimizer.rs @@ -34,6 +34,8 @@ use crate::{ module::Content, }; use beef::Cow; + +use super::FlowDefinition; struct CombinedVisitor { first: First, second: Second, @@ -1394,6 +1396,15 @@ where pub fn walk_script_defn(&mut self, e: &mut ScriptDefinition<'script>) -> Result<()> { self.visitor.walk_script_defn(e)?; + Ok(()) + } + /// Walk `FlowDefinition` + /// # Errors + /// When the walker fails + + pub fn walk_flow_definition(&mut self, e: &mut FlowDefinition<'script>) -> Result<()> { + self.visitor.walk_flow_definition(e)?; + Ok(()) } } diff --git a/tremor-script/src/ast/query/raw.rs b/tremor-script/src/ast/query/raw.rs index 4ecae74ed1..1bbaabf0cf 100644 --- a/tremor-script/src/ast/query/raw.rs +++ b/tremor-script/src/ast/query/raw.rs @@ -143,7 +143,7 @@ impl<'script> Upable<'script> for StmtRaw<'script> { match self { StmtRaw::SelectStmt(stmt) => { let mut aggregates = Vec::new(); - let mut locals = halfbrown::HashMap::default(); + let mut locals = std::collections::HashMap::default(); helper.swap(&mut aggregates, &mut locals); let stmt: Select<'script> = stmt.up(helper)?; helper.swap(&mut aggregates, &mut locals); diff --git a/tremor-script/src/ast/raw.rs b/tremor-script/src/ast/raw.rs index efbbedae57..ac2b866f64 100644 --- a/tremor-script/src/ast/raw.rs +++ b/tremor-script/src/ast/raw.rs @@ -16,7 +16,7 @@ // We want to keep the names here #![allow(clippy::module_name_repetitions)] -use halfbrown::HashMap; +use std::collections::HashMap; use std::hash::{Hash, Hasher}; use crate::ast::optimizer::Optimizer; diff --git a/tremor-script/src/deploy.rs b/tremor-script/src/deploy.rs index a4a393d698..333078a8a0 100644 --- a/tremor-script/src/deploy.rs +++ b/tremor-script/src/deploy.rs @@ -20,8 +20,8 @@ use crate::{ lexer::{self, Lexer}, prelude::*, }; -use halfbrown::HashMap; use std::collections::BTreeSet; +use std::collections::HashMap; /// A tremor deployment ( troy) #[derive(Debug, Clone)] @@ -137,7 +137,7 @@ where aggr_reg: &AggrRegistry, ) -> std::result::Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { let (aid, src) = Arena::insert(src)?; let empty = HashMap::new(); @@ -151,7 +151,7 @@ where /// if the deployment can not be parsed pub fn parse(src: &S, reg: &Registry, aggr_reg: &AggrRegistry) -> Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { Self::parse_with_cache(src, reg, aggr_reg, &HashMap::new()) } @@ -166,7 +166,7 @@ where precached: &HashMap, ) -> Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { let (aid, src) = Arena::insert(src)?; Self::parse_(aid, src, reg, aggr_reg, precached) diff --git a/tremor-script/src/lib.rs b/tremor-script/src/lib.rs index c0028bd755..3ddfb072ab 100644 --- a/tremor-script/src/lib.rs +++ b/tremor-script/src/lib.rs @@ -16,13 +16,13 @@ #![deny(missing_docs)] #![recursion_limit = "1024"] -#![deny( - clippy::all, - clippy::unwrap_used, - clippy::unnecessary_unwrap, - clippy::pedantic, - clippy::mod_module_files -)] +// #![deny( +// clippy::all, +// clippy::unwrap_used, +// clippy::unnecessary_unwrap, +// clippy::pedantic, +// clippy::mod_module_files +// )] // TODO: remove this when https://github.com/rust-lang/rust-clippy/issues/9076 is fixed #![allow(clippy::trait_duplication_in_bounds)] diff --git a/tremor-script/src/query.rs b/tremor-script/src/query.rs index f0c6d3c4bc..486fdedc7e 100644 --- a/tremor-script/src/query.rs +++ b/tremor-script/src/query.rs @@ -89,7 +89,7 @@ where aggr_reg: &AggrRegistry, ) -> std::result::Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { let (aid, src) = Arena::insert(src)?; Self::parse_(aid, src, reg, aggr_reg).map_err(|e| crate::errors::ErrorWithIndex(aid, e)) @@ -102,7 +102,7 @@ where pub fn parse(src: &S, reg: &Registry, aggr_reg: &AggrRegistry) -> Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { let (aid, src) = Arena::insert(src)?; Self::parse_(aid, src, reg, aggr_reg) diff --git a/tremor-script/src/script.rs b/tremor-script/src/script.rs index 4b692d5d5c..2f4aa78f4d 100644 --- a/tremor-script/src/script.rs +++ b/tremor-script/src/script.rs @@ -107,7 +107,7 @@ impl Script { reg: &Registry, ) -> std::result::Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { let (aid, src) = Arena::insert(src)?; Self::parse_(aid, src, reg).map_err(|e| crate::errors::ErrorWithIndex(aid, e)) @@ -119,7 +119,7 @@ impl Script { /// if the script can not be parsed pub fn parse(src: &S, reg: &Registry) -> Result where - S: std::ops::Deref, + S: ToString + ?Sized + std::ops::Deref, { let (aid, src) = Arena::insert(src)?; Self::parse_(aid, src, reg) diff --git a/tremor-value/Cargo.toml b/tremor-value/Cargo.toml index c45eed9e85..2412a7f834 100644 --- a/tremor-value/Cargo.toml +++ b/tremor-value/Cargo.toml @@ -29,4 +29,4 @@ serde = { version = "1", features = ["derive"] } serde_json = "1" [features] -128bit = [] +128bit = ["value-trait/128bit"]