mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-09-23 01:03:32 +08:00
feat: Phase 2 - Add DDL operations and real MQ broker integration
Implements comprehensive DDL support for MQ topic management: New Components: - Real MQ broker connectivity via BrokerClient - CREATE TABLE → ConfigureTopic gRPC calls - DROP TABLE → DeleteTopic operations - DESCRIBE table → Schema introspection - SQL type mapping (SQL ↔ MQ schema types) Enhanced Features: - Live topic discovery from MQ broker - Fallback to cached/sample data when broker unavailable - MySQL-compatible DESCRIBE output - Schema validation and error handling - CREATE TABLE with column definitions Key Infrastructure: - broker_client.go: gRPC communication with MQ broker - sql_types.go: Bidirectional SQL/MQ type conversion - describe.go: Table schema introspection - Enhanced engine.go: Full DDL routing and execution Supported SQL Operations: ✅ SHOW DATABASES, SHOW TABLES (live + fallback) ✅ CREATE TABLE table_name (col1 INT, col2 VARCHAR(50), ...) ✅ DROP TABLE table_name ✅ DESCRIBE table_name / SHOW COLUMNS FROM table_name Known Limitations: - SQL parser issues with reserved keywords (e.g., 'timestamp') - Requires running MQ broker for full functionality - ALTER TABLE not yet implemented - DeleteTopic method needs broker-side implementation Architecture Decisions: - Broker discovery via filer lock mechanism (same as shell commands) - Graceful fallback when broker unavailable - ConfigureTopic for CREATE TABLE with 6 default partitions - Schema versioning ready for ALTER TABLE support Testing: - Unit tests updated with filer address parameter - Integration tests for DDL operations - Error handling for connection failures Next Phase: SELECT query execution with Parquet scanning
This commit is contained in:
97
weed/query/engine/describe.go
Normal file
97
weed/query/engine/describe.go
Normal file
@@ -0,0 +1,97 @@
|
||||
package engine
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/query/sqltypes"
|
||||
"github.com/xwb1989/sqlparser"
|
||||
)
|
||||
|
||||
// executeDescribeStatement handles DESCRIBE table commands
|
||||
// Assumption: DESCRIBE shows table schema in MySQL-compatible format
|
||||
func (e *SQLEngine) executeDescribeStatement(ctx context.Context, tableName string, database string) (*QueryResult, error) {
|
||||
if database == "" {
|
||||
database = e.catalog.GetCurrentDatabase()
|
||||
if database == "" {
|
||||
database = "default"
|
||||
}
|
||||
}
|
||||
|
||||
// Get topic schema from broker
|
||||
recordType, err := e.catalog.brokerClient.GetTopicSchema(ctx, database, tableName)
|
||||
if err != nil {
|
||||
return &QueryResult{Error: err}, err
|
||||
}
|
||||
|
||||
// Format schema as DESCRIBE output
|
||||
result := &QueryResult{
|
||||
Columns: []string{"Field", "Type", "Null", "Key", "Default", "Extra"},
|
||||
Rows: make([][]sqltypes.Value, len(recordType.Fields)),
|
||||
}
|
||||
|
||||
for i, field := range recordType.Fields {
|
||||
sqlType := e.convertMQTypeToSQL(field.Type)
|
||||
|
||||
result.Rows[i] = []sqltypes.Value{
|
||||
sqltypes.NewVarChar(field.Name), // Field
|
||||
sqltypes.NewVarChar(sqlType), // Type
|
||||
sqltypes.NewVarChar("YES"), // Null (assume nullable)
|
||||
sqltypes.NewVarChar(""), // Key (no keys for now)
|
||||
sqltypes.NewVarChar("NULL"), // Default
|
||||
sqltypes.NewVarChar(""), // Extra
|
||||
}
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// Enhanced executeShowStatementWithDescribe handles SHOW statements including DESCRIBE
|
||||
func (e *SQLEngine) executeShowStatementWithDescribe(ctx context.Context, stmt *sqlparser.Show) (*QueryResult, error) {
|
||||
switch strings.ToUpper(stmt.Type) {
|
||||
case "DATABASES":
|
||||
return e.showDatabases(ctx)
|
||||
case "TABLES":
|
||||
// TODO: Parse FROM clause properly for database specification
|
||||
return e.showTables(ctx, "")
|
||||
case "COLUMNS":
|
||||
// SHOW COLUMNS FROM table is equivalent to DESCRIBE
|
||||
if stmt.OnTable.Name.String() != "" {
|
||||
tableName := stmt.OnTable.Name.String()
|
||||
database := ""
|
||||
if stmt.OnTable.Qualifier.String() != "" {
|
||||
database = stmt.OnTable.Qualifier.String()
|
||||
}
|
||||
return e.executeDescribeStatement(ctx, tableName, database)
|
||||
}
|
||||
fallthrough
|
||||
default:
|
||||
err := fmt.Errorf("unsupported SHOW statement: %s", stmt.Type)
|
||||
return &QueryResult{Error: err}, err
|
||||
}
|
||||
}
|
||||
|
||||
// Add support for DESCRIBE as a separate statement type
|
||||
// This would be called from ExecuteSQL if we detect a DESCRIBE statement
|
||||
func (e *SQLEngine) handleDescribeCommand(ctx context.Context, sql string) (*QueryResult, error) {
|
||||
// Simple parsing for "DESCRIBE table_name" format
|
||||
// TODO: Use proper SQL parser for more robust parsing
|
||||
parts := strings.Fields(strings.TrimSpace(sql))
|
||||
if len(parts) < 2 {
|
||||
err := fmt.Errorf("DESCRIBE requires a table name")
|
||||
return &QueryResult{Error: err}, err
|
||||
}
|
||||
|
||||
tableName := parts[1]
|
||||
database := ""
|
||||
|
||||
// Handle database.table format
|
||||
if strings.Contains(tableName, ".") {
|
||||
parts := strings.SplitN(tableName, ".", 2)
|
||||
database = parts[0]
|
||||
tableName = parts[1]
|
||||
}
|
||||
|
||||
return e.executeDescribeStatement(ctx, tableName, database)
|
||||
}
|
Reference in New Issue
Block a user