From e0d7daf2d2e68cb492db62d200808068f903f226 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 25 Aug 2024 21:02:25 +0800 Subject: [PATCH 01/10] add new route for fetching table fragments by id --- src/meta/src/dashboard/mod.rs | 38 +++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 1229554032614..3677786b72202 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -53,6 +53,8 @@ pub(super) mod handlers { use axum::Json; use futures::future::join_all; use itertools::Itertools; + use risingwave_common::bail; + use risingwave_common::catalog::TableId; use risingwave_common_heap_profiling::COLLAPSED_SUFFIX; use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::{PbDatabase, PbSchema, Sink, Source, Subscription, Table, View}; @@ -211,6 +213,41 @@ pub(super) mod handlers { Ok(Json(table_fragments)) } + pub async fn list_fragments_by_job_id( + Extension(srv): Extension, + Path(job_id): Path, + ) -> Result> { + let table_fragments = match &srv.metadata_manager { + MetadataManager::V1(mgr) => { + if let Some(tf) = mgr + .fragment_manager + .get_fragment_read_guard() + .await + .table_fragments() + .get(&TableId::new(job_id)) + { + tf.to_protobuf() + } else { + bail!("job_id {} not found", job_id) + } + } + MetadataManager::V2(mgr) => { + let mut table_fragments = mgr + .catalog_controller + .table_fragments() + .await + .map_err(err)?; + if let Some(tf) = table_fragments.remove(&(job_id as i32)) { + tf + } else { + bail!("job_id {} not found", job_id) + } + } + }; + + Ok(Json(table_fragments)) + } + pub async fn list_users(Extension(srv): Extension) -> Result>> { let users = match &srv.metadata_manager { MetadataManager::V1(mgr) => mgr.catalog_manager.list_users().await, @@ -428,6 +465,7 @@ impl DashboardService { let api_router = Router::new() .route("/clusters/:ty", get(list_clusters)) .route("/fragments2", get(list_fragments)) + .route("/fragments/job_id/:job_id", get(list_fragments_by_job_id)) .route("/views", get(list_views)) .route("/materialized_views", get(list_materialized_views)) .route("/tables", get(list_tables)) From 6345700570ac4da75dbf8a3ae20eca6083ad9d0e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 25 Aug 2024 21:24:53 +0800 Subject: [PATCH 02/10] add fetch for fragments by id --- dashboard/lib/api/streaming.ts | 7 +++++++ dashboard/pages/fragment_graph.tsx | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/dashboard/lib/api/streaming.ts b/dashboard/lib/api/streaming.ts index 948cd567d3f2b..5b3a6c51825c2 100644 --- a/dashboard/lib/api/streaming.ts +++ b/dashboard/lib/api/streaming.ts @@ -34,6 +34,13 @@ import { ColumnCatalog, Field } from "../../proto/gen/plan_common" import { UserInfo } from "../../proto/gen/user" import api from "./api" +export async function getFragmentsByJobId(jobId: number): Promise { + let route = "/fragments/job_id/" + jobId.toString() + console.log("route: ", route) + let tableFragments: TableFragments = TableFragments.fromJSON(await api.get(route)) + return tableFragments +} + export async function getFragments(): Promise { let fragmentList: TableFragments[] = (await api.get("/fragments2")).map( TableFragments.fromJSON diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 1c5f6d7746535..78aa97f44a89b 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -45,7 +45,7 @@ import { fetchEmbeddedBackPressure, fetchPrometheusBackPressure, } from "../lib/api/metric" -import { getFragments, getStreamingJobs } from "../lib/api/streaming" +import { getFragments, getFragmentsByJobId, getStreamingJobs } from "../lib/api/streaming" import { FragmentBox } from "../lib/layout" import { TableFragments, TableFragments_Fragment } from "../proto/gen/meta" import { Dispatcher, MergeNode, StreamNode } from "../proto/gen/stream_plan" From f100b7394fae966e4877d40c1cdfb066ea3351a1 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 25 Aug 2024 23:32:56 +0800 Subject: [PATCH 03/10] use useEffect for updating fragmentGraph + introduce fragmentids - need to add fragment id, actor id proto --- dashboard/lib/api/streaming.ts | 7 ++++++ dashboard/pages/fragment_graph.tsx | 37 ++++++++++++++++++------------ src/meta/src/dashboard/mod.rs | 27 ++++++++++++++++++++++ 3 files changed, 56 insertions(+), 15 deletions(-) diff --git a/dashboard/lib/api/streaming.ts b/dashboard/lib/api/streaming.ts index 5b3a6c51825c2..3639c61d83b0e 100644 --- a/dashboard/lib/api/streaming.ts +++ b/dashboard/lib/api/streaming.ts @@ -34,6 +34,8 @@ import { ColumnCatalog, Field } from "../../proto/gen/plan_common" import { UserInfo } from "../../proto/gen/user" import api from "./api" +// NOTE(kwannoel): This can be optimized further, instead of fetching the entire TableFragments struct, +// We can fetch the fields we need from TableFragments, in a truncated struct. export async function getFragmentsByJobId(jobId: number): Promise { let route = "/fragments/job_id/" + jobId.toString() console.log("route: ", route) @@ -41,6 +43,11 @@ export async function getFragmentsByJobId(jobId: number): Promise>> { + let fragmentIds: Map> = (await api.get("/fragments/ids")) + return fragmentIds +} + export async function getFragments(): Promise { let fragmentList: TableFragments[] = (await api.get("/fragments2")).map( TableFragments.fromJSON diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 78aa97f44a89b..3eaf7d7f00bcd 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -45,7 +45,7 @@ import { fetchEmbeddedBackPressure, fetchPrometheusBackPressure, } from "../lib/api/metric" -import { getFragments, getFragmentsByJobId, getStreamingJobs } from "../lib/api/streaming" +import { getFragments, getFragmentIds, getFragmentsByJobId, getStreamingJobs } from "../lib/api/streaming" import { FragmentBox } from "../lib/layout" import { TableFragments, TableFragments_Fragment } from "../proto/gen/meta" import { Dispatcher, MergeNode, StreamNode } from "../proto/gen/stream_plan" @@ -194,28 +194,34 @@ interface EmbeddedBackPressureInfo { export default function Streaming() { const { response: relationList } = useFetch(getStreamingJobs) + // const { response: fragmentList } = useFetch(getFragments) const { response: fragmentList } = useFetch(getFragments) + const { response: relationIdToFragmentIds } = useFetch(getFragmentIds) const [relationId, setRelationId] = useQueryState("id", parseAsInteger) const [selectedFragmentId, setSelectedFragmentId] = useState() + const [tableFragments, setTableFragments] = useState() const toast = useErrorToast() + useEffect(() => { + if (relationId) { + getFragmentsByJobId(relationId).then(tf => { + setTableFragments(tf) + }) + } + }, [relationId]) + const fragmentDependencyCallback = useCallback(() => { - if (fragmentList) { - if (relationId) { - const fragments = fragmentList.find((x) => x.tableId === relationId) - if (fragments) { - const fragmentDep = buildFragmentDependencyAsEdges(fragments) - return { - fragments, - fragmentDep, - fragmentDepDag: dagStratify()(fragmentDep), - } - } + if (tableFragments) { + const fragmentDep = buildFragmentDependencyAsEdges(tableFragments) + return { + fragments: tableFragments, + fragmentDep, + fragmentDepDag: dagStratify()(fragmentDep), } } - }, [fragmentList, relationId]) + }, [tableFragments]) useEffect(() => { if (relationList) { @@ -258,9 +264,10 @@ export default function Streaming() { if (fragmentList) { for (const tf of fragmentList) { for (const fragmentId in tf.fragments) { - if (tf.fragments[fragmentId].fragmentId == searchFragIdInt) { + const fragment = tf.fragments[fragmentId] + if (fragment.fragmentId == searchFragIdInt) { setRelationId(tf.tableId) - setSelectedFragmentId(searchFragIdInt) + setSelectedFragmentId(fragment.fragmentId) return } } diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 3677786b72202..7945d73eaa221 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -49,6 +49,7 @@ pub struct DashboardService { pub type Service = Arc; pub(super) mod handlers { + use std::collections::{HashMap, HashSet}; use anyhow::Context; use axum::Json; use futures::future::join_all; @@ -213,6 +214,31 @@ pub(super) mod handlers { Ok(Json(table_fragments)) } + pub async fn list_fragment_ids( + Extension(srv): Extension, + ) -> Result>>> { + let table_fragments = match &srv.metadata_manager { + MetadataManager::V1(mgr) => mgr + .fragment_manager + .get_fragment_read_guard() + .await + .table_fragments() + .iter() + .map(|(id, tf)| (id.table_id, tf.fragments.keys().copied().collect::>())) + .collect(), + MetadataManager::V2(mgr) => mgr + .catalog_controller + .table_fragments() + .await + .map_err(err)? + .iter() + .map(|(id, tf)| (*id as u32, tf.fragments.keys().copied().collect::>())) + .collect(), + }; + + Ok(Json(table_fragments)) + } + pub async fn list_fragments_by_job_id( Extension(srv): Extension, Path(job_id): Path, @@ -466,6 +492,7 @@ impl DashboardService { .route("/clusters/:ty", get(list_clusters)) .route("/fragments2", get(list_fragments)) .route("/fragments/job_id/:job_id", get(list_fragments_by_job_id)) + .route("/fragments/ids", get(list_fragment_ids)) .route("/views", get(list_views)) .route("/materialized_views", get(list_materialized_views)) .route("/tables", get(list_tables)) From dda5f8ca7c9572497022303eb559f06ac4b8ff18 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Aug 2024 18:06:27 +0800 Subject: [PATCH 04/10] update api --- proto/meta.proto | 17 ++++++++ src/meta/src/dashboard/mod.rs | 75 ++++++++++++++++++++++++----------- 2 files changed, 69 insertions(+), 23 deletions(-) diff --git a/proto/meta.proto b/proto/meta.proto index d69b8fd349893..b5ebb0ae01336 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -771,3 +771,20 @@ service EventLogService { rpc ListEventLog(ListEventLogRequest) returns (ListEventLogResponse); rpc AddEventLog(AddEventLogRequest) returns (AddEventLogResponse); } + +message ActorIds { + repeated uint32 ids = 1; +} + +message FragmentIdToActorIdMap { + map map = 1; +} + +/// Provides all the ids: relation_id, fragment_id, actor_id +/// in an hierarchical format. +/// relation_id -> [fragment_id] +/// fragment_id -> [actor_id] +message RelationIdInfos { + // relation_id -> FragmentIdToActorIdMap + map map = 1; +} \ No newline at end of file diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 7945d73eaa221..cfdf2bea0f322 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -49,7 +49,8 @@ pub struct DashboardService { pub type Service = Arc; pub(super) mod handlers { - use std::collections::{HashMap, HashSet}; + use std::collections::HashMap; + use anyhow::Context; use axum::Json; use futures::future::join_all; @@ -61,7 +62,9 @@ pub(super) mod handlers { use risingwave_pb::catalog::{PbDatabase, PbSchema, Sink, Source, Subscription, Table, View}; use risingwave_pb::common::{WorkerNode, WorkerType}; use risingwave_pb::meta::list_object_dependencies_response::PbObjectDependencies; - use risingwave_pb::meta::PbTableFragments; + use risingwave_pb::meta::{ + ActorIds, FragmentIdToActorIdMap, PbTableFragments, RelationIdInfos, + }; use risingwave_pb::monitor_service::{ GetBackPressureResponse, HeapProfilingResponse, ListHeapProfilingResponse, StackTraceResponse, @@ -214,29 +217,55 @@ pub(super) mod handlers { Ok(Json(table_fragments)) } - pub async fn list_fragment_ids( + pub async fn get_relation_id_infos( Extension(srv): Extension, - ) -> Result>>> { - let table_fragments = match &srv.metadata_manager { - MetadataManager::V1(mgr) => mgr - .fragment_manager - .get_fragment_read_guard() - .await - .table_fragments() - .iter() - .map(|(id, tf)| (id.table_id, tf.fragments.keys().copied().collect::>())) - .collect(), - MetadataManager::V2(mgr) => mgr - .catalog_controller - .table_fragments() - .await - .map_err(err)? - .iter() - .map(|(id, tf)| (*id as u32, tf.fragments.keys().copied().collect::>())) - .collect(), + ) -> Result> { + let map = match &srv.metadata_manager { + MetadataManager::V1(mgr) => { + let core = mgr.fragment_manager.get_fragment_read_guard().await; + let table_fragments = core.table_fragments(); + let mut map = HashMap::new(); + for (id, tf) in table_fragments { + let mut fragment_id_to_actor_ids = HashMap::new(); + for (fragment_id, fragment) in &tf.fragments { + let actor_ids = fragment.actors.iter().map(|a| a.actor_id).collect_vec(); + fragment_id_to_actor_ids.insert(*fragment_id, ActorIds { ids: actor_ids }); + } + map.insert( + id.table_id, + FragmentIdToActorIdMap { + map: fragment_id_to_actor_ids, + }, + ); + } + map + } + MetadataManager::V2(mgr) => { + let table_fragments = mgr + .catalog_controller + .table_fragments() + .await + .map_err(err)?; + let mut map = HashMap::new(); + for (id, tf) in table_fragments { + let mut fragment_id_to_actor_ids = HashMap::new(); + for (fragment_id, fragment) in &tf.fragments { + let actor_ids = fragment.actors.iter().map(|a| a.actor_id).collect_vec(); + fragment_id_to_actor_ids.insert(*fragment_id, ActorIds { ids: actor_ids }); + } + map.insert( + id as u32, + FragmentIdToActorIdMap { + map: fragment_id_to_actor_ids, + }, + ); + } + map + } }; + let relation_id_infos = RelationIdInfos { map }; - Ok(Json(table_fragments)) + Ok(Json(relation_id_infos)) } pub async fn list_fragments_by_job_id( @@ -492,7 +521,7 @@ impl DashboardService { .route("/clusters/:ty", get(list_clusters)) .route("/fragments2", get(list_fragments)) .route("/fragments/job_id/:job_id", get(list_fragments_by_job_id)) - .route("/fragments/ids", get(list_fragment_ids)) + .route("/relation_id_infos", get(get_relation_id_infos)) .route("/views", get(list_views)) .route("/materialized_views", get(list_materialized_views)) .route("/tables", get(list_tables)) From 10364e39915c3ac59c9242c29eecea6e19fcb83f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Aug 2024 18:36:46 +0800 Subject: [PATCH 05/10] fix endpoint --- dashboard/lib/api/streaming.ts | 6 +++--- dashboard/pages/fragment_graph.tsx | 12 +++++++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/dashboard/lib/api/streaming.ts b/dashboard/lib/api/streaming.ts index 3639c61d83b0e..7613a597f378a 100644 --- a/dashboard/lib/api/streaming.ts +++ b/dashboard/lib/api/streaming.ts @@ -27,7 +27,7 @@ import { View, } from "../../proto/gen/catalog" import { - ListObjectDependenciesResponse_ObjectDependencies as ObjectDependencies, + ListObjectDependenciesResponse_ObjectDependencies as ObjectDependencies, RelationIdInfos, TableFragments, } from "../../proto/gen/meta" import { ColumnCatalog, Field } from "../../proto/gen/plan_common" @@ -43,8 +43,8 @@ export async function getFragmentsByJobId(jobId: number): Promise>> { - let fragmentIds: Map> = (await api.get("/fragments/ids")) +export async function getRelationIdInfos(): Promise { + let fragmentIds: RelationIdInfos = (await api.get("/relation_id_infos")) return fragmentIds } diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 3eaf7d7f00bcd..5b73cbb39958d 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -45,7 +45,13 @@ import { fetchEmbeddedBackPressure, fetchPrometheusBackPressure, } from "../lib/api/metric" -import { getFragments, getFragmentIds, getFragmentsByJobId, getStreamingJobs } from "../lib/api/streaming" +import { + getFragments, + getFragmentIds, + getFragmentsByJobId, + getStreamingJobs, + getRelationIdInfos +} from "../lib/api/streaming" import { FragmentBox } from "../lib/layout" import { TableFragments, TableFragments_Fragment } from "../proto/gen/meta" import { Dispatcher, MergeNode, StreamNode } from "../proto/gen/stream_plan" @@ -194,9 +200,9 @@ interface EmbeddedBackPressureInfo { export default function Streaming() { const { response: relationList } = useFetch(getStreamingJobs) - // const { response: fragmentList } = useFetch(getFragments) const { response: fragmentList } = useFetch(getFragments) - const { response: relationIdToFragmentIds } = useFetch(getFragmentIds) + const { response: fragmentList } = useFetch(getFragments) + const { response: relationIdInfos } = useFetch(getRelationIdInfos) const [relationId, setRelationId] = useQueryState("id", parseAsInteger) const [selectedFragmentId, setSelectedFragmentId] = useState() From 185f6c984bfec1ea673e31b9a7cef1ed96d31b68 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Aug 2024 19:07:38 +0800 Subject: [PATCH 06/10] use relationIdInfos to search --- dashboard/pages/fragment_graph.tsx | 39 ++++++++++++++---------------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 5b73cbb39958d..6b8f8b89e8cbe 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -200,8 +200,6 @@ interface EmbeddedBackPressureInfo { export default function Streaming() { const { response: relationList } = useFetch(getStreamingJobs) - const { response: fragmentList } = useFetch(getFragments) - const { response: fragmentList } = useFetch(getFragments) const { response: relationIdInfos } = useFetch(getRelationIdInfos) const [relationId, setRelationId] = useQueryState("id", parseAsInteger) @@ -267,39 +265,38 @@ export default function Streaming() { const handleSearchFragment = () => { const searchFragIdInt = parseInt(searchFragId) - if (fragmentList) { - for (const tf of fragmentList) { - for (const fragmentId in tf.fragments) { - const fragment = tf.fragments[fragmentId] - if (fragment.fragmentId == searchFragIdInt) { - setRelationId(tf.tableId) - setSelectedFragmentId(fragment.fragmentId) + if (relationIdInfos) { + let map = relationIdInfos.map + for (const relationId in map) { + const fragmentIdToRelationId = map[relationId].map + for (const fragmentId in fragmentIdToRelationId) { + if (parseInt(fragmentId) == searchFragIdInt) { + setRelationId(parseInt(relationId)) + setSelectedFragmentId(searchFragIdInt) return } } } } - toast(new Error(`Fragment ${searchFragIdInt} not found`)) } const handleSearchActor = () => { const searchActorIdInt = parseInt(searchActorId) - if (fragmentList) { - for (const tf of fragmentList) { - for (const fragmentId in tf.fragments) { - const fragment = tf.fragments[fragmentId] - for (const actor of fragment.actors) { - if (actor.actorId == searchActorIdInt) { - setRelationId(tf.tableId) - setSelectedFragmentId(fragment.fragmentId) - return - } + if (relationIdInfos) { + let map = relationIdInfos.map + for (const relationId in map) { + const fragmentIdToRelationId = map[relationId].map + for (const fragmentId in fragmentIdToRelationId) { + let actorIds = fragmentIdToRelationId[fragmentId].ids + if (actorIds.includes(searchActorIdInt)) { + setRelationId(parseInt(relationId)) + setSelectedFragmentId(fragmentId) + return } } } } - toast(new Error(`Actor ${searchActorIdInt} not found`)) } From e5a29276126c9c617db7f5703d3359092a9046d3 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Aug 2024 19:17:20 +0800 Subject: [PATCH 07/10] fmt --- dashboard/lib/api/streaming.ts | 14 +++++++++----- dashboard/pages/fragment_graph.tsx | 6 ++---- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/dashboard/lib/api/streaming.ts b/dashboard/lib/api/streaming.ts index 7613a597f378a..95ad89fdf5c58 100644 --- a/dashboard/lib/api/streaming.ts +++ b/dashboard/lib/api/streaming.ts @@ -27,7 +27,8 @@ import { View, } from "../../proto/gen/catalog" import { - ListObjectDependenciesResponse_ObjectDependencies as ObjectDependencies, RelationIdInfos, + ListObjectDependenciesResponse_ObjectDependencies as ObjectDependencies, + RelationIdInfos, TableFragments, } from "../../proto/gen/meta" import { ColumnCatalog, Field } from "../../proto/gen/plan_common" @@ -36,15 +37,18 @@ import api from "./api" // NOTE(kwannoel): This can be optimized further, instead of fetching the entire TableFragments struct, // We can fetch the fields we need from TableFragments, in a truncated struct. -export async function getFragmentsByJobId(jobId: number): Promise { +export async function getFragmentsByJobId( + jobId: number +): Promise { let route = "/fragments/job_id/" + jobId.toString() - console.log("route: ", route) - let tableFragments: TableFragments = TableFragments.fromJSON(await api.get(route)) + let tableFragments: TableFragments = TableFragments.fromJSON( + await api.get(route) + ) return tableFragments } export async function getRelationIdInfos(): Promise { - let fragmentIds: RelationIdInfos = (await api.get("/relation_id_infos")) + let fragmentIds: RelationIdInfos = await api.get("/relation_id_infos") return fragmentIds } diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 6b8f8b89e8cbe..345589490a28d 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -46,11 +46,9 @@ import { fetchPrometheusBackPressure, } from "../lib/api/metric" import { - getFragments, - getFragmentIds, getFragmentsByJobId, + getRelationIdInfos, getStreamingJobs, - getRelationIdInfos } from "../lib/api/streaming" import { FragmentBox } from "../lib/layout" import { TableFragments, TableFragments_Fragment } from "../proto/gen/meta" @@ -210,7 +208,7 @@ export default function Streaming() { useEffect(() => { if (relationId) { - getFragmentsByJobId(relationId).then(tf => { + getFragmentsByJobId(relationId).then((tf) => { setTableFragments(tf) }) } From 64a8cca3407decdd13545644ed9bfc04c6f12c10 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Aug 2024 19:32:26 +0800 Subject: [PATCH 08/10] fix --- dashboard/pages/fragment_graph.tsx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 345589490a28d..50931aaa64737 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -289,7 +289,7 @@ export default function Streaming() { let actorIds = fragmentIdToRelationId[fragmentId].ids if (actorIds.includes(searchActorIdInt)) { setRelationId(parseInt(relationId)) - setSelectedFragmentId(fragmentId) + setSelectedFragmentId(parseInt(fragmentId)) return } } From e145467e79fa1ecf372498187d353a4b88157976 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 27 Aug 2024 19:58:16 +0800 Subject: [PATCH 09/10] fix newline --- proto/meta.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/proto/meta.proto b/proto/meta.proto index b5ebb0ae01336..953fd29d54b4b 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -787,4 +787,4 @@ message FragmentIdToActorIdMap { message RelationIdInfos { // relation_id -> FragmentIdToActorIdMap map map = 1; -} \ No newline at end of file +} From 165bdaaa902b9c1526a26612aee5f83d7b79db9e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 28 Aug 2024 15:58:21 +0800 Subject: [PATCH 10/10] set to undefined while waiting to fetch new graph --- dashboard/pages/fragment_graph.tsx | 1 + 1 file changed, 1 insertion(+) diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 50931aaa64737..0cf4ccec94f1b 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -208,6 +208,7 @@ export default function Streaming() { useEffect(() => { if (relationId) { + setTableFragments(undefined) getFragmentsByJobId(relationId).then((tf) => { setTableFragments(tf) })