diff --git a/e2e_test/ddl/search_path.slt b/e2e_test/ddl/search_path.slt
index 06db7f3f45c90..e01ed5d602936 100644
--- a/e2e_test/ddl/search_path.slt
+++ b/e2e_test/ddl/search_path.slt
@@ -76,6 +76,11 @@ select a from test order by a;
 1
 2
 
+# Issue #15195
+# index shall be created in `search_path_test2` (same as table) rather than `search_path_test1` (first in path)
+statement ok
+create index if not exists index1_test_a on test(a);
+
 statement ok
 drop table test;
 
diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs
index 7fc3641770a9e..30849c40d4f84 100644
--- a/src/compute/tests/cdc_tests.rs
+++ b/src/compute/tests/cdc_tests.rs
@@ -44,31 +44,22 @@ use risingwave_stream::error::StreamResult;
 use risingwave_stream::executor::monitor::StreamingMetrics;
 use risingwave_stream::executor::test_utils::MockSource;
 use risingwave_stream::executor::{
-    expect_first_barrier, ActorContext, AddMutation, Barrier, BoxedExecutor as StreamBoxedExecutor,
-    BoxedMessageStream, CdcBackfillExecutor, Executor, ExecutorInfo, ExternalStorageTable,
-    MaterializeExecutor, Message, Mutation, PkIndices, PkIndicesRef, StreamExecutorError,
+    expect_first_barrier, ActorContext, AddMutation, Barrier, BoxedMessageStream,
+    CdcBackfillExecutor, Execute, Executor as StreamExecutor, ExecutorInfo, ExternalStorageTable,
+    MaterializeExecutor, Message, Mutation, StreamExecutorError,
 };
 
 // mock upstream binlog offset starting from "1.binlog, pos=0"
 pub struct MockOffsetGenExecutor {
-    upstream: Option<StreamBoxedExecutor>,
-
-    schema: Schema,
-
-    pk_indices: PkIndices,
-
-    identity: String,
+    upstream: Option<StreamExecutor>,
 
     start_offset: u32,
 }
 
 impl MockOffsetGenExecutor {
-    pub fn new(upstream: StreamBoxedExecutor, schema: Schema, pk_indices: PkIndices) -> Self {
+    pub fn new(upstream: StreamExecutor) -> Self {
         Self {
             upstream: Some(upstream),
-            schema,
-            pk_indices,
-            identity: "MockOffsetGenExecutor".to_string(),
             start_offset: 0,
         }
     }
@@ -131,22 +122,10 @@ impl MockOffsetGenExecutor {
     }
 }
 
-impl Executor for MockOffsetGenExecutor {
+impl Execute for MockOffsetGenExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.identity
-    }
 }
 
 #[tokio::test]
@@ -154,21 +133,26 @@ async fn test_cdc_backfill() -> StreamResult<()> {
     use risingwave_common::types::DataType;
     let memory_state_store = MemoryStateStore::new();
 
-    let table_id = TableId::new(1002);
-    let schema = Schema::new(vec![
-        Field::unnamed(DataType::Jsonb),   // payload
-        Field::unnamed(DataType::Varchar), // _rw_offset
-    ]);
-    let column_ids = vec![0.into(), 1.into()];
-
-    let pk_indices = vec![0];
-
-    let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone());
-    let _actor_ctx = ActorContext::for_test(0x3a3a3a);
+    let (mut tx, source) = MockSource::channel();
+    let source = source.into_executor(
+        Schema::new(vec![
+            Field::unnamed(DataType::Jsonb), // payload
+        ]),
+        vec![0],
+    );
 
     // mock upstream offset (start from "1.binlog, pos=0") for ingested chunks
-    let mock_offset_executor =
-        MockOffsetGenExecutor::new(Box::new(source), schema.clone(), pk_indices.clone());
+    let mock_offset_executor = StreamExecutor::new(
+        ExecutorInfo {
+            schema: Schema::new(vec![
+                Field::unnamed(DataType::Jsonb),   // payload
+                Field::unnamed(DataType::Varchar), // _rw_offset
+            ]),
+            pk_indices: vec![0],
+            identity: "MockOffsetGenExecutor".to_string(),
+        },
+        MockOffsetGenExecutor::new(source).boxed(),
+    );
 
     let binlog_file = String::from("1.binlog");
     // mock binlog watermarks for backfill
@@ -188,13 +172,15 @@ async fn test_cdc_backfill() -> StreamResult<()> {
         Field::with_name(DataType::Int64, "id"), // primary key
         Field::with_name(DataType::Float64, "price"),
     ]);
+    let table_pk_indices = vec![0];
+    let table_pk_order_types = vec![OrderType::ascending()];
     let external_table = ExternalStorageTable::new(
-        table_id,
+        TableId::new(1234),
         table_name,
         ExternalTableReaderImpl::Mock(MockExternalTableReader::new(binlog_watermarks)),
         table_schema.clone(),
-        vec![OrderType::ascending()],
-        pk_indices,
+        table_pk_order_types,
+        table_pk_indices.clone(),
         vec![0, 1],
     );
 
@@ -224,32 +210,35 @@ async fn test_cdc_backfill() -> StreamResult<()> {
         vec![0_usize],
     )
     .await;
-    let info = ExecutorInfo {
-        schema: table_schema.clone(),
-        pk_indices: vec![0],
-        identity: "CdcBackfillExecutor".to_string(),
-    };
-    let cdc_backfill = CdcBackfillExecutor::new(
-        ActorContext::for_test(actor_id),
-        info,
-        external_table,
-        Box::new(mock_offset_executor),
-        vec![0, 1],
-        None,
-        Arc::new(StreamingMetrics::unused()),
-        state_table,
-        4, // 4 rows in a snapshot chunk
-        false,
+
+    let cdc_backfill = StreamExecutor::new(
+        ExecutorInfo {
+            schema: table_schema.clone(),
+            pk_indices: table_pk_indices,
+            identity: "CdcBackfillExecutor".to_string(),
+        },
+        CdcBackfillExecutor::new(
+            ActorContext::for_test(actor_id),
+            external_table,
+            mock_offset_executor,
+            vec![0, 1],
+            None,
+            Arc::new(StreamingMetrics::unused()),
+            state_table,
+            4, // 4 rows in a snapshot chunk
+            false,
+        )
+        .boxed(),
     );
 
     // Create a `MaterializeExecutor` to write the changes to storage.
+    let materialize_table_id = TableId::new(5678);
     let mut materialize = MaterializeExecutor::for_test(
-        Box::new(cdc_backfill),
+        cdc_backfill,
         memory_state_store.clone(),
-        table_id,
+        materialize_table_id,
         vec![ColumnOrder::new(0, OrderType::ascending())],
-        column_ids.clone(),
-        4,
+        vec![0.into(), 1.into()],
         Arc::new(AtomicU64::new(0)),
         ConflictBehavior::Overwrite,
     )
@@ -354,7 +343,7 @@ async fn test_cdc_backfill() -> StreamResult<()> {
     // Since we have not polled `Materialize`, we cannot scan anything from this table
     let table = StorageTable::for_test(
         memory_state_store.clone(),
-        table_id,
+        materialize_table_id,
         column_descs.clone(),
         vec![OrderType::ascending()],
         vec![0],
diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs
index 21fbe90b514e0..5e9fb3dfea86a 100644
--- a/src/compute/tests/integration_tests.rs
+++ b/src/compute/tests/integration_tests.rs
@@ -56,7 +56,7 @@ use risingwave_stream::executor::monitor::StreamingMetrics;
 use risingwave_stream::executor::row_id_gen::RowIdGenExecutor;
 use risingwave_stream::executor::source_executor::SourceExecutor;
 use risingwave_stream::executor::{
-    ActorContext, Barrier, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices,
+    ActorContext, Barrier, Execute, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices,
 };
 use tokio::sync::mpsc::unbounded_channel;
 
@@ -162,56 +162,58 @@ async fn test_table_materialize() -> StreamResult<()> {
     let system_params_manager = LocalSystemParamsManager::for_test();
 
     // Create a `SourceExecutor` to read the changes.
-    let source_executor = SourceExecutor::<PanicStateStore>::new(
-        actor_ctx.clone(),
+    let source_executor = Executor::new(
         ExecutorInfo {
             schema: all_schema.clone(),
             pk_indices: pk_indices.clone(),
             identity: format!("SourceExecutor {:X}", 1),
         },
-        None, // There is no external stream source.
-        Arc::new(StreamingMetrics::unused()),
-        barrier_rx,
-        system_params_manager.get_params(),
-        SourceCtrlOpts::default(),
-        ConnectorParams::default(),
+        SourceExecutor::<PanicStateStore>::new(
+            actor_ctx.clone(),
+            None, // There is no external stream source.
+            Arc::new(StreamingMetrics::unused()),
+            barrier_rx,
+            system_params_manager.get_params(),
+            SourceCtrlOpts::default(),
+            ConnectorParams::default(),
+        )
+        .boxed(),
     );
 
     // Create a `DmlExecutor` to accept data change from users.
-    let dml_executor = DmlExecutor::new(
+    let dml_executor = Executor::new(
         ExecutorInfo {
             schema: all_schema.clone(),
             pk_indices: pk_indices.clone(),
             identity: format!("DmlExecutor {:X}", 2),
         },
-        Box::new(source_executor),
-        dml_manager.clone(),
-        table_id,
-        INITIAL_TABLE_VERSION_ID,
-        column_descs.clone(),
-        1024,
+        DmlExecutor::new(
+            source_executor,
+            dml_manager.clone(),
+            table_id,
+            INITIAL_TABLE_VERSION_ID,
+            column_descs.clone(),
+            1024,
+        )
+        .boxed(),
     );
 
-    let row_id_gen_executor = RowIdGenExecutor::new(
-        actor_ctx,
+    let row_id_gen_executor = Executor::new(
         ExecutorInfo {
             schema: all_schema.clone(),
             pk_indices: pk_indices.clone(),
             identity: format!("RowIdGenExecutor {:X}", 3),
         },
-        Box::new(dml_executor),
-        row_id_index,
-        vnodes,
+        RowIdGenExecutor::new(actor_ctx, dml_executor, row_id_index, vnodes).boxed(),
     );
 
     // Create a `MaterializeExecutor` to write the changes to storage.
     let mut materialize = MaterializeExecutor::for_test(
-        Box::new(row_id_gen_executor),
+        row_id_gen_executor,
         memory_state_store.clone(),
         table_id,
         vec![ColumnOrder::new(0, OrderType::ascending())],
         all_column_ids.clone(),
-        4,
         Arc::new(AtomicU64::new(0)),
         ConflictBehavior::NoCheck,
     )
diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs
index af9d9f52b1752..c687f56fad8e8 100644
--- a/src/frontend/src/handler/create_index.rs
+++ b/src/frontend/src/handler/create_index.rs
@@ -45,15 +45,11 @@ use crate::session::SessionImpl;
 use crate::stream_fragmenter::build_graph;
 use crate::TableCatalog;
 
-pub(crate) fn gen_create_index_plan(
+pub(crate) fn resolve_index_schema(
     session: &SessionImpl,
-    context: OptimizerContextRef,
     index_name: ObjectName,
     table_name: ObjectName,
-    columns: Vec<OrderByExpr>,
-    include: Vec<Ident>,
-    distributed_by: Vec<ast::Expr>,
-) -> Result<(PlanRef, PbTable, PbIndex)> {
+) -> Result<(String, Arc<TableCatalog>, String)> {
     let db_name = session.database();
     let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, table_name)?;
     let search_path = session.config().search_path();
@@ -63,12 +59,22 @@ pub(crate) fn gen_create_index_plan(
     let index_table_name = Binder::resolve_index_name(index_name)?;
 
     let catalog_reader = session.env().catalog_reader();
-    let (table, schema_name) = {
-        let read_guard = catalog_reader.read_guard();
-        let (table, schema_name) =
-            read_guard.get_table_by_name(db_name, schema_path, &table_name)?;
-        (table.clone(), schema_name.to_string())
-    };
+    let read_guard = catalog_reader.read_guard();
+    let (table, schema_name) = read_guard.get_table_by_name(db_name, schema_path, &table_name)?;
+    Ok((schema_name.to_string(), table.clone(), index_table_name))
+}
+
+pub(crate) fn gen_create_index_plan(
+    session: &SessionImpl,
+    context: OptimizerContextRef,
+    schema_name: String,
+    table: Arc<TableCatalog>,
+    index_table_name: String,
+    columns: Vec<OrderByExpr>,
+    include: Vec<Ident>,
+    distributed_by: Vec<ast::Expr>,
+) -> Result<(PlanRef, PbTable, PbIndex)> {
+    let table_name = table.name.clone();
 
     if table.is_index() {
         return Err(
@@ -404,22 +410,27 @@ pub async fn handle_create_index(
     let session = handler_args.session.clone();
 
     let (graph, index_table, index) = {
-        {
-            if let Either::Right(resp) = session.check_relation_name_duplicated(
-                index_name.clone(),
-                StatementType::CREATE_INDEX,
-                if_not_exists,
-            )? {
-                return Ok(resp);
-            }
+        let (schema_name, table, index_table_name) =
+            resolve_index_schema(&session, index_name, table_name)?;
+        let qualified_index_name = ObjectName(vec![
+            Ident::with_quote_unchecked('"', &schema_name),
+            Ident::with_quote_unchecked('"', &index_table_name),
+        ]);
+        if let Either::Right(resp) = session.check_relation_name_duplicated(
+            qualified_index_name,
+            StatementType::CREATE_INDEX,
+            if_not_exists,
+        )? {
+            return Ok(resp);
         }
 
         let context = OptimizerContext::from_handler_args(handler_args);
         let (plan, index_table, index) = gen_create_index_plan(
             &session,
             context.into(),
-            index_name.clone(),
-            table_name,
+            schema_name,
+            table,
+            index_table_name,
             columns,
             include,
             distributed_by,
@@ -437,7 +448,7 @@ pub async fn handle_create_index(
 
     tracing::trace!(
         "name={}, graph=\n{}",
-        index_name,
+        index.name,
         serde_json::to_string_pretty(&graph).unwrap()
     );
 
diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs
index b966cca8f50cf..b46882156a24c 100644
--- a/src/frontend/src/handler/explain.rs
+++ b/src/frontend/src/handler/explain.rs
@@ -18,7 +18,7 @@ use risingwave_common::types::Fields;
 use risingwave_sqlparser::ast::{ExplainOptions, ExplainType, Statement};
 use thiserror_ext::AsReport;
 
-use super::create_index::gen_create_index_plan;
+use super::create_index::{gen_create_index_plan, resolve_index_schema};
 use super::create_mv::gen_create_mv_plan;
 use super::create_sink::{gen_sink_plan, get_partition_compute_info};
 use super::create_table::ColumnIdGenerator;
@@ -133,15 +133,20 @@ async fn do_handle_explain(
                         include,
                         distributed_by,
                         ..
-                    } => gen_create_index_plan(
-                        &session,
-                        context.clone(),
-                        name,
-                        table_name,
-                        columns,
-                        include,
-                        distributed_by,
-                    )
+                    } => {
+                        let (schema_name, table, index_table_name) =
+                            resolve_index_schema(&session, name, table_name)?;
+                        gen_create_index_plan(
+                            &session,
+                            context.clone(),
+                            schema_name,
+                            table,
+                            index_table_name,
+                            columns,
+                            include,
+                            distributed_by,
+                        )
+                    }
                     .map(|x| x.0),
 
                     // -- Batch Queries --
diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs
index 06710d42d9d25..0039b9cccc2e0 100644
--- a/src/meta/model_v2/src/table.rs
+++ b/src/meta/model_v2/src/table.rs
@@ -24,7 +24,7 @@ use crate::{
     TableId, TableVersion,
 };
 
-#[derive(Clone, Debug, PartialEq, Copy, Eq, EnumIter, DeriveActiveEnum)]
+#[derive(Clone, Debug, PartialEq, Hash, Copy, Eq, EnumIter, DeriveActiveEnum)]
 #[sea_orm(rs_type = "String", db_type = "String(None)")]
 pub enum TableType {
     #[sea_orm(string_value = "TABLE")]
diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs
index 4792293863237..3397317fe1939 100644
--- a/src/meta/node/src/server.rs
+++ b/src/meta/node/src/server.rs
@@ -478,20 +478,13 @@ pub async fn start_service_as_election_leader(
         prometheus_http_query::Client::from_str(x).unwrap()
     });
     let prometheus_selector = opts.prometheus_selector.unwrap_or_default();
-    let diagnose_command = match &metadata_manager {
-        MetadataManager::V1(mgr) => Some(Arc::new(
-            risingwave_meta::manager::diagnose::DiagnoseCommand::new(
-                mgr.cluster_manager.clone(),
-                mgr.catalog_manager.clone(),
-                mgr.fragment_manager.clone(),
-                hummock_manager.clone(),
-                env.event_log_manager_ref(),
-                prometheus_client.clone(),
-                prometheus_selector.clone(),
-            ),
-        )),
-        MetadataManager::V2(_) => None,
-    };
+    let diagnose_command = Arc::new(risingwave_meta::manager::diagnose::DiagnoseCommand::new(
+        metadata_manager.clone(),
+        hummock_manager.clone(),
+        env.event_log_manager_ref(),
+        prometheus_client.clone(),
+        prometheus_selector.clone(),
+    ));
 
     let trace_state = otlp_embedded::State::new(otlp_embedded::Config {
         max_length: opts.cached_traces_num,
diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs
index a1efaa756bb44..fae19874ab407 100644
--- a/src/meta/src/controller/catalog.rs
+++ b/src/meta/src/controller/catalog.rs
@@ -2446,6 +2446,18 @@ impl CatalogController {
     }
 }
 
+/// `CatalogStats` is a struct to store the statistics of all catalogs.
+pub struct CatalogStats {
+    pub table_num: u64,
+    pub mview_num: u64,
+    pub index_num: u64,
+    pub source_num: u64,
+    pub sink_num: u64,
+    pub function_num: u64,
+    pub streaming_job_num: u64,
+    pub actor_num: u64,
+}
+
 impl CatalogControllerInner {
     pub async fn snapshot(&self) -> MetaResult<(Catalog, Vec<PbUserInfo>)> {
         let databases = self.list_databases().await?;
@@ -2476,6 +2488,40 @@ impl CatalogControllerInner {
         ))
     }
 
+    pub async fn stats(&self) -> MetaResult<CatalogStats> {
+        let mut table_num_map: HashMap<_, _> = Table::find()
+            .select_only()
+            .column(table::Column::TableType)
+            .column_as(table::Column::TableId.count(), "num")
+            .group_by(table::Column::TableType)
+            .having(table::Column::TableType.ne(TableType::Internal))
+            .into_tuple::<(TableType, i64)>()
+            .all(&self.db)
+            .await?
+            .into_iter()
+            .map(|(table_type, num)| (table_type, num as u64))
+            .collect();
+
+        let source_num = Source::find().count(&self.db).await?;
+        let sink_num = Sink::find().count(&self.db).await?;
+        let function_num = Function::find().count(&self.db).await?;
+        let streaming_job_num = StreamingJob::find().count(&self.db).await?;
+        let actor_num = Actor::find().count(&self.db).await?;
+
+        Ok(CatalogStats {
+            table_num: table_num_map.remove(&TableType::Table).unwrap_or(0),
+            mview_num: table_num_map
+                .remove(&TableType::MaterializedView)
+                .unwrap_or(0),
+            index_num: table_num_map.remove(&TableType::Index).unwrap_or(0),
+            source_num,
+            sink_num,
+            function_num,
+            streaming_job_num,
+            actor_num,
+        })
+    }
+
     async fn list_databases(&self) -> MetaResult<Vec<PbDatabase>> {
         let db_objs = Database::find()
             .find_also_related(Object)
diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs
index 833e642a83e74..c9beada284d5d 100644
--- a/src/meta/src/controller/fragment.rs
+++ b/src/meta/src/controller/fragment.rs
@@ -702,6 +702,23 @@ impl CatalogController {
         Ok(count > 0)
     }
 
+    pub async fn worker_actor_count(&self) -> MetaResult<HashMap<WorkerId, usize>> {
+        let inner = self.inner.read().await;
+        let actor_cnt: Vec<(WorkerId, i64)> = Actor::find()
+            .select_only()
+            .column(actor::Column::WorkerId)
+            .column_as(actor::Column::ActorId.count(), "count")
+            .group_by(actor::Column::WorkerId)
+            .into_tuple()
+            .all(&inner.db)
+            .await?;
+
+        Ok(actor_cnt
+            .into_iter()
+            .map(|(worker_id, count)| (worker_id, count as usize))
+            .collect())
+    }
+
     // TODO: This function is too heavy, we should avoid using it and implement others on demand.
     pub async fn table_fragments(&self) -> MetaResult<BTreeMap<ObjectId, PbTableFragments>> {
         let inner = self.inner.read().await;
diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs
index b5104e557a1b2..b4ae8e2c04b36 100644
--- a/src/meta/src/dashboard/mod.rs
+++ b/src/meta/src/dashboard/mod.rs
@@ -47,7 +47,7 @@ pub struct DashboardService {
     pub metadata_manager: MetadataManager,
     pub compute_clients: ComputeClientPool,
     pub ui_path: Option<String>,
-    pub diagnose_command: Option<DiagnoseCommandRef>,
+    pub diagnose_command: DiagnoseCommandRef,
     pub trace_state: otlp_embedded::StateRef,
 }
 
@@ -355,13 +355,7 @@ pub(super) mod handlers {
     }
 
     pub async fn diagnose(Extension(srv): Extension<Service>) -> Result<String> {
-        let report = if let Some(cmd) = &srv.diagnose_command {
-            cmd.report().await
-        } else {
-            "Not supported in sql-backend".to_string()
-        };
-
-        Ok(report)
+        Ok(srv.diagnose_command.report().await)
     }
 
     pub async fn get_back_pressure(
diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs
index fdae16efe5a7b..ecfac68833d0e 100644
--- a/src/meta/src/manager/catalog/fragment.rs
+++ b/src/meta/src/manager/catalog/fragment.rs
@@ -858,6 +858,22 @@ impl FragmentManager {
         actor_maps
     }
 
+    pub async fn node_actor_count(&self) -> HashMap<WorkerId, usize> {
+        let mut actor_count = HashMap::new();
+
+        let map = &self.core.read().await.table_fragments;
+        for fragments in map.values() {
+            for actor_status in fragments.actor_status.values() {
+                if let Some(pu) = &actor_status.parallel_unit {
+                    let e = actor_count.entry(pu.worker_node_id).or_insert(0);
+                    *e += 1;
+                }
+            }
+        }
+
+        actor_count
+    }
+
     // edit the `rate_limit` of the `Source` node in given `source_id`'s fragments
     // return the actor_ids to be applied
     pub async fn update_source_rate_limit_by_source_id(
diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs
index a8c1d050ab748..76184de9eb7ce 100644
--- a/src/meta/src/manager/diagnose.rs
+++ b/src/meta/src/manager/diagnose.rs
@@ -13,7 +13,7 @@
 // limitations under the License.
 
 use std::cmp::{Ordering, Reverse};
-use std::collections::{BinaryHeap, HashMap};
+use std::collections::BinaryHeap;
 use std::fmt::Write;
 use std::sync::Arc;
 
@@ -27,17 +27,16 @@ use risingwave_pb::meta::EventLog;
 use risingwave_pb::monitor_service::StackTraceResponse;
 use risingwave_rpc_client::ComputeClientPool;
 use serde_json::json;
+use thiserror_ext::AsReport;
 
 use crate::hummock::HummockManagerRef;
 use crate::manager::event_log::EventLogMangerRef;
-use crate::manager::{CatalogManagerRef, ClusterManagerRef, FragmentManagerRef};
+use crate::manager::MetadataManager;
 
 pub type DiagnoseCommandRef = Arc<DiagnoseCommand>;
 
 pub struct DiagnoseCommand {
-    cluster_manager: ClusterManagerRef,
-    catalog_manager: CatalogManagerRef,
-    fragment_manager: FragmentManagerRef,
+    metadata_manager: MetadataManager,
     hummock_manger: HummockManagerRef,
     event_log_manager: EventLogMangerRef,
     prometheus_client: Option<prometheus_http_query::Client>,
@@ -46,18 +45,14 @@ pub struct DiagnoseCommand {
 
 impl DiagnoseCommand {
     pub fn new(
-        cluster_manager: ClusterManagerRef,
-        catalog_manager: CatalogManagerRef,
-        fragment_manager: FragmentManagerRef,
+        metadata_manager: MetadataManager,
         hummock_manger: HummockManagerRef,
         event_log_manager: EventLogMangerRef,
         prometheus_client: Option<prometheus_http_query::Client>,
         prometheus_selector: String,
     ) -> Self {
         Self {
-            cluster_manager,
-            catalog_manager,
-            fragment_manager,
+            metadata_manager,
             hummock_manger,
             event_log_manager,
             prometheus_client,
@@ -90,49 +85,60 @@ impl DiagnoseCommand {
 
     #[cfg_attr(coverage, coverage(off))]
     async fn write_catalog(&self, s: &mut String) {
+        match &self.metadata_manager {
+            MetadataManager::V1(_) => self.write_catalog_v1(s).await,
+            MetadataManager::V2(_) => self.write_catalog_v2(s).await,
+        }
+    }
+
+    #[cfg_attr(coverage, coverage(off))]
+    async fn write_catalog_v1(&self, s: &mut String) {
+        let mgr = self.metadata_manager.as_v1_ref();
         let _ = writeln!(s, "number of fragment: {}", self.fragment_num().await);
         let _ = writeln!(s, "number of actor: {}", self.actor_num().await);
         let _ = writeln!(
             s,
             "number of source: {}",
-            self.catalog_manager.source_count().await
+            mgr.catalog_manager.source_count().await
         );
         let _ = writeln!(
             s,
             "number of table: {}",
-            self.catalog_manager.table_count().await
+            mgr.catalog_manager.table_count().await
         );
         let _ = writeln!(
             s,
             "number of materialized view: {}",
-            self.catalog_manager.materialized_view_count().await
+            mgr.catalog_manager.materialized_view_count().await
         );
         let _ = writeln!(
             s,
             "number of sink: {}",
-            self.catalog_manager.sink_count().await
+            mgr.catalog_manager.sink_count().await
         );
         let _ = writeln!(
             s,
             "number of index: {}",
-            self.catalog_manager.index_count().await
+            mgr.catalog_manager.index_count().await
         );
         let _ = writeln!(
             s,
             "number of function: {}",
-            self.catalog_manager.function_count().await
+            mgr.catalog_manager.function_count().await
         );
     }
 
     #[cfg_attr(coverage, coverage(off))]
     async fn fragment_num(&self) -> usize {
-        let core = self.fragment_manager.get_fragment_read_guard().await;
+        let mgr = self.metadata_manager.as_v1_ref();
+        let core = mgr.fragment_manager.get_fragment_read_guard().await;
         core.table_fragments().len()
     }
 
     #[cfg_attr(coverage, coverage(off))]
     async fn actor_num(&self) -> usize {
-        let core = self.fragment_manager.get_fragment_read_guard().await;
+        let mgr = self.metadata_manager.as_v1_ref();
+        let core = mgr.fragment_manager.get_fragment_read_guard().await;
         core.table_fragments()
             .values()
             .map(|t| t.actor_status.len())
@@ -140,25 +146,38 @@ impl DiagnoseCommand {
     }
 
     #[cfg_attr(coverage, coverage(off))]
-    async fn write_worker_nodes(&self, s: &mut String) {
-        let mut worker_actor_count: HashMap<u32, usize> = HashMap::new();
-        for f in self
-            .fragment_manager
-            .get_fragment_read_guard()
-            .await
-            .table_fragments()
-            .values()
-        {
-            for a in f.actor_status.values() {
-                if let Some(pu) = &a.parallel_unit {
-                    let e = worker_actor_count.entry(pu.worker_node_id).or_insert(0);
-                    *e += 1;
-                }
+    async fn write_catalog_v2(&self, s: &mut String) {
+        let mgr = self.metadata_manager.as_v2_ref();
+        let guard = mgr.catalog_controller.get_inner_read_guard().await;
+        let stat = match guard.stats().await {
+            Ok(stat) => stat,
+            Err(err) => {
+                tracing::warn!(error=?err.as_report(), "failed to get catalog stats");
+                return;
             }
-        }
+        };
+        let _ = writeln!(s, "number of fragment: {}", stat.streaming_job_num);
+        let _ = writeln!(s, "number of actor: {}", stat.actor_num);
+        let _ = writeln!(s, "number of source: {}", stat.source_num);
+        let _ = writeln!(s, "number of table: {}", stat.table_num);
+        let _ = writeln!(s, "number of materialized view: {}", stat.mview_num);
+        let _ = writeln!(s, "number of sink: {}", stat.sink_num);
+        let _ = writeln!(s, "number of index: {}", stat.index_num);
+        let _ = writeln!(s, "number of function: {}", stat.function_num);
+    }
+
+    #[cfg_attr(coverage, coverage(off))]
+    async fn write_worker_nodes(&self, s: &mut String) {
+        let Ok(worker_actor_count) = self.metadata_manager.worker_actor_count().await else {
+            tracing::warn!("failed to get worker actor count");
+            return;
+        };
 
         use comfy_table::{Row, Table};
-        let worker_nodes = self.cluster_manager.list_worker_node(None, None).await;
+        let Ok(worker_nodes) = self.metadata_manager.list_worker_node(None, None).await else {
+            tracing::warn!("failed to get worker nodes");
+            return;
+        };
         let mut table = Table::new();
         table.set_header({
             let mut row = Row::new();
@@ -636,10 +655,14 @@ impl DiagnoseCommand {
     #[cfg_attr(coverage, coverage(off))]
     async fn write_await_tree(&self, s: &mut String) {
         // Most lines of code are copied from dashboard::handlers::dump_await_tree_all, because the latter cannot be called directly from here.
-        let worker_nodes = self
-            .cluster_manager
+        let Ok(worker_nodes) = self
+            .metadata_manager
             .list_worker_node(Some(WorkerType::ComputeNode), None)
-            .await;
+            .await
+        else {
+            tracing::warn!("failed to get worker nodes");
+            return;
+        };
 
         let mut all = Default::default();
 
diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs
index eef1c0c101256..fd9b51fa8ba09 100644
--- a/src/meta/src/manager/metadata.rs
+++ b/src/meta/src/manager/metadata.rs
@@ -637,6 +637,19 @@ impl MetadataManager {
         }
     }
 
+    pub async fn worker_actor_count(&self) -> MetaResult<HashMap<WorkerId, usize>> {
+        match &self {
+            MetadataManager::V1(mgr) => Ok(mgr.fragment_manager.node_actor_count().await),
+            MetadataManager::V2(mgr) => {
+                let actor_cnt = mgr.catalog_controller.worker_actor_count().await?;
+                Ok(actor_cnt
+                    .into_iter()
+                    .map(|(id, cnt)| (id as WorkerId, cnt))
+                    .collect())
+            }
+        }
+    }
+
     pub async fn count_streaming_job(&self) -> MetaResult<usize> {
         match self {
             MetadataManager::V1(mgr) => Ok(mgr.fragment_manager.count_streaming_job().await),
diff --git a/src/stream/benches/stream_hash_agg.rs b/src/stream/benches/stream_hash_agg.rs
index 90ac7ef7725e7..2a0d1a93cc6b7 100644
--- a/src/stream/benches/stream_hash_agg.rs
+++ b/src/stream/benches/stream_hash_agg.rs
@@ -26,7 +26,7 @@ use risingwave_storage::memory::MemoryStateStore;
 use risingwave_storage::StateStore;
 use risingwave_stream::executor::test_utils::agg_executor::new_boxed_hash_agg_executor;
 use risingwave_stream::executor::test_utils::*;
-use risingwave_stream::executor::{BoxedExecutor, PkIndices};
+use risingwave_stream::executor::{Executor, PkIndices};
 use tokio::runtime::Runtime;
 
 risingwave_expr_impl::enable!();
@@ -47,7 +47,7 @@ fn bench_hash_agg(c: &mut Criterion) {
 
 /// This aims to mirror `q17`'s aggregator.
 /// We can include more executor patterns as needed.
-fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor {
+fn setup_bench_hash_agg<S: StateStore>(store: S) -> Executor {
     // ---- Define hash agg executor parameters ----
     let input_data_types = vec![
         // to_char(date_time)
@@ -119,7 +119,8 @@ fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor {
     );
 
     // ---- Create MockSourceExecutor ----
-    let (mut tx, source) = MockSource::channel(schema, PkIndices::new());
+    let (mut tx, source) = MockSource::channel();
+    let source = source.into_executor(schema, PkIndices::new());
     tx.push_barrier(1, false);
     for chunk in chunks {
         tx.push_chunk(chunk);
@@ -134,7 +135,7 @@ fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor {
 
     block_on(new_boxed_hash_agg_executor(
         store,
-        Box::new(source),
+        source,
         false,
         agg_calls,
         row_count_index,
@@ -146,7 +147,7 @@ fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor {
     ))
 }
 
-pub async fn execute_executor(executor: BoxedExecutor) {
+pub async fn execute_executor(executor: Executor) {
     let mut stream = executor.execute();
     while let Some(ret) = stream.next().await {
         _ = black_box(ret.unwrap());
diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs
index 603cb7f244f87..91c414877a76b 100644
--- a/src/stream/src/executor/agg_common.rs
+++ b/src/stream/src/executor/agg_common.rs
@@ -29,7 +29,7 @@ pub struct AggExecutorArgs<S: StateStore, E: AggExecutorExtraArgs> {
     pub version: PbAggNodeVersion,
 
     // basic
-    pub input: Box<dyn Executor>,
+    pub input: Executor,
     pub actor_ctx: ActorContextRef,
     pub info: ExecutorInfo,
 
diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs
index 0afea6720cad2..2dd277449be13 100644
--- a/src/stream/src/executor/backfill/arrangement_backfill.rs
+++ b/src/stream/src/executor/backfill/arrangement_backfill.rs
@@ -22,7 +22,6 @@ use futures_async_stream::try_stream;
 use itertools::Itertools;
 use risingwave_common::array::{DataChunk, Op, StreamChunk};
 use risingwave_common::bail;
-use risingwave_common::catalog::Schema;
 use risingwave_common::hash::{VirtualNode, VnodeBitmapExt};
 use risingwave_common::row::OwnedRow;
 use risingwave_common::util::chunk_coalesce::DataChunkBuilder;
@@ -40,8 +39,8 @@ use crate::executor::backfill::utils::{
 };
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::{
-    expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo,
-    HashMap, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult,
+    expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, HashMap, Message,
+    StreamExecutorError, StreamExecutorResult,
 };
 use crate::task::{ActorId, CreateMviewProgress};
 
@@ -57,7 +56,7 @@ pub struct ArrangementBackfillExecutor<S: StateStore, SD: ValueRowSerde> {
     upstream_table: ReplicatedStateTable<S, SD>,
 
     /// Upstream with the same schema with the upstream table.
-    upstream: BoxedExecutor,
+    upstream: Executor,
 
     /// Internal state table for persisting state of backfill state.
     state_table: StateTable<S>,
@@ -69,8 +68,6 @@ pub struct ArrangementBackfillExecutor<S: StateStore, SD: ValueRowSerde> {
 
     actor_id: ActorId,
 
-    info: ExecutorInfo,
-
     metrics: Arc<StreamingMetrics>,
 
     chunk_size: usize,
@@ -86,9 +83,8 @@ where
     #[allow(clippy::too_many_arguments)]
     #[allow(dead_code)]
     pub fn new(
-        info: ExecutorInfo,
         upstream_table: ReplicatedStateTable<S, SD>,
-        upstream: BoxedExecutor,
+        upstream: Executor,
         state_table: StateTable<S>,
         output_indices: Vec<usize>,
         progress: CreateMviewProgress,
@@ -97,7 +93,6 @@ where
         rate_limit: Option<usize>,
     ) -> Self {
         Self {
-            info,
             upstream_table,
             upstream,
             state_table,
@@ -710,7 +705,7 @@ where
     }
 }
 
-impl<S, SD> Executor for ArrangementBackfillExecutor<S, SD>
+impl<S, SD> Execute for ArrangementBackfillExecutor<S, SD>
 where
     S: StateStore,
     SD: ValueRowSerde,
@@ -718,16 +713,4 @@ where
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs
index bb99e2ecab369..e4b83622951f6 100644
--- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs
+++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs
@@ -45,8 +45,8 @@ use crate::executor::backfill::utils::{
 };
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor,
-    ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult,
+    expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message,
+    StreamExecutorError, StreamExecutorResult,
 };
 use crate::task::CreateMviewProgress;
 
@@ -55,13 +55,12 @@ const METADATA_STATE_LEN: usize = 4;
 
 pub struct CdcBackfillExecutor<S: StateStore> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// The external table to be backfilled
     external_table: ExternalStorageTable,
 
     /// Upstream changelog stream which may contain metadata columns, e.g. `_rw_offset`
-    upstream: BoxedExecutor,
+    upstream: Executor,
 
     /// The column indices need to be forwarded to the downstream from the upstream and table scan.
     /// User may select a subset of columns from the upstream table.
@@ -83,9 +82,8 @@ impl<S: StateStore> CdcBackfillExecutor<S> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         actor_ctx: ActorContextRef,
-        info: ExecutorInfo,
         external_table: ExternalStorageTable,
-        upstream: BoxedExecutor,
+        upstream: Executor,
         output_indices: Vec<usize>,
         progress: Option<CreateMviewProgress>,
         metrics: Arc<StreamingMetrics>,
@@ -95,7 +93,6 @@ impl<S: StateStore> CdcBackfillExecutor<S> {
     ) -> Self {
         Self {
             actor_ctx,
-            info,
             external_table,
             upstream,
             output_indices,
@@ -609,22 +606,10 @@ fn get_rw_columns(schema: &Schema) -> Vec<SourceColumnDesc> {
         .collect_vec()
 }
 
-impl<S: StateStore> Executor for CdcBackfillExecutor<S> {
+impl<S: StateStore> Execute for CdcBackfillExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 #[cfg(test)]
@@ -639,7 +624,6 @@ mod tests {
 
     use crate::executor::backfill::cdc::cdc_backfill::transform_upstream;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::Executor;
 
     #[tokio::test]
     async fn test_transform_upstream_chunk() {
@@ -649,7 +633,8 @@ mod tests {
             Field::unnamed(DataType::Varchar), // _rw_table_name
         ]);
         let pk_indices = vec![1];
-        let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone());
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema.clone(), pk_indices.clone());
         // let payload = r#"{"before": null,"after":{"O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" },"source":{"version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null},"op":"r","ts_ms":1695277757017,"transaction":null}"#.to_string();
         let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#;
 
diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs
index 824bc51e13b00..4aea8f63b26ea 100644
--- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs
+++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs
@@ -20,7 +20,6 @@ use futures::stream::select_with_strategy;
 use futures::{stream, StreamExt};
 use futures_async_stream::try_stream;
 use risingwave_common::array::{DataChunk, Op, StreamChunk};
-use risingwave_common::catalog::Schema;
 use risingwave_common::hash::VnodeBitmapExt;
 use risingwave_common::row::{OwnedRow, Row};
 use risingwave_common::types::Datum;
@@ -39,8 +38,8 @@ use crate::executor::backfill::utils::{
 };
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::{
-    expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo,
-    Message, Mutation, PkIndicesRef, StreamExecutorError, StreamExecutorResult,
+    expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, Message, Mutation,
+    StreamExecutorError, StreamExecutorResult,
 };
 use crate::task::{ActorId, CreateMviewProgress};
 
@@ -76,12 +75,10 @@ pub struct BackfillState {
 /// in the same worker, so that we can read uncommitted data from the upstream table without
 /// waiting.
 pub struct BackfillExecutor<S: StateStore> {
-    info: ExecutorInfo,
-
     /// Upstream table
     upstream_table: StorageTable<S>,
     /// Upstream with the same schema with the upstream table.
-    upstream: BoxedExecutor,
+    upstream: Executor,
 
     /// Internal state table for persisting state of backfill state.
     state_table: Option<StateTable<S>>,
@@ -110,9 +107,8 @@ where
 {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        info: ExecutorInfo,
         upstream_table: StorageTable<S>,
-        upstream: BoxedExecutor,
+        upstream: Executor,
         state_table: Option<StateTable<S>>,
         output_indices: Vec<usize>,
         progress: CreateMviewProgress,
@@ -122,7 +118,6 @@ where
     ) -> Self {
         let actor_id = progress.actor_id();
         Self {
-            info,
             upstream_table,
             upstream,
             state_table,
@@ -742,23 +737,11 @@ where
     }
 }
 
-impl<S> Executor for BackfillExecutor<S>
+impl<S> Execute for BackfillExecutor<S>
 where
     S: StateStore,
 {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
diff --git a/src/stream/src/executor/barrier_recv.rs b/src/stream/src/executor/barrier_recv.rs
index 66480ef1cb591..4ca07a80382a5 100644
--- a/src/stream/src/executor/barrier_recv.rs
+++ b/src/stream/src/executor/barrier_recv.rs
@@ -13,52 +13,37 @@
 // limitations under the License.
 
 use futures::StreamExt;
-use risingwave_common::catalog::Schema;
 use tokio::sync::mpsc::UnboundedReceiver;
 use tokio_stream::wrappers::UnboundedReceiverStream;
 
 use super::{
-    ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message,
-    PkIndices, PkIndicesRef, StreamExecutorError,
+    ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Execute, Message,
+    StreamExecutorError,
 };
 
 /// The executor only for receiving barrier from the meta service. It always resides in the leaves
 /// of the streaming graph.
 pub struct BarrierRecvExecutor {
     _ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// The barrier receiver registered in the local barrier manager.
     barrier_receiver: UnboundedReceiver<Barrier>,
 }
 
 impl BarrierRecvExecutor {
-    pub fn new(
-        ctx: ActorContextRef,
-        info: ExecutorInfo,
-        barrier_receiver: UnboundedReceiver<Barrier>,
-    ) -> Self {
+    pub fn new(ctx: ActorContextRef, barrier_receiver: UnboundedReceiver<Barrier>) -> Self {
         Self {
             _ctx: ctx,
-            info,
             barrier_receiver,
         }
     }
 
     pub fn for_test(barrier_receiver: UnboundedReceiver<Barrier>) -> Self {
-        Self::new(
-            ActorContext::for_test(0),
-            ExecutorInfo {
-                schema: Schema::empty().clone(),
-                pk_indices: PkIndices::new(),
-                identity: "BarrierRecvExecutor".to_string(),
-            },
-            barrier_receiver,
-        )
+        Self::new(ActorContext::for_test(0), barrier_receiver)
     }
 }
 
-impl Executor for BarrierRecvExecutor {
+impl Execute for BarrierRecvExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         UnboundedReceiverStream::new(self.barrier_receiver)
             .map(|barrier| Ok(Message::Barrier(barrier)))
@@ -69,18 +54,6 @@ impl Executor for BarrierRecvExecutor {
             }))
             .boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 #[cfg(test)]
diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs
index 533a327432f43..7c92bcd732423 100644
--- a/src/stream/src/executor/batch_query.rs
+++ b/src/stream/src/executor/batch_query.rs
@@ -24,7 +24,7 @@ use risingwave_storage::table::collect_data_chunk;
 use risingwave_storage::StateStore;
 
 use super::error::StreamExecutorError;
-use super::{Executor, ExecutorInfo, Message};
+use super::{Execute, Message};
 use crate::executor::BoxedMessageStream;
 
 pub struct BatchQueryExecutor<S: StateStore> {
@@ -34,18 +34,18 @@ pub struct BatchQueryExecutor<S: StateStore> {
     /// The number of tuples in one [`StreamChunk`]
     batch_size: usize,
 
-    info: ExecutorInfo,
+    schema: Schema,
 }
 
 impl<S> BatchQueryExecutor<S>
 where
     S: StateStore,
 {
-    pub fn new(table: StorageTable<S>, batch_size: usize, info: ExecutorInfo) -> Self {
+    pub fn new(table: StorageTable<S>, batch_size: usize, schema: Schema) -> Self {
         Self {
             table,
             batch_size,
-            info,
+            schema,
         }
     }
 
@@ -62,7 +62,7 @@ where
         pin_mut!(iter);
 
         while let Some(data_chunk) =
-            collect_data_chunk(&mut iter, self.schema(), Some(self.batch_size))
+            collect_data_chunk(&mut iter, &self.schema, Some(self.batch_size))
                 .instrument_await("batch_query_executor_collect_chunk")
                 .await?
         {
@@ -73,7 +73,7 @@ where
     }
 }
 
-impl<S> Executor for BatchQueryExecutor<S>
+impl<S> Execute for BatchQueryExecutor<S>
 where
     S: StateStore,
 {
@@ -81,18 +81,6 @@ where
         unreachable!("should call `execute_with_epoch`")
     }
 
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
     fn execute_with_epoch(self: Box<Self>, epoch: u64) -> BoxedMessageStream {
         self.execute_inner(epoch).boxed()
     }
@@ -100,9 +88,6 @@ where
 
 #[cfg(test)]
 mod test {
-
-    use std::vec;
-
     use futures_async_stream::for_await;
 
     use super::*;
@@ -114,15 +99,10 @@ mod test {
         let test_batch_count = 5;
         let table = gen_basic_table(test_batch_count * test_batch_size).await;
 
-        let info = ExecutorInfo {
-            schema: table.schema().clone(),
-            pk_indices: vec![0, 1],
-            identity: "BatchQuery".to_owned(),
-        };
-
-        let executor = Box::new(BatchQueryExecutor::new(table, test_batch_size, info));
-
-        let stream = executor.execute_with_epoch(u64::MAX);
+        let schema = table.schema().clone();
+        let stream = BatchQueryExecutor::new(table, test_batch_size, schema)
+            .boxed()
+            .execute_with_epoch(u64::MAX);
         let mut batch_cnt = 0;
 
         #[for_await]
diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs
index 0228f826a4bac..3e65bf878b552 100644
--- a/src/stream/src/executor/chain.rs
+++ b/src/stream/src/executor/chain.rs
@@ -14,10 +14,9 @@
 
 use futures::StreamExt;
 use futures_async_stream::try_stream;
-use risingwave_common::catalog::Schema;
 
 use super::error::StreamExecutorError;
-use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message};
+use super::{expect_first_barrier, Execute, Executor, Message};
 use crate::task::{ActorId, CreateMviewProgress};
 
 /// [`ChainExecutor`] is an executor that enables synchronization between the existing stream and
@@ -25,11 +24,9 @@ use crate::task::{ActorId, CreateMviewProgress};
 /// feature. It pipes new data of existing MVs to newly created MV only all of the old data in the
 /// existing MVs are dispatched.
 pub struct ChainExecutor {
-    info: ExecutorInfo,
+    snapshot: Executor,
 
-    snapshot: BoxedExecutor,
-
-    upstream: BoxedExecutor,
+    upstream: Executor,
 
     progress: CreateMviewProgress,
 
@@ -41,14 +38,12 @@ pub struct ChainExecutor {
 
 impl ChainExecutor {
     pub fn new(
-        info: ExecutorInfo,
-        snapshot: BoxedExecutor,
-        upstream: BoxedExecutor,
+        snapshot: Executor,
+        upstream: Executor,
         progress: CreateMviewProgress,
         upstream_only: bool,
     ) -> Self {
         Self {
-            info,
             snapshot,
             upstream,
             actor_id: progress.actor_id(),
@@ -104,22 +99,10 @@ impl ChainExecutor {
     }
 }
 
-impl Executor for ChainExecutor {
+impl Execute for ChainExecutor {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 #[cfg(test)]
@@ -135,9 +118,7 @@ mod test {
 
     use super::ChainExecutor;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::{
-        AddMutation, Barrier, Executor, ExecutorInfo, Message, Mutation, PkIndices,
-    };
+    use crate::executor::{AddMutation, Barrier, Execute, Message, Mutation, PkIndices};
     use crate::task::{CreateMviewProgress, LocalBarrierManager};
 
     #[tokio::test]
@@ -147,48 +128,35 @@ mod test {
         let actor_id = progress.actor_id();
 
         let schema = Schema::new(vec![Field::unnamed(DataType::Int64)]);
-        let first = Box::new(
-            MockSource::with_chunks(
-                schema.clone(),
-                PkIndices::new(),
-                vec![
-                    StreamChunk::from_pretty("I\n + 1"),
-                    StreamChunk::from_pretty("I\n + 2"),
-                ],
-            )
-            .stop_on_finish(false),
-        );
-
-        let second = Box::new(MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1).with_mutation(Mutation::Add(
-                    AddMutation {
-                        adds: maplit::hashmap! {
-                            0 => vec![Dispatcher {
-                                downstream_actor_id: vec![actor_id],
-                                ..Default::default()
-                            }],
-                        },
-                        added_actors: maplit::hashset! { actor_id },
-                        splits: Default::default(),
-                        pause: false,
+        let first = MockSource::with_chunks(vec![
+            StreamChunk::from_pretty("I\n + 1"),
+            StreamChunk::from_pretty("I\n + 2"),
+        ])
+        .stop_on_finish(false)
+        .into_executor(schema.clone(), PkIndices::new());
+
+        let second = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1).with_mutation(Mutation::Add(
+                AddMutation {
+                    adds: maplit::hashmap! {
+                        0 => vec![Dispatcher {
+                            downstream_actor_id: vec![actor_id],
+                            ..Default::default()
+                        }],
                     },
-                ))),
-                Message::Chunk(StreamChunk::from_pretty("I\n + 3")),
-                Message::Chunk(StreamChunk::from_pretty("I\n + 4")),
-            ],
-        ));
-
-        let info = ExecutorInfo {
-            schema,
-            pk_indices: PkIndices::new(),
-            identity: "ChainExecutor".to_string(),
-        };
-        let chain = ChainExecutor::new(info, first, second, progress, false);
-
-        let mut chain = Box::new(chain).execute();
+                    added_actors: maplit::hashset! { actor_id },
+                    splits: Default::default(),
+                    pause: false,
+                },
+            ))),
+            Message::Chunk(StreamChunk::from_pretty("I\n + 3")),
+            Message::Chunk(StreamChunk::from_pretty("I\n + 4")),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
+
+        let chain = ChainExecutor::new(first, second, progress, false);
+
+        let mut chain = chain.boxed().execute();
         chain.next().await;
 
         let mut count = 0;
diff --git a/src/stream/src/executor/dedup/append_only_dedup.rs b/src/stream/src/executor/dedup/append_only_dedup.rs
index 6049b963359ab..34b102ff1e030 100644
--- a/src/stream/src/executor/dedup/append_only_dedup.rs
+++ b/src/stream/src/executor/dedup/append_only_dedup.rs
@@ -19,7 +19,6 @@ use futures_async_stream::try_stream;
 use itertools::Itertools;
 use risingwave_common::array::{Op, StreamChunk};
 use risingwave_common::buffer::BitmapBuilder;
-use risingwave_common::catalog::Schema;
 use risingwave_common::row::{OwnedRow, Row, RowExt};
 use risingwave_storage::StateStore;
 
@@ -29,39 +28,39 @@ use crate::common::table::state_table::StateTable;
 use crate::executor::error::StreamExecutorError;
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor,
-    ExecutorInfo, Message, PkIndicesRef, StreamExecutorResult,
+    expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message,
+    StreamExecutorResult,
 };
 use crate::task::AtomicU64Ref;
 
 /// [`AppendOnlyDedupExecutor`] drops any message that has duplicate pk columns with previous
 /// messages. It only accepts append-only input, and its output will be append-only as well.
 pub struct AppendOnlyDedupExecutor<S: StateStore> {
-    input: Option<BoxedExecutor>,
+    ctx: ActorContextRef,
+
+    input: Option<Executor>,
+    dedup_cols: Vec<usize>,
     state_table: StateTable<S>,
     cache: DedupCache<OwnedRow>,
-
-    info: ExecutorInfo,
-    ctx: ActorContextRef,
 }
 
 impl<S: StateStore> AppendOnlyDedupExecutor<S> {
     pub fn new(
-        input: BoxedExecutor,
-        state_table: StateTable<S>,
-        info: ExecutorInfo,
         ctx: ActorContextRef,
+        input: Executor,
+        dedup_cols: Vec<usize>,
+        state_table: StateTable<S>,
         watermark_epoch: AtomicU64Ref,
         metrics: Arc<StreamingMetrics>,
     ) -> Self {
         let metrics_info =
             MetricsInfo::new(metrics, state_table.table_id(), ctx.id, "AppendOnly Dedup");
         Self {
+            ctx,
             input: Some(input),
+            dedup_cols,
             state_table,
             cache: DedupCache::new(watermark_epoch, metrics_info),
-            info,
-            ctx,
         }
     }
 
@@ -90,7 +89,7 @@ impl<S: StateStore> AppendOnlyDedupExecutor<S> {
                         .data_chunk()
                         .rows_with_holes()
                         .map(|row_ref| {
-                            row_ref.map(|row| row.project(self.pk_indices()).to_owned_row())
+                            row_ref.map(|row| row.project(&self.dedup_cols).to_owned_row())
                         })
                         .collect_vec();
 
@@ -185,22 +184,10 @@ impl<S: StateStore> AppendOnlyDedupExecutor<S> {
     }
 }
 
-impl<S: StateStore> Executor for AppendOnlyDedupExecutor<S> {
+impl<S: StateStore> Execute for AppendOnlyDedupExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.executor_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 #[cfg(test)]
@@ -230,7 +217,8 @@ mod tests {
             Field::unnamed(DataType::Int64),
             Field::unnamed(DataType::Int64),
         ]);
-        let pk_indices = vec![0];
+        let dedup_col_indices = vec![0];
+        let pk_indices = dedup_col_indices.clone();
         let order_types = vec![OrderType::ascending()];
 
         let state_store = MemoryStateStore::new();
@@ -243,20 +231,17 @@ mod tests {
         )
         .await;
 
-        let (mut tx, input) = MockSource::channel(schema.clone(), pk_indices.clone());
-        let info = ExecutorInfo {
-            schema,
-            pk_indices,
-            identity: "AppendOnlyDedupExecutor".to_string(),
-        };
-        let mut dedup_executor = Box::new(AppendOnlyDedupExecutor::new(
-            Box::new(input),
-            state_table,
-            info,
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, pk_indices);
+        let mut dedup_executor = AppendOnlyDedupExecutor::new(
             ActorContext::for_test(123),
+            source,
+            dedup_col_indices,
+            state_table,
             Arc::new(AtomicU64::new(0)),
             Arc::new(StreamingMetrics::unused()),
-        ))
+        )
+        .boxed()
         .execute();
 
         tx.push_barrier(1, false);
diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs
index a21012c5f1664..ee347ee489258 100644
--- a/src/stream/src/executor/dispatch.rs
+++ b/src/stream/src/executor/dispatch.rs
@@ -35,17 +35,17 @@ use tokio::time::Instant;
 use tracing::{event, Instrument};
 
 use super::exchange::output::{new_output, BoxedOutput};
-use super::{AddMutation, UpdateMutation, Watermark};
+use super::{AddMutation, Executor, UpdateMutation, Watermark};
 use crate::error::StreamResult;
 use crate::executor::monitor::StreamingMetrics;
-use crate::executor::{Barrier, BoxedExecutor, Message, Mutation, StreamConsumer};
+use crate::executor::{Barrier, Message, Mutation, StreamConsumer};
 use crate::task::{ActorId, DispatcherId, SharedContext};
 
 /// [`DispatchExecutor`] consumes messages and send them into downstream actors. Usually,
 /// data chunks will be dispatched with some specified policy, while control message
 /// such as barriers will be distributed to all receivers.
 pub struct DispatchExecutor {
-    input: BoxedExecutor,
+    input: Executor,
     inner: DispatchExecutorInner,
 }
 
@@ -341,7 +341,7 @@ impl DispatchExecutorInner {
 
 impl DispatchExecutor {
     pub fn new(
-        input: BoxedExecutor,
+        input: Executor,
         dispatchers: Vec<DispatcherImpl>,
         actor_id: u32,
         fragment_id: u32,
@@ -1042,6 +1042,7 @@ mod tests {
     use crate::executor::exchange::output::Output;
     use crate::executor::exchange::permit::channel_for_test;
     use crate::executor::receiver::ReceiverExecutor;
+    use crate::executor::Execute;
     use crate::task::test_utils::helper_make_local_actor;
 
     #[derive(Debug)]
@@ -1152,7 +1153,7 @@ mod tests {
         let (tx, rx) = channel_for_test();
         let actor_id = 233;
         let fragment_id = 666;
-        let input = Box::new(ReceiverExecutor::for_test(rx));
+        let input = Executor::new(Default::default(), ReceiverExecutor::for_test(rx).boxed());
         let ctx = Arc::new(SharedContext::for_test());
         let metrics = Arc::new(StreamingMetrics::unused());
 
diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs
index 8e0a186e9c81c..0463c695639d7 100644
--- a/src/stream/src/executor/dml.rs
+++ b/src/stream/src/executor/dml.rs
@@ -19,25 +19,20 @@ use either::Either;
 use futures::{StreamExt, TryStreamExt};
 use futures_async_stream::try_stream;
 use risingwave_common::array::StreamChunk;
-use risingwave_common::catalog::{ColumnDesc, Schema, TableId, TableVersionId};
+use risingwave_common::catalog::{ColumnDesc, TableId, TableVersionId};
 use risingwave_common::transaction::transaction_id::TxnId;
 use risingwave_common::transaction::transaction_message::TxnMsg;
 use risingwave_dml::dml_manager::DmlManagerRef;
 
 use super::error::StreamExecutorError;
-use super::{
-    expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message,
-    Mutation, PkIndicesRef,
-};
+use super::{expect_first_barrier, BoxedMessageStream, Execute, Executor, Message, Mutation};
 use crate::common::StreamChunkBuilder;
 use crate::executor::stream_reader::StreamReaderWithPause;
 
 /// [`DmlExecutor`] accepts both stream data and batch data for data manipulation on a specific
 /// table. The two streams will be merged into one and then sent to downstream.
 pub struct DmlExecutor {
-    info: ExecutorInfo,
-
-    upstream: BoxedExecutor,
+    upstream: Executor,
 
     /// Stores the information of batch data channels.
     dml_manager: DmlManagerRef,
@@ -71,10 +66,8 @@ struct TxnBuffer {
 }
 
 impl DmlExecutor {
-    #[allow(clippy::too_many_arguments)]
     pub fn new(
-        info: ExecutorInfo,
-        upstream: BoxedExecutor,
+        upstream: Executor,
         dml_manager: DmlManagerRef,
         table_id: TableId,
         table_version_id: TableVersionId,
@@ -82,7 +75,6 @@ impl DmlExecutor {
         chunk_size: usize,
     ) -> Self {
         Self {
-            info,
             upstream,
             dml_manager,
             table_id,
@@ -277,22 +269,10 @@ impl DmlExecutor {
     }
 }
 
-impl Executor for DmlExecutor {
+impl Execute for DmlExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 #[cfg(test)]
@@ -300,7 +280,7 @@ mod tests {
     use std::sync::Arc;
 
     use risingwave_common::array::StreamChunk;
-    use risingwave_common::catalog::{ColumnId, Field, INITIAL_TABLE_VERSION_ID};
+    use risingwave_common::catalog::{ColumnId, Field, Schema, INITIAL_TABLE_VERSION_ID};
     use risingwave_common::test_prelude::StreamChunkTestExt;
     use risingwave_common::transaction::transaction_id::TxnId;
     use risingwave_common::types::DataType;
@@ -326,23 +306,18 @@ mod tests {
         let pk_indices = vec![0];
         let dml_manager = Arc::new(DmlManager::for_test());
 
-        let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone());
-        let info = ExecutorInfo {
-            schema,
-            pk_indices,
-            identity: "DmlExecutor".to_string(),
-        };
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, pk_indices);
 
-        let dml_executor = Box::new(DmlExecutor::new(
-            info,
-            Box::new(source),
+        let dml_executor = DmlExecutor::new(
+            source,
             dml_manager.clone(),
             table_id,
             INITIAL_TABLE_VERSION_ID,
             column_descs,
             1024,
-        ));
-        let mut dml_executor = dml_executor.execute();
+        );
+        let mut dml_executor = dml_executor.boxed().execute();
 
         let stream_chunk1 = StreamChunk::from_pretty(
             " I I
diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs
index d3299d99e8a33..6bcd3e97ed563 100644
--- a/src/stream/src/executor/dynamic_filter.rs
+++ b/src/stream/src/executor/dynamic_filter.rs
@@ -38,10 +38,7 @@ use risingwave_storage::StateStore;
 use super::barrier_align::*;
 use super::error::StreamExecutorError;
 use super::monitor::StreamingMetrics;
-use super::{
-    ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message,
-    PkIndicesRef,
-};
+use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message};
 use crate::common::table::state_table::{StateTable, WatermarkCacheParameterizedStateTable};
 use crate::common::StreamChunkBuilder;
 use crate::executor::expect_first_barrier_from_aligned_stream;
@@ -49,10 +46,12 @@ use crate::task::ActorEvalErrorReport;
 
 pub struct DynamicFilterExecutor<S: StateStore, const USE_WATERMARK_CACHE: bool> {
     ctx: ActorContextRef,
-    info: ExecutorInfo,
 
-    source_l: Option<BoxedExecutor>,
-    source_r: Option<BoxedExecutor>,
+    eval_error_report: ActorEvalErrorReport,
+
+    schema: Schema,
+    source_l: Option<Executor>,
+    source_r: Option<Executor>,
     key_l: usize,
     comparator: ExprNodeType,
     left_table: WatermarkCacheParameterizedStateTable<S, USE_WATERMARK_CACHE>,
@@ -71,9 +70,9 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
-        source_l: BoxedExecutor,
-        source_r: BoxedExecutor,
+        info: &ExecutorInfo,
+        source_l: Executor,
+        source_r: Executor,
         key_l: usize,
         comparator: ExprNodeType,
         state_table_l: WatermarkCacheParameterizedStateTable<S, USE_WATERMARK_CACHE>,
@@ -83,9 +82,14 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US
         condition_always_relax: bool,
         cleaned_by_watermark: bool,
     ) -> Self {
+        let eval_error_report = ActorEvalErrorReport {
+            actor_context: ctx.clone(),
+            identity: Arc::from(info.identity.as_str()),
+        };
         Self {
             ctx,
-            info,
+            eval_error_report,
+            schema: info.schema.clone(),
             source_l: Some(source_l),
             source_r: Some(source_r),
             key_l,
@@ -263,7 +267,7 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US
     }
 
     #[try_stream(ok = Message, error = StreamExecutorError)]
-    async fn into_stream(mut self) {
+    async fn execute_inner(mut self) {
         let input_l = self.source_l.take().unwrap();
         let input_r = self.source_r.take().unwrap();
 
@@ -273,10 +277,7 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US
         // The types are aligned by frontend.
         assert_eq!(l_data_type, r_data_type);
         let dynamic_cond = {
-            let eval_error_report = ActorEvalErrorReport {
-                actor_context: self.ctx.clone(),
-                identity: Arc::from(self.info.identity.as_str()),
-            };
+            let eval_error_report = self.eval_error_report.clone();
             move |literal: Datum| {
                 literal.map(|scalar| {
                     build_func_non_strict(
@@ -319,7 +320,7 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US
         yield Message::Barrier(barrier);
 
         let mut stream_chunk_builder =
-            StreamChunkBuilder::new(self.chunk_size, self.info.schema.data_types());
+            StreamChunkBuilder::new(self.chunk_size, self.schema.data_types());
 
         let watermark_can_clean_state = !matches!(self.comparator, LessThan | LessThanOrEqual);
         let mut unused_clean_hint = None;
@@ -487,23 +488,11 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US
     }
 }
 
-impl<S: StateStore, const USE_WATERMARK_CACHE: bool> Executor
+impl<S: StateStore, const USE_WATERMARK_CACHE: bool> Execute
     for DynamicFilterExecutor<S, USE_WATERMARK_CACHE>
 {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
-        self.into_stream().boxed()
-    }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
+        self.execute_inner().boxed()
     }
 }
 
@@ -561,21 +550,20 @@ mod tests {
         let schema = Schema {
             fields: vec![Field::unnamed(DataType::Int64)],
         };
-        let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![0]);
-        let (tx_r, source_r) = MockSource::channel(schema, vec![]);
-
-        let schema = source_l.schema().clone();
-        let info = ExecutorInfo {
-            schema,
-            pk_indices: vec![0],
-            identity: "DynamicFilterExecutor".to_string(),
-        };
+        let (tx_l, source_l) = MockSource::channel();
+        let source_l = source_l.into_executor(schema.clone(), vec![0]);
+        let (tx_r, source_r) = MockSource::channel();
+        let source_r = source_r.into_executor(schema, vec![]);
 
         let executor = DynamicFilterExecutor::<MemoryStateStore, false>::new(
             ActorContext::for_test(123),
-            info,
-            Box::new(source_l),
-            Box::new(source_r),
+            &ExecutorInfo {
+                schema: source_l.schema().clone(),
+                pk_indices: vec![0],
+                identity: "DynamicFilterExecutor".to_string(),
+            },
+            source_l,
+            source_r,
             0,
             comparator,
             mem_state_l,
@@ -585,7 +573,7 @@ mod tests {
             always_relax,
             false,
         );
-        (tx_l, tx_r, Box::new(executor).execute())
+        (tx_l, tx_r, executor.boxed().execute())
     }
 
     #[tokio::test]
diff --git a/src/stream/src/executor/expand.rs b/src/stream/src/executor/expand.rs
index cb8c45732d17f..2375ebca3d0df 100644
--- a/src/stream/src/executor/expand.rs
+++ b/src/stream/src/executor/expand.rs
@@ -16,26 +16,19 @@ use std::fmt::Debug;
 
 use futures::StreamExt;
 use futures_async_stream::try_stream;
-use risingwave_common::array::{Array, I64Array};
-use risingwave_common::catalog::Schema;
+use risingwave_common::array::{Array, I64Array, StreamChunk};
 
 use super::error::StreamExecutorError;
-use super::*;
+use super::{BoxedMessageStream, Execute, Executor, Message};
 
 pub struct ExpandExecutor {
-    info: ExecutorInfo,
-    input: BoxedExecutor,
+    input: Executor,
     column_subsets: Vec<Vec<usize>>,
 }
 
 impl ExpandExecutor {
-    pub fn new(
-        info: ExecutorInfo,
-        input: Box<dyn Executor>,
-        column_subsets: Vec<Vec<usize>>,
-    ) -> Self {
+    pub fn new(input: Executor, column_subsets: Vec<Vec<usize>>) -> Self {
         Self {
-            info,
             input,
             column_subsets,
         }
@@ -73,19 +66,7 @@ impl Debug for ExpandExecutor {
     }
 }
 
-impl Executor for ExpandExecutor {
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
+impl Execute for ExpandExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
@@ -100,7 +81,7 @@ mod tests {
 
     use super::ExpandExecutor;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::{Executor, ExecutorInfo, PkIndices};
+    use crate::executor::{Execute, PkIndices};
 
     #[tokio::test]
     async fn test_expand() {
@@ -111,29 +92,19 @@ mod tests {
             + 6 6 3
             - 7 5 4",
         );
-        let input_schema = Schema {
-            fields: vec![
+        let source = MockSource::with_chunks(vec![chunk1]).into_executor(
+            Schema::new(vec![
                 Field::unnamed(DataType::Int64),
                 Field::unnamed(DataType::Int64),
                 Field::unnamed(DataType::Int64),
-            ],
-        };
-        let source = MockSource::with_chunks(input_schema.clone(), PkIndices::new(), vec![chunk1]);
-        let schema = {
-            let mut fields = input_schema.into_fields();
-            fields.extend(fields.clone());
-            fields.push(Field::with_name(DataType::Int64, "flag"));
-            Schema::new(fields)
-        };
-        let info = ExecutorInfo {
-            schema,
-            pk_indices: vec![],
-            identity: "ExpandExecutor".to_string(),
-        };
+            ]),
+            PkIndices::new(),
+        );
 
         let column_subsets = vec![vec![0, 1], vec![1, 2]];
-        let expand = Box::new(ExpandExecutor::new(info, Box::new(source), column_subsets));
-        let mut expand = expand.execute();
+        let mut expand = ExpandExecutor::new(source, column_subsets)
+            .boxed()
+            .execute();
 
         let chunk = expand.next().await.unwrap().unwrap().into_chunk().unwrap();
         assert_eq!(
diff --git a/src/stream/src/executor/filter.rs b/src/stream/src/executor/filter.rs
index e51307b2ffee0..ffc8847c42abf 100644
--- a/src/stream/src/executor/filter.rs
+++ b/src/stream/src/executor/filter.rs
@@ -15,13 +15,17 @@
 use std::fmt::{Debug, Formatter};
 use std::sync::Arc;
 
+use futures::StreamExt;
+use futures_async_stream::try_stream;
 use risingwave_common::array::{Array, ArrayImpl, Op, StreamChunk};
 use risingwave_common::buffer::BitmapBuilder;
-use risingwave_common::catalog::Schema;
 use risingwave_common::util::iter_util::ZipEqFast;
 use risingwave_expr::expr::NonStrictExpression;
 
-use super::*;
+use super::{
+    ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError,
+    StreamExecutorResult,
+};
 
 /// `FilterExecutor` filters data with the `expr`. The `expr` takes a chunk of data,
 /// and returns a boolean array on whether each item should be retained. And then,
@@ -29,8 +33,7 @@ use super::*;
 /// to the result of the expression.
 pub struct FilterExecutor {
     _ctx: ActorContextRef,
-    info: ExecutorInfo,
-    input: BoxedExecutor,
+    input: Executor,
 
     /// Expression of the current filter, note that the filter must always have the same output for
     /// the same input.
@@ -38,15 +41,9 @@ pub struct FilterExecutor {
 }
 
 impl FilterExecutor {
-    pub fn new(
-        ctx: ActorContextRef,
-        info: ExecutorInfo,
-        input: Box<dyn Executor>,
-        expr: NonStrictExpression,
-    ) -> Self {
+    pub fn new(ctx: ActorContextRef, input: Executor, expr: NonStrictExpression) -> Self {
         Self {
             _ctx: ctx,
-            info,
             input,
             expr,
         }
@@ -135,19 +132,7 @@ impl Debug for FilterExecutor {
     }
 }
 
-impl Executor for FilterExecutor {
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
+impl Execute for FilterExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
@@ -218,23 +203,14 @@ mod tests {
             ],
         };
         let pk_indices = PkIndices::new();
-        let source =
-            MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk1, chunk2]);
-        let info = ExecutorInfo {
-            schema,
-            pk_indices,
-            identity: "FilterExecutor".to_string(),
-        };
+        let source = MockSource::with_chunks(vec![chunk1, chunk2])
+            .into_executor(schema.clone(), pk_indices.clone());
 
         let test_expr = build_from_pretty("(greater_than:boolean $0:int8 $1:int8)");
 
-        let filter = Box::new(FilterExecutor::new(
-            ActorContext::for_test(123),
-            info,
-            Box::new(source),
-            test_expr,
-        ));
-        let mut filter = filter.execute();
+        let mut filter = FilterExecutor::new(ActorContext::for_test(123), source, test_expr)
+            .boxed()
+            .execute();
 
         let chunk = filter.next().await.unwrap().unwrap().into_chunk().unwrap();
         assert_eq!(
diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs
index 53d0f15a602ae..4e4212a1e246e 100644
--- a/src/stream/src/executor/flow_control.rs
+++ b/src/stream/src/executor/flow_control.rs
@@ -15,11 +15,14 @@
 use std::fmt::{Debug, Formatter};
 use std::num::NonZeroU32;
 
+use futures::StreamExt;
+use futures_async_stream::try_stream;
 use governor::clock::MonotonicClock;
 use governor::{Quota, RateLimiter};
-use risingwave_common::catalog::Schema;
 
-use super::*;
+use super::{
+    ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError,
+};
 
 /// Flow Control Executor is used to control the rate of the input executor.
 ///
@@ -31,27 +34,16 @@ use super::*;
 ///
 /// It is used to throttle problematic MVs that are consuming too much resources.
 pub struct FlowControlExecutor {
-    input: BoxedExecutor,
+    input: Executor,
     actor_ctx: ActorContextRef,
-    identity: String,
     rate_limit: Option<u32>,
 }
 
 impl FlowControlExecutor {
-    pub fn new(
-        input: Box<dyn Executor>,
-        actor_ctx: ActorContextRef,
-        rate_limit: Option<u32>,
-    ) -> Self {
-        let identity = if rate_limit.is_some() {
-            format!("{} (flow controlled)", input.identity())
-        } else {
-            input.identity().to_owned()
-        };
+    pub fn new(input: Executor, actor_ctx: ActorContextRef, rate_limit: Option<u32>) -> Self {
         Self {
             input,
             actor_ctx,
-            identity,
             rate_limit,
         }
     }
@@ -128,20 +120,8 @@ impl Debug for FlowControlExecutor {
     }
 }
 
-impl Executor for FlowControlExecutor {
+impl Execute for FlowControlExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        self.input.schema()
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        self.input.pk_indices()
-    }
-
-    fn identity(&self) -> &str {
-        &self.identity
-    }
 }
diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs
index 5078fd0bc5154..ccaa4d98ce11d 100644
--- a/src/stream/src/executor/hash_agg.rs
+++ b/src/stream/src/executor/hash_agg.rs
@@ -39,8 +39,7 @@ use super::aggregation::{
 };
 use super::sort_buffer::SortBuffer;
 use super::{
-    expect_first_barrier, ActorContextRef, ExecutorInfo, PkIndicesRef, StreamExecutorResult,
-    Watermark,
+    expect_first_barrier, ActorContextRef, Executor, ExecutorInfo, StreamExecutorResult, Watermark,
 };
 use crate::cache::{cache_may_stale, new_with_hasher, ManagedLruCache};
 use crate::common::metrics::MetricsInfo;
@@ -49,7 +48,7 @@ use crate::common::StreamChunkBuilder;
 use crate::error::StreamResult;
 use crate::executor::aggregation::AggGroup as GenericAggGroup;
 use crate::executor::error::StreamExecutorError;
-use crate::executor::{BoxedMessageStream, Executor, Message};
+use crate::executor::{BoxedMessageStream, Execute, Message};
 use crate::task::AtomicU64Ref;
 
 type AggGroup<S> = GenericAggGroup<S, OnlyOutputIfHasInput>;
@@ -74,7 +73,7 @@ type AggGroupCache<K, S> = ManagedLruCache<K, Option<BoxedAggGroup<S>>, Precompu
 ///   all modifications will be flushed to the storage backend. Meanwhile, the executor will go
 ///   through `group_change_set`, and produce a stream chunk based on the state changes.
 pub struct HashAggExecutor<K: HashKey, S: StateStore> {
-    input: Box<dyn Executor>,
+    input: Executor,
     inner: ExecutorInner<K, S>,
 }
 
@@ -193,27 +192,15 @@ impl ExecutionStats {
     }
 }
 
-impl<K: HashKey, S: StateStore> Executor for HashAggExecutor<K, S> {
+impl<K: HashKey, S: StateStore> Execute for HashAggExecutor<K, S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.inner.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.inner.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info.identity
-    }
 }
 
 impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> {
     pub fn new(args: AggExecutorArgs<S, HashAggExecutorExtraArgs>) -> StreamResult<Self> {
-        let input_info = args.input.info();
+        let input_info = args.input.info().clone();
 
         let group_key_len = args.extra.group_key_indices.len();
         // NOTE: we assume the prefix of table pk is exactly the group key
diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs
index 471d382a763d0..3727da397d01f 100644
--- a/src/stream/src/executor/hash_join.rs
+++ b/src/stream/src/executor/hash_join.rs
@@ -22,7 +22,6 @@ use futures_async_stream::try_stream;
 use itertools::Itertools;
 use multimap::MultiMap;
 use risingwave_common::array::{Op, RowRef, StreamChunk};
-use risingwave_common::catalog::Schema;
 use risingwave_common::hash::{HashKey, NullBitmap};
 use risingwave_common::row::{OwnedRow, Row};
 use risingwave_common::types::{DataType, DefaultOrd, ToOwnedDatum};
@@ -42,8 +41,7 @@ use super::join::{JoinTypePrimitive, SideTypePrimitive, *};
 use super::monitor::StreamingMetrics;
 use super::watermark::*;
 use super::{
-    ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message,
-    PkIndicesRef, Watermark,
+    ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message, Watermark,
 };
 use crate::common::table::state_table::StateTable;
 use crate::executor::expect_first_barrier_from_aligned_stream;
@@ -143,9 +141,9 @@ pub struct HashJoinExecutor<K: HashKey, S: StateStore, const T: JoinTypePrimitiv
     info: ExecutorInfo,
 
     /// Left input executor
-    input_l: Option<BoxedExecutor>,
+    input_l: Option<Executor>,
     /// Right input executor
-    input_r: Option<BoxedExecutor>,
+    input_r: Option<Executor>,
     /// The data types of the formed new columns
     actual_output_data_types: Vec<DataType>,
     /// The parameters of the left join executor
@@ -191,22 +189,10 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> std::fmt::Debug
     }
 }
 
-impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Executor for HashJoinExecutor<K, S, T> {
+impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Execute for HashJoinExecutor<K, S, T> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.into_stream().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 struct EqJoinArgs<'a, K: HashKey, S: StateStore> {
@@ -227,8 +213,8 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> HashJoinExecutor<K,
     pub fn new(
         ctx: ActorContextRef,
         info: ExecutorInfo,
-        input_l: BoxedExecutor,
-        input_r: BoxedExecutor,
+        input_l: Executor,
+        input_r: Executor,
         params_l: JoinParams,
         params_r: JoinParams,
         null_safe: Vec<bool>,
@@ -1156,8 +1142,10 @@ mod tests {
                 Field::unnamed(DataType::Int64),
             ],
         };
-        let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![1]);
-        let (tx_r, source_r) = MockSource::channel(schema, vec![1]);
+        let (tx_l, source_l) = MockSource::channel();
+        let source_l = source_l.into_executor(schema.clone(), vec![1]);
+        let (tx_r, source_r) = MockSource::channel();
+        let source_r = source_r.into_executor(schema, vec![1]);
         let params_l = JoinParams::new(vec![0], vec![1]);
         let params_r = JoinParams::new(vec![0], vec![1]);
         let cond = with_condition.then(|| create_cond(condition_text));
@@ -1200,8 +1188,8 @@ mod tests {
         let executor = HashJoinExecutor::<Key64, MemoryStateStore, T>::new(
             ActorContext::for_test(123),
             info,
-            Box::new(source_l),
-            Box::new(source_r),
+            source_l,
+            source_r,
             params_l,
             params_r,
             vec![null_safe],
@@ -1217,7 +1205,7 @@ mod tests {
             Arc::new(StreamingMetrics::unused()),
             1024,
         );
-        (tx_l, tx_r, Box::new(executor).execute())
+        (tx_l, tx_r, executor.boxed().execute())
     }
 
     async fn create_classical_executor<const T: JoinTypePrimitive>(
@@ -1238,8 +1226,10 @@ mod tests {
                 Field::unnamed(DataType::Int64),
             ],
         };
-        let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![0]);
-        let (tx_r, source_r) = MockSource::channel(schema, vec![0]);
+        let (tx_l, source_l) = MockSource::channel();
+        let source_l = source_l.into_executor(schema.clone(), vec![0]);
+        let (tx_r, source_r) = MockSource::channel();
+        let source_r = source_r.into_executor(schema, vec![0]);
         let params_l = JoinParams::new(vec![0, 1], vec![]);
         let params_r = JoinParams::new(vec![0, 1], vec![]);
         let cond = with_condition.then(|| create_cond(None));
@@ -1290,8 +1280,8 @@ mod tests {
         let executor = HashJoinExecutor::<Key128, MemoryStateStore, T>::new(
             ActorContext::for_test(123),
             info,
-            Box::new(source_l),
-            Box::new(source_r),
+            source_l,
+            source_r,
             params_l,
             params_r,
             vec![false],
@@ -1307,7 +1297,7 @@ mod tests {
             Arc::new(StreamingMetrics::unused()),
             1024,
         );
-        (tx_l, tx_r, Box::new(executor).execute())
+        (tx_l, tx_r, executor.boxed().execute())
     }
 
     #[tokio::test]
diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs
index 4bcca4d593072..801f3daa9f53a 100644
--- a/src/stream/src/executor/hop_window.rs
+++ b/src/stream/src/executor/hop_window.rs
@@ -23,13 +23,12 @@ use risingwave_expr::expr::NonStrictExpression;
 use risingwave_expr::ExprError;
 
 use super::error::StreamExecutorError;
-use super::{ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message};
+use super::{ActorContextRef, Execute, Executor, Message};
 use crate::common::StreamChunkBuilder;
 
 pub struct HopWindowExecutor {
     _ctx: ActorContextRef,
-    pub info: ExecutorInfo,
-    pub input: BoxedExecutor,
+    pub input: Executor,
     pub time_col_idx: usize,
     pub window_slide: Interval,
     pub window_size: Interval,
@@ -43,8 +42,7 @@ impl HopWindowExecutor {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
-        input: BoxedExecutor,
+        input: Executor,
         time_col_idx: usize,
         window_slide: Interval,
         window_size: Interval,
@@ -55,7 +53,6 @@ impl HopWindowExecutor {
     ) -> Self {
         HopWindowExecutor {
             _ctx: ctx,
-            info,
             input,
             time_col_idx,
             window_slide,
@@ -68,22 +65,10 @@ impl HopWindowExecutor {
     }
 }
 
-impl Executor for HopWindowExecutor {
+impl Execute for HopWindowExecutor {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &risingwave_common::catalog::Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 impl HopWindowExecutor {
@@ -253,12 +238,13 @@ mod tests {
     use risingwave_expr::expr::test_utils::make_hop_window_expression;
     use risingwave_expr::expr::NonStrictExpression;
 
+    use super::*;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::{ActorContext, Executor, ExecutorInfo, StreamChunk};
+    use crate::executor::{ActorContext, Execute, StreamChunk};
 
     const CHUNK_SIZE: usize = 256;
 
-    fn create_executor(output_indices: Vec<usize>) -> Box<dyn Executor> {
+    fn create_executor(output_indices: Vec<usize>) -> Box<dyn Execute> {
         let field1 = Field::unnamed(DataType::Int64);
         let field2 = Field::unnamed(DataType::Int64);
         let field3 = Field::with_name(DataType::Timestamp, "created_at");
@@ -278,7 +264,7 @@ mod tests {
                 .replace('^', "2022-02-02T"),
         );
         let input =
-            MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk]).boxed();
+            MockSource::with_chunks(vec![chunk]).into_executor(schema.clone(), pk_indices.clone());
         let window_slide = Interval::from_minutes(15);
         let window_size = Interval::from_minutes(30);
         let window_offset = Interval::from_minutes(0);
@@ -291,14 +277,8 @@ mod tests {
         )
         .unwrap();
 
-        super::HopWindowExecutor::new(
+        HopWindowExecutor::new(
             ActorContext::for_test(123),
-            ExecutorInfo {
-                // TODO: the schema is incorrect, but it seems useless here.
-                schema,
-                pk_indices,
-                identity: "HopWindowExecutor".to_string(),
-            },
             input,
             2,
             window_slide,
diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs
index 0759f4bd2d813..691ba0f46f341 100644
--- a/src/stream/src/executor/integration_tests.rs
+++ b/src/stream/src/executor/integration_tests.rs
@@ -35,7 +35,7 @@ use crate::executor::receiver::ReceiverExecutor;
 use crate::executor::test_utils::agg_executor::{
     generate_agg_schema, new_boxed_simple_agg_executor,
 };
-use crate::executor::{Executor, MergeExecutor, ProjectExecutor, StatelessSimpleAggExecutor};
+use crate::executor::{Execute, MergeExecutor, ProjectExecutor, StatelessSimpleAggExecutor};
 use crate::task::{LocalBarrierManager, SharedContext};
 
 /// This test creates a merger-dispatcher pair, and run a sum. Each chunk
@@ -50,27 +50,25 @@ async fn test_merger_sum_aggr() {
     let actor_ctx = ActorContext::for_test(0);
     // `make_actor` build an actor to do local aggregation
     let make_actor = |input_rx| {
-        let _schema = Schema {
+        let input_schema = Schema {
             fields: vec![Field::unnamed(DataType::Int64)],
         };
-        let input = ReceiverExecutor::for_test(input_rx);
+        let input = Executor::new(
+            ExecutorInfo {
+                schema: input_schema,
+                pk_indices: PkIndices::new(),
+                identity: "ReceiverExecutor".to_string(),
+            },
+            ReceiverExecutor::for_test(input_rx).boxed(),
+        );
         let agg_calls = vec![
             AggCall::from_pretty("(count:int8)"),
             AggCall::from_pretty("(sum:int8 $0:int8)"),
         ];
         let schema = generate_agg_schema(&input, &agg_calls, None);
         // for the local aggregator, we need two states: row count and sum
-        let aggregator = StatelessSimpleAggExecutor::new(
-            actor_ctx.clone(),
-            ExecutorInfo {
-                schema,
-                pk_indices: vec![],
-                identity: format!("StatelessSimpleAggExecutor {:X}", 1),
-            },
-            input.boxed(),
-            agg_calls,
-        )
-        .unwrap();
+        let aggregator =
+            StatelessSimpleAggExecutor::new(actor_ctx.clone(), input, schema, agg_calls).unwrap();
         let (tx, rx) = channel_for_test();
         let consumer = SenderConsumer {
             input: aggregator.boxed(),
@@ -108,13 +106,15 @@ async fn test_merger_sum_aggr() {
 
     // create a round robin dispatcher, which dispatches messages to the actors
     let (input, rx) = channel_for_test();
-    let schema = Schema {
-        fields: vec![
-            Field::unnamed(DataType::Int64),
-            Field::unnamed(DataType::Int64),
-        ],
-    };
-    let receiver_op = Box::new(ReceiverExecutor::for_test(rx));
+    let receiver_op = Executor::new(
+        ExecutorInfo {
+            // input schema of local simple agg
+            schema: Schema::new(vec![Field::unnamed(DataType::Int64)]),
+            pk_indices: PkIndices::new(),
+            identity: "ReceiverExecutor".to_string(),
+        },
+        ReceiverExecutor::for_test(rx).boxed(),
+    );
     let dispatcher = DispatchExecutor::new(
         receiver_op,
         vec![DispatcherImpl::RoundRobin(RoundRobinDataDispatcher::new(
@@ -138,14 +138,25 @@ async fn test_merger_sum_aggr() {
     handles.push(tokio::spawn(actor.run()));
 
     // use a merge operator to collect data from dispatchers before sending them to aggregator
-    let merger = MergeExecutor::for_test(outputs, schema);
+    let merger = Executor::new(
+        ExecutorInfo {
+            // output schema of local simple agg
+            schema: Schema::new(vec![
+                Field::unnamed(DataType::Int64),
+                Field::unnamed(DataType::Int64),
+            ]),
+            pk_indices: PkIndices::new(),
+            identity: "MergeExecutor".to_string(),
+        },
+        MergeExecutor::for_test(outputs).boxed(),
+    );
 
     // for global aggregator, we need to sum data and sum row count
     let is_append_only = false;
     let aggregator = new_boxed_simple_agg_executor(
         actor_ctx.clone(),
         MemoryStateStore::new(),
-        merger.boxed(),
+        merger,
         is_append_only,
         vec![
             AggCall::from_pretty("(sum0:int8 $0:int8)"),
@@ -160,13 +171,6 @@ async fn test_merger_sum_aggr() {
 
     let projection = ProjectExecutor::new(
         actor_ctx.clone(),
-        ExecutorInfo {
-            schema: Schema {
-                fields: vec![Field::unnamed(DataType::Int64)],
-            },
-            pk_indices: vec![],
-            identity: format!("ProjectExecutor {:X}", 3),
-        },
         aggregator,
         vec![
             // TODO: use the new streaming_if_null expression here, and add `None` tests
@@ -232,7 +236,7 @@ async fn test_merger_sum_aggr() {
 }
 
 struct MockConsumer {
-    input: BoxedExecutor,
+    input: Box<dyn Execute>,
     data: Arc<Mutex<Vec<StreamChunk>>>,
 }
 
@@ -259,7 +263,7 @@ impl StreamConsumer for MockConsumer {
 
 /// `SenderConsumer` consumes data from input executor and send it into a channel.
 pub struct SenderConsumer {
-    input: BoxedExecutor,
+    input: Box<dyn Execute>,
     channel: BoxedOutput,
 }
 
diff --git a/src/stream/src/executor/lookup.rs b/src/stream/src/executor/lookup.rs
index 87cc163ea3dff..2c1de3170a801 100644
--- a/src/stream/src/executor/lookup.rs
+++ b/src/stream/src/executor/lookup.rs
@@ -14,11 +14,10 @@
 
 use async_trait::async_trait;
 use futures::StreamExt;
-use risingwave_common::catalog::Schema;
 use risingwave_common::types::DataType;
 use risingwave_storage::StateStore;
 
-use crate::executor::{Barrier, BoxedMessageStream, Executor, PkIndicesRef};
+use crate::executor::{Barrier, BoxedMessageStream, Execute};
 
 mod cache;
 mod sides;
@@ -28,7 +27,7 @@ mod impl_;
 
 pub use impl_::LookupExecutorParams;
 
-use super::{ActorContextRef, ExecutorInfo};
+use super::{ActorContextRef, Executor};
 
 #[cfg(test)]
 mod tests;
@@ -42,8 +41,6 @@ mod tests;
 pub struct LookupExecutor<S: StateStore> {
     ctx: ActorContextRef,
 
-    info: ExecutorInfo,
-
     /// the data types of the produced data chunk inside lookup (before reordering)
     chunk_data_types: Vec<DataType>,
 
@@ -54,10 +51,10 @@ pub struct LookupExecutor<S: StateStore> {
     stream: StreamJoinSide,
 
     /// The executor for arrangement.
-    arrangement_executor: Option<Box<dyn Executor>>,
+    arrangement_executor: Option<Executor>,
 
     /// The executor for stream.
-    stream_executor: Option<Box<dyn Executor>>,
+    stream_executor: Option<Executor>,
 
     /// The last received barrier.
     last_barrier: Option<Barrier>,
@@ -83,20 +80,8 @@ pub struct LookupExecutor<S: StateStore> {
 }
 
 #[async_trait]
-impl<S: StateStore> Executor for LookupExecutor<S> {
+impl<S: StateStore> Execute for LookupExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs
index a503a73a0c8a0..db2511e19e7bd 100644
--- a/src/stream/src/executor/lookup/impl_.rs
+++ b/src/stream/src/executor/lookup/impl_.rs
@@ -46,11 +46,11 @@ pub struct LookupExecutorParams<S: StateStore> {
 
     /// The side for arrangement. Currently, it should be a
     /// `MaterializeExecutor`.
-    pub arrangement: Box<dyn Executor>,
+    pub arrangement: Executor,
 
     /// The side for stream. It can be any stream, but it will generally be a
     /// `MaterializeExecutor`.
-    pub stream: Box<dyn Executor>,
+    pub stream: Executor,
 
     /// Should be the same as [`ColumnDesc`] in the arrangement.
     ///
@@ -198,7 +198,6 @@ impl<S: StateStore> LookupExecutor<S> {
 
         Self {
             ctx,
-            info,
             chunk_data_types,
             last_barrier: None,
             stream_executor: Some(stream),
diff --git a/src/stream/src/executor/lookup/sides.rs b/src/stream/src/executor/lookup/sides.rs
index 26bfdd39ec401..6e64519503a93 100644
--- a/src/stream/src/executor/lookup/sides.rs
+++ b/src/stream/src/executor/lookup/sides.rs
@@ -211,10 +211,7 @@ pub async fn align_barrier(mut left: BoxedMessageStream, mut right: BoxedMessage
 /// * Barrier (prev = `[2`], current = `[3`])
 /// * `[Msg`] Arrangement (batch)
 #[try_stream(ok = ArrangeMessage, error = StreamExecutorError)]
-pub async fn stream_lookup_arrange_prev_epoch(
-    stream: Box<dyn Executor>,
-    arrangement: Box<dyn Executor>,
-) {
+pub async fn stream_lookup_arrange_prev_epoch(stream: Executor, arrangement: Executor) {
     let mut input = pin!(align_barrier(stream.execute(), arrangement.execute()));
     let mut arrange_buf = vec![];
     let mut stream_side_end = false;
@@ -295,10 +292,7 @@ pub async fn stream_lookup_arrange_prev_epoch(
 /// * `[Do`] lookup `a` in arrangement of epoch `[2`] (current epoch)
 /// * Barrier (prev = `[2`], current = `[3`])
 #[try_stream(ok = ArrangeMessage, error = StreamExecutorError)]
-pub async fn stream_lookup_arrange_this_epoch(
-    stream: Box<dyn Executor>,
-    arrangement: Box<dyn Executor>,
-) {
+pub async fn stream_lookup_arrange_this_epoch(stream: Executor, arrangement: Executor) {
     let mut input = pin!(align_barrier(stream.execute(), arrangement.execute()));
     let mut stream_buf = vec![];
     let mut arrange_buf = vec![];
@@ -432,14 +426,16 @@ mod tests {
                 Field::unnamed(DataType::Int64),
             ],
         };
-        let (mut tx_l, source_l) = MockSource::channel(schema.clone(), vec![1]);
-        let (tx_r, source_r) = MockSource::channel(schema, vec![1]);
-
-        let mut stream = stream_lookup_arrange_this_epoch(
-            Box::new(source_l.stop_on_finish(false)),
-            Box::new(source_r.stop_on_finish(false)),
-        )
-        .boxed();
+        let (mut tx_l, source_l) = MockSource::channel();
+        let source_l = source_l
+            .stop_on_finish(false)
+            .into_executor(schema.clone(), vec![1]);
+        let (tx_r, source_r) = MockSource::channel();
+        let source_r = source_r
+            .stop_on_finish(false)
+            .into_executor(schema, vec![1]);
+
+        let mut stream = stream_lookup_arrange_this_epoch(source_l, source_r).boxed();
 
         // Simulate recovery test
         drop(tx_r);
diff --git a/src/stream/src/executor/lookup/tests.rs b/src/stream/src/executor/lookup/tests.rs
index 7d152e8a5f4e7..9b335fcca6428 100644
--- a/src/stream/src/executor/lookup/tests.rs
+++ b/src/stream/src/executor/lookup/tests.rs
@@ -30,8 +30,8 @@ use crate::executor::lookup::impl_::LookupExecutorParams;
 use crate::executor::lookup::LookupExecutor;
 use crate::executor::test_utils::*;
 use crate::executor::{
-    ActorContext, Barrier, BoxedMessageStream, Executor, ExecutorInfo, MaterializeExecutor,
-    Message, PkIndices,
+    ActorContext, Barrier, BoxedMessageStream, Execute, Executor, ExecutorInfo,
+    MaterializeExecutor, Message, PkIndices,
 };
 
 fn arrangement_col_descs() -> Vec<ColumnDesc> {
@@ -68,10 +68,7 @@ fn arrangement_col_arrange_rules_join_key() -> Vec<ColumnOrder> {
 /// | +  | 2337  | 8    | 3       |
 /// | -  | 2333  | 6    | 3       |
 /// | b  |       |      | 3 -> 4  |
-async fn create_arrangement(
-    table_id: TableId,
-    memory_state_store: MemoryStateStore,
-) -> Box<dyn Executor + Send> {
+async fn create_arrangement(table_id: TableId, memory_state_store: MemoryStateStore) -> Executor {
     // Two columns of int32 type, the second column is arrange key.
     let columns = arrangement_col_descs();
 
@@ -101,30 +98,32 @@ async fn create_arrangement(
             .collect_vec(),
     );
 
-    let source = MockSource::with_messages(
-        schema,
-        vec![0],
-        vec![
-            Message::Barrier(Barrier::new_test_barrier(2)),
-            Message::Chunk(chunk1),
-            Message::Barrier(Barrier::new_test_barrier(3)),
-            Message::Chunk(chunk2),
-            Message::Barrier(Barrier::new_test_barrier(4)),
-        ],
-    );
+    let source = MockSource::with_messages(vec![
+        Message::Barrier(Barrier::new_test_barrier(2)),
+        Message::Chunk(chunk1),
+        Message::Barrier(Barrier::new_test_barrier(3)),
+        Message::Chunk(chunk2),
+        Message::Barrier(Barrier::new_test_barrier(4)),
+    ])
+    .into_executor(schema, vec![0]);
 
-    Box::new(
+    Executor::new(
+        ExecutorInfo {
+            schema: source.schema().clone(),
+            pk_indices: source.pk_indices().to_vec(),
+            identity: "MaterializeExecutor".to_string(),
+        },
         MaterializeExecutor::for_test(
-            Box::new(source),
+            source,
             memory_state_store,
             table_id,
             arrangement_col_arrange_rules(),
             column_ids,
-            1,
             Arc::new(AtomicU64::new(0)),
             ConflictBehavior::NoCheck,
         )
-        .await,
+        .await
+        .boxed(),
     )
 }
 
@@ -139,7 +138,7 @@ async fn create_arrangement(
 /// | b  |       |      | 2 -> 3  |
 /// | -  | 6     | 1    | 3       |
 /// | b  |       |      | 3 -> 4  |
-fn create_source() -> Box<dyn Executor + Send> {
+fn create_source() -> Executor {
     let columns = vec![
         ColumnDesc::new_atomic(DataType::Int64, "join_column", 1),
         ColumnDesc::new_atomic(DataType::Int64, "rowid_column", 2),
@@ -163,19 +162,14 @@ fn create_source() -> Box<dyn Executor + Send> {
             .collect_vec(),
     );
 
-    let source = MockSource::with_messages(
-        schema,
-        PkIndices::new(),
-        vec![
-            Message::Barrier(Barrier::new_test_barrier(2)),
-            Message::Chunk(chunk1),
-            Message::Barrier(Barrier::new_test_barrier(3)),
-            Message::Chunk(chunk2),
-            Message::Barrier(Barrier::new_test_barrier(4)),
-        ],
-    );
-
-    Box::new(source)
+    MockSource::with_messages(vec![
+        Message::Barrier(Barrier::new_test_barrier(2)),
+        Message::Chunk(chunk1),
+        Message::Barrier(Barrier::new_test_barrier(3)),
+        Message::Chunk(chunk2),
+        Message::Barrier(Barrier::new_test_barrier(4)),
+    ])
+    .into_executor(schema, PkIndices::new())
 }
 
 async fn next_msg(buffer: &mut Vec<Message>, executor: &mut BoxedMessageStream) {
diff --git a/src/stream/src/executor/lookup_union.rs b/src/stream/src/executor/lookup_union.rs
index c52d562277459..2ffddbda45b46 100644
--- a/src/stream/src/executor/lookup_union.rs
+++ b/src/stream/src/executor/lookup_union.rs
@@ -18,35 +18,28 @@ use futures::future::{join_all, select, Either};
 use futures::{FutureExt, SinkExt, StreamExt};
 use futures_async_stream::try_stream;
 use itertools::Itertools;
-use risingwave_common::catalog::Schema;
 
 use super::error::StreamExecutorError;
-use super::*;
-use crate::executor::{BoxedMessageStream, ExecutorInfo};
+use super::{Barrier, BoxedMessageStream, Execute, Executor, Message};
 
 /// Merges data from multiple inputs with order. If `order = [2, 1, 0]`, then
 /// it will first pipe data from the third input; after the third input gets a barrier, it will then
 /// pipe the second, and finally the first. In the future we could have more efficient
 /// implementation.
 pub struct LookupUnionExecutor {
-    info: ExecutorInfo,
-    inputs: Vec<BoxedExecutor>,
+    inputs: Vec<Executor>,
     order: Vec<usize>,
 }
 
 impl std::fmt::Debug for LookupUnionExecutor {
     fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
-        f.debug_struct("LookupUnionExecutor")
-            .field("schema", &self.info.schema)
-            .field("pk_indices", &self.info.pk_indices)
-            .finish()
+        f.debug_struct("LookupUnionExecutor").finish()
     }
 }
 
 impl LookupUnionExecutor {
-    pub fn new(info: ExecutorInfo, inputs: Vec<BoxedExecutor>, order: Vec<u32>) -> Self {
+    pub fn new(inputs: Vec<Executor>, order: Vec<u32>) -> Self {
         Self {
-            info,
             inputs,
             order: order.iter().map(|x| *x as _).collect(),
         }
@@ -54,22 +47,10 @@ impl LookupUnionExecutor {
 }
 
 #[async_trait]
-impl Executor for LookupUnionExecutor {
+impl Execute for LookupUnionExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 impl LookupUnionExecutor {
@@ -154,52 +135,36 @@ mod tests {
         let schema = Schema {
             fields: vec![Field::unnamed(DataType::Int64)],
         };
-        let source0 = MockSource::with_messages(
-            schema.clone(),
-            vec![0],
-            vec![
-                Message::Chunk(StreamChunk::from_pretty("I\n + 1")),
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(StreamChunk::from_pretty("I\n + 2")),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(StreamChunk::from_pretty("I\n + 3")),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-            ],
-        )
-        .stop_on_finish(false);
-        let source1 = MockSource::with_messages(
-            schema.clone(),
-            vec![0],
-            vec![
-                Message::Chunk(StreamChunk::from_pretty("I\n + 11")),
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(StreamChunk::from_pretty("I\n + 12")),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-            ],
-        )
-        .stop_on_finish(false);
-        let source2 = MockSource::with_messages(
-            schema,
-            vec![0],
-            vec![
-                Message::Chunk(StreamChunk::from_pretty("I\n + 21")),
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(StreamChunk::from_pretty("I\n + 22")),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-            ],
-        )
-        .stop_on_finish(false);
-
-        let executor = Box::new(LookupUnionExecutor::new(
-            ExecutorInfo {
-                schema: source0.schema().clone(),
-                pk_indices: vec![0],
-                identity: "LookupUnionExecutor".to_string(),
-            },
-            vec![Box::new(source0), Box::new(source1), Box::new(source2)],
-            vec![2, 1, 0],
-        ))
-        .execute();
+        let source0 = MockSource::with_messages(vec![
+            Message::Chunk(StreamChunk::from_pretty("I\n + 1")),
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(StreamChunk::from_pretty("I\n + 2")),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(StreamChunk::from_pretty("I\n + 3")),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+        ])
+        .stop_on_finish(false)
+        .into_executor(schema.clone(), vec![0]);
+        let source1 = MockSource::with_messages(vec![
+            Message::Chunk(StreamChunk::from_pretty("I\n + 11")),
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(StreamChunk::from_pretty("I\n + 12")),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+        ])
+        .stop_on_finish(false)
+        .into_executor(schema.clone(), vec![0]);
+        let source2 = MockSource::with_messages(vec![
+            Message::Chunk(StreamChunk::from_pretty("I\n + 21")),
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(StreamChunk::from_pretty("I\n + 22")),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+        ])
+        .stop_on_finish(false)
+        .into_executor(schema, vec![0]);
+
+        let executor = LookupUnionExecutor::new(vec![source0, source1, source2], vec![2, 1, 0])
+            .boxed()
+            .execute();
 
         let outputs: Vec<_> = executor.try_collect().await.unwrap();
         assert_eq!(
diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs
index 145a937fcdb2e..e5b88dd274e1e 100644
--- a/src/stream/src/executor/merge.rs
+++ b/src/stream/src/executor/merge.rs
@@ -20,7 +20,6 @@ use anyhow::Context as _;
 use futures::stream::{FusedStream, FuturesUnordered, StreamFuture};
 use futures::{pin_mut, Stream, StreamExt};
 use futures_async_stream::try_stream;
-use risingwave_common::catalog::Schema;
 use tokio::time::Instant;
 
 use super::error::StreamExecutorError;
@@ -38,9 +37,6 @@ pub struct MergeExecutor {
     /// The context of the actor.
     actor_context: ActorContextRef,
 
-    /// Logical Operator Info
-    info: ExecutorInfo,
-
     /// Upstream channels.
     upstreams: Vec<BoxedInput>,
 
@@ -61,7 +57,6 @@ impl MergeExecutor {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
         fragment_id: FragmentId,
         upstream_fragment_id: FragmentId,
         inputs: Vec<BoxedInput>,
@@ -71,7 +66,6 @@ impl MergeExecutor {
     ) -> Self {
         Self {
             actor_context: ctx,
-            info,
             upstreams: inputs,
             fragment_id,
             upstream_fragment_id,
@@ -81,17 +75,12 @@ impl MergeExecutor {
     }
 
     #[cfg(test)]
-    pub fn for_test(inputs: Vec<super::exchange::permit::Receiver>, schema: Schema) -> Self {
+    pub fn for_test(inputs: Vec<super::exchange::permit::Receiver>) -> Self {
         use super::exchange::input::LocalInput;
         use crate::executor::exchange::input::Input;
 
         Self::new(
             ActorContext::for_test(114),
-            ExecutorInfo {
-                schema,
-                pk_indices: vec![],
-                identity: "MergeExecutor".to_string(),
-            },
             514,
             1919,
             inputs
@@ -245,22 +234,10 @@ impl MergeExecutor {
     }
 }
 
-impl Executor for MergeExecutor {
+impl Execute for MergeExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 /// A stream for merging messages from multiple upstreams.
@@ -464,7 +441,7 @@ mod tests {
     use super::*;
     use crate::executor::exchange::input::RemoteInput;
     use crate::executor::exchange::permit::channel_for_test;
-    use crate::executor::{Barrier, Executor, Mutation};
+    use crate::executor::{Barrier, Execute, Mutation};
     use crate::task::test_utils::helper_make_local_actor;
 
     fn build_test_chunk(epoch: u64) -> StreamChunk {
@@ -483,7 +460,7 @@ mod tests {
             txs.push(tx);
             rxs.push(rx);
         }
-        let merger = MergeExecutor::for_test(rxs, Schema::default());
+        let merger = MergeExecutor::for_test(rxs);
         let mut handles = Vec::with_capacity(CHANNEL_NUMBER);
 
         let epochs = (10..1000u64).step_by(10).collect_vec();
@@ -556,8 +533,6 @@ mod tests {
 
     #[tokio::test]
     async fn test_configuration_change() {
-        let schema = Schema { fields: vec![] };
-
         let actor_id = 233;
         let (untouched, old, new) = (234, 235, 238); // upstream actors
         let ctx = Arc::new(SharedContext::for_test());
@@ -592,13 +567,8 @@ mod tests {
             .try_collect()
             .unwrap();
 
-        let merge = MergeExecutor::new(
+        let mut merge = MergeExecutor::new(
             ActorContext::for_test(actor_id),
-            ExecutorInfo {
-                schema,
-                pk_indices: vec![],
-                identity: "MergeExecutor".to_string(),
-            },
             fragment_id,
             upstream_fragment_id,
             inputs,
@@ -609,8 +579,6 @@ mod tests {
         .boxed()
         .execute();
 
-        pin_mut!(merge);
-
         // 2. Take downstream receivers.
         let txs = [untouched, old, new]
             .into_iter()
diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs
index 40ae252c03cf0..1e82768096f36 100644
--- a/src/stream/src/executor/mod.rs
+++ b/src/stream/src/executor/mod.rs
@@ -20,7 +20,6 @@ use await_tree::InstrumentAwait;
 use enum_as_inner::EnumAsInner;
 use futures::stream::BoxStream;
 use futures::{Stream, StreamExt};
-use futures_async_stream::try_stream;
 use itertools::Itertools;
 use risingwave_common::array::StreamChunk;
 use risingwave_common::buffer::Bitmap;
@@ -154,7 +153,6 @@ pub use wrapper::WrapperExecutor;
 
 use self::barrier_align::AlignedMessageStream;
 
-pub type BoxedExecutor = Box<dyn Executor>;
 pub type MessageStreamItem = StreamExecutorResult<Message>;
 pub type BoxedMessageStream = BoxStream<'static, MessageStreamItem>;
 
@@ -166,48 +164,27 @@ pub trait MessageStream = futures::Stream<Item = MessageStreamItem> + Send;
 /// Static information of an executor.
 #[derive(Debug, Default, Clone)]
 pub struct ExecutorInfo {
-    /// See [`Executor::schema`].
+    /// The schema of the OUTPUT of the executor.
     pub schema: Schema,
 
-    /// See [`Executor::pk_indices`].
+    /// The primary key indices of the OUTPUT of the executor.
+    /// Schema is used by both OLAP and streaming, therefore
+    /// pk indices are maintained independently.
     pub pk_indices: PkIndices,
 
-    /// See [`Executor::identity`].
+    /// Identity of the executor.
     pub identity: String,
 }
 
-/// `Executor` supports handling of control messages.
-pub trait Executor: Send + 'static {
+/// [`Execute`] describes the methods an executor should implement to handle control messages.
+pub trait Execute: Send + 'static {
     fn execute(self: Box<Self>) -> BoxedMessageStream;
 
-    /// Return the schema of the OUTPUT of the executor.
-    fn schema(&self) -> &Schema;
-
-    /// Return the primary key indices of the OUTPUT of the executor.
-    /// Schema is used by both OLAP and streaming, therefore
-    /// pk indices are maintained independently.
-    fn pk_indices(&self) -> PkIndicesRef<'_>;
-
-    /// Identity of the executor.
-    fn identity(&self) -> &str;
-
     fn execute_with_epoch(self: Box<Self>, _epoch: u64) -> BoxedMessageStream {
         self.execute()
     }
 
-    #[inline(always)]
-    fn info(&self) -> ExecutorInfo {
-        let schema = self.schema().to_owned();
-        let pk_indices = self.pk_indices().to_owned();
-        let identity = self.identity().to_owned();
-        ExecutorInfo {
-            schema,
-            pk_indices,
-            identity,
-        }
-    }
-
-    fn boxed(self) -> BoxedExecutor
+    fn boxed(self) -> Box<dyn Execute>
     where
         Self: Sized + Send + 'static,
     {
@@ -215,12 +192,64 @@ pub trait Executor: Send + 'static {
     }
 }
 
-impl std::fmt::Debug for BoxedExecutor {
+/// [`Executor`] combines the static information ([`ExecutorInfo`]) and the executable object to
+/// handle messages ([`Execute`]).
+pub struct Executor {
+    info: ExecutorInfo,
+    execute: Box<dyn Execute>,
+}
+
+impl Executor {
+    pub fn new(info: ExecutorInfo, execute: Box<dyn Execute>) -> Self {
+        Self { info, execute }
+    }
+
+    pub fn info(&self) -> &ExecutorInfo {
+        &self.info
+    }
+
+    pub fn schema(&self) -> &Schema {
+        &self.info.schema
+    }
+
+    pub fn pk_indices(&self) -> PkIndicesRef<'_> {
+        &self.info.pk_indices
+    }
+
+    pub fn identity(&self) -> &str {
+        &self.info.identity
+    }
+
+    pub fn execute(self) -> BoxedMessageStream {
+        self.execute.execute()
+    }
+
+    pub fn execute_with_epoch(self, epoch: u64) -> BoxedMessageStream {
+        self.execute.execute_with_epoch(epoch)
+    }
+}
+
+impl std::fmt::Debug for Executor {
     fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
         f.write_str(self.identity())
     }
 }
 
+impl From<(ExecutorInfo, Box<dyn Execute>)> for Executor {
+    fn from((info, execute): (ExecutorInfo, Box<dyn Execute>)) -> Self {
+        Self::new(info, execute)
+    }
+}
+
+impl<E> From<(ExecutorInfo, E)> for Executor
+where
+    E: Execute,
+{
+    fn from((info, execute): (ExecutorInfo, E)) -> Self {
+        Self::new(info, execute.boxed())
+    }
+}
+
 pub const INVALID_EPOCH: u64 = 0;
 
 type UpstreamFragmentId = FragmentId;
diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs
index 69e238a476f74..c96f875765e8e 100644
--- a/src/stream/src/executor/mview/materialize.rs
+++ b/src/stream/src/executor/mview/materialize.rs
@@ -42,16 +42,16 @@ use crate::common::table::state_table::StateTableInner;
 use crate::executor::error::StreamExecutorError;
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::{
-    expect_first_barrier, ActorContext, ActorContextRef, AddMutation, BoxedExecutor,
-    BoxedMessageStream, Executor, ExecutorInfo, Message, Mutation, PkIndicesRef,
-    StreamExecutorResult, UpdateMutation,
+    expect_first_barrier, ActorContext, ActorContextRef, AddMutation, BoxedMessageStream, Execute,
+    Executor, Message, Mutation, StreamExecutorResult, UpdateMutation,
 };
 use crate::task::{ActorId, AtomicU64Ref};
 
 /// `MaterializeExecutor` materializes changes in stream into a materialized view on storage.
 pub struct MaterializeExecutor<S: StateStore, SD: ValueRowSerde> {
-    input: BoxedExecutor,
-    info: ExecutorInfo,
+    input: Executor,
+
+    schema: Schema,
 
     state_table: StateTableInner<S, SD>,
 
@@ -71,8 +71,8 @@ impl<S: StateStore, SD: ValueRowSerde> MaterializeExecutor<S, SD> {
     /// should be `None`.
     #[allow(clippy::too_many_arguments)]
     pub async fn new(
-        input: BoxedExecutor,
-        info: ExecutorInfo,
+        input: Executor,
+        schema: Schema,
         store: S,
         arrange_key: Vec<ColumnOrder>,
         actor_context: ActorContextRef,
@@ -99,7 +99,7 @@ impl<S: StateStore, SD: ValueRowSerde> MaterializeExecutor<S, SD> {
 
         Self {
             input,
-            info,
+            schema,
             state_table,
             arrange_key_indices,
             actor_context,
@@ -115,7 +115,7 @@ impl<S: StateStore, SD: ValueRowSerde> MaterializeExecutor<S, SD> {
         let actor_id_str = self.actor_context.id.to_string();
         let fragment_id_str = self.actor_context.fragment_id.to_string();
 
-        let data_types = self.schema().data_types().clone();
+        let data_types = self.schema.data_types();
         let mut input = self.input.execute();
 
         let barrier = expect_first_barrier(&mut input).await?;
@@ -264,12 +264,11 @@ impl<S: StateStore> MaterializeExecutor<S, BasicSerde> {
     /// Create a new `MaterializeExecutor` without distribution info for test purpose.
     #[allow(clippy::too_many_arguments)]
     pub async fn for_test(
-        input: BoxedExecutor,
+        input: Executor,
         store: S,
         table_id: TableId,
         keys: Vec<ColumnOrder>,
         column_ids: Vec<ColumnId>,
-        executor_id: u64,
         watermark_epoch: AtomicU64Ref,
         conflict_behavior: ConflictBehavior,
     ) -> Self {
@@ -293,14 +292,10 @@ impl<S: StateStore> MaterializeExecutor<S, BasicSerde> {
 
         Self {
             input,
+            schema,
             state_table,
             arrange_key_indices: arrange_columns.clone(),
             actor_context: ActorContext::for_test(0),
-            info: ExecutorInfo {
-                schema,
-                pk_indices: arrange_columns,
-                identity: format!("MaterializeExecutor {:X}", executor_id),
-            },
             materialize_cache: MaterializeCache::new(watermark_epoch, MetricsInfo::for_test()),
             conflict_behavior,
         }
@@ -429,32 +424,15 @@ impl MaterializeBuffer {
         self.buffer
     }
 }
-impl<S: StateStore, SD: ValueRowSerde> Executor for MaterializeExecutor<S, SD> {
+impl<S: StateStore, SD: ValueRowSerde> Execute for MaterializeExecutor<S, SD> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.info.clone()
-    }
 }
 
 impl<S: StateStore, SD: ValueRowSerde> std::fmt::Debug for MaterializeExecutor<S, SD> {
     fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
         f.debug_struct("MaterializeExecutor")
-            .field("info", &self.info())
             .field("arrange_key_indices", &self.arrange_key_indices)
             .finish()
     }
@@ -676,17 +654,14 @@ mod tests {
         );
 
         // Prepare stream executors.
-        let source = MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(chunk1),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(chunk2),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(chunk1),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(chunk2),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
 
         let order_types = vec![OrderType::ascending()];
         let column_descs = vec![
@@ -703,19 +678,17 @@ mod tests {
             vec![0, 1],
         );
 
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store,
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                ConflictBehavior::NoCheck,
-            )
-            .await,
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store,
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            ConflictBehavior::NoCheck,
         )
+        .await
+        .boxed()
         .execute();
         materialize_executor.next().await.transpose().unwrap();
 
@@ -784,17 +757,14 @@ mod tests {
         );
 
         // Prepare stream executors.
-        let source = MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(chunk1),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(chunk2),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(chunk1),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(chunk2),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
 
         let order_types = vec![OrderType::ascending()];
         let column_descs = vec![
@@ -811,19 +781,17 @@ mod tests {
             vec![0, 1],
         );
 
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store,
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                ConflictBehavior::Overwrite,
-            )
-            .await,
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store,
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            ConflictBehavior::Overwrite,
         )
+        .await
+        .boxed()
         .execute();
         materialize_executor.next().await.transpose().unwrap();
 
@@ -880,18 +848,15 @@ mod tests {
         );
 
         // Prepare stream executors.
-        let source = MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(chunk1),
-                Message::Chunk(chunk2),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(chunk3),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(chunk1),
+            Message::Chunk(chunk2),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(chunk3),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
 
         let order_types = vec![OrderType::ascending()];
         let column_descs = vec![
@@ -908,19 +873,17 @@ mod tests {
             vec![0, 1],
         );
 
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store,
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                ConflictBehavior::Overwrite,
-            )
-            .await,
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store,
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            ConflictBehavior::Overwrite,
         )
+        .await
+        .boxed()
         .execute();
         materialize_executor.next().await.transpose().unwrap();
 
@@ -1012,19 +975,16 @@ mod tests {
         );
 
         // Prepare stream executors.
-        let source = MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(chunk1),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(chunk2),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-                Message::Chunk(chunk3),
-                Message::Barrier(Barrier::new_test_barrier(4)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(chunk1),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(chunk2),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+            Message::Chunk(chunk3),
+            Message::Barrier(Barrier::new_test_barrier(4)),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
 
         let order_types = vec![OrderType::ascending()];
         let column_descs = vec![
@@ -1041,19 +1001,17 @@ mod tests {
             vec![0, 1],
         );
 
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store,
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                ConflictBehavior::Overwrite,
-            )
-            .await,
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store,
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            ConflictBehavior::Overwrite,
         )
+        .await
+        .boxed()
         .execute();
         materialize_executor.next().await.transpose().unwrap();
 
@@ -1196,18 +1154,15 @@ mod tests {
         );
 
         // Prepare stream executors.
-        let source = MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(chunk1),
-                Message::Chunk(chunk2),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(chunk3),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(chunk1),
+            Message::Chunk(chunk2),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(chunk3),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
 
         let order_types = vec![OrderType::ascending()];
         let column_descs = vec![
@@ -1224,19 +1179,17 @@ mod tests {
             vec![0, 1],
         );
 
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store,
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                ConflictBehavior::IgnoreConflict,
-            )
-            .await,
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store,
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            ConflictBehavior::IgnoreConflict,
         )
+        .await
+        .boxed()
         .execute();
         materialize_executor.next().await.transpose().unwrap();
 
@@ -1307,15 +1260,12 @@ mod tests {
         );
 
         // Prepare stream executors.
-        let source = MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(chunk1),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(chunk1),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
 
         let order_types = vec![OrderType::ascending()];
         let column_descs = vec![
@@ -1332,19 +1282,17 @@ mod tests {
             vec![0, 1],
         );
 
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store,
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                ConflictBehavior::IgnoreConflict,
-            )
-            .await,
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store,
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            ConflictBehavior::IgnoreConflict,
         )
+        .await
+        .boxed()
         .execute();
         let _msg1 = materialize_executor
             .next()
@@ -1426,19 +1374,16 @@ mod tests {
         );
 
         // Prepare stream executors.
-        let source = MockSource::with_messages(
-            schema.clone(),
-            PkIndices::new(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(chunk1),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(chunk2),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-                Message::Chunk(chunk3),
-                Message::Barrier(Barrier::new_test_barrier(4)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(chunk1),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(chunk2),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+            Message::Chunk(chunk3),
+            Message::Barrier(Barrier::new_test_barrier(4)),
+        ])
+        .into_executor(schema.clone(), PkIndices::new());
 
         let order_types = vec![OrderType::ascending()];
         let column_descs = vec![
@@ -1455,19 +1400,17 @@ mod tests {
             vec![0, 1],
         );
 
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store,
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                ConflictBehavior::IgnoreConflict,
-            )
-            .await,
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store,
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            ConflictBehavior::IgnoreConflict,
         )
+        .await
+        .boxed()
         .execute();
         materialize_executor.next().await.transpose().unwrap();
 
@@ -1635,21 +1578,20 @@ mod tests {
             .chain(iter::once(Message::Barrier(Barrier::new_test_barrier(2))))
             .collect();
         // Prepare stream executors.
-        let source = MockSource::with_messages(schema.clone(), PkIndices::new(), messages);
-
-        let mut materialize_executor = Box::new(
-            MaterializeExecutor::for_test(
-                Box::new(source),
-                memory_state_store.clone(),
-                table_id,
-                vec![ColumnOrder::new(0, OrderType::ascending())],
-                column_ids,
-                1,
-                Arc::new(AtomicU64::new(0)),
-                conflict_behavior,
-            )
-            .await,
+        let source =
+            MockSource::with_messages(messages).into_executor(schema.clone(), PkIndices::new());
+
+        let mut materialize_executor = MaterializeExecutor::for_test(
+            source,
+            memory_state_store.clone(),
+            table_id,
+            vec![ColumnOrder::new(0, OrderType::ascending())],
+            column_ids,
+            Arc::new(AtomicU64::new(0)),
+            conflict_behavior,
         )
+        .await
+        .boxed()
         .execute();
         materialize_executor.expect_barrier().await;
 
diff --git a/src/stream/src/executor/no_op.rs b/src/stream/src/executor/no_op.rs
index f116eced8864d..ac12bf99d5d7f 100644
--- a/src/stream/src/executor/no_op.rs
+++ b/src/stream/src/executor/no_op.rs
@@ -12,44 +12,23 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-use risingwave_common::catalog::Schema;
-
-use super::{
-    ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef,
-};
+use super::{ActorContextRef, BoxedMessageStream, Execute, Executor};
 
 /// No-op executor directly forwards the input stream. Currently used to break the multiple edges in
 /// the fragment graph.
 pub struct NoOpExecutor {
     _ctx: ActorContextRef,
-    info: ExecutorInfo,
-    input: BoxedExecutor,
+    input: Executor,
 }
 
 impl NoOpExecutor {
-    pub fn new(ctx: ActorContextRef, info: ExecutorInfo, input: BoxedExecutor) -> Self {
-        Self {
-            _ctx: ctx,
-            info,
-            input,
-        }
+    pub fn new(ctx: ActorContextRef, input: Executor) -> Self {
+        Self { _ctx: ctx, input }
     }
 }
 
-impl Executor for NoOpExecutor {
+impl Execute for NoOpExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.input.execute()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs
index 9d00646a9bcc1..755e48f325965 100644
--- a/src/stream/src/executor/now.rs
+++ b/src/stream/src/executor/now.rs
@@ -18,7 +18,6 @@ use std::ops::Bound::Unbounded;
 use futures::{pin_mut, StreamExt};
 use futures_async_stream::try_stream;
 use risingwave_common::array::{Op, StreamChunk};
-use risingwave_common::catalog::Schema;
 use risingwave_common::row::{self, OwnedRow};
 use risingwave_common::types::{DataType, Datum};
 use risingwave_storage::StateStore;
@@ -26,13 +25,12 @@ use tokio::sync::mpsc::UnboundedReceiver;
 use tokio_stream::wrappers::UnboundedReceiverStream;
 
 use super::{
-    Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, Mutation, PkIndicesRef,
-    StreamExecutorError, Watermark,
+    Barrier, BoxedMessageStream, Execute, Message, Mutation, StreamExecutorError, Watermark,
 };
 use crate::common::table::state_table::StateTable;
 
 pub struct NowExecutor<S: StateStore> {
-    info: ExecutorInfo,
+    data_types: Vec<DataType>,
 
     /// Receiver of barrier channel.
     barrier_receiver: UnboundedReceiver<Barrier>,
@@ -42,24 +40,23 @@ pub struct NowExecutor<S: StateStore> {
 
 impl<S: StateStore> NowExecutor<S> {
     pub fn new(
-        info: ExecutorInfo,
+        data_types: Vec<DataType>,
         barrier_receiver: UnboundedReceiver<Barrier>,
         state_table: StateTable<S>,
     ) -> Self {
         Self {
-            info,
+            data_types,
             barrier_receiver,
             state_table,
         }
     }
 
     #[try_stream(ok = Message, error = StreamExecutorError)]
-    async fn into_stream(self) {
+    async fn execute_inner(self) {
         let Self {
+            data_types,
             barrier_receiver,
             mut state_table,
-            info,
-            ..
         } = self;
 
         // Whether the executor is paused.
@@ -131,15 +128,12 @@ impl<S: StateStore> NowExecutor<S> {
                 let row = row::once(&timestamp);
                 state_table.update(last_row, row);
 
-                StreamChunk::from_rows(
-                    &[(Op::Delete, last_row), (Op::Insert, row)],
-                    &info.schema.data_types(),
-                )
+                StreamChunk::from_rows(&[(Op::Delete, last_row), (Op::Insert, row)], &data_types)
             } else {
                 let row = row::once(&timestamp);
                 state_table.insert(row);
 
-                StreamChunk::from_rows(&[(Op::Insert, row)], &info.schema.data_types())
+                StreamChunk::from_rows(&[(Op::Insert, row)], &data_types)
             };
 
             yield Message::Chunk(stream_chunk);
@@ -155,28 +149,16 @@ impl<S: StateStore> NowExecutor<S> {
     }
 }
 
-impl<S: StateStore> Executor for NowExecutor<S> {
+impl<S: StateStore> Execute for NowExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
-        self.into_stream().boxed()
-    }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
+        self.execute_inner().boxed()
     }
 }
 
 #[cfg(test)]
 mod tests {
     use risingwave_common::array::StreamChunk;
-    use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId};
+    use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId};
     use risingwave_common::test_prelude::StreamChunkTestExt;
     use risingwave_common::types::{DataType, ScalarImpl};
     use risingwave_storage::memory::MemoryStateStore;
@@ -186,8 +168,7 @@ mod tests {
     use crate::common::table::state_table::StateTable;
     use crate::executor::test_utils::StreamExecutorTestExt;
     use crate::executor::{
-        Barrier, BoxedMessageStream, Executor, ExecutorInfo, Mutation, StreamExecutorResult,
-        Watermark,
+        Barrier, BoxedMessageStream, Execute, Mutation, StreamExecutorResult, Watermark,
     };
 
     #[tokio::test]
@@ -410,22 +391,8 @@ mod tests {
 
         let (sender, barrier_receiver) = unbounded_channel();
 
-        let schema = Schema::new(vec![Field {
-            data_type: DataType::Timestamptz,
-            name: String::from("now"),
-            sub_fields: vec![],
-            type_name: String::default(),
-        }]);
-
-        let now_executor = NowExecutor::new(
-            ExecutorInfo {
-                schema,
-                pk_indices: vec![],
-                identity: "NowExecutor".to_string(),
-            },
-            barrier_receiver,
-            state_table,
-        );
-        (sender, Box::new(now_executor).execute())
+        let now_executor =
+            NowExecutor::new(vec![DataType::Timestamptz], barrier_receiver, state_table);
+        (sender, now_executor.boxed().execute())
     }
 }
diff --git a/src/stream/src/executor/over_window/eowc.rs b/src/stream/src/executor/over_window/eowc.rs
index 601a2f536aa00..32f1fafe9b8db 100644
--- a/src/stream/src/executor/over_window/eowc.rs
+++ b/src/stream/src/executor/over_window/eowc.rs
@@ -39,8 +39,8 @@ use crate::cache::{new_unbounded, ManagedLruCache};
 use crate::common::metrics::MetricsInfo;
 use crate::common::table::state_table::StateTable;
 use crate::executor::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor,
-    ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult,
+    expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message,
+    StreamExecutorError, StreamExecutorResult,
 };
 use crate::task::AtomicU64Ref;
 
@@ -95,14 +95,14 @@ type PartitionCache = ManagedLruCache<MemcmpEncoded, Partition>; // TODO(rc): us
 /// - `WindowState` should output agg result for `curr output row`.
 /// - Recover: iterate through state table, push rows to `WindowState`, ignore ready windows.
 pub struct EowcOverWindowExecutor<S: StateStore> {
-    input: Box<dyn Executor>,
+    input: Executor,
     inner: ExecutorInner<S>,
 }
 
 struct ExecutorInner<S: StateStore> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
+    schema: Schema,
     calls: Vec<WindowFuncCall>,
     input_pk_indices: Vec<usize>,
     partition_key_indices: Vec<usize>,
@@ -117,30 +117,18 @@ struct ExecutionVars<S: StateStore> {
     _phantom: PhantomData<S>,
 }
 
-impl<S: StateStore> Executor for EowcOverWindowExecutor<S> {
+impl<S: StateStore> Execute for EowcOverWindowExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.executor_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.inner.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.inner.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info.identity
-    }
 }
 
 pub struct EowcOverWindowExecutorArgs<S: StateStore> {
     pub actor_ctx: ActorContextRef,
-    pub info: ExecutorInfo,
 
-    pub input: BoxedExecutor,
+    pub input: Executor,
 
+    pub schema: Schema,
     pub calls: Vec<WindowFuncCall>,
     pub partition_key_indices: Vec<usize>,
     pub order_key_index: usize,
@@ -150,13 +138,13 @@ pub struct EowcOverWindowExecutorArgs<S: StateStore> {
 
 impl<S: StateStore> EowcOverWindowExecutor<S> {
     pub fn new(args: EowcOverWindowExecutorArgs<S>) -> Self {
-        let input_info = args.input.info();
+        let input_info = args.input.info().clone();
 
         Self {
             input: args.input,
             inner: ExecutorInner {
                 actor_ctx: args.actor_ctx,
-                info: args.info,
+                schema: args.schema,
                 calls: args.calls,
                 input_pk_indices: input_info.pk_indices,
                 partition_key_indices: args.partition_key_indices,
@@ -237,7 +225,7 @@ impl<S: StateStore> EowcOverWindowExecutor<S> {
         vars: &mut ExecutionVars<S>,
         chunk: StreamChunk,
     ) -> StreamExecutorResult<Option<StreamChunk>> {
-        let mut builders = this.info.schema.create_array_builders(chunk.capacity()); // just an estimate
+        let mut builders = this.schema.create_array_builders(chunk.capacity()); // just an estimate
 
         // We assume that the input is sorted by order key.
         for record in chunk.records() {
@@ -318,7 +306,7 @@ impl<S: StateStore> EowcOverWindowExecutor<S> {
                     for key in keys_to_evict {
                         let order_key = memcmp_encoding::decode_row(
                             &key.order_key,
-                            &[this.info.schema[this.order_key_index].data_type()],
+                            &[this.schema[this.order_key_index].data_type()],
                             &[OrderType::ascending()],
                         )?;
                         let state_row_pk = (&partition_key).chain(order_key).chain(key.pk);
diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs
index df4e3ac5c84c2..a7245c57f368c 100644
--- a/src/stream/src/executor/over_window/general.rs
+++ b/src/stream/src/executor/over_window/general.rs
@@ -23,6 +23,7 @@ use futures_async_stream::try_stream;
 use itertools::Itertools;
 use risingwave_common::array::stream_record::Record;
 use risingwave_common::array::{Op, RowRef, StreamChunk};
+use risingwave_common::catalog::Schema;
 use risingwave_common::row::{OwnedRow, Row, RowExt};
 use risingwave_common::session_config::OverWindowCachePolicy as CachePolicy;
 use risingwave_common::types::{DataType, DefaultOrdered};
@@ -46,8 +47,8 @@ use crate::common::StreamChunkBuilder;
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::over_window::over_partition::AffectedRange;
 use crate::executor::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message,
-    StreamExecutorError, StreamExecutorResult,
+    expect_first_barrier, ActorContextRef, Execute, Executor, Message, StreamExecutorError,
+    StreamExecutorResult,
 };
 use crate::task::AtomicU64Ref;
 
@@ -57,14 +58,14 @@ use crate::task::AtomicU64Ref;
 /// - State table schema = output schema, state table pk = `partition key | order key | input pk`.
 /// - Output schema = input schema + window function results.
 pub struct OverWindowExecutor<S: StateStore> {
-    input: Box<dyn Executor>,
+    input: Executor,
     inner: ExecutorInner<S>,
 }
 
 struct ExecutorInner<S: StateStore> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
+    schema: Schema,
     calls: Vec<WindowFuncCall>,
     partition_key_indices: Vec<usize>,
     order_key_indices: Vec<usize>,
@@ -98,22 +99,10 @@ struct ExecutionStats {
     cache_lookup: u64,
 }
 
-impl<S: StateStore> Executor for OverWindowExecutor<S> {
+impl<S: StateStore> Execute for OverWindowExecutor<S> {
     fn execute(self: Box<Self>) -> crate::executor::BoxedMessageStream {
         self.executor_inner().boxed()
     }
-
-    fn schema(&self) -> &risingwave_common::catalog::Schema {
-        &self.inner.info.schema
-    }
-
-    fn pk_indices(&self) -> crate::executor::PkIndicesRef<'_> {
-        &self.inner.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info.identity
-    }
 }
 
 impl<S: StateStore> ExecutorInner<S> {
@@ -145,10 +134,10 @@ impl<S: StateStore> ExecutorInner<S> {
 
 pub struct OverWindowExecutorArgs<S: StateStore> {
     pub actor_ctx: ActorContextRef,
-    pub info: ExecutorInfo,
 
-    pub input: BoxedExecutor,
+    pub input: Executor,
 
+    pub schema: Schema,
     pub calls: Vec<WindowFuncCall>,
     pub partition_key_indices: Vec<usize>,
     pub order_key_indices: Vec<usize>,
@@ -164,7 +153,7 @@ pub struct OverWindowExecutorArgs<S: StateStore> {
 
 impl<S: StateStore> OverWindowExecutor<S> {
     pub fn new(args: OverWindowExecutorArgs<S>) -> Self {
-        let input_info = args.input.info();
+        let input_info = args.input.info().clone();
         let input_schema = &input_info.schema;
 
         let has_unbounded_frame = args
@@ -195,7 +184,7 @@ impl<S: StateStore> OverWindowExecutor<S> {
             input: args.input,
             inner: ExecutorInner {
                 actor_ctx: args.actor_ctx,
-                info: args.info,
+                schema: args.schema,
                 calls: args.calls,
                 partition_key_indices: args.partition_key_indices,
                 order_key_indices: args.order_key_indices,
@@ -328,8 +317,7 @@ impl<S: StateStore> OverWindowExecutor<S> {
         // `input pk` => `Record`
         let mut key_change_update_buffer: BTreeMap<DefaultOrdered<OwnedRow>, Record<OwnedRow>> =
             BTreeMap::new();
-        let mut chunk_builder =
-            StreamChunkBuilder::new(this.chunk_size, this.info.schema.data_types());
+        let mut chunk_builder = StreamChunkBuilder::new(this.chunk_size, this.schema.data_types());
 
         // Prepare things needed by metrics.
         let actor_id = this.actor_ctx.id.to_string();
diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs
index efc381560e75d..c44d93e22639a 100644
--- a/src/stream/src/executor/project.rs
+++ b/src/stream/src/executor/project.rs
@@ -14,27 +14,30 @@
 
 use std::fmt::{Debug, Formatter};
 
+use futures::StreamExt;
+use futures_async_stream::try_stream;
 use multimap::MultiMap;
 use risingwave_common::array::StreamChunk;
-use risingwave_common::catalog::Schema;
 use risingwave_common::row::{Row, RowExt};
-use risingwave_common::types::ToOwnedDatum;
+use risingwave_common::types::{ScalarImpl, ToOwnedDatum};
 use risingwave_common::util::iter_util::ZipEqFast;
 use risingwave_expr::expr::NonStrictExpression;
 
-use super::*;
+use super::{
+    ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError,
+    StreamExecutorResult, Watermark,
+};
 
 /// `ProjectExecutor` project data with the `expr`. The `expr` takes a chunk of data,
 /// and returns a new data chunk. And then, `ProjectExecutor` will insert, delete
 /// or update element into next operator according to the result of the expression.
 pub struct ProjectExecutor {
-    input: BoxedExecutor,
+    input: Executor,
     inner: Inner,
 }
 
 struct Inner {
     _ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// Expressions of the current projection.
     exprs: Vec<NonStrictExpression>,
@@ -55,8 +58,7 @@ impl ProjectExecutor {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
-        input: Box<dyn Executor>,
+        input: Executor,
         exprs: Vec<NonStrictExpression>,
         watermark_derivations: MultiMap<usize, usize>,
         nondecreasing_expr_indices: Vec<usize>,
@@ -67,7 +69,6 @@ impl ProjectExecutor {
             input,
             inner: Inner {
                 _ctx: ctx,
-                info,
                 exprs,
                 watermark_derivations,
                 nondecreasing_expr_indices,
@@ -86,19 +87,7 @@ impl Debug for ProjectExecutor {
     }
 }
 
-impl Executor for ProjectExecutor {
-    fn schema(&self) -> &Schema {
-        &self.inner.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.inner.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info.identity
-    }
-
+impl Execute for ProjectExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.inner.execute(self.input).boxed()
     }
@@ -142,8 +131,8 @@ impl Inner {
                 ret.push(derived_watermark);
             } else {
                 warn!(
-                    "{} derive a NULL watermark with the expression {}!",
-                    self.info.identity, out_col_idx
+                    "a NULL watermark is derived with the expression {}!",
+                    out_col_idx
                 );
             }
         }
@@ -151,7 +140,7 @@ impl Inner {
     }
 
     #[try_stream(ok = Message, error = StreamExecutorError)]
-    async fn execute(mut self, input: BoxedExecutor) {
+    async fn execute(mut self, input: Executor) {
         #[for_await]
         for msg in input.execute() {
             let msg = msg?;
@@ -241,28 +230,20 @@ mod tests {
             ],
         };
         let pk_indices = vec![0];
-        let (mut tx, source) = MockSource::channel(schema, pk_indices);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, pk_indices);
 
         let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)");
 
-        let info = ExecutorInfo {
-            schema: Schema {
-                fields: vec![Field::unnamed(DataType::Int64)],
-            },
-            pk_indices: vec![],
-            identity: "ProjectExecutor".to_string(),
-        };
-
-        let project = Box::new(ProjectExecutor::new(
+        let project = ProjectExecutor::new(
             ActorContext::for_test(123),
-            info,
-            Box::new(source),
+            source,
             vec![test_expr],
             MultiMap::new(),
             vec![],
             0.0,
-        ));
-        let mut project = project.execute();
+        );
+        let mut project = project.boxed().execute();
 
         tx.push_barrier(1, false);
         let barrier = project.next().await.unwrap().unwrap();
@@ -329,34 +310,22 @@ mod tests {
                 Field::unnamed(DataType::Int64),
             ],
         };
-        let (mut tx, source) = MockSource::channel(schema, PkIndices::new());
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, PkIndices::new());
 
         let a_expr = build_from_pretty("(add:int8 $0:int8 1:int8)");
         let b_expr = build_from_pretty("(subtract:int8 $0:int8 1:int8)");
         let c_expr = NonStrictExpression::for_test(DummyNondecreasingExpr);
 
-        let info = ExecutorInfo {
-            schema: Schema {
-                fields: vec![
-                    Field::unnamed(DataType::Int64),
-                    Field::unnamed(DataType::Int64),
-                    Field::unnamed(DataType::Int64),
-                ],
-            },
-            pk_indices: vec![],
-            identity: "ProjectExecutor".to_string(),
-        };
-
-        let project = Box::new(ProjectExecutor::new(
+        let project = ProjectExecutor::new(
             ActorContext::for_test(123),
-            info,
-            Box::new(source),
+            source,
             vec![a_expr, b_expr, c_expr],
             MultiMap::from_iter(vec![(0, 0), (0, 1)].into_iter()),
             vec![2],
             0.0,
-        ));
-        let mut project = project.execute();
+        );
+        let mut project = project.boxed().execute();
 
         tx.push_barrier(1, false);
         tx.push_int64_watermark(0, 100);
diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs
index 5167a71db2cb8..9fadb5949dac2 100644
--- a/src/stream/src/executor/project_set.rs
+++ b/src/stream/src/executor/project_set.rs
@@ -20,7 +20,6 @@ use futures_async_stream::try_stream;
 use multimap::MultiMap;
 use risingwave_common::array::{Op, StreamChunk};
 use risingwave_common::bail;
-use risingwave_common::catalog::Schema;
 use risingwave_common::row::{Row, RowExt};
 use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum};
 use risingwave_common::util::iter_util::ZipEqFast;
@@ -28,10 +27,7 @@ use risingwave_expr::expr::{LogReport, NonStrictExpression};
 use risingwave_expr::table_function::ProjectSetSelectItem;
 
 use super::error::StreamExecutorError;
-use super::{
-    ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef,
-    StreamExecutorResult, Watermark,
-};
+use super::{ActorContextRef, Execute, Executor, Message, StreamExecutorResult, Watermark};
 use crate::common::StreamChunkBuilder;
 
 const PROJ_ROW_ID_OFFSET: usize = 1;
@@ -40,13 +36,12 @@ const PROJ_ROW_ID_OFFSET: usize = 1;
 /// and returns a new data chunk. And then, `ProjectSetExecutor` will insert, delete
 /// or update element into next operator according to the result of the expression.
 pub struct ProjectSetExecutor {
-    input: BoxedExecutor,
+    input: Executor,
     inner: Inner,
 }
 
 struct Inner {
     _ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// Expressions of the current project_section.
     select_list: Vec<ProjectSetSelectItem>,
@@ -62,8 +57,7 @@ impl ProjectSetExecutor {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
-        input: Box<dyn Executor>,
+        input: Executor,
         select_list: Vec<ProjectSetSelectItem>,
         chunk_size: usize,
         watermark_derivations: MultiMap<usize, usize>,
@@ -71,7 +65,6 @@ impl ProjectSetExecutor {
     ) -> Self {
         let inner = Inner {
             _ctx: ctx,
-            info,
             select_list,
             chunk_size,
             watermark_derivations,
@@ -90,27 +83,15 @@ impl Debug for ProjectSetExecutor {
     }
 }
 
-impl Executor for ProjectSetExecutor {
+impl Execute for ProjectSetExecutor {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.inner.execute(self.input).boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.inner.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.inner.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info.identity
-    }
 }
 
 impl Inner {
     #[try_stream(ok = Message, error = StreamExecutorError)]
-    async fn execute(self, input: BoxedExecutor) {
+    async fn execute(self, input: Executor) {
         assert!(!self.select_list.is_empty());
         // First column will be `projected_row_id`, which represents the index in the
         // output table
@@ -270,8 +251,8 @@ impl Inner {
                 ret.push(derived_watermark);
             } else {
                 warn!(
-                    "{} derive a NULL watermark with the expression {}!",
-                    self.info.identity, expr_idx
+                    "a NULL watermark is derived with the expression {}!",
+                    expr_idx
                 );
             }
         }
diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs
index 1bb83009a203e..0b51bab665d41 100644
--- a/src/stream/src/executor/rearranged_chain.rs
+++ b/src/stream/src/executor/rearranged_chain.rs
@@ -19,12 +19,9 @@ use futures::stream::select_with_strategy;
 use futures::{stream, StreamExt};
 use futures_async_stream::try_stream;
 use risingwave_common::array::StreamChunk;
-use risingwave_common::catalog::Schema;
 
 use super::error::StreamExecutorError;
-use super::{
-    expect_first_barrier, Barrier, BoxedExecutor, Executor, ExecutorInfo, Message, MessageStream,
-};
+use super::{expect_first_barrier, Barrier, Execute, Executor, Message, MessageStream};
 use crate::task::{ActorId, CreateMviewProgress};
 
 /// `ChainExecutor` is an executor that enables synchronization between the existing stream and
@@ -35,11 +32,9 @@ use crate::task::{ActorId, CreateMviewProgress};
 /// [`RearrangedChainExecutor`] resolves the latency problem when creating MV with a huge amount of
 /// existing data, by rearranging the barrier from the upstream. Check the design doc for details.
 pub struct RearrangedChainExecutor {
-    info: ExecutorInfo,
+    snapshot: Executor,
 
-    snapshot: BoxedExecutor,
-
-    upstream: BoxedExecutor,
+    upstream: Executor,
 
     progress: CreateMviewProgress,
 
@@ -84,14 +79,8 @@ impl RearrangedMessage {
 }
 
 impl RearrangedChainExecutor {
-    pub fn new(
-        info: ExecutorInfo,
-        snapshot: BoxedExecutor,
-        upstream: BoxedExecutor,
-        progress: CreateMviewProgress,
-    ) -> Self {
+    pub fn new(snapshot: Executor, upstream: Executor, progress: CreateMviewProgress) -> Self {
         Self {
-            info,
             snapshot,
             upstream,
             actor_id: progress.actor_id(),
@@ -288,26 +277,10 @@ impl RearrangedChainExecutor {
     }
 }
 
-impl Executor for RearrangedChainExecutor {
+impl Execute for RearrangedChainExecutor {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.info.clone()
-    }
 }
 
 // TODO: add new unit tests for rearranged chain
diff --git a/src/stream/src/executor/receiver.rs b/src/stream/src/executor/receiver.rs
index 273a3fcf1b339..62f2dd694f910 100644
--- a/src/stream/src/executor/receiver.rs
+++ b/src/stream/src/executor/receiver.rs
@@ -17,7 +17,6 @@ use anyhow::Context;
 use futures::StreamExt;
 use futures_async_stream::try_stream;
 use itertools::Itertools;
-use risingwave_common::catalog::Schema;
 use tokio::time::Instant;
 
 use super::exchange::input::BoxedInput;
@@ -25,17 +24,12 @@ use super::ActorContextRef;
 use crate::executor::exchange::input::new_input;
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::utils::ActorInputMetrics;
-use crate::executor::{
-    expect_first_barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef,
-};
+use crate::executor::{expect_first_barrier, BoxedMessageStream, Execute, Message};
 use crate::task::{FragmentId, SharedContext};
 /// `ReceiverExecutor` is used along with a channel. After creating a mpsc channel,
 /// there should be a `ReceiverExecutor` running in the background, so as to push
 /// messages down to the executors.
 pub struct ReceiverExecutor {
-    /// Logical Operator Info
-    info: ExecutorInfo,
-
     /// Input from upstream.
     input: BoxedInput,
 
@@ -57,10 +51,7 @@ pub struct ReceiverExecutor {
 
 impl std::fmt::Debug for ReceiverExecutor {
     fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
-        f.debug_struct("ReceiverExecutor")
-            .field("schema", &self.info.schema)
-            .field("pk_indices", &self.info.pk_indices)
-            .finish()
+        f.debug_struct("ReceiverExecutor").finish()
     }
 }
 
@@ -68,7 +59,6 @@ impl ReceiverExecutor {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
         fragment_id: FragmentId,
         upstream_fragment_id: FragmentId,
         input: BoxedInput,
@@ -78,7 +68,6 @@ impl ReceiverExecutor {
     ) -> Self {
         Self {
             input,
-            info,
             actor_context: ctx,
             upstream_fragment_id,
             metrics,
@@ -95,11 +84,6 @@ impl ReceiverExecutor {
 
         Self::new(
             ActorContext::for_test(114),
-            ExecutorInfo {
-                schema: Schema::default(),
-                pk_indices: vec![],
-                identity: "ReceiverExecutor".to_string(),
-            },
             514,
             1919,
             LocalInput::new(input, 0).boxed_input(),
@@ -110,7 +94,7 @@ impl ReceiverExecutor {
     }
 }
 
-impl Executor for ReceiverExecutor {
+impl Execute for ReceiverExecutor {
     fn execute(mut self: Box<Self>) -> BoxedMessageStream {
         let actor_id = self.actor_context.id;
 
@@ -207,22 +191,6 @@ impl Executor for ReceiverExecutor {
 
         stream.boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.info.clone()
-    }
 }
 
 #[cfg(test)]
@@ -235,13 +203,11 @@ mod tests {
     use risingwave_pb::stream_plan::update_mutation::MergeUpdate;
 
     use super::*;
-    use crate::executor::{ActorContext, Barrier, Executor, Mutation, UpdateMutation};
+    use crate::executor::{ActorContext, Barrier, Execute, Mutation, UpdateMutation};
     use crate::task::test_utils::helper_make_local_actor;
 
     #[tokio::test]
     async fn test_configuration_change() {
-        let schema = Schema { fields: vec![] };
-
         let actor_id = 233;
         let (old, new) = (114, 514); // old and new upstream actor id
 
@@ -271,15 +237,8 @@ mod tests {
         )
         .unwrap();
 
-        let info = ExecutorInfo {
-            schema,
-            pk_indices: vec![],
-            identity: "ReceiverExecutor".to_string(),
-        };
-
         let receiver = ReceiverExecutor::new(
             ActorContext::for_test(actor_id),
-            info,
             fragment_id,
             upstream_fragment_id,
             input,
diff --git a/src/stream/src/executor/row_id_gen.rs b/src/stream/src/executor/row_id_gen.rs
index fe0ed6d908925..0a0fe1ffe84e1 100644
--- a/src/stream/src/executor/row_id_gen.rs
+++ b/src/stream/src/executor/row_id_gen.rs
@@ -19,23 +19,19 @@ use risingwave_common::array::{
     Array, ArrayBuilder, ArrayRef, Op, SerialArrayBuilder, StreamChunk,
 };
 use risingwave_common::buffer::Bitmap;
-use risingwave_common::catalog::Schema;
 use risingwave_common::hash::VnodeBitmapExt;
 use risingwave_common::types::Serial;
 use risingwave_common::util::iter_util::ZipEqFast;
 use risingwave_common::util::row_id::RowIdGenerator;
 
-use super::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, PkIndicesRef,
-};
+use super::{expect_first_barrier, ActorContextRef, Execute, Executor};
 use crate::executor::{Message, StreamExecutorError};
 
 /// [`RowIdGenExecutor`] generates row id for data, where the user has not specified a pk.
 pub struct RowIdGenExecutor {
     ctx: ActorContextRef,
-    info: ExecutorInfo,
 
-    upstream: Option<BoxedExecutor>,
+    upstream: Option<Executor>,
 
     row_id_index: usize,
 
@@ -45,14 +41,12 @@ pub struct RowIdGenExecutor {
 impl RowIdGenExecutor {
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
-        upstream: BoxedExecutor,
+        upstream: Executor,
         row_id_index: usize,
         vnodes: Bitmap,
     ) -> Self {
         Self {
             ctx,
-            info,
             upstream: Some(upstream),
             row_id_index,
             row_id_generator: Self::new_generator(&vnodes),
@@ -126,22 +120,10 @@ impl RowIdGenExecutor {
     }
 }
 
-impl Executor for RowIdGenExecutor {
+impl Execute for RowIdGenExecutor {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 #[cfg(test)]
@@ -154,7 +136,7 @@ mod tests {
 
     use super::*;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::{ActorContext, Executor};
+    use crate::executor::{ActorContext, Execute};
 
     #[tokio::test]
     async fn test_row_id_gen_executor() {
@@ -165,20 +147,16 @@ mod tests {
         let pk_indices = vec![0];
         let row_id_index = 0;
         let row_id_generator = Bitmap::ones(VirtualNode::COUNT);
-        let (mut tx, upstream) = MockSource::channel(schema.clone(), pk_indices.clone());
-        let row_id_gen_executor = Box::new(RowIdGenExecutor::new(
+        let (mut tx, upstream) = MockSource::channel();
+        let upstream = upstream.into_executor(schema.clone(), pk_indices.clone());
+
+        let row_id_gen_executor = RowIdGenExecutor::new(
             ActorContext::for_test(233),
-            ExecutorInfo {
-                schema,
-                pk_indices,
-                identity: "RowIdGenExecutor".to_string(),
-            },
-            Box::new(upstream),
+            upstream,
             row_id_index,
             row_id_generator,
-        ));
-
-        let mut row_id_gen_executor = row_id_gen_executor.execute();
+        );
+        let mut row_id_gen_executor = row_id_gen_executor.boxed().execute();
 
         // Init barrier
         tx.push_barrier(1, false);
diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs
index b8915a070dbc5..a863fc0432876 100644
--- a/src/stream/src/executor/simple_agg.rs
+++ b/src/stream/src/executor/simple_agg.rs
@@ -47,7 +47,7 @@ use crate::task::AtomicU64Ref;
 /// Therefore, we "automatically" implemented a window function inside
 /// `SimpleAggExecutor`.
 pub struct SimpleAggExecutor<S: StateStore> {
-    input: Box<dyn Executor>,
+    input: Executor,
     inner: ExecutorInner<S>,
 }
 
@@ -111,27 +111,15 @@ struct ExecutionVars<S: StateStore> {
     state_changed: bool,
 }
 
-impl<S: StateStore> Executor for SimpleAggExecutor<S> {
+impl<S: StateStore> Execute for SimpleAggExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.inner.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.inner.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info.identity
-    }
 }
 
 impl<S: StateStore> SimpleAggExecutor<S> {
     pub fn new(args: AggExecutorArgs<S, SimpleAggExecutorExtraArgs>) -> StreamResult<Self> {
-        let input_info = args.input.info();
+        let input_info = args.input.info().clone();
         Ok(Self {
             input: args.input,
             inner: ExecutorInner {
@@ -335,7 +323,8 @@ mod tests {
                 Field::unnamed(DataType::Int64),
             ],
         };
-        let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, vec![2]);
         tx.push_barrier(1, false);
         tx.push_barrier(2, false);
         tx.push_chunk(StreamChunk::from_pretty(
@@ -364,7 +353,7 @@ mod tests {
         let simple_agg = new_boxed_simple_agg_executor(
             ActorContext::for_test(123),
             store,
-            Box::new(source),
+            source,
             false,
             agg_calls,
             0,
diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs
index 4e64c3456bc8f..7928308c3f94a 100644
--- a/src/stream/src/executor/sink.rs
+++ b/src/stream/src/executor/sink.rs
@@ -34,7 +34,7 @@ use risingwave_connector::sink::{
 use thiserror_ext::AsReport;
 
 use super::error::{StreamExecutorError, StreamExecutorResult};
-use super::{BoxedExecutor, Executor, ExecutorInfo, Message, PkIndices};
+use super::{Execute, Executor, ExecutorInfo, Message, PkIndices};
 use crate::executor::{
     expect_first_barrier, ActorContextRef, BoxedMessageStream, MessageStream, Mutation,
 };
@@ -43,7 +43,7 @@ use crate::task::ActorId;
 pub struct SinkExecutor<F: LogStoreFactory> {
     actor_context: ActorContextRef,
     info: ExecutorInfo,
-    input: BoxedExecutor,
+    input: Executor,
     sink: SinkImpl,
     input_columns: Vec<ColumnCatalog>,
     sink_param: SinkParam,
@@ -83,7 +83,7 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
     pub async fn new(
         actor_context: ActorContextRef,
         info: ExecutorInfo,
-        input: BoxedExecutor,
+        input: Executor,
         sink_writer_param: SinkWriterParam,
         sink_param: SinkParam,
         columns: Vec<ColumnCatalog>,
@@ -412,22 +412,10 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
     }
 }
 
-impl<F: LogStoreFactory> Executor for SinkExecutor<F> {
+impl<F: LogStoreFactory> Execute for SinkExecutor<F> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 #[cfg(test)]
@@ -475,28 +463,25 @@ mod test {
             .collect();
         let pk_indices = vec![0];
 
-        let mock = MockSource::with_messages(
-            schema.clone(),
-            pk_indices.clone(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
-                    " I I I
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
+                " I I I
                     + 3 2 1",
-                ))),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
-                    "  I I I
+            ))),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
+                "  I I I
                     U- 3 2 1
                     U+ 3 4 1
                      + 5 6 7",
-                ))),
-                Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
-                    " I I I
+            ))),
+            Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
+                " I I I
                     - 5 6 7",
-                ))),
-            ],
-        );
+            ))),
+        ])
+        .into_executor(schema.clone(), pk_indices.clone());
 
         let sink_param = SinkParam {
             sink_id: 0.into(),
@@ -522,7 +507,7 @@ mod test {
         let sink_executor = SinkExecutor::new(
             ActorContext::for_test(0),
             info,
-            Box::new(mock),
+            source,
             SinkWriterParam::for_test(),
             sink_param,
             columns.clone(),
@@ -531,7 +516,7 @@ mod test {
         .await
         .unwrap();
 
-        let mut executor = SinkExecutor::execute(Box::new(sink_executor));
+        let mut executor = sink_executor.boxed().execute();
 
         // Barrier message.
         executor.next().await.unwrap().unwrap();
@@ -598,32 +583,29 @@ mod test {
             .map(|column| Field::from(column.column_desc.clone()))
             .collect();
 
-        let mock = MockSource::with_messages(
-            schema.clone(),
-            vec![0, 1],
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
-                    " I I I
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
+                " I I I
                     + 1 1 10",
-                ))),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
-                    " I I I
+            ))),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
+                " I I I
                     + 1 3 30",
-                ))),
-                Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
-                    " I I I
+            ))),
+            Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
+                " I I I
                     + 1 2 20
                     - 1 2 20",
-                ))),
-                Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
-                    " I I I
+            ))),
+            Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty(
+                " I I I
                     - 1 1 10",
-                ))),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-            ],
-        );
+            ))),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+        ])
+        .into_executor(schema.clone(), vec![0, 1]);
 
         let sink_param = SinkParam {
             sink_id: 0.into(),
@@ -649,7 +631,7 @@ mod test {
         let sink_executor = SinkExecutor::new(
             ActorContext::for_test(0),
             info,
-            Box::new(mock),
+            source,
             SinkWriterParam::for_test(),
             sink_param,
             columns.clone(),
@@ -658,7 +640,7 @@ mod test {
         .await
         .unwrap();
 
-        let mut executor = SinkExecutor::execute(Box::new(sink_executor));
+        let mut executor = sink_executor.boxed().execute();
 
         // Barrier message.
         executor.next().await.unwrap().unwrap();
@@ -739,15 +721,12 @@ mod test {
             .collect();
         let pk_indices = vec![0];
 
-        let mock = MockSource::with_messages(
-            schema.clone(),
-            pk_indices.clone(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-            ],
-        );
+        let source = MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+        ])
+        .into_executor(schema.clone(), pk_indices.clone());
 
         let sink_param = SinkParam {
             sink_id: 0.into(),
@@ -773,7 +752,7 @@ mod test {
         let sink_executor = SinkExecutor::new(
             ActorContext::for_test(0),
             info,
-            Box::new(mock),
+            source,
             SinkWriterParam::for_test(),
             sink_param,
             columns,
@@ -782,7 +761,7 @@ mod test {
         .await
         .unwrap();
 
-        let mut executor = SinkExecutor::execute(Box::new(sink_executor));
+        let mut executor = sink_executor.boxed().execute();
 
         // Barrier message.
         assert_eq!(
diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs
index f07967ce45299..72474eea32977 100644
--- a/src/stream/src/executor/sort.rs
+++ b/src/stream/src/executor/sort.rs
@@ -20,23 +20,23 @@ use risingwave_storage::StateStore;
 
 use super::sort_buffer::SortBuffer;
 use super::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor,
-    ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, Watermark,
+    expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message,
+    StreamExecutorError, Watermark,
 };
 use crate::common::table::state_table::StateTable;
 use crate::common::StreamChunkBuilder;
 
 pub struct SortExecutor<S: StateStore> {
-    input: BoxedExecutor,
+    input: Executor,
     inner: ExecutorInner<S>,
 }
 
 pub struct SortExecutorArgs<S: StateStore> {
     pub actor_ctx: ActorContextRef,
-    pub info: ExecutorInfo,
 
-    pub input: BoxedExecutor,
+    pub input: Executor,
 
+    pub schema: Schema,
     pub buffer_table: StateTable<S>,
     pub chunk_size: usize,
     pub sort_column_index: usize,
@@ -44,8 +44,8 @@ pub struct SortExecutorArgs<S: StateStore> {
 
 struct ExecutorInner<S: StateStore> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
+    schema: Schema,
     buffer_table: StateTable<S>,
     chunk_size: usize,
     sort_column_index: usize,
@@ -55,22 +55,10 @@ struct ExecutionVars<S: StateStore> {
     buffer: SortBuffer<S>,
 }
 
-impl<S: StateStore> Executor for SortExecutor<S> {
+impl<S: StateStore> Execute for SortExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.executor_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.inner.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.inner.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info.identity
-    }
 }
 
 impl<S: StateStore> SortExecutor<S> {
@@ -79,7 +67,7 @@ impl<S: StateStore> SortExecutor<S> {
             input: args.input,
             inner: ExecutorInner {
                 actor_ctx: args.actor_ctx,
-                info: args.info,
+                schema: args.schema,
                 buffer_table: args.buffer_table,
                 chunk_size: args.chunk_size,
                 sort_column_index: args.sort_column_index,
@@ -114,7 +102,7 @@ impl<S: StateStore> SortExecutor<S> {
                     if col_idx == this.sort_column_index =>
                 {
                     let mut chunk_builder =
-                        StreamChunkBuilder::new(this.chunk_size, this.info.schema.data_types());
+                        StreamChunkBuilder::new(this.chunk_size, this.schema.data_types());
 
                     #[for_await]
                     for row in vars
@@ -172,7 +160,7 @@ mod tests {
 
     use super::*;
     use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt};
-    use crate::executor::{ActorContext, BoxedMessageStream, Executor};
+    use crate::executor::{ActorContext, BoxedMessageStream, Execute};
 
     async fn create_executor<S: StateStore>(
         sort_column_index: usize,
@@ -202,15 +190,12 @@ mod tests {
         )
         .await;
 
-        let (tx, source) = MockSource::channel(input_schema, input_pk_indices);
+        let (tx, source) = MockSource::channel();
+        let source = source.into_executor(input_schema, input_pk_indices);
         let sort_executor = SortExecutor::new(SortExecutorArgs {
             actor_ctx: ActorContext::for_test(123),
-            info: ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "SortExecutor".to_string(),
-            },
-            input: source.boxed(),
+            schema: source.schema().clone(),
+            input: source,
             buffer_table,
             chunk_size: 1024,
             sort_column_index,
diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs
index 36513fc4aba36..73d1b5b42f1c6 100644
--- a/src/stream/src/executor/source/fetch_executor.rs
+++ b/src/stream/src/executor/source/fetch_executor.rs
@@ -22,7 +22,7 @@ use futures::stream::{self, StreamExt};
 use futures::{pin_mut, TryStreamExt};
 use futures_async_stream::try_stream;
 use risingwave_common::array::StreamChunk;
-use risingwave_common::catalog::{ColumnId, Schema, TableId};
+use risingwave_common::catalog::{ColumnId, TableId};
 use risingwave_common::hash::VnodeBitmapExt;
 use risingwave_common::row::{OwnedRow, Row};
 use risingwave_common::types::{ScalarRef, ScalarRefImpl};
@@ -39,8 +39,13 @@ use risingwave_storage::store::PrefetchOptions;
 use risingwave_storage::StateStore;
 use thiserror_ext::AsReport;
 
+use super::{get_split_offset_col_idx, SourceStateTableHandler};
 use crate::executor::stream_reader::StreamReaderWithPause;
-use crate::executor::*;
+use crate::executor::{
+    expect_first_barrier, get_split_offset_mapping_from_chunk, prune_additional_cols,
+    ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError,
+    StreamExecutorResult, StreamSourceCore,
+};
 
 const SPLIT_BATCH_SIZE: usize = 1000;
 
@@ -48,13 +53,12 @@ type SplitBatch = Option<Vec<SplitImpl>>;
 
 pub struct FsFetchExecutor<S: StateStore, Src: OpendalSource> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// Streaming source for external
     stream_source_core: Option<StreamSourceCore<S>>,
 
     /// Upstream list executor.
-    upstream: Option<BoxedExecutor>,
+    upstream: Option<Executor>,
 
     // control options for connector level
     source_ctrl_opts: SourceCtrlOpts,
@@ -69,15 +73,13 @@ impl<S: StateStore, Src: OpendalSource> FsFetchExecutor<S, Src> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         actor_ctx: ActorContextRef,
-        info: ExecutorInfo,
         stream_source_core: StreamSourceCore<S>,
-        upstream: BoxedExecutor,
+        upstream: Executor,
         source_ctrl_opts: SourceCtrlOpts,
         connector_params: ConnectorParams,
     ) -> Self {
         Self {
             actor_ctx,
-            info,
             stream_source_core: Some(stream_source_core),
             upstream: Some(upstream),
             source_ctrl_opts,
@@ -354,22 +356,10 @@ impl<S: StateStore, Src: OpendalSource> FsFetchExecutor<S, Src> {
     }
 }
 
-impl<S: StateStore, Src: OpendalSource> Executor for FsFetchExecutor<S, Src> {
+impl<S: StateStore, Src: OpendalSource> Execute for FsFetchExecutor<S, Src> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.into_stream().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 impl<S: StateStore, Src: OpendalSource> Debug for FsFetchExecutor<S, Src> {
@@ -378,7 +368,6 @@ impl<S: StateStore, Src: OpendalSource> Debug for FsFetchExecutor<S, Src> {
             f.debug_struct("FsFetchExecutor")
                 .field("source_id", &core.source_id)
                 .field("column_ids", &core.column_ids)
-                .field("pk_indices", &self.info.pk_indices)
                 .finish()
         } else {
             f.debug_struct("FsFetchExecutor").finish()
diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs
index 95894429e9361..3f2d13b8be76f 100644
--- a/src/stream/src/executor/source/fs_source_executor.rs
+++ b/src/stream/src/executor/source/fs_source_executor.rs
@@ -22,7 +22,6 @@ use anyhow::anyhow;
 use either::Either;
 use futures::{StreamExt, TryStreamExt};
 use futures_async_stream::try_stream;
-use risingwave_common::catalog::Schema;
 use risingwave_common::system_param::local_manager::SystemParamsReaderRef;
 use risingwave_common::system_param::reader::SystemParamsRead;
 use risingwave_connector::error::ConnectorError;
@@ -50,7 +49,6 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5;
 /// such as s3.
 pub struct FsSourceExecutor<S: StateStore> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// Streaming source  for external
     stream_source_core: StreamSourceCore<S>,
@@ -71,7 +69,6 @@ impl<S: StateStore> FsSourceExecutor<S> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         actor_ctx: ActorContextRef,
-        info: ExecutorInfo,
         stream_source_core: StreamSourceCore<S>,
         metrics: Arc<StreamingMetrics>,
         barrier_receiver: UnboundedReceiver<Barrier>,
@@ -80,7 +77,6 @@ impl<S: StateStore> FsSourceExecutor<S> {
     ) -> StreamResult<Self> {
         Ok(Self {
             actor_ctx,
-            info,
             stream_source_core,
             metrics,
             barrier_receiver: Some(barrier_receiver),
@@ -481,22 +477,10 @@ impl<S: StateStore> FsSourceExecutor<S> {
     }
 }
 
-impl<S: StateStore> Executor for FsSourceExecutor<S> {
+impl<S: StateStore> Execute for FsSourceExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.into_stream().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 impl<S: StateStore> Debug for FsSourceExecutor<S> {
@@ -504,7 +488,6 @@ impl<S: StateStore> Debug for FsSourceExecutor<S> {
         f.debug_struct("FsSourceExecutor")
             .field("source_id", &self.stream_source_core.source_id)
             .field("column_ids", &self.stream_source_core.column_ids)
-            .field("pk_indices", &self.info.pk_indices)
             .finish()
     }
 }
diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs
index f2a6e47ae8641..7996848a749e3 100644
--- a/src/stream/src/executor/source/list_executor.rs
+++ b/src/stream/src/executor/source/list_executor.rs
@@ -20,7 +20,6 @@ use either::Either;
 use futures::{StreamExt, TryStreamExt};
 use futures_async_stream::try_stream;
 use risingwave_common::array::Op;
-use risingwave_common::catalog::Schema;
 use risingwave_common::system_param::local_manager::SystemParamsReaderRef;
 use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder};
 use risingwave_connector::source::SourceCtrlOpts;
@@ -39,7 +38,6 @@ const CHUNK_SIZE: usize = 1024;
 #[allow(dead_code)]
 pub struct FsListExecutor<S: StateStore> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// Streaming source for external
     stream_source_core: Option<StreamSourceCore<S>>,
@@ -64,7 +62,6 @@ impl<S: StateStore> FsListExecutor<S> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         actor_ctx: ActorContextRef,
-        info: ExecutorInfo,
         stream_source_core: Option<StreamSourceCore<S>>,
         metrics: Arc<StreamingMetrics>,
         barrier_receiver: UnboundedReceiver<Barrier>,
@@ -74,7 +71,6 @@ impl<S: StateStore> FsListExecutor<S> {
     ) -> Self {
         Self {
             actor_ctx,
-            info,
             stream_source_core,
             metrics,
             barrier_receiver: Some(barrier_receiver),
@@ -193,22 +189,10 @@ impl<S: StateStore> FsListExecutor<S> {
     }
 }
 
-impl<S: StateStore> Executor for FsListExecutor<S> {
+impl<S: StateStore> Execute for FsListExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.into_stream().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 impl<S: StateStore> Debug for FsListExecutor<S> {
@@ -217,7 +201,6 @@ impl<S: StateStore> Debug for FsListExecutor<S> {
             f.debug_struct("FsListExecutor")
                 .field("source_id", &core.source_id)
                 .field("column_ids", &core.column_ids)
-                .field("pk_indices", &self.info.pk_indices)
                 .finish()
         } else {
             f.debug_struct("FsListExecutor").finish()
diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs
index e2567bb141492..a695b3c8b6b4b 100644
--- a/src/stream/src/executor/source/source_executor.rs
+++ b/src/stream/src/executor/source/source_executor.rs
@@ -43,7 +43,6 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5;
 
 pub struct SourceExecutor<S: StateStore> {
     actor_ctx: ActorContextRef,
-    info: ExecutorInfo,
 
     /// Streaming source for external
     stream_source_core: Option<StreamSourceCore<S>>,
@@ -68,7 +67,6 @@ impl<S: StateStore> SourceExecutor<S> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
         actor_ctx: ActorContextRef,
-        info: ExecutorInfo,
         stream_source_core: Option<StreamSourceCore<S>>,
         metrics: Arc<StreamingMetrics>,
         barrier_receiver: UnboundedReceiver<Barrier>,
@@ -78,7 +76,6 @@ impl<S: StateStore> SourceExecutor<S> {
     ) -> Self {
         Self {
             actor_ctx,
-            info,
             stream_source_core,
             metrics,
             barrier_receiver: Some(barrier_receiver),
@@ -527,8 +524,7 @@ impl<S: StateStore> SourceExecutor<S> {
                         // chunks.
                         self_paused = true;
                         tracing::warn!(
-                            "source {} paused, wait barrier for {:?}",
-                            self.info.identity,
+                            "source paused, wait barrier for {:?}",
                             last_barrier_time.elapsed()
                         );
                         stream.pause_stream();
@@ -616,7 +612,7 @@ impl<S: StateStore> SourceExecutor<S> {
     }
 }
 
-impl<S: StateStore> Executor for SourceExecutor<S> {
+impl<S: StateStore> Execute for SourceExecutor<S> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         if self.stream_source_core.is_some() {
             self.execute_with_stream_source().boxed()
@@ -624,18 +620,6 @@ impl<S: StateStore> Executor for SourceExecutor<S> {
             self.execute_without_stream_source().boxed()
         }
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 impl<S: StateStore> Debug for SourceExecutor<S> {
@@ -644,7 +628,6 @@ impl<S: StateStore> Debug for SourceExecutor<S> {
             f.debug_struct("SourceExecutor")
                 .field("source_id", &core.source_id)
                 .field("column_ids", &core.column_ids)
-                .field("pk_indices", &self.info.pk_indices)
                 .finish()
         } else {
             f.debug_struct("SourceExecutor").finish()
@@ -683,7 +666,6 @@ mod tests {
             fields: vec![Field::with_name(DataType::Int32, "sequence_int")],
         };
         let row_id_index = None;
-        let pk_indices = vec![0];
         let source_info = StreamSourceInfo {
             row_format: PbRowFormatType::Native as i32,
             ..Default::default()
@@ -720,11 +702,6 @@ mod tests {
 
         let executor = SourceExecutor::new(
             ActorContext::for_test(0),
-            ExecutorInfo {
-                schema,
-                pk_indices,
-                identity: "SourceExecutor".to_string(),
-            },
             Some(core),
             Arc::new(StreamingMetrics::unused()),
             barrier_rx,
@@ -732,7 +709,7 @@ mod tests {
             SourceCtrlOpts::default(),
             ConnectorParams::default(),
         );
-        let mut executor = Box::new(executor).execute();
+        let mut executor = executor.boxed().execute();
 
         let init_barrier = Barrier::new_test_barrier(1).with_mutation(Mutation::Add(AddMutation {
             adds: HashMap::new(),
@@ -776,7 +753,6 @@ mod tests {
             fields: vec![Field::with_name(DataType::Int32, "v1")],
         };
         let row_id_index = None;
-        let pk_indices = vec![0_usize];
         let source_info = StreamSourceInfo {
             row_format: PbRowFormatType::Native as i32,
             ..Default::default()
@@ -814,11 +790,6 @@ mod tests {
 
         let executor = SourceExecutor::new(
             ActorContext::for_test(0),
-            ExecutorInfo {
-                schema,
-                pk_indices,
-                identity: "SourceExecutor".to_string(),
-            },
             Some(core),
             Arc::new(StreamingMetrics::unused()),
             barrier_rx,
@@ -826,7 +797,7 @@ mod tests {
             SourceCtrlOpts::default(),
             ConnectorParams::default(),
         );
-        let mut handler = Box::new(executor).execute();
+        let mut handler = executor.boxed().execute();
 
         let init_barrier = Barrier::new_test_barrier(1).with_mutation(Mutation::Add(AddMutation {
             adds: HashMap::new(),
diff --git a/src/stream/src/executor/stateless_simple_agg.rs b/src/stream/src/executor/stateless_simple_agg.rs
index 8a6334b7743b5..1ad25fe5c7f6c 100644
--- a/src/stream/src/executor/stateless_simple_agg.rs
+++ b/src/stream/src/executor/stateless_simple_agg.rs
@@ -16,7 +16,6 @@ use futures::StreamExt;
 use futures_async_stream::try_stream;
 use itertools::Itertools;
 use risingwave_common::array::{Op, StreamChunk};
-use risingwave_common::catalog::Schema;
 use risingwave_common::util::iter_util::ZipEqFast;
 use risingwave_expr::aggregate::{
     build_retractable, AggCall, AggregateState, BoxedAggregateFunction,
@@ -29,28 +28,16 @@ use crate::error::StreamResult;
 
 pub struct StatelessSimpleAggExecutor {
     _ctx: ActorContextRef,
-    pub(super) info: ExecutorInfo,
-    pub(super) input: Box<dyn Executor>,
+    pub(super) input: Executor,
+    pub(super) schema: Schema,
     pub(super) aggs: Vec<BoxedAggregateFunction>,
     pub(super) agg_calls: Vec<AggCall>,
 }
 
-impl Executor for StatelessSimpleAggExecutor {
+impl Execute for StatelessSimpleAggExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 impl StatelessSimpleAggExecutor {
@@ -73,7 +60,7 @@ impl StatelessSimpleAggExecutor {
         let StatelessSimpleAggExecutor {
             _ctx,
             input,
-            info,
+            schema,
             aggs,
             agg_calls,
         } = self;
@@ -94,7 +81,7 @@ impl StatelessSimpleAggExecutor {
                     if is_dirty {
                         is_dirty = false;
 
-                        let mut builders = info.schema.create_array_builders(1);
+                        let mut builders = schema.create_array_builders(1);
                         for ((agg, state), builder) in aggs
                             .iter()
                             .zip_eq_fast(states.iter_mut())
@@ -124,15 +111,15 @@ impl StatelessSimpleAggExecutor {
 impl StatelessSimpleAggExecutor {
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
-        input: Box<dyn Executor>,
+        input: Executor,
+        schema: Schema,
         agg_calls: Vec<AggCall>,
     ) -> StreamResult<Self> {
         let aggs = agg_calls.iter().map(build_retractable).try_collect()?;
         Ok(StatelessSimpleAggExecutor {
             _ctx: ctx,
-            info,
             input,
+            schema,
             aggs,
             agg_calls,
         })
@@ -150,34 +137,24 @@ mod tests {
     use super::*;
     use crate::executor::test_utils::agg_executor::generate_agg_schema;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::{Executor, StatelessSimpleAggExecutor};
+    use crate::executor::{Execute, StatelessSimpleAggExecutor};
 
     #[tokio::test]
     async fn test_no_chunk() {
         let schema = schema_test_utils::ii();
-        let (mut tx, source) = MockSource::channel(schema, vec![2]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, vec![2]);
         tx.push_barrier(1, false);
         tx.push_barrier(2, false);
         tx.push_barrier(3, false);
 
         let agg_calls = vec![AggCall::from_pretty("(count:int8)")];
         let schema = generate_agg_schema(&source, &agg_calls, None);
-        let info = ExecutorInfo {
-            schema,
-            pk_indices: vec![],
-            identity: "StatelessSimpleAggExecutor".to_string(),
-        };
 
-        let simple_agg = Box::new(
-            StatelessSimpleAggExecutor::new(
-                ActorContext::for_test(123),
-                info,
-                Box::new(source),
-                agg_calls,
-            )
-            .unwrap(),
-        );
-        let mut simple_agg = simple_agg.execute();
+        let simple_agg =
+            StatelessSimpleAggExecutor::new(ActorContext::for_test(123), source, schema, agg_calls)
+                .unwrap();
+        let mut simple_agg = simple_agg.boxed().execute();
 
         assert_matches!(
             simple_agg.next().await.unwrap().unwrap(),
@@ -196,7 +173,8 @@ mod tests {
     #[tokio::test]
     async fn test_local_simple_agg() {
         let schema = schema_test_utils::iii();
-        let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk\
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, vec![2]);
         tx.push_barrier(1, false);
         tx.push_chunk(StreamChunk::from_pretty(
             "   I   I    I
@@ -220,22 +198,11 @@ mod tests {
             AggCall::from_pretty("(sum:int8 $1:int8)"),
         ];
         let schema = generate_agg_schema(&source, &agg_calls, None);
-        let info = ExecutorInfo {
-            schema,
-            pk_indices: vec![],
-            identity: "StatelessSimpleAggExecutor".to_string(),
-        };
 
-        let simple_agg = Box::new(
-            StatelessSimpleAggExecutor::new(
-                ActorContext::for_test(123),
-                info,
-                Box::new(source),
-                agg_calls,
-            )
-            .unwrap(),
-        );
-        let mut simple_agg = simple_agg.execute();
+        let simple_agg =
+            StatelessSimpleAggExecutor::new(ActorContext::for_test(123), source, schema, agg_calls)
+                .unwrap();
+        let mut simple_agg = simple_agg.boxed().execute();
 
         // Consume the init barrier
         simple_agg.next().await.unwrap().unwrap();
diff --git a/src/stream/src/executor/subscription.rs b/src/stream/src/executor/subscription.rs
index f1a2923c9085c..dfe352909fd8e 100644
--- a/src/stream/src/executor/subscription.rs
+++ b/src/stream/src/executor/subscription.rs
@@ -16,15 +16,14 @@ use core::time::Duration;
 
 use futures::prelude::stream::StreamExt;
 use futures_async_stream::try_stream;
-use risingwave_common::catalog::Schema;
 use risingwave_common::types::Timestamptz;
 use risingwave_common::util::epoch::Epoch;
 use risingwave_storage::store::LocalStateStore;
 use tokio::time::Instant;
 
 use super::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor,
-    ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult,
+    expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message,
+    StreamExecutorError, StreamExecutorResult,
 };
 use crate::common::log_store_impl::kv_log_store::ReaderTruncationOffsetType;
 use crate::common::log_store_impl::subscription_log_store::SubscriptionLogStoreWriter;
@@ -33,8 +32,7 @@ const EXECUTE_GC_INTERVAL: u64 = 3600;
 
 pub struct SubscriptionExecutor<LS: LocalStateStore> {
     actor_context: ActorContextRef,
-    info: ExecutorInfo,
-    input: BoxedExecutor,
+    input: Executor,
     log_store: SubscriptionLogStoreWriter<LS>,
     retention_seconds: u64,
 }
@@ -44,14 +42,12 @@ impl<LS: LocalStateStore> SubscriptionExecutor<LS> {
     #[expect(clippy::unused_async)]
     pub async fn new(
         actor_context: ActorContextRef,
-        info: ExecutorInfo,
-        input: BoxedExecutor,
+        input: Executor,
         log_store: SubscriptionLogStoreWriter<LS>,
         retention_seconds: u64,
     ) -> StreamExecutorResult<Self> {
         Ok(Self {
             actor_context,
-            info,
             input,
             log_store,
             retention_seconds,
@@ -115,24 +111,8 @@ impl<LS: LocalStateStore> SubscriptionExecutor<LS> {
         }
     }
 }
-impl<LS: LocalStateStore> Executor for SubscriptionExecutor<LS> {
+impl<LS: LocalStateStore> Execute for SubscriptionExecutor<LS> {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.info.clone()
-    }
 }
diff --git a/src/stream/src/executor/subtask.rs b/src/stream/src/executor/subtask.rs
index 0130a73fc411a..fea9644b151f3 100644
--- a/src/stream/src/executor/subtask.rs
+++ b/src/stream/src/executor/subtask.rs
@@ -20,7 +20,7 @@ use tokio::sync::mpsc::error::SendError;
 use tokio_stream::wrappers::ReceiverStream;
 
 use super::actor::spawn_blocking_drop_stream;
-use super::{BoxedExecutor, Executor, ExecutorInfo, Message, MessageStreamItem};
+use super::{Execute, Executor, Message, MessageStreamItem};
 use crate::task::ActorId;
 
 /// Handle used to drive the subtask.
@@ -29,31 +29,13 @@ pub type SubtaskHandle = impl Future<Output = ()> + Send + 'static;
 /// The thin wrapper for subtask-wrapped executor, containing a channel to receive the messages from
 /// the subtask.
 pub struct SubtaskRxExecutor {
-    info: ExecutorInfo,
-
     rx: mpsc::Receiver<MessageStreamItem>,
 }
 
-impl Executor for SubtaskRxExecutor {
+impl Execute for SubtaskRxExecutor {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         ReceiverStream::new(self.rx).boxed()
     }
-
-    fn schema(&self) -> &risingwave_common::catalog::Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.info.clone()
-    }
 }
 
 /// Wrap an executor into a subtask and a thin receiver executor, connected by a channel with a
@@ -62,15 +44,9 @@ impl Executor for SubtaskRxExecutor {
 /// Used when there're multiple stateful executors in an actor. These subtasks can be concurrently
 /// executed to improve the I/O performance, while the computing resource can be still bounded to a
 /// single thread.
-pub fn wrap(input: BoxedExecutor, actor_id: ActorId) -> (SubtaskHandle, SubtaskRxExecutor) {
+pub fn wrap(input: Executor, actor_id: ActorId) -> (SubtaskHandle, SubtaskRxExecutor) {
     let (tx, rx) = mpsc::channel(1);
-    let rx_executor = SubtaskRxExecutor {
-        info: ExecutorInfo {
-            identity: "SubtaskRxExecutor".to_owned(),
-            ..input.info()
-        },
-        rx,
-    };
+    let rx_executor = SubtaskRxExecutor { rx };
 
     let handle = async move {
         let mut input = input.execute();
diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs
index da0ac7b45dbdc..f6c614f41aef4 100644
--- a/src/stream/src/executor/temporal_join.rs
+++ b/src/stream/src/executor/temporal_join.rs
@@ -25,7 +25,6 @@ use futures_async_stream::{for_await, try_stream};
 use local_stats_alloc::{SharedStatsAlloc, StatsAlloc};
 use lru::DefaultHasher;
 use risingwave_common::array::{Op, StreamChunk};
-use risingwave_common::catalog::Schema;
 use risingwave_common::estimate_size::{EstimateSize, KvSize};
 use risingwave_common::hash::{HashKey, NullBitmap};
 use risingwave_common::row::{OwnedRow, Row, RowExt};
@@ -40,21 +39,21 @@ use risingwave_storage::StateStore;
 
 use super::join::{JoinType, JoinTypePrimitive};
 use super::{
-    Barrier, Executor, ExecutorInfo, Message, MessageStream, StreamExecutorError,
+    Barrier, Execute, ExecutorInfo, Message, MessageStream, StreamExecutorError,
     StreamExecutorResult,
 };
 use crate::cache::{cache_may_stale, new_with_hasher_in, ManagedLruCache};
 use crate::common::metrics::MetricsInfo;
 use crate::executor::join::builder::JoinStreamChunkBuilder;
 use crate::executor::monitor::StreamingMetrics;
-use crate::executor::{ActorContextRef, BoxedExecutor, Watermark};
+use crate::executor::{ActorContextRef, Executor, Watermark};
 use crate::task::AtomicU64Ref;
 
 pub struct TemporalJoinExecutor<K: HashKey, S: StateStore, const T: JoinTypePrimitive> {
     ctx: ActorContextRef,
     info: ExecutorInfo,
-    left: BoxedExecutor,
-    right: BoxedExecutor,
+    left: Executor,
+    right: Executor,
     right_table: TemporalSide<K, S>,
     left_join_keys: Vec<usize>,
     right_join_keys: Vec<usize>,
@@ -261,7 +260,7 @@ async fn internal_messages_until_barrier(stream: impl MessageStream, expected_ba
 // any number of `InternalMessage::Chunk(left_chunk)` and followed by
 // `InternalMessage::Barrier(right_chunks, barrier)`.
 #[try_stream(ok = InternalMessage, error = StreamExecutorError)]
-async fn align_input(left: Box<dyn Executor>, right: Box<dyn Executor>) {
+async fn align_input(left: Executor, right: Executor) {
     let mut left = pin!(left.execute());
     let mut right = pin!(right.execute());
     // Keep producing intervals until stream exhaustion or errors.
@@ -313,8 +312,8 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> TemporalJoinExecutor
     pub fn new(
         ctx: ActorContextRef,
         info: ExecutorInfo,
-        left: BoxedExecutor,
-        right: BoxedExecutor,
+        left: Executor,
+        right: Executor,
         table: StorageTable<S>,
         left_join_keys: Vec<usize>,
         right_join_keys: Vec<usize>,
@@ -475,22 +474,10 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> TemporalJoinExecutor
     }
 }
 
-impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Executor
+impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Execute
     for TemporalJoinExecutor<K, S, T>
 {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.into_stream().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs
index cc7505164d154..0be0c9b5648b1 100644
--- a/src/stream/src/executor/test_utils.rs
+++ b/src/stream/src/executor/test_utils.rs
@@ -21,8 +21,8 @@ use tokio::sync::mpsc;
 
 use super::error::StreamExecutorError;
 use super::{
-    Barrier, BoxedMessageStream, Executor, Message, MessageStream, PkIndices, StreamChunk,
-    StreamExecutorResult, Watermark,
+    Barrier, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message, MessageStream,
+    StreamChunk, StreamExecutorResult, Watermark,
 };
 
 pub mod prelude {
@@ -40,12 +40,10 @@ pub mod prelude {
     pub use crate::common::table::state_table::StateTable;
     pub use crate::executor::test_utils::expr::build_from_pretty;
     pub use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt};
-    pub use crate::executor::{ActorContext, BoxedMessageStream, Executor, PkIndices};
+    pub use crate::executor::{ActorContext, BoxedMessageStream, Execute, PkIndices};
 }
 
 pub struct MockSource {
-    schema: Schema,
-    pk_indices: PkIndices,
     rx: mpsc::UnboundedReceiver<Message>,
 
     /// Whether to send a `Stop` barrier on stream finish.
@@ -107,20 +105,15 @@ impl MessageSender {
 
 impl std::fmt::Debug for MockSource {
     fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
-        f.debug_struct("MockSource")
-            .field("schema", &self.schema)
-            .field("pk_indices", &self.pk_indices)
-            .finish()
+        f.debug_struct("MockSource").finish()
     }
 }
 
 impl MockSource {
     #[allow(dead_code)]
-    pub fn channel(schema: Schema, pk_indices: PkIndices) -> (MessageSender, Self) {
+    pub fn channel() -> (MessageSender, Self) {
         let (tx, rx) = mpsc::unbounded_channel();
         let source = Self {
-            schema,
-            pk_indices,
             rx,
             stop_on_finish: true,
         };
@@ -128,16 +121,16 @@ impl MockSource {
     }
 
     #[allow(dead_code)]
-    pub fn with_messages(schema: Schema, pk_indices: PkIndices, msgs: Vec<Message>) -> Self {
-        let (tx, source) = Self::channel(schema, pk_indices);
+    pub fn with_messages(msgs: Vec<Message>) -> Self {
+        let (tx, source) = Self::channel();
         for msg in msgs {
             tx.0.send(msg).unwrap();
         }
         source
     }
 
-    pub fn with_chunks(schema: Schema, pk_indices: PkIndices, chunks: Vec<StreamChunk>) -> Self {
-        let (tx, source) = Self::channel(schema, pk_indices);
+    pub fn with_chunks(chunks: Vec<StreamChunk>) -> Self {
+        let (tx, source) = Self::channel();
         for chunk in chunks {
             tx.0.send(Message::Chunk(chunk)).unwrap();
         }
@@ -153,6 +146,17 @@ impl MockSource {
         }
     }
 
+    pub fn into_executor(self, schema: Schema, pk_indices: Vec<usize>) -> Executor {
+        Executor::new(
+            ExecutorInfo {
+                schema,
+                pk_indices,
+                identity: "MockSource".to_string(),
+            },
+            self.boxed(),
+        )
+    }
+
     #[try_stream(ok = Message, error = StreamExecutorError)]
     async fn execute_inner(mut self: Box<Self>) {
         let mut epoch = 1;
@@ -168,22 +172,10 @@ impl MockSource {
     }
 }
 
-impl Executor for MockSource {
+impl Execute for MockSource {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        "MockSource"
-    }
 }
 
 /// `row_nonnull` builds a `OwnedRow` with concrete values.
@@ -295,14 +287,14 @@ pub mod agg_executor {
     };
     use crate::executor::aggregation::AggStateStorage;
     use crate::executor::{
-        ActorContext, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, HashAggExecutor,
-        PkIndices, SimpleAggExecutor,
+        ActorContext, ActorContextRef, Executor, ExecutorInfo, HashAggExecutor, PkIndices,
+        SimpleAggExecutor,
     };
 
-    /// Generate agg executor's schema from `input`, `agg_calls` and `group_key_indices`.
+    /// Generate aggExecuter's schema from `input`, `agg_calls` and `group_key_indices`.
     /// For [`crate::executor::HashAggExecutor`], the group key indices should be provided.
     pub fn generate_agg_schema(
-        input: &dyn Executor,
+        input_ref: &Executor,
         agg_calls: &[AggCall],
         group_key_indices: Option<&[usize]>,
     ) -> Schema {
@@ -313,7 +305,7 @@ pub mod agg_executor {
         let fields = if let Some(key_indices) = group_key_indices {
             let keys = key_indices
                 .iter()
-                .map(|idx| input.schema().fields[*idx].clone());
+                .map(|idx| input_ref.schema().fields[*idx].clone());
 
             keys.chain(aggs).collect()
         } else {
@@ -331,7 +323,7 @@ pub mod agg_executor {
         agg_call: &AggCall,
         group_key_indices: &[usize],
         pk_indices: &[usize],
-        input_ref: &dyn Executor,
+        input_ref: &Executor,
         is_append_only: bool,
     ) -> AggStateStorage<S> {
         match agg_call.kind {
@@ -402,7 +394,7 @@ pub mod agg_executor {
         table_id: TableId,
         agg_calls: &[AggCall],
         group_key_indices: &[usize],
-        input_ref: &dyn Executor,
+        input_ref: &Executor,
     ) -> StateTable<S> {
         let input_fields = input_ref.schema().fields();
 
@@ -441,7 +433,7 @@ pub mod agg_executor {
     #[allow(clippy::too_many_arguments)]
     pub async fn new_boxed_hash_agg_executor<S: StateStore>(
         store: S,
-        input: Box<dyn Executor>,
+        input: Executor,
         is_append_only: bool,
         agg_calls: Vec<AggCall>,
         row_count_index: usize,
@@ -450,7 +442,7 @@ pub mod agg_executor {
         extreme_cache_size: usize,
         emit_on_window_close: bool,
         executor_id: u64,
-    ) -> Box<dyn Executor> {
+    ) -> Executor {
         let mut storages = Vec::with_capacity(agg_calls.iter().len());
         for (idx, agg_call) in agg_calls.iter().enumerate() {
             storages.push(
@@ -460,7 +452,7 @@ pub mod agg_executor {
                     agg_call,
                     &group_key_indices,
                     &pk_indices,
-                    input.as_ref(),
+                    &input,
                     is_append_only,
                 )
                 .await,
@@ -472,23 +464,23 @@ pub mod agg_executor {
             TableId::new(agg_calls.len() as u32),
             &agg_calls,
             &group_key_indices,
-            input.as_ref(),
+            &input,
         )
         .await;
 
-        let schema = generate_agg_schema(input.as_ref(), &agg_calls, Some(&group_key_indices));
+        let schema = generate_agg_schema(&input, &agg_calls, Some(&group_key_indices));
         let info = ExecutorInfo {
             schema,
             pk_indices,
             identity: format!("HashAggExecutor {:X}", executor_id),
         };
 
-        HashAggExecutor::<SerializedKey, S>::new(AggExecutorArgs {
+        let exec = HashAggExecutor::<SerializedKey, S>::new(AggExecutorArgs {
             version: PbAggNodeVersion::Max,
 
             input,
             actor_ctx: ActorContext::for_test(123),
-            info,
+            info: info.clone(),
 
             extreme_cache_size,
 
@@ -506,21 +498,21 @@ pub mod agg_executor {
                 emit_on_window_close,
             },
         })
-        .unwrap()
-        .boxed()
+        .unwrap();
+        (info, exec).into()
     }
 
     #[allow(clippy::too_many_arguments)]
     pub async fn new_boxed_simple_agg_executor<S: StateStore>(
         actor_ctx: ActorContextRef,
         store: S,
-        input: BoxedExecutor,
+        input: Executor,
         is_append_only: bool,
         agg_calls: Vec<AggCall>,
         row_count_index: usize,
         pk_indices: PkIndices,
         executor_id: u64,
-    ) -> Box<dyn Executor> {
+    ) -> Executor {
         let storages = future::join_all(agg_calls.iter().enumerate().map(|(idx, agg_call)| {
             create_agg_state_storage(
                 store.clone(),
@@ -528,7 +520,7 @@ pub mod agg_executor {
                 agg_call,
                 &[],
                 &pk_indices,
-                input.as_ref(),
+                &input,
                 is_append_only,
             )
         }))
@@ -539,23 +531,23 @@ pub mod agg_executor {
             TableId::new(agg_calls.len() as u32),
             &agg_calls,
             &[],
-            input.as_ref(),
+            &input,
         )
         .await;
 
-        let schema = generate_agg_schema(input.as_ref(), &agg_calls, None);
+        let schema = generate_agg_schema(&input, &agg_calls, None);
         let info = ExecutorInfo {
             schema,
             pk_indices,
             identity: format!("SimpleAggExecutor {:X}", executor_id),
         };
 
-        SimpleAggExecutor::new(AggExecutorArgs {
+        let exec = SimpleAggExecutor::new(AggExecutorArgs {
             version: PbAggNodeVersion::Max,
 
             input,
             actor_ctx,
-            info,
+            info: info.clone(),
 
             extreme_cache_size: 1024,
 
@@ -567,8 +559,8 @@ pub mod agg_executor {
             watermark_epoch: Arc::new(AtomicU64::new(0)),
             extra: SimpleAggExecutorExtraArgs {},
         })
-        .unwrap()
-        .boxed()
+        .unwrap();
+        (info, exec).into()
     }
 }
 
diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs
index ed0f64d098a7a..d4c0fa029341e 100644
--- a/src/stream/src/executor/top_n/group_top_n.rs
+++ b/src/stream/src/executor/top_n/group_top_n.rs
@@ -17,6 +17,7 @@ use std::sync::Arc;
 
 use risingwave_common::array::{Op, StreamChunk};
 use risingwave_common::buffer::Bitmap;
+use risingwave_common::catalog::Schema;
 use risingwave_common::hash::HashKey;
 use risingwave_common::row::RowExt;
 use risingwave_common::util::epoch::EpochPair;
@@ -32,7 +33,7 @@ use crate::common::metrics::MetricsInfo;
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
 use crate::executor::error::StreamExecutorResult;
-use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark};
+use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark};
 use crate::task::AtomicU64Ref;
 
 pub type GroupTopNExecutor<K, S, const WITH_TIES: bool> =
@@ -41,9 +42,9 @@ pub type GroupTopNExecutor<K, S, const WITH_TIES: bool> =
 impl<K: HashKey, S: StateStore, const WITH_TIES: bool> GroupTopNExecutor<K, S, WITH_TIES> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        input: Box<dyn Executor>,
+        input: Executor,
         ctx: ActorContextRef,
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -55,7 +56,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> GroupTopNExecutor<K, S, W
             input,
             ctx: ctx.clone(),
             inner: InnerGroupTopNExecutor::new(
-                info,
+                schema,
                 storage_key,
                 offset_and_limit,
                 order_by,
@@ -69,7 +70,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> GroupTopNExecutor<K, S, W
 }
 
 pub struct InnerGroupTopNExecutor<K: HashKey, S: StateStore, const WITH_TIES: bool> {
-    info: ExecutorInfo,
+    schema: Schema,
 
     /// `LIMIT XXX`. None means no limit.
     limit: usize,
@@ -97,7 +98,7 @@ pub struct InnerGroupTopNExecutor<K: HashKey, S: StateStore, const WITH_TIES: bo
 impl<K: HashKey, S: StateStore, const WITH_TIES: bool> InnerGroupTopNExecutor<K, S, WITH_TIES> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -113,12 +114,11 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> InnerGroupTopNExecutor<K,
             "GroupTopN",
         );
 
-        let cache_key_serde =
-            create_cache_key_serde(&storage_key, &info.schema, &order_by, &group_by);
+        let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &group_by);
         let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone());
 
         Ok(Self {
-            info,
+            schema,
             offset: offset_and_limit.0,
             limit: offset_and_limit.1,
             managed_state,
@@ -191,7 +191,7 @@ where
                     .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str])
                     .inc();
                 let mut topn_cache =
-                    TopNCache::new(self.offset, self.limit, self.info().schema.data_types());
+                    TopNCache::new(self.offset, self.limit, self.schema.data_types());
                 self.managed_state
                     .init_topn_cache(Some(group_key), &mut topn_cache)
                     .await?;
@@ -227,7 +227,7 @@ where
             .group_top_n_cached_entry_count
             .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str])
             .set(self.caches.len() as i64);
-        generate_output(res_rows, res_ops, &self.info().schema)
+        generate_output(res_rows, res_ops, &self.schema)
     }
 
     async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
@@ -238,10 +238,6 @@ where
         self.managed_state.try_flush().await
     }
 
-    fn info(&self) -> &ExecutorInfo {
-        &self.info
-    }
-
     fn update_epoch(&mut self, epoch: u64) {
         self.caches.update_epoch(epoch);
     }
@@ -289,7 +285,7 @@ mod tests {
     use super::*;
     use crate::executor::test_utils::top_n_executor::create_in_memory_state_table;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::{ActorContext, Barrier, Message};
+    use crate::executor::{ActorContext, Barrier, Execute, Message};
 
     fn create_schema() -> Schema {
         Schema {
@@ -355,24 +351,21 @@ mod tests {
         vec![chunk0, chunk1, chunk2, chunk3]
     }
 
-    fn create_source() -> Box<MockSource> {
+    fn create_source() -> Executor {
         let mut chunks = create_stream_chunks();
         let schema = create_schema();
-        Box::new(MockSource::with_messages(
-            schema,
-            pk_indices(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(std::mem::take(&mut chunks[0])),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(std::mem::take(&mut chunks[1])),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-                Message::Chunk(std::mem::take(&mut chunks[2])),
-                Message::Barrier(Barrier::new_test_barrier(4)),
-                Message::Chunk(std::mem::take(&mut chunks[3])),
-                Message::Barrier(Barrier::new_test_barrier(5)),
-            ],
-        ))
+        MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(std::mem::take(&mut chunks[0])),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(std::mem::take(&mut chunks[1])),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+            Message::Chunk(std::mem::take(&mut chunks[2])),
+            Message::Barrier(Barrier::new_test_barrier(4)),
+            Message::Chunk(std::mem::take(&mut chunks[3])),
+            Message::Barrier(Barrier::new_test_barrier(5)),
+        ])
+        .into_executor(schema, pk_indices())
     }
 
     #[tokio::test]
@@ -388,26 +381,20 @@ mod tests {
             &pk_indices(),
         )
         .await;
-        let info = ExecutorInfo {
-            schema: source.schema().clone(),
-            pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix
-            identity: "GroupTopNExecutor 1".to_string(),
-        };
-        let top_n_executor = Box::new(
-            GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new(
-                source as Box<dyn Executor>,
-                ActorContext::for_test(0),
-                info,
-                storage_key(),
-                (0, 2),
-                order_by_1(),
-                vec![1],
-                state_table,
-                Arc::new(AtomicU64::new(0)),
-            )
-            .unwrap(),
-        );
-        let mut top_n_executor = top_n_executor.execute();
+        let schema = source.schema().clone();
+        let top_n_executor = GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new(
+            source,
+            ActorContext::for_test(0),
+            schema,
+            storage_key(),
+            (0, 2),
+            order_by_1(),
+            vec![1],
+            state_table,
+            Arc::new(AtomicU64::new(0)),
+        )
+        .unwrap();
+        let mut top_n_executor = top_n_executor.boxed().execute();
 
         // consume the init barrier
         top_n_executor.next().await.unwrap().unwrap();
@@ -490,26 +477,20 @@ mod tests {
             &pk_indices(),
         )
         .await;
-        let info = ExecutorInfo {
-            schema: source.schema().clone(),
-            pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix
-            identity: "GroupTopNExecutor 1".to_string(),
-        };
-        let top_n_executor = Box::new(
-            GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new(
-                source as Box<dyn Executor>,
-                ActorContext::for_test(0),
-                info,
-                storage_key(),
-                (1, 2),
-                order_by_1(),
-                vec![1],
-                state_table,
-                Arc::new(AtomicU64::new(0)),
-            )
-            .unwrap(),
-        );
-        let mut top_n_executor = top_n_executor.execute();
+        let schema = source.schema().clone();
+        let top_n_executor = GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new(
+            source,
+            ActorContext::for_test(0),
+            schema,
+            storage_key(),
+            (1, 2),
+            order_by_1(),
+            vec![1],
+            state_table,
+            Arc::new(AtomicU64::new(0)),
+        )
+        .unwrap();
+        let mut top_n_executor = top_n_executor.boxed().execute();
 
         // consume the init barrier
         top_n_executor.next().await.unwrap().unwrap();
@@ -585,26 +566,20 @@ mod tests {
             &pk_indices(),
         )
         .await;
-        let info = ExecutorInfo {
-            schema: source.schema().clone(),
-            pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix
-            identity: "GroupTopNExecutor 1".to_string(),
-        };
-        let top_n_executor = Box::new(
-            GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new(
-                source as Box<dyn Executor>,
-                ActorContext::for_test(0),
-                info,
-                storage_key(),
-                (0, 2),
-                order_by_2(),
-                vec![1, 2],
-                state_table,
-                Arc::new(AtomicU64::new(0)),
-            )
-            .unwrap(),
-        );
-        let mut top_n_executor = top_n_executor.execute();
+        let schema = source.schema().clone();
+        let top_n_executor = GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new(
+            source,
+            ActorContext::for_test(0),
+            schema,
+            storage_key(),
+            (0, 2),
+            order_by_2(),
+            vec![1, 2],
+            state_table,
+            Arc::new(AtomicU64::new(0)),
+        )
+        .unwrap();
+        let mut top_n_executor = top_n_executor.boxed().execute();
 
         // consume the init barrier
         top_n_executor.next().await.unwrap().unwrap();
diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs
index 44522ddfb31ff..94f1df73770ab 100644
--- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs
+++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs
@@ -16,6 +16,7 @@ use std::sync::Arc;
 
 use risingwave_common::array::{Op, StreamChunk};
 use risingwave_common::buffer::Bitmap;
+use risingwave_common::catalog::Schema;
 use risingwave_common::hash::HashKey;
 use risingwave_common::row::{RowDeserializer, RowExt};
 use risingwave_common::util::epoch::EpochPair;
@@ -31,7 +32,7 @@ use crate::common::metrics::MetricsInfo;
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
 use crate::executor::error::StreamExecutorResult;
-use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark};
+use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark};
 use crate::task::AtomicU64Ref;
 
 /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need
@@ -45,9 +46,9 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool>
 {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        input: Box<dyn Executor>,
+        input: Executor,
         ctx: ActorContextRef,
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -59,7 +60,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool>
             input,
             ctx: ctx.clone(),
             inner: InnerAppendOnlyGroupTopNExecutor::new(
-                info,
+                schema,
                 storage_key,
                 offset_and_limit,
                 order_by,
@@ -73,7 +74,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool>
 }
 
 pub struct InnerAppendOnlyGroupTopNExecutor<K: HashKey, S: StateStore, const WITH_TIES: bool> {
-    info: ExecutorInfo,
+    schema: Schema,
 
     /// `LIMIT XXX`. None means no limit.
     limit: usize,
@@ -103,7 +104,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool>
 {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -119,12 +120,11 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool>
             "GroupTopN",
         );
 
-        let cache_key_serde =
-            create_cache_key_serde(&storage_key, &info.schema, &order_by, &group_by);
+        let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &group_by);
         let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone());
 
         Ok(Self {
-            info,
+            schema,
             offset: offset_and_limit.0,
             limit: offset_and_limit.1,
             managed_state,
@@ -147,7 +147,7 @@ where
         let mut res_rows = Vec::with_capacity(self.limit);
         let keys = K::build(&self.group_by, chunk.data_chunk())?;
 
-        let data_types = self.info().schema.data_types();
+        let data_types = self.schema.data_types();
         let row_deserializer = RowDeserializer::new(data_types.clone());
         let table_id_str = self.managed_state.table().table_id().to_string();
         let actor_id_str = self.ctx.id.to_string();
@@ -197,7 +197,7 @@ where
             .group_top_n_appendonly_cached_entry_count
             .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str])
             .set(self.caches.len() as i64);
-        generate_output(res_rows, res_ops, &self.info().schema)
+        generate_output(res_rows, res_ops, &self.schema)
     }
 
     async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
@@ -208,10 +208,6 @@ where
         self.managed_state.try_flush().await
     }
 
-    fn info(&self) -> &ExecutorInfo {
-        &self.info
-    }
-
     fn update_vnode_bitmap(&mut self, vnode_bitmap: Arc<Bitmap>) {
         let cache_may_stale = self.managed_state.update_vnode_bitmap(vnode_bitmap);
         if cache_may_stale {
diff --git a/src/stream/src/executor/top_n/top_n_appendonly.rs b/src/stream/src/executor/top_n/top_n_appendonly.rs
index 8da016fd8c7ac..095222939a6e8 100644
--- a/src/stream/src/executor/top_n/top_n_appendonly.rs
+++ b/src/stream/src/executor/top_n/top_n_appendonly.rs
@@ -13,6 +13,7 @@
 // limitations under the License.
 
 use risingwave_common::array::{Op, StreamChunk};
+use risingwave_common::catalog::Schema;
 use risingwave_common::row::{RowDeserializer, RowExt};
 use risingwave_common::util::epoch::EpochPair;
 use risingwave_common::util::sort_util::ColumnOrder;
@@ -24,7 +25,7 @@ use super::{ManagedTopNState, TopNCache, NO_GROUP_KEY};
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
 use crate::executor::error::StreamExecutorResult;
-use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark};
+use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark};
 
 /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need
 /// to keep all the rows seen. As long as a record
@@ -38,9 +39,9 @@ pub type AppendOnlyTopNExecutor<S, const WITH_TIES: bool> =
 impl<S: StateStore, const WITH_TIES: bool> AppendOnlyTopNExecutor<S, WITH_TIES> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        input: Box<dyn Executor>,
+        input: Executor,
         ctx: ActorContextRef,
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -50,7 +51,7 @@ impl<S: StateStore, const WITH_TIES: bool> AppendOnlyTopNExecutor<S, WITH_TIES>
             input,
             ctx,
             inner: InnerAppendOnlyTopNExecutor::new(
-                info,
+                schema,
                 storage_key,
                 offset_and_limit,
                 order_by,
@@ -61,7 +62,7 @@ impl<S: StateStore, const WITH_TIES: bool> AppendOnlyTopNExecutor<S, WITH_TIES>
 }
 
 pub struct InnerAppendOnlyTopNExecutor<S: StateStore, const WITH_TIES: bool> {
-    info: ExecutorInfo,
+    schema: Schema,
 
     /// The storage key indices of the `TopNExecutor`
     storage_key_indices: PkIndices,
@@ -80,7 +81,7 @@ pub struct InnerAppendOnlyTopNExecutor<S: StateStore, const WITH_TIES: bool> {
 impl<S: StateStore, const WITH_TIES: bool> InnerAppendOnlyTopNExecutor<S, WITH_TIES> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -89,12 +90,12 @@ impl<S: StateStore, const WITH_TIES: bool> InnerAppendOnlyTopNExecutor<S, WITH_T
         let num_offset = offset_and_limit.0;
         let num_limit = offset_and_limit.1;
 
-        let cache_key_serde = create_cache_key_serde(&storage_key, &info.schema, &order_by, &[]);
+        let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &[]);
         let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone());
-        let data_types = info.schema.data_types();
+        let data_types = schema.data_types();
 
         Ok(Self {
-            info,
+            schema,
             managed_state,
             storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(),
             cache: TopNCache::new(num_offset, num_limit, data_types),
@@ -111,7 +112,7 @@ where
     async fn apply_chunk(&mut self, chunk: StreamChunk) -> StreamExecutorResult<StreamChunk> {
         let mut res_ops = Vec::with_capacity(self.cache.limit);
         let mut res_rows = Vec::with_capacity(self.cache.limit);
-        let data_types = self.info().schema.data_types();
+        let data_types = self.schema.data_types();
         let row_deserializer = RowDeserializer::new(data_types);
         // apply the chunk to state table
         for (op, row_ref) in chunk.rows() {
@@ -128,7 +129,7 @@ where
             )?;
         }
 
-        generate_output(res_rows, res_ops, &self.info().schema)
+        generate_output(res_rows, res_ops, &self.schema)
     }
 
     async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
@@ -139,10 +140,6 @@ where
         self.managed_state.try_flush().await
     }
 
-    fn info(&self) -> &ExecutorInfo {
-        &self.info
-    }
-
     async fn init(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
         self.managed_state.init_epoch(epoch);
         self.managed_state
@@ -170,7 +167,7 @@ mod tests {
     use super::AppendOnlyTopNExecutor;
     use crate::executor::test_utils::top_n_executor::create_in_memory_state_table;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::{ActorContext, Barrier, Executor, ExecutorInfo, Message, PkIndices};
+    use crate::executor::{ActorContext, Barrier, Execute, Executor, Message, PkIndices};
 
     fn create_stream_chunks() -> Vec<StreamChunk> {
         let chunk1 = StreamChunk::from_pretty(
@@ -223,21 +220,17 @@ mod tests {
         vec![0, 1]
     }
 
-    fn create_source() -> Box<MockSource> {
+    fn create_source() -> Executor {
         let mut chunks = create_stream_chunks();
-        let schema = create_schema();
-        Box::new(MockSource::with_messages(
-            schema,
-            pk_indices(),
-            vec![
-                Message::Barrier(Barrier::new_test_barrier(1)),
-                Message::Chunk(std::mem::take(&mut chunks[0])),
-                Message::Barrier(Barrier::new_test_barrier(2)),
-                Message::Chunk(std::mem::take(&mut chunks[1])),
-                Message::Barrier(Barrier::new_test_barrier(3)),
-                Message::Chunk(std::mem::take(&mut chunks[2])),
-            ],
-        ))
+        MockSource::with_messages(vec![
+            Message::Barrier(Barrier::new_test_barrier(1)),
+            Message::Chunk(std::mem::take(&mut chunks[0])),
+            Message::Barrier(Barrier::new_test_barrier(2)),
+            Message::Chunk(std::mem::take(&mut chunks[1])),
+            Message::Barrier(Barrier::new_test_barrier(3)),
+            Message::Chunk(std::mem::take(&mut chunks[2])),
+        ])
+        .into_executor(create_schema(), pk_indices())
     }
 
     #[tokio::test]
@@ -251,24 +244,18 @@ mod tests {
         )
         .await;
 
-        let info = ExecutorInfo {
-            schema: source.schema().clone(),
-            pk_indices: source.pk_indices().to_vec(),
-            identity: "AppendOnlyTopNExecutor 1".to_string(),
-        };
-        let top_n_executor = Box::new(
-            AppendOnlyTopNExecutor::<_, false>::new(
-                source as Box<dyn Executor>,
-                ActorContext::for_test(0),
-                info,
-                storage_key,
-                (0, 5),
-                order_by(),
-                state_table,
-            )
-            .unwrap(),
-        );
-        let mut top_n_executor = top_n_executor.execute();
+        let schema = source.schema().clone();
+        let top_n_executor = AppendOnlyTopNExecutor::<_, false>::new(
+            source,
+            ActorContext::for_test(0),
+            schema,
+            storage_key,
+            (0, 5),
+            order_by(),
+            state_table,
+        )
+        .unwrap();
+        let mut top_n_executor = top_n_executor.boxed().execute();
 
         // consume the init epoch
         top_n_executor.next().await.unwrap().unwrap();
@@ -338,24 +325,18 @@ mod tests {
         )
         .await;
 
-        let info = ExecutorInfo {
-            schema: source.schema().clone(),
-            pk_indices: source.pk_indices().to_vec(),
-            identity: "AppendOnlyTopNExecutor 1".to_string(),
-        };
-        let top_n_executor = Box::new(
-            AppendOnlyTopNExecutor::<_, false>::new(
-                source as Box<dyn Executor>,
-                ActorContext::for_test(0),
-                info,
-                storage_key(),
-                (3, 4),
-                order_by(),
-                state_table,
-            )
-            .unwrap(),
-        );
-        let mut top_n_executor = top_n_executor.execute();
+        let schema = source.schema().clone();
+        let top_n_executor = AppendOnlyTopNExecutor::<_, false>::new(
+            source,
+            ActorContext::for_test(0),
+            schema,
+            storage_key(),
+            (3, 4),
+            order_by(),
+            state_table,
+        )
+        .unwrap();
+        let mut top_n_executor = top_n_executor.boxed().execute();
 
         // consume the init epoch
         top_n_executor.next().await.unwrap().unwrap();
diff --git a/src/stream/src/executor/top_n/top_n_plain.rs b/src/stream/src/executor/top_n/top_n_plain.rs
index 5df41cfca3a05..b09f13764ee79 100644
--- a/src/stream/src/executor/top_n/top_n_plain.rs
+++ b/src/stream/src/executor/top_n/top_n_plain.rs
@@ -13,6 +13,7 @@
 // limitations under the License.
 
 use risingwave_common::array::{Op, StreamChunk};
+use risingwave_common::catalog::Schema;
 use risingwave_common::row::RowExt;
 use risingwave_common::util::epoch::EpochPair;
 use risingwave_common::util::sort_util::ColumnOrder;
@@ -23,7 +24,7 @@ use super::{ManagedTopNState, TopNCache, TopNCacheTrait};
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
 use crate::executor::error::StreamExecutorResult;
-use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark};
+use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark};
 
 /// `TopNExecutor` works with input with modification, it keeps all the data
 /// records/rows that have been seen, and returns topN records overall.
@@ -33,9 +34,9 @@ pub type TopNExecutor<S, const WITH_TIES: bool> =
 impl<S: StateStore, const WITH_TIES: bool> TopNExecutor<S, WITH_TIES> {
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        input: Box<dyn Executor>,
+        input: Executor,
         ctx: ActorContextRef,
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -45,7 +46,7 @@ impl<S: StateStore, const WITH_TIES: bool> TopNExecutor<S, WITH_TIES> {
             input,
             ctx,
             inner: InnerTopNExecutor::new(
-                info,
+                schema,
                 storage_key,
                 offset_and_limit,
                 order_by,
@@ -61,16 +62,16 @@ impl<S: StateStore> TopNExecutor<S, true> {
     #[allow(clippy::too_many_arguments)]
     #[cfg(test)]
     pub fn new_with_ties_for_test(
-        input: Box<dyn Executor>,
+        input: Executor,
         ctx: ActorContextRef,
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
         state_table: StateTable<S>,
     ) -> StreamResult<Self> {
         let mut inner =
-            InnerTopNExecutor::new(info, storage_key, offset_and_limit, order_by, state_table)?;
+            InnerTopNExecutor::new(schema, storage_key, offset_and_limit, order_by, state_table)?;
 
         inner.cache.high_capacity = 2;
 
@@ -79,7 +80,7 @@ impl<S: StateStore> TopNExecutor<S, true> {
 }
 
 pub struct InnerTopNExecutor<S: StateStore, const WITH_TIES: bool> {
-    info: ExecutorInfo,
+    schema: Schema,
 
     /// The storage key indices of the `TopNExecutor`
     storage_key_indices: PkIndices,
@@ -103,7 +104,7 @@ impl<S: StateStore, const WITH_TIES: bool> InnerTopNExecutor<S, WITH_TIES> {
     /// into `CacheKey`.
     #[allow(clippy::too_many_arguments)]
     pub fn new(
-        info: ExecutorInfo,
+        schema: Schema,
         storage_key: Vec<ColumnOrder>,
         offset_and_limit: (usize, usize),
         order_by: Vec<ColumnOrder>,
@@ -112,12 +113,12 @@ impl<S: StateStore, const WITH_TIES: bool> InnerTopNExecutor<S, WITH_TIES> {
         let num_offset = offset_and_limit.0;
         let num_limit = offset_and_limit.1;
 
-        let cache_key_serde = create_cache_key_serde(&storage_key, &info.schema, &order_by, &[]);
+        let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &[]);
         let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone());
-        let data_types = info.schema.data_types();
+        let data_types = schema.data_types();
 
         Ok(Self {
-            info,
+            schema,
             managed_state,
             storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(),
             cache: TopNCache::new(num_offset, num_limit, data_types),
@@ -162,7 +163,7 @@ where
                 }
             }
         }
-        generate_output(res_rows, res_ops, &self.info().schema)
+        generate_output(res_rows, res_ops, &self.schema)
     }
 
     async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
@@ -173,10 +174,6 @@ where
         self.managed_state.try_flush().await
     }
 
-    fn info(&self) -> &ExecutorInfo {
-        &self.info
-    }
-
     async fn init(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> {
         self.managed_state.init_epoch(epoch);
         self.managed_state
@@ -206,7 +203,7 @@ mod tests {
 
     mod test1 {
         use super::*;
-        use crate::executor::ActorContext;
+        use crate::executor::{ActorContext, Execute};
         fn create_stream_chunks() -> Vec<StreamChunk> {
             let chunk1 = StreamChunk::from_pretty(
                 "  I I
@@ -265,24 +262,21 @@ mod tests {
             vec![0, 1]
         }
 
-        fn create_source() -> Box<MockSource> {
+        fn create_source() -> Executor {
             let mut chunks = create_stream_chunks();
             let schema = create_schema();
-            Box::new(MockSource::with_messages(
-                schema,
-                pk_indices(),
-                vec![
-                    Message::Barrier(Barrier::new_test_barrier(1)),
-                    Message::Chunk(std::mem::take(&mut chunks[0])),
-                    Message::Barrier(Barrier::new_test_barrier(2)),
-                    Message::Chunk(std::mem::take(&mut chunks[1])),
-                    Message::Barrier(Barrier::new_test_barrier(3)),
-                    Message::Chunk(std::mem::take(&mut chunks[2])),
-                    Message::Barrier(Barrier::new_test_barrier(4)),
-                    Message::Chunk(std::mem::take(&mut chunks[3])),
-                    Message::Barrier(Barrier::new_test_barrier(5)),
-                ],
-            ))
+            MockSource::with_messages(vec![
+                Message::Barrier(Barrier::new_test_barrier(1)),
+                Message::Chunk(std::mem::take(&mut chunks[0])),
+                Message::Barrier(Barrier::new_test_barrier(2)),
+                Message::Chunk(std::mem::take(&mut chunks[1])),
+                Message::Barrier(Barrier::new_test_barrier(3)),
+                Message::Chunk(std::mem::take(&mut chunks[2])),
+                Message::Barrier(Barrier::new_test_barrier(4)),
+                Message::Chunk(std::mem::take(&mut chunks[3])),
+                Message::Barrier(Barrier::new_test_barrier(5)),
+            ])
+            .into_executor(schema, pk_indices())
         }
 
         #[tokio::test]
@@ -295,24 +289,18 @@ mod tests {
             )
             .await;
 
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::<_, false>::new(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (3, 1000),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::<_, false>::new(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (3, 1000),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -396,24 +384,18 @@ mod tests {
                 &pk_indices(),
             )
             .await;
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::<_, false>::new(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (0, 4),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::<_, false>::new(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (0, 4),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -509,24 +491,18 @@ mod tests {
                 &pk_indices(),
             )
             .await;
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::<_, true>::new(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (0, 4),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::<_, true>::new(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (0, 4),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -621,24 +597,18 @@ mod tests {
                 &pk_indices(),
             )
             .await;
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::<_, false>::new(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (3, 4),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::<_, false>::new(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (3, 4),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -715,9 +685,9 @@ mod tests {
 
         use super::*;
         use crate::executor::test_utils::top_n_executor::create_in_memory_state_table_from_state_store;
-        use crate::executor::ActorContext;
+        use crate::executor::{ActorContext, Execute};
 
-        fn create_source_new() -> Box<MockSource> {
+        fn create_source_new() -> Executor {
             let mut chunks = vec![
                 StreamChunk::from_pretty(
                     " I I I I
@@ -746,21 +716,18 @@ mod tests {
                     Field::unnamed(DataType::Int64),
                 ],
             };
-            Box::new(MockSource::with_messages(
-                schema,
-                pk_indices(),
-                vec![
-                    Message::Barrier(Barrier::new_test_barrier(1)),
-                    Message::Chunk(std::mem::take(&mut chunks[0])),
-                    Message::Chunk(std::mem::take(&mut chunks[1])),
-                    Message::Chunk(std::mem::take(&mut chunks[2])),
-                    Message::Chunk(std::mem::take(&mut chunks[3])),
-                    Message::Barrier(Barrier::new_test_barrier(2)),
-                ],
-            ))
+            MockSource::with_messages(vec![
+                Message::Barrier(Barrier::new_test_barrier(1)),
+                Message::Chunk(std::mem::take(&mut chunks[0])),
+                Message::Chunk(std::mem::take(&mut chunks[1])),
+                Message::Chunk(std::mem::take(&mut chunks[2])),
+                Message::Chunk(std::mem::take(&mut chunks[3])),
+                Message::Barrier(Barrier::new_test_barrier(2)),
+            ])
+            .into_executor(schema, pk_indices())
         }
 
-        fn create_source_new_before_recovery() -> Box<MockSource> {
+        fn create_source_new_before_recovery() -> Executor {
             let mut chunks = [
                 StreamChunk::from_pretty(
                     " I I I I
@@ -779,19 +746,16 @@ mod tests {
                     Field::unnamed(DataType::Int64),
                 ],
             };
-            Box::new(MockSource::with_messages(
-                schema,
-                pk_indices(),
-                vec![
-                    Message::Barrier(Barrier::new_test_barrier(1)),
-                    Message::Chunk(std::mem::take(&mut chunks[0])),
-                    Message::Chunk(std::mem::take(&mut chunks[1])),
-                    Message::Barrier(Barrier::new_test_barrier(2)),
-                ],
-            ))
+            MockSource::with_messages(vec![
+                Message::Barrier(Barrier::new_test_barrier(1)),
+                Message::Chunk(std::mem::take(&mut chunks[0])),
+                Message::Chunk(std::mem::take(&mut chunks[1])),
+                Message::Barrier(Barrier::new_test_barrier(2)),
+            ])
+            .into_executor(schema, pk_indices())
         }
 
-        fn create_source_new_after_recovery() -> Box<MockSource> {
+        fn create_source_new_after_recovery() -> Executor {
             let mut chunks = [
                 StreamChunk::from_pretty(
                     " I I I I
@@ -812,16 +776,13 @@ mod tests {
                     Field::unnamed(DataType::Int64),
                 ],
             };
-            Box::new(MockSource::with_messages(
-                schema,
-                pk_indices(),
-                vec![
-                    Message::Barrier(Barrier::new_test_barrier(3)),
-                    Message::Chunk(std::mem::take(&mut chunks[0])),
-                    Message::Chunk(std::mem::take(&mut chunks[1])),
-                    Message::Barrier(Barrier::new_test_barrier(4)),
-                ],
-            ))
+            MockSource::with_messages(vec![
+                Message::Barrier(Barrier::new_test_barrier(3)),
+                Message::Chunk(std::mem::take(&mut chunks[0])),
+                Message::Chunk(std::mem::take(&mut chunks[1])),
+                Message::Barrier(Barrier::new_test_barrier(4)),
+            ])
+            .into_executor(schema, pk_indices())
         }
 
         fn storage_key() -> Vec<ColumnOrder> {
@@ -853,24 +814,18 @@ mod tests {
                 &pk_indices(),
             )
             .await;
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::<_, false>::new(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (1, 3),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::<_, false>::new(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (1, 3),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -937,24 +892,18 @@ mod tests {
             )
             .await;
             let source = create_source_new_before_recovery();
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::<_, false>::new(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (1, 3),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::<_, false>::new(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (1, 3),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -997,24 +946,18 @@ mod tests {
 
             // recovery
             let source = create_source_new_after_recovery();
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor_after_recovery = Box::new(
-                TopNExecutor::<_, false>::new(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (1, 3),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor_after_recovery.execute();
+            let schema = source.schema().clone();
+            let top_n_executor_after_recovery = TopNExecutor::<_, false>::new(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (1, 3),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor_after_recovery.boxed().execute();
 
             // barrier
             assert_matches!(
@@ -1057,9 +1000,9 @@ mod tests {
 
         use super::*;
         use crate::executor::test_utils::top_n_executor::create_in_memory_state_table_from_state_store;
-        use crate::executor::ActorContext;
+        use crate::executor::{ActorContext, Execute};
 
-        fn create_source() -> Box<MockSource> {
+        fn create_source() -> Executor {
             let mut chunks = vec![
                 StreamChunk::from_pretty(
                     "  I I
@@ -1094,18 +1037,15 @@ mod tests {
                     Field::unnamed(DataType::Int64),
                 ],
             };
-            Box::new(MockSource::with_messages(
-                schema,
-                pk_indices(),
-                vec![
-                    Message::Barrier(Barrier::new_test_barrier(1)),
-                    Message::Chunk(std::mem::take(&mut chunks[0])),
-                    Message::Chunk(std::mem::take(&mut chunks[1])),
-                    Message::Chunk(std::mem::take(&mut chunks[2])),
-                    Message::Chunk(std::mem::take(&mut chunks[3])),
-                    Message::Barrier(Barrier::new_test_barrier(2)),
-                ],
-            ))
+            MockSource::with_messages(vec![
+                Message::Barrier(Barrier::new_test_barrier(1)),
+                Message::Chunk(std::mem::take(&mut chunks[0])),
+                Message::Chunk(std::mem::take(&mut chunks[1])),
+                Message::Chunk(std::mem::take(&mut chunks[2])),
+                Message::Chunk(std::mem::take(&mut chunks[3])),
+                Message::Barrier(Barrier::new_test_barrier(2)),
+            ])
+            .into_executor(schema, pk_indices())
         }
 
         fn storage_key() -> Vec<ColumnOrder> {
@@ -1131,24 +1071,18 @@ mod tests {
                 &pk_indices(),
             )
             .await;
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::new_with_ties_for_test(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (0, 3),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::new_with_ties_for_test(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (0, 3),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -1208,7 +1142,7 @@ mod tests {
             );
         }
 
-        fn create_source_before_recovery() -> Box<MockSource> {
+        fn create_source_before_recovery() -> Executor {
             let mut chunks = [
                 StreamChunk::from_pretty(
                     "  I I
@@ -1235,19 +1169,16 @@ mod tests {
                     Field::unnamed(DataType::Int64),
                 ],
             };
-            Box::new(MockSource::with_messages(
-                schema,
-                pk_indices(),
-                vec![
-                    Message::Barrier(Barrier::new_test_barrier(1)),
-                    Message::Chunk(std::mem::take(&mut chunks[0])),
-                    Message::Chunk(std::mem::take(&mut chunks[1])),
-                    Message::Barrier(Barrier::new_test_barrier(2)),
-                ],
-            ))
+            MockSource::with_messages(vec![
+                Message::Barrier(Barrier::new_test_barrier(1)),
+                Message::Chunk(std::mem::take(&mut chunks[0])),
+                Message::Chunk(std::mem::take(&mut chunks[1])),
+                Message::Barrier(Barrier::new_test_barrier(2)),
+            ])
+            .into_executor(schema, pk_indices())
         }
 
-        fn create_source_after_recovery() -> Box<MockSource> {
+        fn create_source_after_recovery() -> Executor {
             let mut chunks = [
                 StreamChunk::from_pretty(
                     " I I
@@ -1264,16 +1195,13 @@ mod tests {
                     Field::unnamed(DataType::Int64),
                 ],
             };
-            Box::new(MockSource::with_messages(
-                schema,
-                pk_indices(),
-                vec![
-                    Message::Barrier(Barrier::new_test_barrier(3)),
-                    Message::Chunk(std::mem::take(&mut chunks[0])),
-                    Message::Chunk(std::mem::take(&mut chunks[1])),
-                    Message::Barrier(Barrier::new_test_barrier(4)),
-                ],
-            ))
+            MockSource::with_messages(vec![
+                Message::Barrier(Barrier::new_test_barrier(3)),
+                Message::Chunk(std::mem::take(&mut chunks[0])),
+                Message::Chunk(std::mem::take(&mut chunks[1])),
+                Message::Barrier(Barrier::new_test_barrier(4)),
+            ])
+            .into_executor(schema, pk_indices())
         }
 
         #[tokio::test]
@@ -1287,24 +1215,18 @@ mod tests {
             )
             .await;
             let source = create_source_before_recovery();
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor = Box::new(
-                TopNExecutor::new_with_ties_for_test(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (0, 3),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor.execute();
+            let schema = source.schema().clone();
+            let top_n_executor = TopNExecutor::new_with_ties_for_test(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (0, 3),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor.boxed().execute();
 
             // consume the init barrier
             top_n_executor.next().await.unwrap().unwrap();
@@ -1350,24 +1272,18 @@ mod tests {
 
             // recovery
             let source = create_source_after_recovery();
-            let info = ExecutorInfo {
-                schema: source.schema().clone(),
-                pk_indices: source.pk_indices().to_vec(),
-                identity: "TopNExecutor 1".to_string(),
-            };
-            let top_n_executor_after_recovery = Box::new(
-                TopNExecutor::new_with_ties_for_test(
-                    source as Box<dyn Executor>,
-                    ActorContext::for_test(0),
-                    info,
-                    storage_key(),
-                    (0, 3),
-                    order_by(),
-                    state_table,
-                )
-                .unwrap(),
-            );
-            let mut top_n_executor = top_n_executor_after_recovery.execute();
+            let schema = source.schema().clone();
+            let top_n_executor_after_recovery = TopNExecutor::new_with_ties_for_test(
+                source,
+                ActorContext::for_test(0),
+                schema,
+                storage_key(),
+                (0, 3),
+                order_by(),
+                state_table,
+            )
+            .unwrap();
+            let mut top_n_executor = top_n_executor_after_recovery.boxed().execute();
 
             // barrier
             assert_matches!(
diff --git a/src/stream/src/executor/top_n/utils.rs b/src/stream/src/executor/top_n/utils.rs
index b2310f0d352b1..bbd956cde2168 100644
--- a/src/stream/src/executor/top_n/utils.rs
+++ b/src/stream/src/executor/top_n/utils.rs
@@ -30,8 +30,8 @@ use risingwave_common::util::sort_util::ColumnOrder;
 use super::CacheKey;
 use crate::executor::error::{StreamExecutorError, StreamExecutorResult};
 use crate::executor::{
-    expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor,
-    ExecutorInfo, Message, PkIndicesRef, Watermark,
+    expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message,
+    Watermark,
 };
 
 pub trait TopNExecutorBase: Send + 'static {
@@ -50,8 +50,6 @@ pub trait TopNExecutorBase: Send + 'static {
     /// Flush the buffered chunk to the storage backend.
     fn try_flush_data(&mut self) -> impl Future<Output = StreamExecutorResult<()>> + Send;
 
-    fn info(&self) -> &ExecutorInfo;
-
     /// Update the vnode bitmap for the state table and manipulate the cache if necessary, only used
     /// by Group Top-N since it's distributed.
     fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc<Bitmap>) {
@@ -72,34 +70,18 @@ pub trait TopNExecutorBase: Send + 'static {
 
 /// The struct wraps a [`TopNExecutorBase`]
 pub struct TopNExecutorWrapper<E> {
-    pub(super) input: BoxedExecutor,
+    pub(super) input: Executor,
     pub(super) ctx: ActorContextRef,
     pub(super) inner: E,
 }
 
-impl<E> Executor for TopNExecutorWrapper<E>
+impl<E> Execute for TopNExecutorWrapper<E>
 where
     E: TopNExecutorBase,
 {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         self.top_n_executor_execute().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.inner.info().schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.inner.info().pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.inner.info().identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.inner.info().clone()
-    }
 }
 
 impl<E> TopNExecutorWrapper<E>
diff --git a/src/stream/src/executor/union.rs b/src/stream/src/executor/union.rs
index a3b3df6873e6b..acd35ce080cda 100644
--- a/src/stream/src/executor/union.rs
+++ b/src/stream/src/executor/union.rs
@@ -17,57 +17,37 @@ use std::pin::Pin;
 use std::task::{Context, Poll};
 
 use futures::stream::{FusedStream, FuturesUnordered};
-use futures::StreamExt;
+use futures::{Stream, StreamExt};
 use futures_async_stream::try_stream;
 use pin_project::pin_project;
-use risingwave_common::catalog::Schema;
 
 use super::watermark::BufferedWatermarks;
-use super::*;
-use crate::executor::{BoxedMessageStream, ExecutorInfo};
+use super::{
+    Barrier, BoxedMessageStream, Execute, Executor, Message, MessageStreamItem, StreamExecutorError,
+};
 
 /// `UnionExecutor` merges data from multiple inputs.
 pub struct UnionExecutor {
-    info: ExecutorInfo,
-    inputs: Vec<BoxedExecutor>,
+    inputs: Vec<Executor>,
 }
 
 impl std::fmt::Debug for UnionExecutor {
     fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
-        f.debug_struct("UnionExecutor")
-            .field("schema", &self.info.schema)
-            .field("pk_indices", &self.info.pk_indices)
-            .finish()
+        f.debug_struct("UnionExecutor").finish()
     }
 }
 
 impl UnionExecutor {
-    pub fn new(info: ExecutorInfo, inputs: Vec<BoxedExecutor>) -> Self {
-        Self { info, inputs }
+    pub fn new(inputs: Vec<Executor>) -> Self {
+        Self { inputs }
     }
 }
 
-impl Executor for UnionExecutor {
+impl Execute for UnionExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         let streams = self.inputs.into_iter().map(|e| e.execute()).collect();
         merge(streams).boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.info.clone()
-    }
 }
 
 #[pin_project]
@@ -170,8 +150,10 @@ mod tests {
     use async_stream::try_stream;
     use risingwave_common::array::stream_chunk::StreamChunkTestExt;
     use risingwave_common::array::StreamChunk;
+    use risingwave_common::types::{DataType, ScalarImpl};
 
     use super::*;
+    use crate::executor::Watermark;
 
     #[tokio::test]
     async fn union() {
diff --git a/src/stream/src/executor/utils.rs b/src/stream/src/executor/utils.rs
index 8ff9f8f29576d..18b64633a3e12 100644
--- a/src/stream/src/executor/utils.rs
+++ b/src/stream/src/executor/utils.rs
@@ -13,40 +13,19 @@
 // limitations under the License.
 
 use futures::StreamExt;
-use risingwave_common::catalog::Schema;
 use risingwave_common::metrics::LabelGuardedIntCounter;
 
 use crate::executor::monitor::StreamingMetrics;
-use crate::executor::{BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef};
+use crate::executor::{BoxedMessageStream, Execute};
 use crate::task::{ActorId, FragmentId};
 
 #[derive(Default)]
-pub struct DummyExecutor {
-    pub info: ExecutorInfo,
-}
+pub struct DummyExecutor;
 
-impl DummyExecutor {
-    pub fn new(info: ExecutorInfo) -> Self {
-        Self { info }
-    }
-}
-
-impl Executor for DummyExecutor {
+impl Execute for DummyExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
         futures::stream::pending().boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
 }
 
 pub(crate) struct ActorInputMetrics {
diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs
index 2a927f1a3a780..d7deee71f1443 100644
--- a/src/stream/src/executor/values.rs
+++ b/src/stream/src/executor/values.rs
@@ -24,10 +24,7 @@ use risingwave_common::util::iter_util::ZipEqFast;
 use risingwave_expr::expr::NonStrictExpression;
 use tokio::sync::mpsc::UnboundedReceiver;
 
-use super::{
-    ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef,
-    StreamExecutorError,
-};
+use super::{ActorContextRef, Barrier, BoxedMessageStream, Execute, Message, StreamExecutorError};
 use crate::task::CreateMviewProgress;
 
 const DEFAULT_CHUNK_SIZE: usize = 1024;
@@ -36,8 +33,8 @@ const DEFAULT_CHUNK_SIZE: usize = 1024;
 /// May refractor with `BarrierRecvExecutor` in the near future.
 pub struct ValuesExecutor {
     ctx: ActorContextRef,
-    info: ExecutorInfo,
 
+    schema: Schema,
     // Receiver of barrier channel.
     barrier_receiver: UnboundedReceiver<Barrier>,
     progress: CreateMviewProgress,
@@ -49,14 +46,14 @@ impl ValuesExecutor {
     /// Currently hard-code the `pk_indices` as the last column.
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
+        schema: Schema,
         progress: CreateMviewProgress,
         rows: Vec<Vec<NonStrictExpression>>,
         barrier_receiver: UnboundedReceiver<Barrier>,
     ) -> Self {
         Self {
             ctx,
-            info,
+            schema,
             progress,
             barrier_receiver,
             rows: rows.into_iter(),
@@ -64,9 +61,9 @@ impl ValuesExecutor {
     }
 
     #[try_stream(ok = Message, error = StreamExecutorError)]
-    async fn into_stream(self) {
+    async fn execute_inner(self) {
         let Self {
-            info,
+            schema,
             mut progress,
             mut barrier_receiver,
             mut rows,
@@ -96,7 +93,7 @@ impl ValuesExecutor {
                 }
             }
 
-            let cardinality = info.schema.len();
+            let cardinality = schema.len();
             ensure!(cardinality > 0);
             while !rows.is_empty() {
                 // We need a one row chunk rather than an empty chunk because constant
@@ -105,7 +102,7 @@ impl ValuesExecutor {
                 let one_row_chunk = DataChunk::new_dummy(1);
 
                 let chunk_size = DEFAULT_CHUNK_SIZE.min(rows.len());
-                let mut array_builders = info.schema.create_array_builders(chunk_size);
+                let mut array_builders = schema.create_array_builders(chunk_size);
                 for row in rows.by_ref().take(chunk_size) {
                     for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) {
                         let out = expr.eval_infallible(&one_row_chunk).await;
@@ -135,21 +132,9 @@ impl ValuesExecutor {
     }
 }
 
-impl Executor for ValuesExecutor {
+impl Execute for ValuesExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
-        self.into_stream().boxed()
-    }
-
-    fn schema(&self) -> &Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
+        self.execute_inner().boxed()
     }
 }
 
@@ -167,7 +152,7 @@ mod tests {
 
     use super::ValuesExecutor;
     use crate::executor::test_utils::StreamExecutorTestExt;
-    use crate::executor::{ActorContext, AddMutation, Barrier, Executor, ExecutorInfo, Mutation};
+    use crate::executor::{ActorContext, AddMutation, Barrier, Execute, Mutation};
     use crate::task::{CreateMviewProgress, LocalBarrierManager};
 
     #[tokio::test]
@@ -206,15 +191,9 @@ mod tests {
             .iter() // for each column
             .map(|col| Field::unnamed(col.return_type()))
             .collect::<Schema>();
-        let pk_indices = vec![schema.len() - 1];
-        let info = ExecutorInfo {
-            schema,
-            pk_indices,
-            identity: "ValuesExecutor".to_string(),
-        };
         let values_executor_struct = ValuesExecutor::new(
             ActorContext::for_test(actor_id),
-            info,
+            schema,
             progress,
             vec![exprs
                 .into_iter()
diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs
index 1aa671b4f671c..4875d8a53c2d1 100644
--- a/src/stream/src/executor/watermark_filter.rs
+++ b/src/stream/src/executor/watermark_filter.rs
@@ -36,9 +36,7 @@ use risingwave_storage::StateStore;
 
 use super::error::StreamExecutorError;
 use super::filter::FilterExecutor;
-use super::{
-    ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, StreamExecutorResult,
-};
+use super::{ActorContextRef, Execute, Executor, ExecutorInfo, Message, StreamExecutorResult};
 use crate::common::table::state_table::StateTable;
 use crate::executor::{expect_first_barrier, Watermark};
 use crate::task::ActorEvalErrorReport;
@@ -48,59 +46,49 @@ use crate::task::ActorEvalErrorReport;
 /// filtered.
 pub struct WatermarkFilterExecutor<S: StateStore> {
     ctx: ActorContextRef,
-    info: ExecutorInfo,
 
-    input: BoxedExecutor,
+    input: Executor,
     /// The expression used to calculate the watermark value.
     watermark_expr: NonStrictExpression,
     /// The column we should generate watermark and filter on.
     event_time_col_idx: usize,
     table: StateTable<S>,
     global_watermark_table: StorageTable<S>,
+
+    eval_error_report: ActorEvalErrorReport,
 }
 
 impl<S: StateStore> WatermarkFilterExecutor<S> {
     pub fn new(
         ctx: ActorContextRef,
-        info: ExecutorInfo,
-        input: BoxedExecutor,
+        info: &ExecutorInfo,
+        input: Executor,
         watermark_expr: NonStrictExpression,
         event_time_col_idx: usize,
         table: StateTable<S>,
         global_watermark_table: StorageTable<S>,
     ) -> Self {
+        let eval_error_report = ActorEvalErrorReport {
+            actor_context: ctx.clone(),
+            identity: Arc::from(info.identity.as_ref()),
+        };
+
         Self {
             ctx,
-            info,
             input,
             watermark_expr,
             event_time_col_idx,
             table,
             global_watermark_table,
+            eval_error_report,
         }
     }
 }
 
-impl<S: StateStore> Executor for WatermarkFilterExecutor<S> {
+impl<S: StateStore> Execute for WatermarkFilterExecutor<S> {
     fn execute(self: Box<Self>) -> super::BoxedMessageStream {
         self.execute_inner().boxed()
     }
-
-    fn schema(&self) -> &risingwave_common::catalog::Schema {
-        &self.info.schema
-    }
-
-    fn pk_indices(&self) -> super::PkIndicesRef<'_> {
-        &self.info.pk_indices
-    }
-
-    fn identity(&self) -> &str {
-        &self.info.identity
-    }
-
-    fn info(&self) -> ExecutorInfo {
-        self.info.clone()
-    }
 }
 
 impl<S: StateStore> WatermarkFilterExecutor<S> {
@@ -111,16 +99,11 @@ impl<S: StateStore> WatermarkFilterExecutor<S> {
             event_time_col_idx,
             watermark_expr,
             ctx,
-            info,
             mut table,
             mut global_watermark_table,
+            eval_error_report,
         } = *self;
 
-        let eval_error_report = ActorEvalErrorReport {
-            actor_context: ctx.clone(),
-            identity: info.identity.into(),
-        };
-
         let watermark_type = watermark_expr.return_type();
         assert_eq!(
             watermark_type,
@@ -456,7 +439,7 @@ mod tests {
 
     async fn create_watermark_filter_executor(
         mem_state: MemoryStateStore,
-    ) -> (BoxedExecutor, MessageSender) {
+    ) -> (Box<dyn Execute>, MessageSender) {
         let schema = Schema {
             fields: vec![
                 Field::unnamed(DataType::Int16),        // pk
@@ -476,7 +459,8 @@ mod tests {
         )
         .await;
 
-        let (tx, source) = MockSource::channel(schema, vec![0]);
+        let (tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, vec![0]);
 
         let info = ExecutorInfo {
             schema: source.schema().clone(),
@@ -487,8 +471,8 @@ mod tests {
         (
             WatermarkFilterExecutor::new(
                 ActorContext::for_test(123),
-                info,
-                source.boxed(),
+                &info,
+                source,
                 watermark_expr,
                 1,
                 table,
diff --git a/src/stream/src/executor/wrapper.rs b/src/stream/src/executor/wrapper.rs
index ebd3f6d9bfa73..dddd94da5ab73 100644
--- a/src/stream/src/executor/wrapper.rs
+++ b/src/stream/src/executor/wrapper.rs
@@ -15,9 +15,8 @@
 use std::sync::Arc;
 
 use futures::StreamExt;
-use risingwave_common::catalog::Schema;
 
-use super::*;
+use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, MessageStream};
 
 mod epoch_check;
 mod epoch_provide;
@@ -27,7 +26,7 @@ mod update_check;
 
 /// [`WrapperExecutor`] will do some sanity checks and logging for the wrapped executor.
 pub struct WrapperExecutor {
-    input: BoxedExecutor,
+    input: Executor,
 
     actor_ctx: ActorContextRef,
 
@@ -36,7 +35,7 @@ pub struct WrapperExecutor {
 
 impl WrapperExecutor {
     pub fn new(
-        input: BoxedExecutor,
+        input: Executor,
         actor_ctx: ActorContextRef,
         enable_executor_row_count: bool,
     ) -> Self {
@@ -88,9 +87,9 @@ impl WrapperExecutor {
     }
 }
 
-impl Executor for WrapperExecutor {
+impl Execute for WrapperExecutor {
     fn execute(self: Box<Self>) -> BoxedMessageStream {
-        let info = Arc::new(self.input.info());
+        let info = Arc::new(self.input.info().clone());
         Self::wrap(
             self.enable_executor_row_count,
             info,
@@ -101,7 +100,7 @@ impl Executor for WrapperExecutor {
     }
 
     fn execute_with_epoch(self: Box<Self>, epoch: u64) -> BoxedMessageStream {
-        let info = Arc::new(self.input.info());
+        let info = Arc::new(self.input.info().clone());
         Self::wrap(
             self.enable_executor_row_count,
             info,
@@ -110,16 +109,4 @@ impl Executor for WrapperExecutor {
         )
         .boxed()
     }
-
-    fn schema(&self) -> &Schema {
-        self.input.schema()
-    }
-
-    fn pk_indices(&self) -> PkIndicesRef<'_> {
-        self.input.pk_indices()
-    }
-
-    fn identity(&self) -> &str {
-        self.input.identity()
-    }
 }
diff --git a/src/stream/src/executor/wrapper/epoch_check.rs b/src/stream/src/executor/wrapper/epoch_check.rs
index 732c67630a345..13d755dcbc60d 100644
--- a/src/stream/src/executor/wrapper/epoch_check.rs
+++ b/src/stream/src/executor/wrapper/epoch_check.rs
@@ -80,18 +80,18 @@ mod tests {
 
     use super::*;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::Executor;
 
     #[tokio::test]
     async fn test_epoch_ok() {
-        let (mut tx, source) = MockSource::channel(Default::default(), vec![]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(Default::default(), vec![]);
         tx.push_barrier(1, false);
         tx.push_chunk(StreamChunk::default());
         tx.push_barrier(2, false);
         tx.push_barrier(3, false);
         tx.push_barrier(4, false);
 
-        let checked = epoch_check(source.info().into(), source.boxed().execute());
+        let checked = epoch_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         assert_matches!(checked.next().await.unwrap().unwrap(), Message::Barrier(b) if b.epoch.curr == 1);
@@ -104,14 +104,15 @@ mod tests {
     #[should_panic]
     #[tokio::test]
     async fn test_epoch_bad() {
-        let (mut tx, source) = MockSource::channel(Default::default(), vec![]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(Default::default(), vec![]);
         tx.push_barrier(100, false);
         tx.push_chunk(StreamChunk::default());
         tx.push_barrier(514, false);
         tx.push_barrier(514, false);
         tx.push_barrier(114, false);
 
-        let checked = epoch_check(source.info().into(), source.boxed().execute());
+        let checked = epoch_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         assert_matches!(checked.next().await.unwrap().unwrap(), Message::Barrier(b) if b.epoch.curr == 100);
@@ -125,11 +126,12 @@ mod tests {
     #[should_panic]
     #[tokio::test]
     async fn test_epoch_first_not_barrier() {
-        let (mut tx, source) = MockSource::channel(Default::default(), vec![]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(Default::default(), vec![]);
         tx.push_chunk(StreamChunk::default());
         tx.push_barrier(114, false);
 
-        let checked = epoch_check(source.info().into(), source.boxed().execute());
+        let checked = epoch_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         checked.next().await.unwrap().unwrap(); // should panic
@@ -137,9 +139,11 @@ mod tests {
 
     #[tokio::test]
     async fn test_empty() {
-        let (_, mut source) = MockSource::channel(Default::default(), vec![]);
-        source = source.stop_on_finish(false);
-        let checked = epoch_check(source.info().into(), source.boxed().execute());
+        let (_, source) = MockSource::channel();
+        let source = source
+            .stop_on_finish(false)
+            .into_executor(Default::default(), vec![]);
+        let checked = epoch_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         assert!(checked.next().await.transpose().unwrap().is_none());
diff --git a/src/stream/src/executor/wrapper/schema_check.rs b/src/stream/src/executor/wrapper/schema_check.rs
index 462ef5962042e..ad5f0acd3dfaa 100644
--- a/src/stream/src/executor/wrapper/schema_check.rs
+++ b/src/stream/src/executor/wrapper/schema_check.rs
@@ -62,7 +62,6 @@ mod tests {
 
     use super::*;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::Executor;
 
     #[tokio::test]
     async fn test_schema_ok() {
@@ -73,7 +72,8 @@ mod tests {
             ],
         };
 
-        let (mut tx, source) = MockSource::channel(schema, vec![1]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, vec![1]);
         tx.push_chunk(StreamChunk::from_pretty(
             "   I     F
             + 100 200.0
@@ -82,7 +82,7 @@ mod tests {
         ));
         tx.push_barrier(1, false);
 
-        let checked = schema_check(source.info().into(), source.boxed().execute());
+        let checked = schema_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         assert_matches!(checked.next().await.unwrap().unwrap(), Message::Chunk(_));
@@ -99,7 +99,8 @@ mod tests {
             ],
         };
 
-        let (mut tx, source) = MockSource::channel(schema, vec![1]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(schema, vec![1]);
         tx.push_chunk(StreamChunk::from_pretty(
             "   I   I
             + 100 200
@@ -108,7 +109,7 @@ mod tests {
         ));
         tx.push_barrier(1, false);
 
-        let checked = schema_check(source.info().into(), source.boxed().execute());
+        let checked = schema_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
         checked.next().await.unwrap().unwrap();
     }
diff --git a/src/stream/src/executor/wrapper/update_check.rs b/src/stream/src/executor/wrapper/update_check.rs
index 4049960845282..37a2ca4f2a942 100644
--- a/src/stream/src/executor/wrapper/update_check.rs
+++ b/src/stream/src/executor/wrapper/update_check.rs
@@ -62,12 +62,12 @@ mod tests {
 
     use super::*;
     use crate::executor::test_utils::MockSource;
-    use crate::executor::Executor;
 
     #[should_panic]
     #[tokio::test]
     async fn test_not_next_to_each_other() {
-        let (mut tx, source) = MockSource::channel(Default::default(), vec![]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(Default::default(), vec![]);
         tx.push_chunk(StreamChunk::from_pretty(
             "     I
             U-  114
@@ -76,7 +76,7 @@ mod tests {
             U+  810",
         ));
 
-        let checked = update_check(source.info().into(), source.boxed().execute());
+        let checked = update_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         checked.next().await.unwrap().unwrap(); // should panic
@@ -85,13 +85,14 @@ mod tests {
     #[should_panic]
     #[tokio::test]
     async fn test_first_one_update_insert() {
-        let (mut tx, source) = MockSource::channel(Default::default(), vec![]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(Default::default(), vec![]);
         tx.push_chunk(StreamChunk::from_pretty(
             "     I
             U+  114",
         ));
 
-        let checked = update_check(source.info().into(), source.boxed().execute());
+        let checked = update_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         checked.next().await.unwrap().unwrap(); // should panic
@@ -100,7 +101,8 @@ mod tests {
     #[should_panic]
     #[tokio::test]
     async fn test_last_one_update_delete() {
-        let (mut tx, source) = MockSource::channel(Default::default(), vec![]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(Default::default(), vec![]);
         tx.push_chunk(StreamChunk::from_pretty(
             "        I
             U-     114
@@ -108,7 +110,7 @@ mod tests {
             U- 1919810",
         ));
 
-        let checked = update_check(source.info().into(), source.boxed().execute());
+        let checked = update_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         checked.next().await.unwrap().unwrap(); // should panic
@@ -116,10 +118,11 @@ mod tests {
 
     #[tokio::test]
     async fn test_empty_chunk() {
-        let (mut tx, source) = MockSource::channel(Default::default(), vec![]);
+        let (mut tx, source) = MockSource::channel();
+        let source = source.into_executor(Default::default(), vec![]);
         tx.push_chunk(StreamChunk::default());
 
-        let checked = update_check(source.info().into(), source.boxed().execute());
+        let checked = update_check(source.info().clone().into(), source.execute());
         pin_mut!(checked);
 
         checked.next().await.unwrap().unwrap();
diff --git a/src/stream/src/from_proto/append_only_dedup.rs b/src/stream/src/from_proto/append_only_dedup.rs
index 4ae2ee24ce798..020d13f3ff96b 100644
--- a/src/stream/src/from_proto/append_only_dedup.rs
+++ b/src/stream/src/from_proto/append_only_dedup.rs
@@ -20,7 +20,7 @@ use risingwave_storage::StateStore;
 use super::ExecutorBuilder;
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
-use crate::executor::{AppendOnlyDedupExecutor, BoxedExecutor};
+use crate::executor::{AppendOnlyDedupExecutor, Executor};
 use crate::task::ExecutorParams;
 
 pub struct AppendOnlyDedupExecutorBuilder;
@@ -32,18 +32,19 @@ impl ExecutorBuilder for AppendOnlyDedupExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let table = node.get_state_table()?;
         let vnodes = params.vnode_bitmap.map(Arc::new);
         let state_table = StateTable::from_table_catalog(table, store, vnodes).await;
-        Ok(Box::new(AppendOnlyDedupExecutor::new(
+        let exec = AppendOnlyDedupExecutor::new(
+            params.actor_context,
             input,
+            params.info.pk_indices.clone(), /* TODO(rc): should change to use `dedup_column_indices`, but need to check backward compatibility */
             state_table,
-            params.info,
-            params.actor_context,
             params.watermark_epoch,
             params.executor_stats.clone(),
-        )))
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/barrier_recv.rs b/src/stream/src/from_proto/barrier_recv.rs
index 032afde2ee1dc..b8c38f0f53d32 100644
--- a/src/stream/src/from_proto/barrier_recv.rs
+++ b/src/stream/src/from_proto/barrier_recv.rs
@@ -27,7 +27,7 @@ impl ExecutorBuilder for BarrierRecvExecutorBuilder {
         params: ExecutorParams,
         _node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         assert!(
             params.input.is_empty(),
             "barrier receiver should not have input"
@@ -38,6 +38,7 @@ impl ExecutorBuilder for BarrierRecvExecutorBuilder {
             .local_barrier_manager
             .register_sender(params.actor_context.id, sender);
 
-        Ok(BarrierRecvExecutor::new(params.actor_context, params.info, barrier_receiver).boxed())
+        let exec = BarrierRecvExecutor::new(params.actor_context, barrier_receiver);
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs
index 4fc492f7c984d..5e86d7e6d5b30 100644
--- a/src/stream/src/from_proto/batch_query.rs
+++ b/src/stream/src/from_proto/batch_query.rs
@@ -30,12 +30,12 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         state_store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         if node.table_desc.is_none() {
             // used in sharing cdc source backfill as a dummy batch plan node
             let mut info = params.info;
             info.identity = "DummyBatchQueryExecutor".to_string();
-            return Ok(Box::new(DummyExecutor::new(info)));
+            return Ok((info, DummyExecutor).into());
         }
 
         let table_desc: &StorageTableDesc = node.get_table_desc()?;
@@ -55,9 +55,11 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder {
         );
         assert_eq!(table.schema().data_types(), params.info.schema.data_types());
 
-        let executor =
-            BatchQueryExecutor::new(table, params.env.config().developer.chunk_size, params.info);
-
-        Ok(executor.boxed())
+        let exec = BatchQueryExecutor::new(
+            table,
+            params.env.config().developer.chunk_size,
+            params.info.schema.clone(),
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/cdc_filter.rs b/src/stream/src/from_proto/cdc_filter.rs
index 5747e1894d4a8..673f6a1c5e712 100644
--- a/src/stream/src/from_proto/cdc_filter.rs
+++ b/src/stream/src/from_proto/cdc_filter.rs
@@ -28,11 +28,12 @@ impl ExecutorBuilder for CdcFilterExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let search_condition =
             build_non_strict_from_prost(node.get_search_condition()?, params.eval_error_report)?;
 
-        Ok(FilterExecutor::new(params.actor_context, params.info, input, search_condition).boxed())
+        let exec = FilterExecutor::new(params.actor_context, input, search_condition);
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/dml.rs b/src/stream/src/from_proto/dml.rs
index d7f21fda7bf2d..cc8836890ce7f 100644
--- a/src/stream/src/from_proto/dml.rs
+++ b/src/stream/src/from_proto/dml.rs
@@ -20,7 +20,7 @@ use risingwave_storage::StateStore;
 use super::ExecutorBuilder;
 use crate::error::StreamResult;
 use crate::executor::dml::DmlExecutor;
-use crate::executor::BoxedExecutor;
+use crate::executor::Executor;
 use crate::task::ExecutorParams;
 
 pub struct DmlExecutorBuilder;
@@ -32,19 +32,19 @@ impl ExecutorBuilder for DmlExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [upstream]: [_; 1] = params.input.try_into().unwrap();
         let table_id = TableId::new(node.table_id);
         let column_descs = node.column_descs.iter().map(Into::into).collect_vec();
 
-        Ok(Box::new(DmlExecutor::new(
-            params.info,
+        let exec = DmlExecutor::new(
             upstream,
             params.env.dml_manager_ref(),
             table_id,
             node.table_version_id,
             column_descs,
             params.env.config().developer.chunk_size,
-        )))
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/dynamic_filter.rs b/src/stream/src/from_proto/dynamic_filter.rs
index babb2680a9575..c09fd767ad9ef 100644
--- a/src/stream/src/from_proto/dynamic_filter.rs
+++ b/src/stream/src/from_proto/dynamic_filter.rs
@@ -32,7 +32,7 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [source_l, source_r]: [_; 2] = params.input.try_into().unwrap();
         let key_l = node.get_left_key() as usize;
 
@@ -62,7 +62,7 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder {
         let left_table = node.get_left_table()?;
         let cleaned_by_watermark = left_table.get_cleaned_by_watermark();
 
-        if cleaned_by_watermark {
+        let exec = if cleaned_by_watermark {
             let state_table_l = WatermarkCacheStateTable::from_table_catalog(
                 node.get_left_table()?,
                 store,
@@ -70,9 +70,9 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder {
             )
             .await;
 
-            Ok(Box::new(DynamicFilterExecutor::new(
+            DynamicFilterExecutor::new(
                 params.actor_context,
-                params.info,
+                &params.info,
                 source_l,
                 source_r,
                 key_l,
@@ -83,14 +83,15 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder {
                 params.env.config().developer.chunk_size,
                 condition_always_relax,
                 cleaned_by_watermark,
-            )))
+            )
+            .boxed()
         } else {
             let state_table_l =
                 StateTable::from_table_catalog(node.get_left_table()?, store, Some(vnodes)).await;
 
-            Ok(Box::new(DynamicFilterExecutor::new(
+            DynamicFilterExecutor::new(
                 params.actor_context,
-                params.info,
+                &params.info,
                 source_l,
                 source_r,
                 key_l,
@@ -101,7 +102,10 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder {
                 params.env.config().developer.chunk_size,
                 condition_always_relax,
                 cleaned_by_watermark,
-            )))
-        }
+            )
+            .boxed()
+        };
+
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/eowc_over_window.rs b/src/stream/src/from_proto/eowc_over_window.rs
index b2136f51c3970..4f6e873d7bcf3 100644
--- a/src/stream/src/from_proto/eowc_over_window.rs
+++ b/src/stream/src/from_proto/eowc_over_window.rs
@@ -21,9 +21,7 @@ use risingwave_storage::StateStore;
 use super::ExecutorBuilder;
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
-use crate::executor::{
-    BoxedExecutor, EowcOverWindowExecutor, EowcOverWindowExecutorArgs, Executor,
-};
+use crate::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs, Executor};
 use crate::task::ExecutorParams;
 
 pub struct EowcOverWindowExecutorBuilder;
@@ -35,7 +33,7 @@ impl ExecutorBuilder for EowcOverWindowExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let calls: Vec<_> = node
             .get_calls()
@@ -56,18 +54,18 @@ impl ExecutorBuilder for EowcOverWindowExecutorBuilder {
         let state_table =
             StateTable::from_table_catalog_inconsistent_op(node.get_state_table()?, store, vnodes)
                 .await;
-        Ok(EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs {
+        let exec = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs {
             actor_ctx: params.actor_context,
-            info: params.info,
 
             input,
 
+            schema: params.info.schema.clone(),
             calls,
             partition_key_indices,
             order_key_index,
             state_table,
             watermark_epoch: params.watermark_epoch,
-        })
-        .boxed())
+        });
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/expand.rs b/src/stream/src/from_proto/expand.rs
index 40bb972cdbd33..34d4313e24cb7 100644
--- a/src/stream/src/from_proto/expand.rs
+++ b/src/stream/src/from_proto/expand.rs
@@ -26,7 +26,7 @@ impl ExecutorBuilder for ExpandExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let column_subsets = node
             .column_subsets
@@ -39,6 +39,6 @@ impl ExecutorBuilder for ExpandExecutorBuilder {
                     .collect_vec()
             })
             .collect_vec();
-        Ok(ExpandExecutor::new(params.info, input, column_subsets).boxed())
+        Ok((params.info, ExpandExecutor::new(input, column_subsets)).into())
     }
 }
diff --git a/src/stream/src/from_proto/filter.rs b/src/stream/src/from_proto/filter.rs
index a82b9cc65dea7..a4efb953064b8 100644
--- a/src/stream/src/from_proto/filter.rs
+++ b/src/stream/src/from_proto/filter.rs
@@ -27,11 +27,12 @@ impl ExecutorBuilder for FilterExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let search_condition =
             build_non_strict_from_prost(node.get_search_condition()?, params.eval_error_report)?;
 
-        Ok(FilterExecutor::new(params.actor_context, params.info, input, search_condition).boxed())
+        let exec = FilterExecutor::new(params.actor_context, input, search_condition);
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/group_top_n.rs b/src/stream/src/from_proto/group_top_n.rs
index a90e13d3f496b..51512cb9d94d3 100644
--- a/src/stream/src/from_proto/group_top_n.rs
+++ b/src/stream/src/from_proto/group_top_n.rs
@@ -14,6 +14,7 @@
 
 use std::sync::Arc;
 
+use risingwave_common::catalog::Schema;
 use risingwave_common::hash::{HashKey, HashKeyDispatcher};
 use risingwave_common::types::DataType;
 use risingwave_common::util::sort_util::ColumnOrder;
@@ -21,7 +22,7 @@ use risingwave_pb::stream_plan::GroupTopNNode;
 
 use super::*;
 use crate::common::table::state_table::StateTable;
-use crate::executor::{ActorContextRef, AppendOnlyGroupTopNExecutor, GroupTopNExecutor};
+use crate::executor::{ActorContextRef, AppendOnlyGroupTopNExecutor, Executor, GroupTopNExecutor};
 use crate::task::AtomicU64Ref;
 
 pub struct GroupTopNExecutorBuilder<const APPEND_ONLY: bool>;
@@ -33,7 +34,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for GroupTopNExecutorBuilder<APPEN
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let group_by: Vec<usize> = node
             .get_group_key()
             .iter()
@@ -61,7 +62,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for GroupTopNExecutorBuilder<APPEN
         let args = GroupTopNExecutorDispatcherArgs {
             input,
             ctx: params.actor_context,
-            info: params.info,
+            schema: params.info.schema.clone(),
             storage_key,
             offset_and_limit: (node.offset as usize, node.limit as usize),
             order_by,
@@ -73,14 +74,14 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for GroupTopNExecutorBuilder<APPEN
             with_ties: node.with_ties,
             append_only: APPEND_ONLY,
         };
-        args.dispatch()
+        Ok((params.info, args.dispatch()?).into())
     }
 }
 
 struct GroupTopNExecutorDispatcherArgs<S: StateStore> {
-    input: BoxedExecutor,
+    input: Executor,
     ctx: ActorContextRef,
-    info: ExecutorInfo,
+    schema: Schema,
     storage_key: Vec<ColumnOrder>,
     offset_and_limit: (usize, usize),
     order_by: Vec<ColumnOrder>,
@@ -94,7 +95,7 @@ struct GroupTopNExecutorDispatcherArgs<S: StateStore> {
 }
 
 impl<S: StateStore> HashKeyDispatcher for GroupTopNExecutorDispatcherArgs<S> {
-    type Output = StreamResult<BoxedExecutor>;
+    type Output = StreamResult<Box<dyn Execute>>;
 
     fn dispatch_impl<K: HashKey>(self) -> Self::Output {
         macro_rules! build {
@@ -102,7 +103,7 @@ impl<S: StateStore> HashKeyDispatcher for GroupTopNExecutorDispatcherArgs<S> {
                 Ok($excutor::<K, S, $with_ties>::new(
                     self.input,
                     self.ctx,
-                    self.info,
+                    self.schema,
                     self.storage_key,
                     self.offset_and_limit,
                     self.order_by,
diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs
index 52390b6806d0e..3331640519617 100644
--- a/src/stream/src/from_proto/hash_agg.rs
+++ b/src/stream/src/from_proto/hash_agg.rs
@@ -35,7 +35,7 @@ pub struct HashAggExecutorDispatcherArgs<S: StateStore> {
 }
 
 impl<S: StateStore> HashKeyDispatcher for HashAggExecutorDispatcherArgs<S> {
-    type Output = StreamResult<BoxedExecutor>;
+    type Output = StreamResult<Box<dyn Execute>>;
 
     fn dispatch_impl<K: HashKey>(self) -> Self::Output {
         Ok(HashAggExecutor::<K, S>::new(self.args)?.boxed())
@@ -55,7 +55,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let group_key_indices = node
             .get_group_key()
             .iter()
@@ -93,13 +93,13 @@ impl ExecutorBuilder for HashAggExecutorBuilder {
             build_distinct_dedup_table_from_proto(node.get_distinct_dedup_tables(), store, vnodes)
                 .await;
 
-        HashAggExecutorDispatcherArgs {
+        let exec = HashAggExecutorDispatcherArgs {
             args: AggExecutorArgs {
                 version: node.version(),
 
                 input,
                 actor_ctx: params.actor_context,
-                info: params.info,
+                info: params.info.clone(),
 
                 extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size,
 
@@ -122,6 +122,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder {
             },
             group_key_types,
         }
-        .dispatch()
+        .dispatch()?;
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs
index 0c50f26f941b6..ad69fd1ecd5ac 100644
--- a/src/stream/src/from_proto/hash_join.rs
+++ b/src/stream/src/from_proto/hash_join.rs
@@ -27,7 +27,7 @@ use super::*;
 use crate::common::table::state_table::StateTable;
 use crate::executor::hash_join::*;
 use crate::executor::monitor::StreamingMetrics;
-use crate::executor::{ActorContextRef, JoinType};
+use crate::executor::{ActorContextRef, Executor, JoinType};
 use crate::task::AtomicU64Ref;
 
 pub struct HashJoinExecutorBuilder;
@@ -39,7 +39,7 @@ impl ExecutorBuilder for HashJoinExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let is_append_only = node.is_append_only;
         let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for hash join"));
 
@@ -136,7 +136,7 @@ impl ExecutorBuilder for HashJoinExecutorBuilder {
 
         let args = HashJoinExecutorDispatcherArgs {
             ctx: params.actor_context,
-            info: params.info,
+            info: params.info.clone(),
             source_l,
             source_r,
             params_l,
@@ -157,15 +157,16 @@ impl ExecutorBuilder for HashJoinExecutorBuilder {
             chunk_size: params.env.config().developer.chunk_size,
         };
 
-        args.dispatch()
+        let exec = args.dispatch()?;
+        Ok((params.info, exec).into())
     }
 }
 
 struct HashJoinExecutorDispatcherArgs<S: StateStore> {
     ctx: ActorContextRef,
     info: ExecutorInfo,
-    source_l: Box<dyn Executor>,
-    source_r: Box<dyn Executor>,
+    source_l: Executor,
+    source_r: Executor,
     params_l: JoinParams,
     params_r: JoinParams,
     null_safe: Vec<bool>,
@@ -185,34 +186,33 @@ struct HashJoinExecutorDispatcherArgs<S: StateStore> {
 }
 
 impl<S: StateStore> HashKeyDispatcher for HashJoinExecutorDispatcherArgs<S> {
-    type Output = StreamResult<BoxedExecutor>;
+    type Output = StreamResult<Box<dyn Execute>>;
 
     fn dispatch_impl<K: HashKey>(self) -> Self::Output {
         /// This macro helps to fill the const generic type parameter.
         macro_rules! build {
             ($join_type:ident) => {
-                Ok(Box::new(
-                    HashJoinExecutor::<K, S, { JoinType::$join_type }>::new(
-                        self.ctx,
-                        self.info,
-                        self.source_l,
-                        self.source_r,
-                        self.params_l,
-                        self.params_r,
-                        self.null_safe,
-                        self.output_indices,
-                        self.cond,
-                        self.inequality_pairs,
-                        self.state_table_l,
-                        self.degree_state_table_l,
-                        self.state_table_r,
-                        self.degree_state_table_r,
-                        self.lru_manager,
-                        self.is_append_only,
-                        self.metrics,
-                        self.chunk_size,
-                    ),
-                ))
+                Ok(HashJoinExecutor::<K, S, { JoinType::$join_type }>::new(
+                    self.ctx,
+                    self.info,
+                    self.source_l,
+                    self.source_r,
+                    self.params_l,
+                    self.params_r,
+                    self.null_safe,
+                    self.output_indices,
+                    self.cond,
+                    self.inequality_pairs,
+                    self.state_table_l,
+                    self.degree_state_table_l,
+                    self.state_table_r,
+                    self.degree_state_table_r,
+                    self.lru_manager,
+                    self.is_append_only,
+                    self.metrics,
+                    self.chunk_size,
+                )
+                .boxed())
             };
         }
         match self.join_type_proto {
diff --git a/src/stream/src/from_proto/hop_window.rs b/src/stream/src/from_proto/hop_window.rs
index a56cbc2d8c976..2598ae927608e 100644
--- a/src/stream/src/from_proto/hop_window.rs
+++ b/src/stream/src/from_proto/hop_window.rs
@@ -27,16 +27,8 @@ impl ExecutorBuilder for HopWindowExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
-        let ExecutorParams {
-            actor_context,
-            info,
-            input,
-            env,
-            ..
-        } = params;
-
-        let input = input.into_iter().next().unwrap();
+    ) -> StreamResult<Executor> {
+        let input = params.input.into_iter().next().unwrap();
         // TODO: reuse the schema derivation with frontend.
         let output_indices = node
             .get_output_indices()
@@ -59,11 +51,10 @@ impl ExecutorBuilder for HopWindowExecutorBuilder {
         let window_slide = node.get_window_slide()?.into();
         let window_size = node.get_window_size()?.into();
 
-        let chunk_size = env.config().developer.chunk_size;
+        let chunk_size = params.env.config().developer.chunk_size;
 
-        Ok(HopWindowExecutor::new(
-            actor_context,
-            info,
+        let exec = HopWindowExecutor::new(
+            params.actor_context,
             input,
             time_col,
             window_slide,
@@ -72,7 +63,7 @@ impl ExecutorBuilder for HopWindowExecutorBuilder {
             window_end_exprs,
             output_indices,
             chunk_size,
-        )
-        .boxed())
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs
index 1f873f31127c5..dc7f7e3c49dfe 100644
--- a/src/stream/src/from_proto/lookup.rs
+++ b/src/stream/src/from_proto/lookup.rs
@@ -30,7 +30,7 @@ impl ExecutorBuilder for LookupExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let lookup = node;
 
         let [stream, arrangement]: [_; 2] = params.input.try_into().unwrap();
@@ -69,9 +69,9 @@ impl ExecutorBuilder for LookupExecutorBuilder {
             table_desc,
         );
 
-        Ok(Box::new(LookupExecutor::new(LookupExecutorParams {
+        let exec = LookupExecutor::new(LookupExecutorParams {
             ctx: params.actor_context,
-            info: params.info,
+            info: params.info.clone(),
             arrangement,
             stream,
             arrangement_col_descs,
@@ -83,6 +83,7 @@ impl ExecutorBuilder for LookupExecutorBuilder {
             storage_table,
             watermark_epoch: params.watermark_epoch,
             chunk_size: params.env.config().developer.chunk_size,
-        })))
+        });
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/lookup_union.rs b/src/stream/src/from_proto/lookup_union.rs
index e9cc0ed33311a..93e847609cae2 100644
--- a/src/stream/src/from_proto/lookup_union.rs
+++ b/src/stream/src/from_proto/lookup_union.rs
@@ -26,7 +26,8 @@ impl ExecutorBuilder for LookupUnionExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
-        Ok(LookupUnionExecutor::new(params.info, params.input, node.order.clone()).boxed())
+    ) -> StreamResult<Executor> {
+        let exec = LookupUnionExecutor::new(params.input, node.order.clone());
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/merge.rs b/src/stream/src/from_proto/merge.rs
index aa0a6a9d2bb92..eded1f59d294e 100644
--- a/src/stream/src/from_proto/merge.rs
+++ b/src/stream/src/from_proto/merge.rs
@@ -27,7 +27,7 @@ impl ExecutorBuilder for MergeExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let upstreams = node.get_upstream_actor_id();
         let upstream_fragment_id = node.get_upstream_fragment_id();
 
@@ -56,10 +56,9 @@ impl ExecutorBuilder for MergeExecutorBuilder {
             DispatcherType::NoShuffle => true,
         };
 
-        if always_single_input {
-            Ok(ReceiverExecutor::new(
+        let exec = if always_single_input {
+            ReceiverExecutor::new(
                 params.actor_context,
-                params.info,
                 params.fragment_id,
                 upstream_fragment_id,
                 inputs.into_iter().exactly_one().unwrap(),
@@ -67,11 +66,10 @@ impl ExecutorBuilder for MergeExecutorBuilder {
                 params.operator_id,
                 params.executor_stats.clone(),
             )
-            .boxed())
+            .boxed()
         } else {
-            Ok(MergeExecutor::new(
+            MergeExecutor::new(
                 params.actor_context,
-                params.info,
                 params.fragment_id,
                 upstream_fragment_id,
                 inputs,
@@ -79,7 +77,8 @@ impl ExecutorBuilder for MergeExecutorBuilder {
                 params.operator_id,
                 params.executor_stats.clone(),
             )
-            .boxed())
-        }
+            .boxed()
+        };
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs
index 46ec1a6cc8e59..c00a9fd844df7 100644
--- a/src/stream/src/from_proto/mod.rs
+++ b/src/stream/src/from_proto/mod.rs
@@ -93,7 +93,7 @@ use self::top_n::*;
 use self::union::*;
 use self::watermark_filter::WatermarkFilterBuilder;
 use crate::error::StreamResult;
-use crate::executor::{BoxedExecutor, Executor, ExecutorInfo};
+use crate::executor::{Execute, Executor, ExecutorInfo};
 use crate::from_proto::subscription::SubscriptionExecutorBuilder;
 use crate::from_proto::values::ValuesExecutorBuilder;
 use crate::task::ExecutorParams;
@@ -101,12 +101,12 @@ use crate::task::ExecutorParams;
 trait ExecutorBuilder {
     type Node;
 
-    /// Create a [`BoxedExecutor`] from [`StreamNode`].
+    /// Create an [`Executor`] from [`StreamNode`].
     fn new_boxed_executor(
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> impl std::future::Future<Output = StreamResult<BoxedExecutor>> + Send;
+    ) -> impl std::future::Future<Output = StreamResult<Executor>> + Send;
 }
 
 macro_rules! build_executor {
@@ -127,7 +127,7 @@ pub async fn create_executor(
     params: ExecutorParams,
     node: &StreamNode,
     store: impl StateStore,
-) -> StreamResult<BoxedExecutor> {
+) -> StreamResult<Executor> {
     build_executor! {
         params,
         node,
diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs
index ecc54b20d8535..80aab576435ca 100644
--- a/src/stream/src/from_proto/mview.rs
+++ b/src/stream/src/from_proto/mview.rs
@@ -32,7 +32,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
 
         let order_key = node
@@ -51,7 +51,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder {
             ($SD:ident) => {
                 MaterializeExecutor::<_, $SD>::new(
                     input,
-                    params.info,
+                    params.info.schema.clone(),
                     store,
                     order_key,
                     params.actor_context,
@@ -66,13 +66,13 @@ impl ExecutorBuilder for MaterializeExecutorBuilder {
             };
         }
 
-        let executor = if versioned {
+        let exec = if versioned {
             new_executor!(ColumnAwareSerde)
         } else {
             new_executor!(BasicSerde)
         };
 
-        Ok(executor)
+        Ok((params.info, exec).into())
     }
 }
 
@@ -85,7 +85,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
 
         let keys = node
@@ -102,9 +102,9 @@ impl ExecutorBuilder for ArrangeExecutorBuilder {
         let vnodes = params.vnode_bitmap.map(Arc::new);
         let conflict_behavior =
             ConflictBehavior::from_protobuf(&table.handle_pk_conflict_behavior());
-        let executor = MaterializeExecutor::<_, BasicSerde>::new(
+        let exec = MaterializeExecutor::<_, BasicSerde>::new(
             input,
-            params.info,
+            params.info.schema.clone(),
             store,
             keys,
             params.actor_context,
@@ -116,6 +116,6 @@ impl ExecutorBuilder for ArrangeExecutorBuilder {
         )
         .await;
 
-        Ok(executor.boxed())
+        Ok((params.info, exec.boxed()).into())
     }
 }
diff --git a/src/stream/src/from_proto/no_op.rs b/src/stream/src/from_proto/no_op.rs
index 606b584d9109c..31494d771b903 100644
--- a/src/stream/src/from_proto/no_op.rs
+++ b/src/stream/src/from_proto/no_op.rs
@@ -17,7 +17,7 @@ use risingwave_storage::StateStore;
 
 use super::ExecutorBuilder;
 use crate::error::StreamResult;
-use crate::executor::{BoxedExecutor, Executor, NoOpExecutor};
+use crate::executor::{Executor, NoOpExecutor};
 use crate::task::ExecutorParams;
 
 pub struct NoOpExecutorBuilder;
@@ -29,8 +29,8 @@ impl ExecutorBuilder for NoOpExecutorBuilder {
         params: ExecutorParams,
         _node: &NoOpNode,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
-        Ok(NoOpExecutor::new(params.actor_context, params.info, input).boxed())
+        Ok((params.info, NoOpExecutor::new(params.actor_context, input)).into())
     }
 }
diff --git a/src/stream/src/from_proto/now.rs b/src/stream/src/from_proto/now.rs
index f622cdef1343e..06de0cacb7197 100644
--- a/src/stream/src/from_proto/now.rs
+++ b/src/stream/src/from_proto/now.rs
@@ -19,7 +19,7 @@ use tokio::sync::mpsc::unbounded_channel;
 use super::ExecutorBuilder;
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
-use crate::executor::{BoxedExecutor, NowExecutor};
+use crate::executor::{Executor, NowExecutor};
 use crate::task::ExecutorParams;
 
 pub struct NowExecutorBuilder;
@@ -31,7 +31,7 @@ impl ExecutorBuilder for NowExecutorBuilder {
         params: ExecutorParams,
         node: &NowNode,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let (sender, barrier_receiver) = unbounded_channel();
         params
             .local_barrier_manager
@@ -40,10 +40,11 @@ impl ExecutorBuilder for NowExecutorBuilder {
         let state_table =
             StateTable::from_table_catalog(node.get_state_table()?, store, None).await;
 
-        Ok(Box::new(NowExecutor::new(
-            params.info,
+        let exec = NowExecutor::new(
+            params.info.schema.data_types(),
             barrier_receiver,
             state_table,
-        )))
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/over_window.rs b/src/stream/src/from_proto/over_window.rs
index d2fc2893b6135..f7ca73c183a81 100644
--- a/src/stream/src/from_proto/over_window.rs
+++ b/src/stream/src/from_proto/over_window.rs
@@ -23,7 +23,7 @@ use risingwave_storage::StateStore;
 use super::ExecutorBuilder;
 use crate::common::table::state_table::StateTable;
 use crate::error::StreamResult;
-use crate::executor::{BoxedExecutor, Executor, OverWindowExecutor, OverWindowExecutorArgs};
+use crate::executor::{Executor, OverWindowExecutor, OverWindowExecutorArgs};
 use crate::task::ExecutorParams;
 
 pub struct OverWindowExecutorBuilder;
@@ -35,7 +35,7 @@ impl ExecutorBuilder for OverWindowExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let calls: Vec<_> = node
             .get_calls()
@@ -60,12 +60,12 @@ impl ExecutorBuilder for OverWindowExecutorBuilder {
         ));
         let state_table =
             StateTable::from_table_catalog(node.get_state_table()?, store, vnodes).await;
-        Ok(OverWindowExecutor::new(OverWindowExecutorArgs {
+        let exec = OverWindowExecutor::new(OverWindowExecutorArgs {
             actor_ctx: params.actor_context,
-            info: params.info,
 
             input,
 
+            schema: params.info.schema.clone(),
             calls,
             partition_key_indices,
             order_key_indices,
@@ -79,7 +79,7 @@ impl ExecutorBuilder for OverWindowExecutorBuilder {
             cache_policy: OverWindowCachePolicy::from_protobuf(
                 node.get_cache_policy().unwrap_or_default(),
             ),
-        })
-        .boxed())
+        });
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/project.rs b/src/stream/src/from_proto/project.rs
index 6fc9ada39aed8..d7f96c4dffcbf 100644
--- a/src/stream/src/from_proto/project.rs
+++ b/src/stream/src/from_proto/project.rs
@@ -30,7 +30,7 @@ impl ExecutorBuilder for ProjectExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let project_exprs: Vec<_> = node
             .get_select_list()
@@ -60,15 +60,14 @@ impl ExecutorBuilder for ProjectExecutorBuilder {
             )
         });
         let materialize_selectivity_threshold = if extremely_light { 0.0 } else { 0.5 };
-        Ok(ProjectExecutor::new(
+        let exec = ProjectExecutor::new(
             params.actor_context,
-            params.info,
             input,
             project_exprs,
             watermark_derivations,
             nondecreasing_expr_indices,
             materialize_selectivity_threshold,
-        )
-        .boxed())
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/project_set.rs b/src/stream/src/from_proto/project_set.rs
index b98e9e467ccf2..c2338394b33ef 100644
--- a/src/stream/src/from_proto/project_set.rs
+++ b/src/stream/src/from_proto/project_set.rs
@@ -29,7 +29,7 @@ impl ExecutorBuilder for ProjectSetExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let select_list: Vec<_> = node
             .get_select_list()
@@ -55,15 +55,14 @@ impl ExecutorBuilder for ProjectSetExecutorBuilder {
             .collect();
 
         let chunk_size = params.env.config().developer.chunk_size;
-        Ok(ProjectSetExecutor::new(
+        let exec = ProjectSetExecutor::new(
             params.actor_context,
-            params.info,
             input,
             select_list,
             chunk_size,
             watermark_derivations,
             nondecreasing_expr_indices,
-        )
-        .boxed())
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/row_id_gen.rs b/src/stream/src/from_proto/row_id_gen.rs
index 1333a99aebfa5..ab87566547ac6 100644
--- a/src/stream/src/from_proto/row_id_gen.rs
+++ b/src/stream/src/from_proto/row_id_gen.rs
@@ -18,7 +18,7 @@ use risingwave_storage::StateStore;
 use super::ExecutorBuilder;
 use crate::error::StreamResult;
 use crate::executor::row_id_gen::RowIdGenExecutor;
-use crate::executor::BoxedExecutor;
+use crate::executor::Executor;
 use crate::task::ExecutorParams;
 
 pub struct RowIdGenExecutorBuilder;
@@ -30,19 +30,18 @@ impl ExecutorBuilder for RowIdGenExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [upstream]: [_; 1] = params.input.try_into().unwrap();
         tracing::debug!("row id gen executor: {:?}", params.vnode_bitmap);
         let vnodes = params
             .vnode_bitmap
             .expect("vnodes not set for row id gen executor");
-        let executor = RowIdGenExecutor::new(
+        let exec = RowIdGenExecutor::new(
             params.actor_context,
-            params.info,
             upstream,
             node.row_id_index as _,
             vnodes,
         );
-        Ok(Box::new(executor))
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/simple_agg.rs b/src/stream/src/from_proto/simple_agg.rs
index 209a08daf6a22..16809edb8bcaf 100644
--- a/src/stream/src/from_proto/simple_agg.rs
+++ b/src/stream/src/from_proto/simple_agg.rs
@@ -34,7 +34,7 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let agg_calls: Vec<AggCall> = node
             .get_agg_calls()
@@ -55,12 +55,12 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder {
             build_distinct_dedup_table_from_proto(node.get_distinct_dedup_tables(), store, None)
                 .await;
 
-        Ok(SimpleAggExecutor::new(AggExecutorArgs {
+        let exec = SimpleAggExecutor::new(AggExecutorArgs {
             version: node.version(),
 
             input,
             actor_ctx: params.actor_context,
-            info: params.info,
+            info: params.info.clone(),
 
             extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size,
 
@@ -71,7 +71,8 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder {
             distinct_dedup_tables,
             watermark_epoch: params.watermark_epoch,
             extra: SimpleAggExecutorExtraArgs {},
-        })?
-        .boxed())
+        })?;
+
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs
index 87d63cd80f599..884925d4a35ab 100644
--- a/src/stream/src/from_proto/sink.rs
+++ b/src/stream/src/from_proto/sink.rs
@@ -103,7 +103,7 @@ impl ExecutorBuilder for SinkExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         state_store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input_executor]: [_; 1] = params.input.try_into().unwrap();
 
         let sink_desc = node.sink_desc.as_ref().unwrap();
@@ -189,23 +189,22 @@ impl ExecutorBuilder for SinkExecutorBuilder {
             connector, sink_id.sink_id, params.executor_id
         );
 
-        match node.log_store_type() {
+        let exec = match node.log_store_type() {
             // Default value is the normal in memory log store to be backward compatible with the
             // previously unset value
             SinkLogStoreType::InMemoryLogStore | SinkLogStoreType::Unspecified => {
                 let factory = BoundedInMemLogStoreFactory::new(1);
-                Ok(Box::new(
-                    SinkExecutor::new(
-                        params.actor_context,
-                        params.info,
-                        input_executor,
-                        sink_write_param,
-                        sink_param,
-                        columns,
-                        factory,
-                    )
-                    .await?,
-                ))
+                SinkExecutor::new(
+                    params.actor_context,
+                    params.info.clone(),
+                    input_executor,
+                    sink_write_param,
+                    sink_param,
+                    columns,
+                    factory,
+                )
+                .await?
+                .boxed()
             }
             SinkLogStoreType::KvLogStore => {
                 let metrics = KvLogStoreMetrics::new(
@@ -230,19 +229,20 @@ impl ExecutorBuilder for SinkExecutorBuilder {
                     pk_info,
                 );
 
-                Ok(Box::new(
-                    SinkExecutor::new(
-                        params.actor_context,
-                        params.info,
-                        input_executor,
-                        sink_write_param,
-                        sink_param,
-                        columns,
-                        factory,
-                    )
-                    .await?,
-                ))
+                SinkExecutor::new(
+                    params.actor_context,
+                    params.info.clone(),
+                    input_executor,
+                    sink_write_param,
+                    sink_param,
+                    columns,
+                    factory,
+                )
+                .await?
+                .boxed()
             }
-        }
+        };
+
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/sort.rs b/src/stream/src/from_proto/sort.rs
index dcb59f77f9549..e901d66e5d8ff 100644
--- a/src/stream/src/from_proto/sort.rs
+++ b/src/stream/src/from_proto/sort.rs
@@ -29,18 +29,19 @@ impl ExecutorBuilder for SortExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for sort"));
         let state_table =
             StateTable::from_table_catalog(node.get_state_table()?, store, Some(vnodes)).await;
-        Ok(Box::new(SortExecutor::new(SortExecutorArgs {
+        let exec = SortExecutor::new(SortExecutorArgs {
             actor_ctx: params.actor_context,
-            info: params.info,
+            schema: params.info.schema.clone(),
             input,
             buffer_table: state_table,
             chunk_size: params.env.config().developer.chunk_size,
             sort_column_index: node.sort_column_index as _,
-        })))
+        });
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs
index 2cedce5a8cd0a..355f7d874dec8 100644
--- a/src/stream/src/from_proto/source/fs_fetch.rs
+++ b/src/stream/src/from_proto/source/fs_fetch.rs
@@ -25,7 +25,7 @@ use risingwave_storage::StateStore;
 
 use crate::error::StreamResult;
 use crate::executor::{
-    BoxedExecutor, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler,
+    Execute, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler,
     StreamSourceCore,
 };
 use crate::from_proto::ExecutorBuilder;
@@ -40,7 +40,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [upstream]: [_; 1] = params.input.try_into().unwrap();
 
         let source = node.node_inner.as_ref().unwrap();
@@ -89,11 +89,10 @@ impl ExecutorBuilder for FsFetchExecutorBuilder {
             state_table_handler,
         );
 
-        let executor = match properties {
+        let exec = match properties {
             risingwave_connector::source::ConnectorProperties::Gcs(_) => {
                 FsFetchExecutor::<_, OpendalGcs>::new(
                     params.actor_context.clone(),
-                    params.info,
                     stream_source_core,
                     upstream,
                     source_ctrl_opts,
@@ -104,7 +103,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder {
             risingwave_connector::source::ConnectorProperties::OpendalS3(_) => {
                 FsFetchExecutor::<_, OpendalS3>::new(
                     params.actor_context.clone(),
-                    params.info,
                     stream_source_core,
                     upstream,
                     source_ctrl_opts,
@@ -115,7 +113,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder {
             risingwave_connector::source::ConnectorProperties::PosixFs(_) => {
                 FsFetchExecutor::<_, OpendalPosixFs>::new(
                     params.actor_context.clone(),
-                    params.info,
                     stream_source_core,
                     upstream,
                     source_ctrl_opts,
@@ -125,7 +122,11 @@ impl ExecutorBuilder for FsFetchExecutorBuilder {
             }
             _ => unreachable!(),
         };
+        let mut info = params.info.clone();
+        info.identity = format!("{} (flow controlled)", info.identity);
+
         let rate_limit = source.rate_limit.map(|x| x as _);
-        Ok(FlowControlExecutor::new(executor, params.actor_context, rate_limit).boxed())
+        let exec = FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit);
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs
index 8ce6b88b0196b..7870bab10151b 100644
--- a/src/stream/src/from_proto/source/trad_source.rs
+++ b/src/stream/src/from_proto/source/trad_source.rs
@@ -46,7 +46,7 @@ impl ExecutorBuilder for SourceExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let (sender, barrier_receiver) = unbounded_channel();
         params
             .local_barrier_manager
@@ -54,7 +54,7 @@ impl ExecutorBuilder for SourceExecutorBuilder {
         let system_params = params.env.system_params_manager_ref().get_params();
 
         if let Some(source) = &node.source_inner {
-            let executor = {
+            let exec = {
                 let source_id = TableId::new(source.source_id);
                 let source_name = source.source_name.clone();
                 let mut source_info = source.get_info()?.clone();
@@ -205,7 +205,6 @@ impl ExecutorBuilder for SourceExecutorBuilder {
                     #[expect(deprecated)]
                     crate::executor::FsSourceExecutor::new(
                         params.actor_context.clone(),
-                        params.info,
                         stream_source_core,
                         params.executor_stats,
                         barrier_receiver,
@@ -216,7 +215,6 @@ impl ExecutorBuilder for SourceExecutorBuilder {
                 } else if is_fs_v2_connector {
                     FsListExecutor::new(
                         params.actor_context.clone(),
-                        params.info.clone(),
                         Some(stream_source_core),
                         params.executor_stats.clone(),
                         barrier_receiver,
@@ -228,7 +226,6 @@ impl ExecutorBuilder for SourceExecutorBuilder {
                 } else {
                     SourceExecutor::new(
                         params.actor_context.clone(),
-                        params.info.clone(),
                         Some(stream_source_core),
                         params.executor_stats.clone(),
                         barrier_receiver,
@@ -239,14 +236,18 @@ impl ExecutorBuilder for SourceExecutorBuilder {
                     .boxed()
                 }
             };
+            let mut info = params.info.clone();
+            info.identity = format!("{} (flow controlled)", info.identity);
+
             let rate_limit = source.rate_limit.map(|x| x as _);
-            Ok(FlowControlExecutor::new(executor, params.actor_context, rate_limit).boxed())
+            let exec =
+                FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit);
+            Ok((params.info, exec).into())
         } else {
             // If there is no external stream source, then no data should be persisted. We pass a
             // `PanicStateStore` type here for indication.
-            Ok(SourceExecutor::<PanicStateStore>::new(
+            let exec = SourceExecutor::<PanicStateStore>::new(
                 params.actor_context,
-                params.info,
                 None,
                 params.executor_stats,
                 barrier_receiver,
@@ -254,8 +255,8 @@ impl ExecutorBuilder for SourceExecutorBuilder {
                 // we don't expect any data in, so no need to set chunk_sizes
                 SourceCtrlOpts::default(),
                 params.env.connector_params(),
-            )
-            .boxed())
+            );
+            Ok((params.info, exec).into())
         }
     }
 }
diff --git a/src/stream/src/from_proto/stateless_simple_agg.rs b/src/stream/src/from_proto/stateless_simple_agg.rs
index 87617b42f83e4..2ce5e612737e8 100644
--- a/src/stream/src/from_proto/stateless_simple_agg.rs
+++ b/src/stream/src/from_proto/stateless_simple_agg.rs
@@ -27,7 +27,7 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let agg_calls: Vec<AggCall> = node
             .get_agg_calls()
@@ -35,9 +35,12 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder {
             .map(AggCall::from_protobuf)
             .try_collect()?;
 
-        Ok(
-            StatelessSimpleAggExecutor::new(params.actor_context, params.info, input, agg_calls)?
-                .boxed(),
-        )
+        let exec = StatelessSimpleAggExecutor::new(
+            params.actor_context,
+            input,
+            params.info.schema.clone(),
+            agg_calls,
+        )?;
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/stream_cdc_scan.rs b/src/stream/src/from_proto/stream_cdc_scan.rs
index 66e08c2bb4695..2736fdd712cb7 100644
--- a/src/stream/src/from_proto/stream_cdc_scan.rs
+++ b/src/stream/src/from_proto/stream_cdc_scan.rs
@@ -23,7 +23,7 @@ use risingwave_pb::stream_plan::StreamCdcScanNode;
 
 use super::*;
 use crate::common::table::state_table::StateTable;
-use crate::executor::{CdcBackfillExecutor, ExternalStorageTable, FlowControlExecutor};
+use crate::executor::{CdcBackfillExecutor, Executor, ExternalStorageTable, FlowControlExecutor};
 
 pub struct StreamCdcScanExecutorBuilder;
 
@@ -34,7 +34,7 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         state_store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [upstream]: [_; 1] = params.input.try_into().unwrap();
 
         let output_indices = node
@@ -95,9 +95,8 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder {
             .map(|x| std::cmp::min(x as usize, chunk_size))
             .unwrap_or(chunk_size);
 
-        let executor = CdcBackfillExecutor::new(
+        let exec = CdcBackfillExecutor::new(
             params.actor_context.clone(),
-            params.info,
             external_table,
             upstream,
             output_indices,
@@ -106,14 +105,15 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder {
             state_table,
             backfill_chunk_size,
             disable_backfill,
-        )
-        .boxed();
+        );
+        let mut info = params.info.clone();
+        info.identity = format!("{} (flow controlled)", info.identity);
 
-        Ok(FlowControlExecutor::new(
-            executor,
+        let exec = FlowControlExecutor::new(
+            (info, exec).into(),
             params.actor_context,
             node.rate_limit.map(|x| x as _),
-        )
-        .boxed())
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs
index c3f15802afc8d..5f0ae484e04c8 100644
--- a/src/stream/src/from_proto/stream_scan.rs
+++ b/src/stream/src/from_proto/stream_scan.rs
@@ -37,7 +37,7 @@ impl ExecutorBuilder for StreamScanExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         state_store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [upstream, snapshot]: [_; 2] = params.input.try_into().unwrap();
         // For reporting the progress.
         let progress = params
@@ -50,13 +50,13 @@ impl ExecutorBuilder for StreamScanExecutorBuilder {
             .map(|&i| i as usize)
             .collect_vec();
 
-        let executor = match node.stream_scan_type() {
+        let exec = match node.stream_scan_type() {
             StreamScanType::Chain | StreamScanType::UpstreamOnly => {
                 let upstream_only = matches!(node.stream_scan_type(), StreamScanType::UpstreamOnly);
-                ChainExecutor::new(params.info, snapshot, upstream, progress, upstream_only).boxed()
+                ChainExecutor::new(snapshot, upstream, progress, upstream_only).boxed()
             }
             StreamScanType::Rearrange => {
-                RearrangedChainExecutor::new(params.info, snapshot, upstream, progress).boxed()
+                RearrangedChainExecutor::new(snapshot, upstream, progress).boxed()
             }
 
             StreamScanType::Backfill => {
@@ -83,7 +83,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder {
                     StorageTable::new_partial(state_store.clone(), column_ids, vnodes, table_desc);
 
                 BackfillExecutor::new(
-                    params.info,
                     upstream_table,
                     upstream,
                     state_table,
@@ -126,7 +125,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder {
                             )
                             .await;
                         ArrangementBackfillExecutor::<_, $SD>::new(
-                            params.info,
                             upstream_table,
                             upstream,
                             state_table,
@@ -147,11 +145,14 @@ impl ExecutorBuilder for StreamScanExecutorBuilder {
             }
             StreamScanType::Unspecified => unreachable!(),
         };
-        Ok(FlowControlExecutor::new(
-            executor,
+        let mut info = params.info.clone();
+        info.identity = format!("{} (flow controlled)", info.identity);
+
+        let exec = FlowControlExecutor::new(
+            (info, exec).into(),
             params.actor_context,
             node.rate_limit.map(|x| x as _),
-        )
-        .boxed())
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/subscription.rs b/src/stream/src/from_proto/subscription.rs
index 4d5e6eb6855b1..ff0c1f8c1d084 100644
--- a/src/stream/src/from_proto/subscription.rs
+++ b/src/stream/src/from_proto/subscription.rs
@@ -21,7 +21,7 @@ use crate::common::log_store_impl::kv_log_store::serde::LogStoreRowSerde;
 use crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V2_INFO;
 use crate::common::log_store_impl::subscription_log_store::SubscriptionLogStoreWriter;
 use crate::error::StreamResult;
-use crate::executor::{BoxedExecutor, SubscriptionExecutor};
+use crate::executor::{Executor, SubscriptionExecutor};
 
 pub struct SubscriptionExecutorBuilder;
 
@@ -32,7 +32,7 @@ impl ExecutorBuilder for SubscriptionExecutorBuilder {
         params: crate::task::ExecutorParams,
         node: &Self::Node,
         state_store: impl risingwave_storage::StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let table_id = TableId::new(node.log_store_table.as_ref().unwrap().id);
         let local_state_store = state_store
@@ -61,15 +61,13 @@ impl ExecutorBuilder for SubscriptionExecutorBuilder {
         let log_store_identity = format!("subscription-executor[{}]", params.executor_id);
         let log_store =
             SubscriptionLogStoreWriter::new(table_id, local_state_store, serde, log_store_identity);
-        Ok(Box::new(
-            SubscriptionExecutor::new(
-                params.actor_context,
-                params.info,
-                input,
-                log_store,
-                node.retention_seconds,
-            )
-            .await?,
-        ))
+        let exec = SubscriptionExecutor::new(
+            params.actor_context,
+            input,
+            log_store,
+            node.retention_seconds,
+        )
+        .await?;
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs
index 7c91b8c1d261d..15badec97e5cc 100644
--- a/src/stream/src/from_proto/temporal_join.rs
+++ b/src/stream/src/from_proto/temporal_join.rs
@@ -35,7 +35,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let table_desc: &StorageTableDesc = node.get_table_desc()?;
         let table = {
             let column_ids = table_desc
@@ -101,7 +101,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder {
 
         let dispatcher_args = TemporalJoinExecutorDispatcherArgs {
             ctx: params.actor_context,
-            info: params.info,
+            info: params.info.clone(),
             left: source_l,
             right: source_r,
             right_table: table,
@@ -119,15 +119,15 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder {
             join_key_data_types,
         };
 
-        dispatcher_args.dispatch()
+        Ok((params.info, dispatcher_args.dispatch()?).into())
     }
 }
 
 struct TemporalJoinExecutorDispatcherArgs<S: StateStore> {
     ctx: ActorContextRef,
     info: ExecutorInfo,
-    left: BoxedExecutor,
-    right: BoxedExecutor,
+    left: Executor,
+    right: Executor,
     right_table: StorageTable<S>,
     left_join_keys: Vec<usize>,
     right_join_keys: Vec<usize>,
@@ -144,7 +144,7 @@ struct TemporalJoinExecutorDispatcherArgs<S: StateStore> {
 }
 
 impl<S: StateStore> HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs<S> {
-    type Output = StreamResult<BoxedExecutor>;
+    type Output = StreamResult<Box<dyn Execute>>;
 
     fn dispatch_impl<K: HashKey>(self) -> Self::Output {
         /// This macro helps to fill the const generic type parameter.
diff --git a/src/stream/src/from_proto/top_n.rs b/src/stream/src/from_proto/top_n.rs
index 5c74f9f97e0eb..10071534e57df 100644
--- a/src/stream/src/from_proto/top_n.rs
+++ b/src/stream/src/from_proto/top_n.rs
@@ -30,7 +30,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONL
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
 
         let table = node.get_table()?;
@@ -52,7 +52,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONL
                 Ok($excutor::<_, $with_ties>::new(
                     input,
                     params.actor_context,
-                    params.info,
+                    params.info.schema.clone(),
                     storage_key,
                     (node.offset as usize, node.limit as usize),
                     order_by,
@@ -62,11 +62,12 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONL
             };
         }
 
-        match (APPEND_ONLY, node.with_ties) {
+        let exec: StreamResult<Box<dyn Execute>> = match (APPEND_ONLY, node.with_ties) {
             (true, true) => build!(AppendOnlyTopNExecutor, true),
             (true, false) => build!(AppendOnlyTopNExecutor, false),
             (false, true) => build!(TopNExecutor, true),
             (false, false) => build!(TopNExecutor, false),
-        }
+        };
+        Ok((params.info, exec?).into())
     }
 }
diff --git a/src/stream/src/from_proto/union.rs b/src/stream/src/from_proto/union.rs
index e11ea8f3f2d21..1360f92c1c83b 100644
--- a/src/stream/src/from_proto/union.rs
+++ b/src/stream/src/from_proto/union.rs
@@ -26,7 +26,7 @@ impl ExecutorBuilder for UnionExecutorBuilder {
         params: ExecutorParams,
         _node: &Self::Node,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
-        Ok(UnionExecutor::new(params.info, params.input).boxed())
+    ) -> StreamResult<Executor> {
+        Ok((params.info, UnionExecutor::new(params.input)).into())
     }
 }
diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs
index b3ed75e4f0903..28e9ca086414b 100644
--- a/src/stream/src/from_proto/values.rs
+++ b/src/stream/src/from_proto/values.rs
@@ -20,7 +20,7 @@ use tokio::sync::mpsc::unbounded_channel;
 
 use super::ExecutorBuilder;
 use crate::error::StreamResult;
-use crate::executor::{BoxedExecutor, ValuesExecutor};
+use crate::executor::{Executor, ValuesExecutor};
 use crate::task::ExecutorParams;
 
 /// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for
@@ -34,7 +34,7 @@ impl ExecutorBuilder for ValuesExecutorBuilder {
         params: ExecutorParams,
         node: &ValuesNode,
         _store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let (sender, barrier_receiver) = unbounded_channel();
         params
             .local_barrier_manager
@@ -55,12 +55,13 @@ impl ExecutorBuilder for ValuesExecutorBuilder {
                     .collect_vec()
             })
             .collect_vec();
-        Ok(Box::new(ValuesExecutor::new(
+        let exec = ValuesExecutor::new(
             params.actor_context,
-            params.info,
+            params.info.schema.clone(),
             progress,
             rows,
             barrier_receiver,
-        )))
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/from_proto/watermark_filter.rs b/src/stream/src/from_proto/watermark_filter.rs
index cc3382bcc65fc..f01695e991487 100644
--- a/src/stream/src/from_proto/watermark_filter.rs
+++ b/src/stream/src/from_proto/watermark_filter.rs
@@ -34,7 +34,7 @@ impl ExecutorBuilder for WatermarkFilterBuilder {
         params: ExecutorParams,
         node: &Self::Node,
         store: impl StateStore,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         let [input]: [_; 1] = params.input.try_into().unwrap();
         let watermark_descs = node.get_watermark_descs().clone();
         let [watermark_desc]: [_; 1] = watermark_descs.try_into().unwrap();
@@ -63,15 +63,15 @@ impl ExecutorBuilder for WatermarkFilterBuilder {
         let table =
             StateTable::from_table_catalog_inconsistent_op(&table, store, Some(vnodes)).await;
 
-        Ok(WatermarkFilterExecutor::new(
+        let exec = WatermarkFilterExecutor::new(
             params.actor_context,
-            params.info,
+            &params.info,
             input,
             watermark_expr,
             event_time_col_idx,
             table,
             global_watermark_table,
-        )
-        .boxed())
+        );
+        Ok((params.info, exec).into())
     }
 }
diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs
index 11eb9a44290cf..ad93575771ea2 100644
--- a/src/stream/src/task/stream_manager.rs
+++ b/src/stream/src/task/stream_manager.rs
@@ -44,7 +44,10 @@ use super::{unique_executor_id, unique_operator_id, BarrierCompleteResult};
 use crate::error::StreamResult;
 use crate::executor::monitor::StreamingMetrics;
 use crate::executor::subtask::SubtaskHandle;
-use crate::executor::*;
+use crate::executor::{
+    Actor, ActorContext, ActorContextRef, Barrier, DispatchExecutor, DispatcherImpl, Executor,
+    ExecutorInfo, WrapperExecutor,
+};
 use crate::from_proto::create_executor;
 use crate::task::barrier_manager::{LocalBarrierEvent, LocalBarrierWorker};
 use crate::task::{
@@ -102,7 +105,7 @@ pub struct ExecutorParams {
     pub op_info: String,
 
     /// The input executor.
-    pub input: Vec<BoxedExecutor>,
+    pub input: Vec<Executor>,
 
     /// FragmentId of the actor
     pub fragment_id: FragmentId,
@@ -351,7 +354,7 @@ impl StreamActorManager {
     /// Create dispatchers with downstream information registered before
     fn create_dispatcher(
         &self,
-        input: BoxedExecutor,
+        input: Executor,
         dispatchers: &[stream_plan::Dispatcher],
         actor_id: ActorId,
         fragment_id: FragmentId,
@@ -384,7 +387,7 @@ impl StreamActorManager {
         vnode_bitmap: Option<Bitmap>,
         has_stateful: bool,
         subtasks: &mut Vec<SubtaskHandle>,
-    ) -> StreamResult<BoxedExecutor> {
+    ) -> StreamResult<Executor> {
         // The "stateful" here means that the executor may issue read operations to the state store
         // massively and continuously. Used to decide whether to apply the optimization of subtasks.
         fn is_stateful_executor(stream_node: &StreamNode) -> bool {
@@ -435,21 +438,22 @@ impl StreamActorManager {
         let schema: Schema = node.fields.iter().map(Field::from).collect();
 
         let identity = format!("{} {:X}", node.get_node_body().unwrap(), executor_id);
+        let info = ExecutorInfo {
+            schema,
+            pk_indices,
+            identity,
+        };
+
         let eval_error_report = ActorEvalErrorReport {
             actor_context: actor_context.clone(),
-            identity: identity.clone().into(),
+            identity: info.identity.clone().into(),
         };
 
         // Build the executor with params.
         let executor_params = ExecutorParams {
             env: env.clone(),
 
-            info: ExecutorInfo {
-                schema: schema.clone(),
-                pk_indices: pk_indices.clone(),
-                identity: identity.clone(),
-            },
-
+            info: info.clone(),
             executor_id,
             operator_id,
             op_info,
@@ -465,26 +469,14 @@ impl StreamActorManager {
         };
 
         let executor = create_executor(executor_params, node, store).await?;
-        assert_eq!(
-            executor.pk_indices(),
-            &pk_indices,
-            "`pk_indices` of {} not consistent with what derived by optimizer",
-            executor.identity()
-        );
-        assert_eq!(
-            executor.schema(),
-            &schema,
-            "`schema` of {} not consistent with what derived by optimizer",
-            executor.identity()
-        );
 
         // Wrap the executor for debug purpose.
-        let executor = WrapperExecutor::new(
+        let wrapped = WrapperExecutor::new(
             executor,
             actor_context.clone(),
             env.config().developer.enable_executor_row_count,
-        )
-        .boxed();
+        );
+        let executor = (info, wrapped).into();
 
         // If there're multiple stateful executors in this actor, we will wrap it into a subtask.
         let executor = if has_stateful && is_stateful {
@@ -510,7 +502,7 @@ impl StreamActorManager {
         env: StreamEnvironment,
         actor_context: &ActorContextRef,
         vnode_bitmap: Option<Bitmap>,
-    ) -> StreamResult<(BoxedExecutor, Vec<SubtaskHandle>)> {
+    ) -> StreamResult<(Executor, Vec<SubtaskHandle>)> {
         let mut subtasks = vec![];
 
         let executor = dispatch_state_store!(env.state_store(), store, {
diff --git a/src/stream/tests/integration_tests/eowc_over_window.rs b/src/stream/tests/integration_tests/eowc_over_window.rs
index 6941d86f3a759..cc674e556ab5e 100644
--- a/src/stream/tests/integration_tests/eowc_over_window.rs
+++ b/src/stream/tests/integration_tests/eowc_over_window.rs
@@ -14,9 +14,7 @@
 
 use risingwave_expr::aggregate::{AggArgs, AggKind};
 use risingwave_expr::window_function::{Frame, FrameBound, WindowFuncCall, WindowFuncKind};
-use risingwave_stream::executor::{
-    EowcOverWindowExecutor, EowcOverWindowExecutorArgs, ExecutorInfo,
-};
+use risingwave_stream::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs};
 
 use crate::prelude::*;
 
@@ -54,7 +52,6 @@ async fn create_executor<S: StateStore>(
         });
         Schema { fields }
     };
-    let output_pk_indices = vec![2];
 
     let state_table = StateTable::new_without_distribution_inconsistent_op(
         store,
@@ -65,17 +62,14 @@ async fn create_executor<S: StateStore>(
     )
     .await;
 
-    let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone());
+    let (tx, source) = MockSource::channel();
+    let source = source.into_executor(input_schema, input_pk_indices.clone());
     let executor = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs {
         actor_ctx: ActorContext::for_test(123),
-        info: ExecutorInfo {
-            schema: output_schema,
-            pk_indices: output_pk_indices,
-            identity: "EowcOverWindowExecutor".to_string(),
-        },
 
-        input: source.boxed(),
+        input: source,
 
+        schema: output_schema,
         calls,
         partition_key_indices,
         order_key_index,
diff --git a/src/stream/tests/integration_tests/hash_agg.rs b/src/stream/tests/integration_tests/hash_agg.rs
index bf82a2986bf7d..aecf0ad4d5251 100644
--- a/src/stream/tests/integration_tests/hash_agg.rs
+++ b/src/stream/tests/integration_tests/hash_agg.rs
@@ -38,10 +38,11 @@ async fn test_hash_agg_count_sum() {
         AggCall::from_pretty("(sum:int8 $2:int8)"),
     ];
 
-    let (mut tx, source) = MockSource::channel(schema, PkIndices::new());
+    let (mut tx, source) = MockSource::channel();
+    let source = source.into_executor(schema, PkIndices::new());
     let hash_agg = new_boxed_hash_agg_executor(
         store,
-        Box::new(source),
+        source,
         false,
         agg_calls,
         0,
@@ -116,10 +117,11 @@ async fn test_hash_agg_min() {
         AggCall::from_pretty("(min:int8 $1:int8)"),
     ];
 
-    let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk
+    let (mut tx, source) = MockSource::channel();
+    let source = source.into_executor(schema, vec![2]);
     let hash_agg = new_boxed_hash_agg_executor(
         store,
-        Box::new(source),
+        source,
         false,
         agg_calls,
         0,
@@ -191,10 +193,11 @@ async fn test_hash_agg_min_append_only() {
         AggCall::from_pretty("(min:int8 $1:int8)"),
     ];
 
-    let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk
+    let (mut tx, source) = MockSource::channel();
+    let source = source.into_executor(schema, vec![2]);
     let hash_agg = new_boxed_hash_agg_executor(
         store,
-        Box::new(source),
+        source,
         true, // is append only
         agg_calls,
         0,
@@ -266,10 +269,11 @@ async fn test_hash_agg_emit_on_window_close() {
     let agg_calls = vec![AggCall::from_pretty("(count:int8)")];
 
     let create_executor = || async {
-        let (tx, source) = MockSource::channel(input_schema.clone(), PkIndices::new());
+        let (tx, source) = MockSource::channel();
+        let source = source.into_executor(input_schema.clone(), PkIndices::new());
         let hash_agg = new_boxed_hash_agg_executor(
             store.clone(),
-            Box::new(source),
+            source,
             false,
             agg_calls.clone(),
             0,
diff --git a/src/stream/tests/integration_tests/hop_window.rs b/src/stream/tests/integration_tests/hop_window.rs
index f0bd65c84b69e..97e2886dbecda 100644
--- a/src/stream/tests/integration_tests/hop_window.rs
+++ b/src/stream/tests/integration_tests/hop_window.rs
@@ -16,7 +16,7 @@ use risingwave_common::types::test_utils::IntervalTestExt;
 use risingwave_common::types::{Interval, Timestamp};
 use risingwave_expr::expr::test_utils::make_hop_window_expression;
 use risingwave_expr::expr::NonStrictExpression;
-use risingwave_stream::executor::{ExecutorInfo, HopWindowExecutor};
+use risingwave_stream::executor::HopWindowExecutor;
 
 use crate::prelude::*;
 
@@ -24,12 +24,15 @@ const TIME_COL_IDX: usize = 2;
 const CHUNK_SIZE: usize = 256;
 
 fn create_executor(output_indices: Vec<usize>) -> (MessageSender, BoxedMessageStream) {
-    let field1 = Field::unnamed(DataType::Int64);
-    let field2 = Field::unnamed(DataType::Int64);
-    let field3 = Field::with_name(DataType::Timestamp, "created_at");
-    let schema = Schema::new(vec![field1, field2, field3]);
-    let pk_indices = vec![0];
-    let (tx, source) = MockSource::channel(schema.clone(), pk_indices.clone());
+    let (tx, source) = MockSource::channel();
+    let source = source.into_executor(
+        Schema::new(vec![
+            Field::unnamed(DataType::Int64),
+            Field::unnamed(DataType::Int64),
+            Field::with_name(DataType::Timestamp, "created_at"),
+        ]),
+        vec![0],
+    );
 
     let window_slide = Interval::from_minutes(15);
     let window_size = Interval::from_minutes(30);
@@ -47,12 +50,7 @@ fn create_executor(output_indices: Vec<usize>) -> (MessageSender, BoxedMessageSt
         tx,
         HopWindowExecutor::new(
             ActorContext::for_test(123),
-            ExecutorInfo {
-                schema,
-                pk_indices,
-                identity: "HopWindowExecutor".to_string(),
-            },
-            Box::new(source),
+            source,
             TIME_COL_IDX,
             window_slide,
             window_size,
diff --git a/src/stream/tests/integration_tests/over_window.rs b/src/stream/tests/integration_tests/over_window.rs
index 78203d79a87ac..0be8e1848e9cd 100644
--- a/src/stream/tests/integration_tests/over_window.rs
+++ b/src/stream/tests/integration_tests/over_window.rs
@@ -18,7 +18,7 @@ use risingwave_expr::window_function::{
     Frame, FrameBound, FrameExclusion, WindowFuncCall, WindowFuncKind,
 };
 use risingwave_stream::executor::monitor::StreamingMetrics;
-use risingwave_stream::executor::{ExecutorInfo, OverWindowExecutor, OverWindowExecutorArgs};
+use risingwave_stream::executor::{OverWindowExecutor, OverWindowExecutorArgs};
 
 use crate::prelude::*;
 
@@ -63,7 +63,6 @@ async fn create_executor<S: StateStore>(
         });
         Schema { fields }
     };
-    let output_pk_indices = vec![2];
 
     let state_table = StateTable::new_without_distribution(
         store,
@@ -74,17 +73,14 @@ async fn create_executor<S: StateStore>(
     )
     .await;
 
-    let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone());
+    let (tx, source) = MockSource::channel();
+    let source = source.into_executor(input_schema, input_pk_indices.clone());
     let executor = OverWindowExecutor::new(OverWindowExecutorArgs {
         actor_ctx: ActorContext::for_test(123),
-        info: ExecutorInfo {
-            schema: output_schema,
-            pk_indices: output_pk_indices,
-            identity: "OverWindowExecutor".to_string(),
-        },
 
-        input: source.boxed(),
+        input: source,
 
+        schema: output_schema,
         calls,
         partition_key_indices,
         order_key_indices,
diff --git a/src/stream/tests/integration_tests/project_set.rs b/src/stream/tests/integration_tests/project_set.rs
index 79fcbfc0d48db..5cff03284f4b6 100644
--- a/src/stream/tests/integration_tests/project_set.rs
+++ b/src/stream/tests/integration_tests/project_set.rs
@@ -14,7 +14,7 @@
 
 use multimap::MultiMap;
 use risingwave_expr::table_function::repeat;
-use risingwave_stream::executor::{ExecutorInfo, ProjectSetExecutor};
+use risingwave_stream::executor::ProjectSetExecutor;
 
 use crate::prelude::*;
 
@@ -27,30 +27,17 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) {
             Field::unnamed(DataType::Int64),
         ],
     };
-    let (tx, source) = MockSource::channel(schema, PkIndices::new());
+    let (tx, source) = MockSource::channel();
+    let source = source.into_executor(schema, PkIndices::new());
 
     let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)").into_inner();
     let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)").into_inner();
     let tf1 = repeat(build_from_pretty("1:int4").into_inner(), 1);
     let tf2 = repeat(build_from_pretty("2:int4").into_inner(), 2);
 
-    let info = ExecutorInfo {
-        schema: Schema {
-            fields: vec![
-                Field::unnamed(DataType::Int64),
-                Field::unnamed(DataType::Int64),
-                Field::unnamed(DataType::Int32),
-                Field::unnamed(DataType::Int32),
-            ],
-        },
-        pk_indices: vec![],
-        identity: "ProjectSetExecutor".to_string(),
-    };
-
-    let project_set = Box::new(ProjectSetExecutor::new(
+    let project_set = ProjectSetExecutor::new(
         ActorContext::for_test(123),
-        info,
-        Box::new(source),
+        source,
         vec![
             test_expr.into(),
             test_expr_watermark.into(),
@@ -60,8 +47,8 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) {
         CHUNK_SIZE,
         MultiMap::from_iter(std::iter::once((0, 1))),
         vec![],
-    ));
-    (tx, project_set.execute())
+    );
+    (tx, project_set.boxed().execute())
 }
 
 #[tokio::test]