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

chore(streaming): enable actor metrics in release mode #2612

Merged
merged 2 commits into from
May 18, 2022
Merged
Show file tree
Hide file tree
Changes from all 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
37 changes: 32 additions & 5 deletions src/stream/src/executor/debug.rs
Original file line number Diff line number Diff line change
Expand Up @@ -58,13 +58,12 @@ impl DebugExecutor {
}
}

#[allow(clippy::redundant_clone)]
#[allow(clippy::let_and_return)]
fn wrap(
fn wrap_debug(
info: Arc<ExecutorInfo>,
extra: DebugExtraInfo,
stream: impl MessageStream,
) -> impl MessageStream {
stream: impl MessageStream + 'static,
) -> impl MessageStream + 'static {
// Trace
let stream = trace::trace(
info.clone(),
Expand All @@ -73,15 +72,43 @@ impl DebugExecutor {
extra.metrics,
stream,
);

// Schema check
let stream = schema_check::schema_check(info.clone(), stream);
// Epoch check
let stream = epoch_check::epoch_check(info.clone(), stream);
// Update check
let stream = update_check::update_check(info.clone(), stream);
let stream = update_check::update_check(info, stream);

stream
}

#[allow(clippy::let_and_return)]
fn wrap_release(
info: Arc<ExecutorInfo>,
extra: DebugExtraInfo,
stream: impl MessageStream + 'static,
) -> impl MessageStream + 'static {
// Metrics
let stream = trace::metrics(extra.actor_id, extra.metrics, stream);

// Epoch check
let stream = epoch_check::epoch_check(info, stream);

stream
}

fn wrap(
info: Arc<ExecutorInfo>,
extra: DebugExtraInfo,
stream: impl MessageStream + 'static,
) -> BoxedMessageStream {
if cfg!(debug_assertions) {
Self::wrap_debug(info, extra, stream).boxed()
} else {
Self::wrap_release(info, extra, stream).boxed()
}
}
}

impl Executor for DebugExecutor {
Expand Down
21 changes: 21 additions & 0 deletions src/stream/src/executor/debug/trace.rs
Original file line number Diff line number Diff line change
Expand Up @@ -61,3 +61,24 @@ pub async fn trace(
yield message;
}
}

/// Streams wrapped by `metrics` will update actor metrics.
#[try_stream(ok = Message, error = StreamExecutorError)]
pub async fn metrics(actor_id: ActorId, metrics: Arc<StreamingMetrics>, input: impl MessageStream) {
let actor_id_string = actor_id.to_string();

pin_mut!(input);

while let Some(message) = input.next().await.transpose()? {
if let Message::Chunk(chunk) = &message {
if chunk.cardinality() > 0 {
metrics
.actor_row_count
.with_label_values(&[&actor_id_string])
.inc_by(chunk.cardinality() as u64);
}
}

yield message;
}
}