1616// under the License.
1717
1818pub use ballista_core:: extension:: { SessionConfigExt , SessionStateExt } ;
19+ use ballista_core:: remote_catalog:: remote_scalar_udf:: RemoteScalarUDF ;
20+ use ballista_core:: remote_catalog:: remote_table_provider:: RemoteTableProvider ;
1921use ballista_core:: serde:: protobuf:: scheduler_grpc_client:: SchedulerGrpcClient ;
22+ use ballista_core:: serde:: protobuf:: { GetCatalogParams , GetRemoteFunctionsParams } ;
23+ use datafusion:: catalog:: {
24+ CatalogProvider , MemoryCatalogProvider , MemorySchemaProvider , SchemaProvider ,
25+ } ;
26+ use datafusion:: execution:: FunctionRegistry ;
27+ use datafusion:: logical_expr:: ScalarUDF ;
2028use datafusion:: {
2129 error:: DataFusionError , execution:: SessionState , prelude:: SessionContext ,
2230} ;
31+ use std:: sync:: Arc ;
2332use url:: Url ;
2433
2534const DEFAULT_SCHEDULER_PORT : u16 = 50050 ;
@@ -86,6 +95,19 @@ pub trait SessionContextExt {
8695 url : & str ,
8796 state : SessionState ,
8897 ) -> datafusion:: error:: Result < SessionContext > ;
98+
99+ /// Populates the local catalog with metadata from the remote scheduler.
100+ /// This allows catalog queries like SHOW TABLES to work on the client.
101+ async fn populate_catalog_from_scheduler (
102+ & self ,
103+ scheduler_url : & str ,
104+ ) -> datafusion:: error:: Result < ( ) > ;
105+
106+ /// Populates local context with functions from the scheduler.
107+ async fn populate_functions_from_scheduler (
108+ & self ,
109+ scheduler_url : & str ,
110+ ) -> datafusion:: error:: Result < ( ) > ;
89111}
90112
91113#[ async_trait:: async_trait]
@@ -100,14 +122,21 @@ impl SessionContextExt for SessionContext {
100122 scheduler_url. clone( )
101123 ) ;
102124
103- let session_state = state. upgrade_for_ballista ( scheduler_url) ?;
125+ let session_state = state. upgrade_for_ballista ( scheduler_url. clone ( ) ) ?;
104126
105127 log:: info!(
106128 "Server side SessionContext created with session id: {}" ,
107129 session_state. session_id( )
108130 ) ;
109131
110- Ok ( SessionContext :: new_with_state ( session_state) )
132+ let ctx = SessionContext :: new_with_state ( session_state) ;
133+
134+ // Populate local catalog from scheduler
135+ ctx. populate_catalog_from_scheduler ( & scheduler_url) . await ?;
136+ ctx. populate_functions_from_scheduler ( & scheduler_url)
137+ . await ?;
138+
139+ Ok ( ctx)
111140 }
112141
113142 async fn remote ( url : & str ) -> datafusion:: error:: Result < SessionContext > {
@@ -117,13 +146,20 @@ impl SessionContextExt for SessionContext {
117146 scheduler_url. clone( )
118147 ) ;
119148
120- let session_state = SessionState :: new_ballista_state ( scheduler_url) ?;
149+ let session_state = SessionState :: new_ballista_state ( scheduler_url. clone ( ) ) ?;
121150 log:: info!(
122151 "Server side SessionContext created with session id: {}" ,
123152 session_state. session_id( )
124153 ) ;
125154
126- Ok ( SessionContext :: new_with_state ( session_state) )
155+ let ctx = SessionContext :: new_with_state ( session_state) ;
156+
157+ // Populate local catalog from scheduler
158+ ctx. populate_catalog_from_scheduler ( & scheduler_url) . await ?;
159+ ctx. populate_functions_from_scheduler ( & scheduler_url)
160+ . await ?;
161+
162+ Ok ( ctx)
127163 }
128164
129165 #[ cfg( feature = "standalone" ) ]
@@ -157,6 +193,118 @@ impl SessionContextExt for SessionContext {
157193
158194 Ok ( SessionContext :: new_with_state ( session_state) )
159195 }
196+
197+ async fn populate_catalog_from_scheduler (
198+ & self ,
199+ scheduler_url : & str ,
200+ ) -> datafusion:: error:: Result < ( ) > {
201+ let mut client = SchedulerGrpcClient :: connect ( scheduler_url. to_string ( ) )
202+ . await
203+ . map_err ( |e| {
204+ DataFusionError :: External (
205+ format ! ( "Failed to connect to scheduler: {}" , e) . into ( ) ,
206+ )
207+ } ) ?;
208+
209+ let request = tonic:: Request :: new ( GetCatalogParams {
210+ session_id : self . state ( ) . session_id ( ) . to_string ( ) ,
211+ } ) ;
212+
213+ let response = client. get_catalog ( request) . await . map_err ( |e| {
214+ DataFusionError :: External ( format ! ( "Failed to fetch catalog: {}" , e) . into ( ) )
215+ } ) ?;
216+
217+ let catalog_result = response. into_inner ( ) ;
218+
219+ log:: info!(
220+ "Received {} catalogs from scheduler" ,
221+ catalog_result. catalogs. len( )
222+ ) ;
223+
224+ for catalog_info in catalog_result. catalogs {
225+ let catalog_name = catalog_info. catalog_name ;
226+
227+ let catalog: Arc < dyn CatalogProvider > =
228+ if let Some ( _existing_catalog) = self . catalog ( & catalog_name) {
229+ continue ;
230+ } else {
231+ let new_catalog: Arc < dyn CatalogProvider > =
232+ Arc :: new ( MemoryCatalogProvider :: new ( ) ) ;
233+ self . register_catalog ( & catalog_name, Arc :: clone ( & new_catalog) ) ;
234+ new_catalog
235+ } ;
236+
237+ for schema_info in catalog_info. schemas {
238+ let schema_name = schema_info. schema_name ;
239+ let schema = if let Some ( _existing_schema) = catalog. schema ( & schema_name)
240+ {
241+ continue ;
242+ } else {
243+ let new_schema: Arc < dyn SchemaProvider > =
244+ Arc :: new ( MemorySchemaProvider :: new ( ) ) ;
245+ catalog. register_schema ( & schema_name, Arc :: clone ( & new_schema) ) ?;
246+ new_schema
247+ } ;
248+
249+ // Bind `RemoteTableProvider` tables
250+ for table_info in schema_info. tables {
251+ if let Some ( proto_schema) = table_info. schema {
252+ let arrow_schema: datafusion:: arrow:: datatypes:: Schema =
253+ ( & proto_schema)
254+ . try_into ( )
255+ . map_err ( |e| DataFusionError :: External ( Box :: new ( e) ) ) ?;
256+
257+ let stub_table = RemoteTableProvider :: new (
258+ & catalog_name,
259+ & schema_name,
260+ & table_info. table_name ,
261+ Arc :: new ( arrow_schema) ,
262+ ) ;
263+
264+ schema. register_table (
265+ table_info. table_name . clone ( ) ,
266+ Arc :: new ( stub_table) ,
267+ ) ?;
268+ }
269+ }
270+ }
271+ }
272+
273+ Ok ( ( ) )
274+ }
275+
276+ async fn populate_functions_from_scheduler (
277+ & self ,
278+ scheduler_url : & str ,
279+ ) -> datafusion:: common:: Result < ( ) > {
280+ let mut client = SchedulerGrpcClient :: connect ( scheduler_url. to_string ( ) )
281+ . await
282+ . map_err ( |e| {
283+ DataFusionError :: External (
284+ format ! ( "Failed to connect to scheduler: {}" , e) . into ( ) ,
285+ )
286+ } ) ?;
287+
288+ let request = tonic:: Request :: new ( GetRemoteFunctionsParams {
289+ session_id : self . state ( ) . session_id ( ) . to_string ( ) ,
290+ } ) ;
291+
292+ let response = client. get_remote_functions ( request) . await . map_err ( |e| {
293+ DataFusionError :: External ( format ! ( "Failed to fetch catalog: {}" , e) . into ( ) )
294+ } ) ?;
295+
296+ let remote_functions = response. into_inner ( ) ;
297+
298+ for udf in remote_functions. udfs {
299+ if self . state ( ) . udf ( & udf. name ) . is_ok ( ) {
300+ continue ;
301+ }
302+
303+ self . register_udf ( ScalarUDF :: new_from_impl ( RemoteScalarUDF :: new ( udf) ?) )
304+ }
305+
306+ Ok ( ( ) )
307+ }
160308}
161309
162310struct Extension { }
0 commit comments