|
| 1 | +use std::sync::Arc; |
| 2 | + |
| 3 | +use datafusion::arrow::array::{ArrayRef, BooleanArray, RecordBatch, StringArray}; |
| 4 | +use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; |
| 5 | +use datafusion::catalog::CatalogProviderList; |
| 6 | +use datafusion::error::Result; |
| 7 | +use datafusion::execution::{SendableRecordBatchStream, TaskContext}; |
| 8 | +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; |
| 9 | +use datafusion::physical_plan::streaming::PartitionStream; |
| 10 | + |
| 11 | +#[derive(Debug, Clone)] |
| 12 | +pub(crate) struct PgTablesTable { |
| 13 | + schema: SchemaRef, |
| 14 | + catalog_list: Arc<dyn CatalogProviderList>, |
| 15 | +} |
| 16 | + |
| 17 | +impl PgTablesTable { |
| 18 | + pub(crate) fn new(catalog_list: Arc<dyn CatalogProviderList>) -> PgTablesTable { |
| 19 | + // Define the schema for pg_class |
| 20 | + // This matches key columns from PostgreSQL's pg_class |
| 21 | + let schema = Arc::new(Schema::new(vec![ |
| 22 | + Field::new("schemaname", DataType::Utf8, false), |
| 23 | + Field::new("tablename", DataType::Utf8, false), |
| 24 | + Field::new("tableowner", DataType::Utf8, false), |
| 25 | + Field::new("tablespace", DataType::Utf8, true), |
| 26 | + Field::new("hasindex", DataType::Boolean, false), |
| 27 | + Field::new("hasrules", DataType::Boolean, false), |
| 28 | + Field::new("hastriggers", DataType::Boolean, false), |
| 29 | + Field::new("rowsecurity", DataType::Boolean, false), |
| 30 | + ])); |
| 31 | + |
| 32 | + Self { |
| 33 | + schema, |
| 34 | + catalog_list, |
| 35 | + } |
| 36 | + } |
| 37 | + |
| 38 | + /// Generate record batches based on the current state of the catalog |
| 39 | + async fn get_data(this: PgTablesTable) -> Result<RecordBatch> { |
| 40 | + // Vectors to store column data |
| 41 | + let mut schema_names = Vec::new(); |
| 42 | + let mut table_names = Vec::new(); |
| 43 | + let mut table_owners = Vec::new(); |
| 44 | + let mut table_spaces: Vec<Option<String>> = Vec::new(); |
| 45 | + let mut has_index = Vec::new(); |
| 46 | + let mut has_rules = Vec::new(); |
| 47 | + let mut has_triggers = Vec::new(); |
| 48 | + let mut row_security = Vec::new(); |
| 49 | + |
| 50 | + // Iterate through all catalogs and schemas |
| 51 | + for catalog_name in this.catalog_list.catalog_names() { |
| 52 | + if let Some(catalog) = this.catalog_list.catalog(&catalog_name) { |
| 53 | + for schema_name in catalog.schema_names() { |
| 54 | + if let Some(schema) = catalog.schema(&schema_name) { |
| 55 | + // Now process all tables in this schema |
| 56 | + for table_name in schema.table_names() { |
| 57 | + schema_names.push(schema_name.to_string()); |
| 58 | + table_names.push(table_name.to_string()); |
| 59 | + table_owners.push("postgres".to_string()); |
| 60 | + table_spaces.push(None); |
| 61 | + has_index.push(false); |
| 62 | + has_rules.push(false); |
| 63 | + has_triggers.push(false); |
| 64 | + row_security.push(false); |
| 65 | + } |
| 66 | + } |
| 67 | + } |
| 68 | + } |
| 69 | + } |
| 70 | + |
| 71 | + // Create Arrow arrays from the collected data |
| 72 | + let arrays: Vec<ArrayRef> = vec![ |
| 73 | + Arc::new(StringArray::from(schema_names)), |
| 74 | + Arc::new(StringArray::from(table_names)), |
| 75 | + Arc::new(StringArray::from(table_owners)), |
| 76 | + Arc::new(StringArray::from(table_spaces)), |
| 77 | + Arc::new(BooleanArray::from(has_index)), |
| 78 | + Arc::new(BooleanArray::from(has_rules)), |
| 79 | + Arc::new(BooleanArray::from(has_triggers)), |
| 80 | + Arc::new(BooleanArray::from(row_security)), |
| 81 | + ]; |
| 82 | + |
| 83 | + // Create a record batch |
| 84 | + let batch = RecordBatch::try_new(this.schema.clone(), arrays)?; |
| 85 | + |
| 86 | + Ok(batch) |
| 87 | + } |
| 88 | +} |
| 89 | + |
| 90 | +impl PartitionStream for PgTablesTable { |
| 91 | + fn schema(&self) -> &SchemaRef { |
| 92 | + &self.schema |
| 93 | + } |
| 94 | + |
| 95 | + fn execute(&self, _ctx: Arc<TaskContext>) -> SendableRecordBatchStream { |
| 96 | + let this = self.clone(); |
| 97 | + Box::pin(RecordBatchStreamAdapter::new( |
| 98 | + this.schema.clone(), |
| 99 | + futures::stream::once(async move { PgTablesTable::get_data(this).await }), |
| 100 | + )) |
| 101 | + } |
| 102 | +} |
0 commit comments