Skip to content

Commit 10e66f7

Browse files
committed
Add routing
1 parent ba3a3ee commit 10e66f7

File tree

6 files changed

+203
-86
lines changed

6 files changed

+203
-86
lines changed

sentry_streams/src/consumer.rs

+101-50
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,8 @@
88
use crate::kafka_config::PyKafkaConsumerConfig;
99
use crate::operators::build;
1010
use crate::operators::RuntimeOperator;
11+
use crate::routes::Route;
12+
use crate::routes::RoutedValue;
1113
use pyo3::prelude::*;
1214
use pyo3::types::PyBytes;
1315
use sentry_arroyo::backends::kafka::types::KafkaPayload;
@@ -39,6 +41,8 @@ pub struct ArroyoConsumer {
3941

4042
topic: String,
4143

44+
source: String,
45+
4246
steps: Vec<Py<RuntimeOperator>>,
4347

4448
/// The ProcessorHandle allows the main thread to stop the StreamingProcessor
@@ -49,10 +53,11 @@ pub struct ArroyoConsumer {
4953
#[pymethods]
5054
impl ArroyoConsumer {
5155
#[new]
52-
fn new(kafka_config: PyKafkaConsumerConfig, topic: String) -> Self {
56+
fn new(source: String, kafka_config: PyKafkaConsumerConfig, topic: String) -> Self {
5357
ArroyoConsumer {
5458
consumer_config: kafka_config,
5559
topic,
60+
source,
5661
steps: Vec::new(),
5762
handle: None,
5863
}
@@ -71,13 +76,8 @@ impl ArroyoConsumer {
7176
fn run(&mut self) {
7277
tracing_subscriber::fmt::init();
7378
println!("Running Arroyo Consumer...");
74-
let steps_copy = Python::with_gil(|py| {
75-
self.steps
76-
.iter()
77-
.map(|step| step.clone_ref(py))
78-
.collect::<Vec<_>>()
79-
});
80-
let factory = ArroyoStreamingFactory { steps: steps_copy };
79+
80+
let factory = ArroyoStreamingFactory::new(self.source.clone(), &self.steps);
8181
let config = self.consumer_config.clone().into();
8282
let processor = StreamProcessor::with_kafka(config, factory, Topic::new(&self.topic), None);
8383
self.handle = Some(processor.get_handle());
@@ -89,7 +89,7 @@ impl ArroyoConsumer {
8989
})
9090
.expect("Error setting Ctrl+C handler");
9191

92-
processor.run();
92+
let _ = processor.run();
9393
}
9494

9595
fn shutdown(&mut self) {
@@ -100,85 +100,126 @@ impl ArroyoConsumer {
100100
}
101101
}
102102

103-
/// Converts a Message<KafkaPayload> to a Message<Py<PyAny>>.
104-
/// It takes the Kafka payload as bytes and turns it into a
105-
/// Python bytes object.
106-
fn to_python(message: Message<KafkaPayload>) -> Message<Py<PyAny>> {
107-
let payload = Python::with_gil(|py| {
108-
let payload = message.payload().payload().unwrap();
109-
let py_bytes = PyBytes::new(py, payload);
110-
py_bytes.into_any().unbind()
103+
/// Converts a Message<KafkaPayload> to a Message<RoutedValue>.
104+
///
105+
/// The messages we send around between steps in the pipeline contain
106+
/// the `Route` object that represent the path the message took when
107+
/// going through branches.
108+
/// The message coming from Kafka is a Message<KafkaPayload>, so we need
109+
/// to turn the content into PyBytes for python to manage the content
110+
/// and we need to wrap the message into a RoutedValue object.
111+
fn to_routed_value(source: &str, message: Message<KafkaPayload>) -> Message<RoutedValue> {
112+
let payload = Python::with_gil(|py| match message.payload().payload() {
113+
Some(payload) => PyBytes::new(py, payload).into_any().unbind(),
114+
None => py.None(),
111115
});
112-
message.replace(payload)
116+
let route = Route::new(source.to_string(), vec![]);
117+
message.replace(RoutedValue { route, payload })
113118
}
114119

115120
/// Builds the Arroyo StreamProcessor for this consumer.
116-
/// It plugs a Commit policy at the end and a translator at the beginning
117-
/// that takes the payload of the Kafka message and turns it into a Py<PyAny>
118-
fn build_chain(steps: &[Py<RuntimeOperator>]) -> Box<dyn ProcessingStrategy<KafkaPayload>> {
119-
let mut next: Box<dyn ProcessingStrategy<Py<PyAny>>> =
120-
Box::new(CommitOffsets::new(Duration::from_secs(5)));
121+
///
122+
/// It wires up all the operators added to the consumer object,
123+
/// it prefix the chain with a step that converts the Message<KafkaPayload>
124+
/// to a Message<RoutedValue> and it adds a termination step provided
125+
/// by the caller. This is generally a CommitOffsets step but it can
126+
/// be customized.
127+
fn build_chain(
128+
source: &str,
129+
steps: &[Py<RuntimeOperator>],
130+
ending_strategy: Box<dyn ProcessingStrategy<RoutedValue>>,
131+
) -> Box<dyn ProcessingStrategy<KafkaPayload>> {
132+
let mut next = ending_strategy;
121133
for step in steps.iter().rev() {
122134
next = build(step, next);
123135
}
124136

125-
let converter = RunTask::new(
126-
|message: Message<KafkaPayload>| Ok(to_python(message)),
127-
next,
128-
);
137+
let copied_source = source.to_string();
138+
let conversion_function =
139+
move |message: Message<KafkaPayload>| Ok(to_routed_value(&copied_source, message));
140+
141+
let converter = RunTask::new(conversion_function, next);
129142

130143
Box::new(converter)
131144
}
132145

133146
struct ArroyoStreamingFactory {
147+
source: String,
134148
steps: Vec<Py<RuntimeOperator>>,
135149
}
136150

151+
impl ArroyoStreamingFactory {
152+
/// Creates a new instance of ArroyoStreamingFactory.
153+
fn new(source: String, steps: &[Py<RuntimeOperator>]) -> Self {
154+
let steps_copy = Python::with_gil(|py| {
155+
steps
156+
.iter()
157+
.map(|step| step.clone_ref(py))
158+
.collect::<Vec<_>>()
159+
});
160+
ArroyoStreamingFactory {
161+
source,
162+
steps: steps_copy,
163+
}
164+
}
165+
}
166+
137167
impl ProcessingStrategyFactory<KafkaPayload> for ArroyoStreamingFactory {
138168
fn create(&self) -> Box<dyn ProcessingStrategy<KafkaPayload>> {
139-
build_chain(&self.steps)
169+
build_chain(
170+
&self.source,
171+
&self.steps,
172+
Box::new(CommitOffsets::new(Duration::from_secs(5))),
173+
)
140174
}
141175
}
142176
#[cfg(test)]
143177
mod tests {
144178
use super::*;
179+
use crate::fake_strategy::assert_messages_match;
180+
use crate::fake_strategy::FakeStrategy;
145181
use crate::operators::RuntimeOperator;
146182
use crate::routes::Route;
183+
use crate::test_operators::make_lambda;
184+
use crate::test_operators::make_msg;
147185
use pyo3::ffi::c_str;
148186
use pyo3::IntoPyObjectExt;
149-
use std::collections::BTreeMap;
187+
use std::ops::Deref;
188+
use std::sync::{Arc, Mutex};
150189

151190
#[test]
152-
fn test_to_python() {
191+
fn test_to_routed_value() {
153192
Python::with_gil(|py| {
154193
let payload_data = b"test_payload";
155-
let message = Message::new_any_message(
156-
KafkaPayload::new(None, None, Some(payload_data.to_vec())),
157-
BTreeMap::new(),
158-
);
194+
let message = make_msg(Some(payload_data.to_vec()));
159195

160-
let python_message = to_python(message);
196+
let python_message = to_routed_value("source", message);
161197

162198
let py_payload = python_message.payload();
163199

164-
let down: &Bound<PyBytes> = py_payload.bind(py).downcast().unwrap();
200+
let down: &Bound<PyBytes> = py_payload.payload.bind(py).downcast().unwrap();
165201
let payload_bytes: &[u8] = down.as_bytes();
166202
assert_eq!(payload_bytes, payload_data);
203+
assert_eq!(py_payload.route.source, "source");
204+
assert_eq!(py_payload.route.waypoints.len(), 0);
205+
});
206+
}
207+
208+
#[test]
209+
fn test_to_none_python() {
210+
Python::with_gil(|py| {
211+
let message = make_msg(None);
212+
let python_message = to_routed_value("source", message);
213+
let py_payload = &python_message.payload().payload;
214+
215+
assert!(py_payload.is_none(py));
167216
});
168217
}
169218

170219
#[test]
171220
fn test_build_chain() {
172221
Python::with_gil(|py| {
173-
let callable = py
174-
.eval(
175-
c_str!("lambda x: x.decode('utf-8') + '_transformed'"),
176-
None,
177-
None,
178-
)
179-
.unwrap()
180-
.into_py_any(py)
181-
.unwrap();
222+
let callable = make_lambda(py, c_str!("lambda x: x.decode('utf-8') + '_transformed'"));
182223

183224
let mut steps: Vec<Py<RuntimeOperator>> = vec![];
184225

@@ -192,12 +233,22 @@ mod tests {
192233
.unwrap();
193234
steps.push(r);
194235

195-
let mut chain = build_chain(&steps);
196-
let message = Message::new_any_message(
197-
KafkaPayload::new(None, None, Some(b"test_payload".to_vec())),
198-
BTreeMap::new(),
199-
);
236+
let submitted_messages = Arc::new(Mutex::new(Vec::new()));
237+
let submitted_messages_clone = submitted_messages.clone();
238+
let next_step = FakeStrategy {
239+
submitted: submitted_messages,
240+
};
241+
242+
let mut chain = build_chain("source", &steps, Box::new(next_step));
243+
let message = make_msg(Some(b"test_payload".to_vec()));
244+
200245
chain.submit(message).unwrap();
246+
247+
let value = "test_payload_transformed".into_py_any(py).unwrap();
248+
let expected_messages = vec![value];
249+
let actual_messages = submitted_messages_clone.lock().unwrap();
250+
251+
assert_messages_match(py, expected_messages, actual_messages.deref());
201252
})
202253
}
203254
}

sentry_streams/src/fake_strategy.rs

+7-3
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
use super::*;
2+
use crate::routes::RoutedValue;
23
use sentry_arroyo::processing::strategies::{
34
CommitRequest, ProcessingStrategy, StrategyError, SubmitError,
45
};
@@ -10,13 +11,16 @@ pub struct FakeStrategy {
1011
pub submitted: Arc<Mutex<Vec<Py<PyAny>>>>,
1112
}
1213

13-
impl ProcessingStrategy<Py<PyAny>> for FakeStrategy {
14+
impl ProcessingStrategy<RoutedValue> for FakeStrategy {
1415
fn poll(&mut self) -> Result<Option<CommitRequest>, StrategyError> {
1516
Ok(None)
1617
}
1718

18-
fn submit(&mut self, message: Message<Py<PyAny>>) -> Result<(), SubmitError<Py<PyAny>>> {
19-
self.submitted.lock().unwrap().push(message.into_payload());
19+
fn submit(&mut self, message: Message<RoutedValue>) -> Result<(), SubmitError<RoutedValue>> {
20+
self.submitted
21+
.lock()
22+
.unwrap()
23+
.push(message.into_payload().payload);
2024
Ok(())
2125
}
2226

sentry_streams/src/lib.rs

+5-1
Original file line numberDiff line numberDiff line change
@@ -1,11 +1,15 @@
11
use pyo3::prelude::*;
22
mod consumer;
3-
mod fake_strategy;
43
mod kafka_config;
54
mod operators;
65
mod routes;
76
mod transformer;
87

8+
#[cfg(test)]
9+
mod fake_strategy;
10+
#[cfg(test)]
11+
mod test_operators;
12+
913
#[pymodule]
1014
fn rust_streams(m: &Bound<'_, PyModule>) -> PyResult<()> {
1115
m.add_class::<routes::Route>()?;

sentry_streams/src/operators.rs

+5-5
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,5 @@
11
use crate::kafka_config::PyKafkaConsumerConfig;
2-
use crate::routes::Route;
2+
use crate::routes::{Route, RoutedValue};
33
use crate::transformer::build_map;
44
use pyo3::prelude::*;
55
use sentry_arroyo::processing::strategies::ProcessingStrategy;
@@ -77,12 +77,12 @@ impl RuntimeOperator {
7777

7878
pub fn build(
7979
step: &Py<RuntimeOperator>,
80-
next: Box<dyn ProcessingStrategy<Py<PyAny>>>,
81-
) -> Box<dyn ProcessingStrategy<Py<PyAny>>> {
80+
next: Box<dyn ProcessingStrategy<RoutedValue>>,
81+
) -> Box<dyn ProcessingStrategy<RoutedValue>> {
8282
match step.get() {
83-
RuntimeOperator::Map { function, .. } => {
83+
RuntimeOperator::Map { function, route } => {
8484
let func_ref = Python::with_gil(|py| function.clone_ref(py));
85-
build_map(func_ref, next)
85+
build_map(route, func_ref, next)
8686
}
8787
RuntimeOperator::StreamSink { .. } => {
8888
// Handle StreamSink step

sentry_streams/src/test_operators.rs

+21
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,21 @@
1+
use pyo3::prelude::*;
2+
use pyo3::IntoPyObjectExt;
3+
use sentry_arroyo::backends::kafka::types::KafkaPayload;
4+
use sentry_arroyo::types::Message;
5+
use std::ffi::CStr;
6+
7+
#[cfg(test)]
8+
pub fn make_lambda(py: Python<'_>, py_code: &CStr) -> Py<PyAny> {
9+
py.eval(py_code, None, None)
10+
.unwrap()
11+
.into_py_any(py)
12+
.unwrap()
13+
}
14+
15+
#[cfg(test)]
16+
pub fn make_msg(payload: Option<Vec<u8>>) -> Message<KafkaPayload> {
17+
Message::new_any_message(
18+
KafkaPayload::new(None, None, payload),
19+
std::collections::BTreeMap::new(),
20+
)
21+
}

0 commit comments

Comments
 (0)