Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Adding Channels and Orchestration #35

Merged
merged 8 commits into from
Sep 6, 2024
Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
160 changes: 112 additions & 48 deletions Cargo.lock

Large diffs are not rendered by default.

3 changes: 2 additions & 1 deletion Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,9 @@ description = "Embeddable stream processing engine"
[workspace.dependencies]
denormalized = { path = "crates/core" }
denormalized-common = { path = "crates/common" }
denormalized-orchestrator = { path = "crates/orchestrator" }

datafusion = { git = "https://github.com/probably-nothing-labs/arrow-datafusion", branch = "main" }
datafusion = { git = "https://github.com/probably-nothing-labs/arrow-datafusion", branch = "amey/patch-with-node-id" }
ameyc marked this conversation as resolved.
Show resolved Hide resolved

arrow = { version = "52.0.0", features = ["prettyprint"] }
arrow-array = { version = "52.0.0", default-features = false, features = [
Expand Down
1 change: 1 addition & 0 deletions crates/core/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ edition = { workspace = true }

[dependencies]
denormalized-common = { workspace = true }
denormalized-orchestrator = { workspace = true }

datafusion = { workspace = true }

Expand Down
43 changes: 40 additions & 3 deletions crates/core/src/datasource/kafka/kafka_stream_read.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5,11 +5,15 @@ use std::time::Duration;
use arrow::datatypes::TimestampMillisecondType;
use arrow_array::{Array, ArrayRef, PrimitiveArray, RecordBatch, StringArray, StructArray};
use arrow_schema::{DataType, Field, SchemaRef, TimeUnit};
use denormalized_orchestrator::channel_manager::{create_channel, get_sender};
use denormalized_orchestrator::orchestrator::{self, OrchestrationMessage};
use log::{debug, error};
use serde::{Deserialize, Serialize};
use serde_json::Value;
use tracing::{debug, error};
//use tracing::{debug, error};

use crate::config_extensions::denormalized_config::DenormalizedConfig;
use crate::physical_plan::stream_table::PartitionStreamExt;
use crate::physical_plan::utils::time::array_to_timestamp_array;
use crate::state_backend::rocksdb_backend::get_global_rocksdb;
use crate::utils::arrow_helpers::json_records_to_arrow_record_batch;
Expand All @@ -24,9 +28,21 @@ use rdkafka::{ClientConfig, Message, Timestamp, TopicPartitionList};

use super::KafkaReadConfig;

#[derive(Clone)]
pub struct KafkaStreamRead {
pub config: Arc<KafkaReadConfig>,
pub assigned_partitions: Vec<i32>,
pub exec_node_id: Option<usize>,
}

impl KafkaStreamRead {
pub fn with_node_id(self, node_id: Option<usize>) -> KafkaStreamRead {
Self {
config: self.config.clone(),
assigned_partitions: self.assigned_partitions.clone(),
exec_node_id: node_id,
}
}
}

#[derive(Debug, Serialize, Deserialize)]
Expand Down Expand Up @@ -122,9 +138,19 @@ impl PartitionStream for KafkaStreamRead {
let timestamp_column: String = self.config.timestamp_column.clone();
let timestamp_unit = self.config.timestamp_unit.clone();
let batch_timeout = Duration::from_millis(100);

let mut channel_tag = String::from("");
if orchestrator::SHOULD_CHECKPOINT {
let node_id = self.exec_node_id.unwrap();
channel_tag = format!("{}_{}", node_id, partition_tag);
create_channel(channel_tag.as_str(), 10);
}
builder.spawn(async move {
let mut epoch = 0;
if orchestrator::SHOULD_CHECKPOINT {
let orchestrator_sender = get_sender("orchestrator");
let msg = OrchestrationMessage::RegisterStream(channel_tag.clone());
orchestrator_sender.as_ref().unwrap().send(msg).unwrap();
}
loop {
let mut last_offsets = HashMap::new();
if let Some(backend) = &state_backend {
Expand Down Expand Up @@ -202,7 +228,7 @@ impl PartitionStream for KafkaStreamRead {
}
}

debug!("Batch size {}", batch.len());
//debug!("Batch size {}", batch.len());

if !batch.is_empty() {
let record_batch: RecordBatch =
Expand Down Expand Up @@ -277,3 +303,14 @@ impl PartitionStream for KafkaStreamRead {
builder.build()
}
}

// Implement this for KafkaStreamRead
impl PartitionStreamExt for KafkaStreamRead {
fn requires_node_id(&self) -> bool {
true
}

fn as_partition_with_node_id(&self) -> Option<&KafkaStreamRead> {
Some(self)
}
}
7 changes: 5 additions & 2 deletions crates/core/src/datasource/kafka/topic_reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,9 @@ use datafusion::common::{not_impl_err, plan_err, Result};
use datafusion::datasource::TableProvider;
use datafusion::logical_expr::{Expr, TableType};
use datafusion::physical_expr::{expressions, LexOrdering, PhysicalSortExpr};
use datafusion::physical_plan::{streaming::StreamingTableExec, ExecutionPlan};
use datafusion::physical_plan::ExecutionPlan;

use crate::physical_plan::stream_table::DenormalizedStreamingTableExec;

use super::{KafkaReadConfig, KafkaStreamRead};

Expand Down Expand Up @@ -37,11 +39,12 @@ impl TopicReader {
let read_stream = Arc::new(KafkaStreamRead {
config: self.0.clone(),
assigned_partitions: vec![part],
exec_node_id: None,
});
partition_streams.push(read_stream as _);
}

Ok(Arc::new(StreamingTableExec::try_new(
Ok(Arc::new(DenormalizedStreamingTableExec::try_new(
self.0.schema.clone(),
partition_streams,
projection,
Expand Down
31 changes: 30 additions & 1 deletion crates/core/src/datastream.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,8 @@
use datafusion::common::runtime::SpawnedTask;
use datafusion::logical_expr::LogicalPlan;
use datafusion::physical_plan::ExecutionPlan;
use datafusion::physical_plan::ExecutionPlanProperties;
use denormalized_orchestrator::orchestrator;
use futures::StreamExt;
use std::{sync::Arc, time::Duration};

Expand All @@ -15,6 +19,7 @@ use crate::context::Context;
use crate::datasource::kafka::{ConnectionOpts, KafkaTopicBuilder};
use crate::logical_plan::StreamingLogicalPlanBuilder;
use crate::physical_plan::utils::time::TimestampUnit;
use denormalized_orchestrator::orchestrator::Orchestrator;

use denormalized_common::error::Result;

Expand Down Expand Up @@ -128,7 +133,18 @@ impl DataStream {

/// execute the stream and print the results to stdout.
/// Mainly used for development and debugging
pub async fn print_stream(self) -> Result<()> {
pub async fn print_stream(&self) -> Result<()> {
if orchestrator::SHOULD_CHECKPOINT {
let plan = self.df.as_ref().clone().create_physical_plan().await?;
let node_ids = extract_node_ids_and_partitions(&plan);
let max_buffer_size = node_ids
.iter()
.map(|x| x.1)
.sum::<usize>();
let mut orchestrator = Orchestrator::default();
SpawnedTask::spawn_blocking(move || orchestrator.run(max_buffer_size));
}

let mut stream: SendableRecordBatchStream =
self.df.as_ref().clone().execute_stream().await?;
loop {
Expand Down Expand Up @@ -232,3 +248,16 @@ impl Joinable for DataStream {
plan
}
}

fn extract_node_ids_and_partitions(plan: &Arc<dyn ExecutionPlan>) -> Vec<(Option<usize>, usize)> {
let node_id = plan.node_id();
let partitions = plan.output_partitioning().partition_count();
let mut traversals: Vec<(Option<usize>, usize)> = vec![];

for child in plan.children() {
let mut traversal = extract_node_ids_and_partitions(child);
traversals.append(&mut traversal);
}
traversals.push((node_id, partitions));
traversals
}
19 changes: 19 additions & 0 deletions crates/core/src/physical_plan/continuous/streaming_window.rs
Original file line number Diff line number Diff line change
Expand Up @@ -509,6 +509,25 @@ impl ExecutionPlan for StreamingWindowExec {
) -> Result<Option<Arc<dyn ExecutionPlan>>> {
Ok(None)
}

fn with_node_id(self: Arc<Self>, _node_id: usize) -> Result<Option<Arc<dyn ExecutionPlan>>> {
let cache = self.properties().clone().with_node_id(_node_id);
let new_exec = StreamingWindowExec {
input: self.input.clone(),
aggregate_expressions: self.aggregate_expressions.clone(),
filter_expressions: self.filter_expressions.clone(),
group_by: self.group_by.clone(),
schema: self.schema.clone(),
input_schema: self.input_schema.clone(),
watermark: Arc::new(Mutex::new(None)),
metrics: ExecutionPlanMetricsSet::new(),
cache,
mode: self.mode,
window_type: self.window_type,
upstream_partitioning: self.upstream_partitioning,
};
Ok(Some(Arc::new(new_exec)))
}
}

impl DisplayAs for StreamingWindowExec {
Expand Down
1 change: 1 addition & 0 deletions crates/core/src/physical_plan/mod.rs
Original file line number Diff line number Diff line change
@@ -1,2 +1,3 @@
pub mod continuous;
pub mod stream_table;
pub mod utils;
Loading
Loading