@@ -31,7 +31,7 @@ use async_trait::async_trait;
3131use datafusion_common:: config:: { ConfigEntry , ConfigOptions } ;
3232use datafusion_common:: error:: Result ;
3333use datafusion_common:: DataFusionError ;
34- use datafusion_execution:: TaskContext ;
34+ use datafusion_execution:: { disk_manager , TaskContext } ;
3535use datafusion_expr:: { AggregateUDF , ScalarUDF , Signature , TypeSignature , WindowUDF } ;
3636use datafusion_expr:: { TableType , Volatility } ;
3737use datafusion_physical_plan:: stream:: RecordBatchStreamAdapter ;
@@ -49,6 +49,7 @@ pub(crate) const DF_SETTINGS: &str = "df_settings";
4949pub ( crate ) const SCHEMATA : & str = "schemata" ;
5050pub ( crate ) const ROUTINES : & str = "routines" ;
5151pub ( crate ) const PARAMETERS : & str = "parameters" ;
52+ pub ( crate ) const TEMPORARY_FILES : & str = "temporary_files" ;
5253
5354/// All information schema tables
5455pub const INFORMATION_SCHEMA_TABLES : & [ & str ] = & [
@@ -59,6 +60,7 @@ pub const INFORMATION_SCHEMA_TABLES: &[&str] = &[
5960 SCHEMATA ,
6061 ROUTINES ,
6162 PARAMETERS ,
63+ TEMPORARY_FILES ,
6264] ;
6365
6466/// Implements the `information_schema` virtual schema and tables
@@ -397,6 +399,19 @@ impl InformationSchemaConfig {
397399 TypeSignature :: Variadic ( _) | TypeSignature :: VariadicAny
398400 )
399401 }
402+
403+ fn make_temporary_files (
404+ & self ,
405+ disk_manager : & disk_manager:: DiskManager ,
406+ builder : & mut InformationSchemaTemporaryFilesBuilder ,
407+ ) -> Result < ( ) , DataFusionError > {
408+ let temporary_files = disk_manager. get_temporary_files ( ) ;
409+
410+ for ( path, size) in temporary_files {
411+ builder. add_file ( & path, size) ;
412+ }
413+ Ok ( ( ) )
414+ }
400415}
401416
402417/// get the arguments and return types of a UDF
@@ -496,6 +511,7 @@ impl SchemaProvider for InformationSchemaProvider {
496511 SCHEMATA => Arc :: new ( InformationSchemata :: new ( config) ) ,
497512 ROUTINES => Arc :: new ( InformationSchemaRoutines :: new ( config) ) ,
498513 PARAMETERS => Arc :: new ( InformationSchemaParameters :: new ( config) ) ,
514+ TEMPORARY_FILES => Arc :: new ( InformationSchemaTemporaryFiles :: new ( config) ) ,
499515 _ => return Ok ( None ) ,
500516 } ;
501517
@@ -1348,3 +1364,70 @@ impl PartitionStream for InformationSchemaParameters {
13481364 ) )
13491365 }
13501366}
1367+
1368+ #[ derive( Debug ) ]
1369+ struct InformationSchemaTemporaryFiles {
1370+ schema : SchemaRef ,
1371+ config : InformationSchemaConfig ,
1372+ }
1373+
1374+ impl InformationSchemaTemporaryFiles {
1375+ fn new ( config : InformationSchemaConfig ) -> Self {
1376+ let schema = Arc :: new ( Schema :: new ( vec ! [
1377+ Field :: new( "path" , DataType :: Utf8 , false ) ,
1378+ Field :: new( "size" , DataType :: UInt64 , false ) ,
1379+ ] ) ) ;
1380+
1381+ Self { schema, config }
1382+ }
1383+
1384+ fn builder ( & self ) -> InformationSchemaTemporaryFilesBuilder {
1385+ InformationSchemaTemporaryFilesBuilder {
1386+ paths : StringBuilder :: new ( ) ,
1387+ sizes : UInt64Builder :: new ( ) ,
1388+ schema : Arc :: clone ( & self . schema ) ,
1389+ }
1390+ }
1391+ }
1392+
1393+ impl PartitionStream for InformationSchemaTemporaryFiles {
1394+ fn schema ( & self ) -> & SchemaRef {
1395+ & self . schema
1396+ }
1397+
1398+ fn execute ( & self , ctx : Arc < TaskContext > ) -> SendableRecordBatchStream {
1399+ let mut builder = self . builder ( ) ;
1400+ let config = self . config . clone ( ) ;
1401+ Box :: pin ( RecordBatchStreamAdapter :: new (
1402+ Arc :: clone ( & self . schema ) ,
1403+ futures:: stream:: once ( async move {
1404+ config. make_temporary_files ( & ctx. runtime_env ( ) . disk_manager , & mut builder) ?;
1405+ Ok ( builder. finish ( ) )
1406+ } ) ,
1407+ ) )
1408+ }
1409+ }
1410+
1411+ struct InformationSchemaTemporaryFilesBuilder {
1412+ schema : SchemaRef ,
1413+ paths : StringBuilder ,
1414+ sizes : UInt64Builder ,
1415+ }
1416+
1417+ impl InformationSchemaTemporaryFilesBuilder {
1418+ fn add_file ( & mut self , path : & str , size : u64 ) {
1419+ self . paths . append_value ( path) ;
1420+ self . sizes . append_value ( size) ;
1421+ }
1422+
1423+ fn finish ( & mut self ) -> RecordBatch {
1424+ RecordBatch :: try_new (
1425+ Arc :: clone ( & self . schema ) ,
1426+ vec ! [
1427+ Arc :: new( self . paths. finish( ) ) ,
1428+ Arc :: new( self . sizes. finish( ) ) ,
1429+ ] ,
1430+ )
1431+ . unwrap ( )
1432+ }
1433+ }
0 commit comments