-
Notifications
You must be signed in to change notification settings - Fork 523
/
create_sink.rs
204 lines (172 loc) · 6.92 KB
/
create_sink.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
// Copyright 2023 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::catalog::{DatabaseId, SchemaId, UserId};
use risingwave_common::error::Result;
use risingwave_connector::sink::catalog::SinkCatalog;
use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism;
use risingwave_sqlparser::ast::{
CreateSink, CreateSinkStatement, ObjectName, Query, Select, SelectItem, SetExpr, TableFactor,
TableWithJoins,
};
use super::create_mv::get_column_names;
use super::RwPgResponse;
use crate::binder::Binder;
use crate::handler::HandlerArgs;
use crate::optimizer::plan_node::Explain;
use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef};
use crate::scheduler::streaming_manager::CreatingStreamingJobInfo;
use crate::session::SessionImpl;
use crate::stream_fragmenter::build_graph;
use crate::Planner;
pub fn gen_sink_query_from_name(from_name: ObjectName) -> Result<Query> {
let table_factor = TableFactor::Table {
name: from_name,
alias: None,
};
let from = vec![TableWithJoins {
relation: table_factor,
joins: vec![],
}];
let select = Select {
from,
projection: vec![SelectItem::Wildcard],
..Default::default()
};
let body = SetExpr::Select(Box::new(select));
Ok(Query {
with: None,
body,
order_by: vec![],
limit: None,
offset: None,
fetch: None,
})
}
pub fn gen_sink_plan(
session: &SessionImpl,
context: OptimizerContextRef,
stmt: CreateSinkStatement,
) -> Result<(PlanRef, SinkCatalog)> {
let db_name = session.database();
let (sink_schema_name, sink_table_name) =
Binder::resolve_schema_qualified_name(db_name, stmt.sink_name.clone())?;
let query = match stmt.sink_from {
CreateSink::From(from_name) => Box::new(gen_sink_query_from_name(from_name)?),
CreateSink::AsQuery(query) => query,
};
let (sink_database_id, sink_schema_id) =
session.get_database_and_schema_id_for_create(sink_schema_name)?;
let definition = context.normalized_sql().to_owned();
let bound = {
let mut binder = Binder::new(session);
binder.bind_query(*query)?
};
// If colume names not specified, use the name in materialized view.
let col_names = get_column_names(&bound, session, stmt.columns)?;
let properties = context.with_options().clone();
let mut plan_root = Planner::new(context).plan_query(bound)?;
if let Some(col_names) = col_names {
plan_root.set_out_names(col_names)?;
};
let sink_plan = plan_root.gen_sink_plan(sink_table_name, definition, properties)?;
let sink_desc = sink_plan.sink_desc().clone();
let sink_catalog = sink_desc.into_catalog(
SchemaId::new(sink_schema_id),
DatabaseId::new(sink_database_id),
UserId::new(session.user_id()),
vec![],
);
let sink_plan: PlanRef = sink_plan.into();
let ctx = sink_plan.ctx();
let explain_trace = ctx.is_explain_trace();
if explain_trace {
ctx.trace("Create Sink:");
ctx.trace(sink_plan.explain_to_string().unwrap());
}
Ok((sink_plan, sink_catalog))
}
pub async fn handle_create_sink(
handle_args: HandlerArgs,
stmt: CreateSinkStatement,
) -> Result<RwPgResponse> {
let session = handle_args.session.clone();
session.check_relation_name_duplicated(stmt.sink_name.clone())?;
let (sink, graph) = {
let context = OptimizerContext::from_handler_args(handle_args);
let (plan, sink) = gen_sink_plan(&session, context.into(), stmt)?;
let mut graph = build_graph(plan);
graph.parallelism = session
.config()
.get_streaming_parallelism()
.map(|parallelism| Parallelism { parallelism });
(sink, graph)
};
let _job_guard =
session
.env()
.creating_streaming_job_tracker()
.guard(CreatingStreamingJobInfo::new(
session.session_id(),
sink.database_id.database_id,
sink.schema_id.schema_id,
sink.name.clone(),
));
let catalog_writer = session.env().catalog_writer();
catalog_writer.create_sink(sink.to_proto(), graph).await?;
Ok(PgResponse::empty_result(StatementType::CREATE_SINK))
}
#[cfg(test)]
pub mod tests {
use risingwave_common::catalog::{DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME};
use crate::catalog::root_catalog::SchemaPath;
use crate::test_utils::{create_proto_file, LocalFrontend, PROTO_FILE_DATA};
#[tokio::test]
async fn test_create_sink_handler() {
let proto_file = create_proto_file(PROTO_FILE_DATA);
let sql = format!(
r#"CREATE SOURCE t1
WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001')
ROW FORMAT PROTOBUF MESSAGE '.test.TestRecord' ROW SCHEMA LOCATION 'file://{}';"#,
proto_file.path().to_str().unwrap()
);
let frontend = LocalFrontend::new(Default::default()).await;
frontend.run_sql(sql).await.unwrap();
let sql = "create materialized view mv1 as select t1.country from t1;";
frontend.run_sql(sql).await.unwrap();
let sql = r#"CREATE SINK snk1 FROM mv1
WITH (connector = 'mysql', mysql.endpoint = '127.0.0.1:3306', mysql.table =
'<table_name>', mysql.database = '<database_name>', mysql.user = '<user_name>',
mysql.password = '<password>', format = 'append_only', force_append_only = 'true');"#.to_string();
frontend.run_sql(sql).await.unwrap();
let session = frontend.session_ref();
let catalog_reader = session.env().catalog_reader().read_guard();
let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
// Check source exists.
let (source, _) = catalog_reader
.get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t1")
.unwrap();
assert_eq!(source.name, "t1");
// Check table exists.
let (table, schema_name) = catalog_reader
.get_table_by_name(DEFAULT_DATABASE_NAME, schema_path, "mv1")
.unwrap();
assert_eq!(table.name(), "mv1");
// Check sink exists.
let (sink, _) = catalog_reader
.get_sink_by_name(DEFAULT_DATABASE_NAME, SchemaPath::Name(schema_name), "snk1")
.unwrap();
assert_eq!(sink.name, "snk1");
}
}