@ -11,6 +11,7 @@ import (
"strings"
"strings"
"time"
"time"
pg_query "github.com/pganalyze/pg_query_go/v6"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
@ -18,10 +19,327 @@ import (
"github.com/seaweedfs/seaweedfs/weed/query/sqltypes"
"github.com/seaweedfs/seaweedfs/weed/query/sqltypes"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/util"
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
"github.com/xwb1989/sqlparser"
"google.golang.org/protobuf/proto"
"google.golang.org/protobuf/proto"
)
)
// PostgreSQL parser compatibility types
type Statement interface {
isStatement ( )
}
type ShowStatement struct {
Type string // "databases", "tables", "columns"
Table string // for SHOW COLUMNS FROM table
Schema string // for database context
OnTable NameRef // for compatibility with existing code that checks OnTable
}
func ( s * ShowStatement ) isStatement ( ) { }
type DDLStatement struct {
Action string // "create", "alter", "drop"
NewName NameRef
TableSpec * TableSpec
}
type NameRef struct {
Name StringGetter
Qualifier StringGetter
}
type StringGetter interface {
String ( ) string
}
type stringValue string
func ( s stringValue ) String ( ) string { return string ( s ) }
type TableSpec struct {
Columns [ ] ColumnDef
}
type ColumnDef struct {
Name StringGetter
Type TypeRef
}
type TypeRef struct {
Type string
}
func ( d * DDLStatement ) isStatement ( ) { }
type SelectStatement struct {
SelectExprs [ ] SelectExpr
From [ ] TableExpr
Where * WhereClause
Limit * LimitClause
}
type WhereClause struct {
Expr ExprNode
}
type LimitClause struct {
Rowcount ExprNode
}
func ( s * SelectStatement ) isStatement ( ) { }
type SelectExpr interface {
isSelectExpr ( )
}
type StarExpr struct { }
func ( s * StarExpr ) isSelectExpr ( ) { }
type AliasedExpr struct {
Expr ExprNode
As AliasRef
}
type AliasRef interface {
IsEmpty ( ) bool
String ( ) string
}
type aliasValue string
func ( a aliasValue ) IsEmpty ( ) bool { return string ( a ) == "" }
func ( a aliasValue ) String ( ) string { return string ( a ) }
func ( a * AliasedExpr ) isSelectExpr ( ) { }
type TableExpr interface {
isTableExpr ( )
}
type AliasedTableExpr struct {
Expr interface { }
}
func ( a * AliasedTableExpr ) isTableExpr ( ) { }
type TableName struct {
Name StringGetter
Qualifier StringGetter
}
type ExprNode interface {
isExprNode ( )
}
type FuncExpr struct {
Name StringGetter
Exprs [ ] SelectExpr
}
func ( f * FuncExpr ) isExprNode ( ) { }
type ColName struct {
Name StringGetter
}
func ( c * ColName ) isExprNode ( ) { }
type ComparisonExpr struct {
Left ExprNode
Right ExprNode
Operator string
}
func ( c * ComparisonExpr ) isExprNode ( ) { }
type AndExpr struct {
Left ExprNode
Right ExprNode
}
func ( a * AndExpr ) isExprNode ( ) { }
type OrExpr struct {
Left ExprNode
Right ExprNode
}
func ( o * OrExpr ) isExprNode ( ) { }
type ParenExpr struct {
Expr ExprNode
}
func ( p * ParenExpr ) isExprNode ( ) { }
type SQLVal struct {
Type int
Val [ ] byte
}
func ( s * SQLVal ) isExprNode ( ) { }
type ValTuple [ ] ExprNode
func ( v ValTuple ) isExprNode ( ) { }
// SQLVal types
const (
IntVal = iota
StrVal
)
// Operator constants
const (
CreateStr = "create"
AlterStr = "alter"
DropStr = "drop"
EqualStr = "="
LessThanStr = "<"
GreaterThanStr = ">"
LessEqualStr = "<="
GreaterEqualStr = ">="
NotEqualStr = "!="
)
// ParseSQL uses PostgreSQL parser to parse SQL statements
func ParseSQL ( sql string ) ( Statement , error ) {
sql = strings . TrimSpace ( sql )
// Parse with pg_query_go
result , err := pg_query . Parse ( sql )
if err != nil {
return nil , fmt . Errorf ( "PostgreSQL parse error: %v" , err )
}
if len ( result . Stmts ) == 0 {
return nil , fmt . Errorf ( "no statements parsed" )
}
// Convert first statement
stmt := result . Stmts [ 0 ]
// Handle SELECT statements
if selectStmt := stmt . Stmt . GetSelectStmt ( ) ; selectStmt != nil {
return convertSelectStatement ( selectStmt ) , nil
}
// Handle DDL statements by parsing SQL text patterns
sqlUpper := strings . ToUpper ( sql )
if strings . HasPrefix ( sqlUpper , "CREATE TABLE" ) {
return parseCreateTableFromSQL ( sql )
}
if strings . HasPrefix ( sqlUpper , "DROP TABLE" ) {
return parseDropTableFromSQL ( sql )
}
if strings . HasPrefix ( sqlUpper , "ALTER TABLE" ) {
return parseAlterTableFromSQL ( sql )
}
// Handle SHOW statements
if strings . HasPrefix ( sqlUpper , "SHOW DATABASES" ) || strings . HasPrefix ( sqlUpper , "SHOW SCHEMAS" ) {
return & ShowStatement { Type : "databases" } , nil
}
if strings . HasPrefix ( sqlUpper , "SHOW TABLES" ) {
return & ShowStatement { Type : "tables" } , nil
}
return nil , fmt . Errorf ( "unsupported statement type" )
}
// Conversion helpers
func convertSelectStatement ( stmt * pg_query . SelectStmt ) * SelectStatement {
s := & SelectStatement {
SelectExprs : [ ] SelectExpr { } ,
From : [ ] TableExpr { } ,
}
// Convert SELECT expressions
for _ , target := range stmt . GetTargetList ( ) {
if resTarget := target . GetResTarget ( ) ; resTarget != nil {
if resTarget . GetVal ( ) . GetColumnRef ( ) != nil {
// This is likely SELECT *
s . SelectExprs = append ( s . SelectExprs , & StarExpr { } )
} else {
expr := & AliasedExpr { }
if resTarget . GetName ( ) != "" {
expr . As = aliasValue ( resTarget . GetName ( ) )
} else {
expr . As = aliasValue ( "" )
}
s . SelectExprs = append ( s . SelectExprs , expr )
}
}
}
// Convert FROM clause
for _ , fromExpr := range stmt . GetFromClause ( ) {
if rangeVar := fromExpr . GetRangeVar ( ) ; rangeVar != nil {
tableName := TableName {
Name : stringValue ( rangeVar . GetRelname ( ) ) ,
Qualifier : stringValue ( rangeVar . GetSchemaname ( ) ) ,
}
s . From = append ( s . From , & AliasedTableExpr { Expr : tableName } )
}
}
return s
}
func parseCreateTableFromSQL ( sql string ) ( * DDLStatement , error ) {
parts := strings . Fields ( sql )
if len ( parts ) < 3 {
return nil , fmt . Errorf ( "invalid CREATE TABLE syntax" )
}
tableName := parts [ 2 ]
// Remove schema prefix if present
if strings . Contains ( tableName , "." ) {
parts := strings . Split ( tableName , "." )
tableName = parts [ len ( parts ) - 1 ]
}
ddl := & DDLStatement { Action : CreateStr }
ddl . NewName . Name = stringValue ( tableName )
return ddl , nil
}
func parseDropTableFromSQL ( sql string ) ( * DDLStatement , error ) {
parts := strings . Fields ( sql )
if len ( parts ) < 3 {
return nil , fmt . Errorf ( "invalid DROP TABLE syntax" )
}
tableName := parts [ 2 ]
if strings . Contains ( tableName , "." ) {
parts := strings . Split ( tableName , "." )
tableName = parts [ len ( parts ) - 1 ]
}
ddl := & DDLStatement { Action : DropStr }
ddl . NewName . Name = stringValue ( tableName )
return ddl , nil
}
func parseAlterTableFromSQL ( sql string ) ( * DDLStatement , error ) {
parts := strings . Fields ( sql )
if len ( parts ) < 3 {
return nil , fmt . Errorf ( "invalid ALTER TABLE syntax" )
}
tableName := parts [ 2 ]
if strings . Contains ( tableName , "." ) {
parts := strings . Split ( tableName , "." )
tableName = parts [ len ( parts ) - 1 ]
}
ddl := & DDLStatement { Action : AlterStr }
ddl . NewName . Name = stringValue ( tableName )
return ddl , nil
}
// debugModeKey is used to store debug mode flag in context
// debugModeKey is used to store debug mode flag in context
type debugModeKey struct { }
type debugModeKey struct { }
@ -87,10 +405,10 @@ func (e *SQLEngine) GetCatalog() *SchemaCatalog {
// ExecuteSQL parses and executes a SQL statement
// ExecuteSQL parses and executes a SQL statement
// Assumptions:
// Assumptions:
// 1. All SQL statements are MySQL-compatible via xwb1989/sqlparser
// 1. All SQL statements are PostgreSQL-compatible via pg_query_go
// 2. DDL operations (CREATE/ALTER/DROP) modify underlying MQ topics
// 2. DDL operations (CREATE/ALTER/DROP) modify underlying MQ topics
// 3. DML operations (SELECT) query Parquet files directly
// 3. DML operations (SELECT) query Parquet files directly
// 4. Error handling follows My SQL conventions
// 4. Error handling follows Postgre SQL conventions
func ( e * SQLEngine ) ExecuteSQL ( ctx context . Context , sql string ) ( * QueryResult , error ) {
func ( e * SQLEngine ) ExecuteSQL ( ctx context . Context , sql string ) ( * QueryResult , error ) {
startTime := time . Now ( )
startTime := time . Now ( )
@ -108,8 +426,8 @@ func (e *SQLEngine) ExecuteSQL(ctx context.Context, sql string) (*QueryResult, e
return e . handleDescribeCommand ( ctx , sqlTrimmed )
return e . handleDescribeCommand ( ctx , sqlTrimmed )
}
}
// Parse the SQL statement
stmt , err := sqlparser . Parse ( sql )
// Parse the SQL statement using PostgreSQL parser
stmt , err := ParseSQL ( sql )
if err != nil {
if err != nil {
return & QueryResult {
return & QueryResult {
Error : fmt . Errorf ( "SQL parse error: %v" , err ) ,
Error : fmt . Errorf ( "SQL parse error: %v" , err ) ,
@ -118,11 +436,11 @@ func (e *SQLEngine) ExecuteSQL(ctx context.Context, sql string) (*QueryResult, e
// Route to appropriate handler based on statement type
// Route to appropriate handler based on statement type
switch stmt := stmt . ( type ) {
switch stmt := stmt . ( type ) {
case * sqlparser . Show :
case * ShowStatement :
return e . executeShowStatementWithDescribe ( ctx , stmt )
return e . executeShowStatementWithDescribe ( ctx , stmt )
case * sqlparser . DDL :
case * DDLStatement :
return e . executeDDLStatement ( ctx , stmt )
return e . executeDDLStatement ( ctx , stmt )
case * sqlparser . Selec t:
case * SelectStatemen t:
return e . executeSelectStatement ( ctx , stmt )
return e . executeSelectStatement ( ctx , stmt )
default :
default :
err := fmt . Errorf ( "unsupported SQL statement type: %T" , stmt )
err := fmt . Errorf ( "unsupported SQL statement type: %T" , stmt )
@ -135,8 +453,8 @@ func (e *SQLEngine) executeExplain(ctx context.Context, actualSQL string, startT
// Enable debug mode for EXPLAIN queries
// Enable debug mode for EXPLAIN queries
ctx = withDebugMode ( ctx )
ctx = withDebugMode ( ctx )
// Parse the actual SQL statement
stmt , err := sqlparser . Parse ( actualSQL )
// Parse the actual SQL statement using PostgreSQL parser
stmt , err := ParseSQL ( actualSQL )
if err != nil {
if err != nil {
return & QueryResult {
return & QueryResult {
Error : fmt . Errorf ( "SQL parse error in EXPLAIN query: %v" , err ) ,
Error : fmt . Errorf ( "SQL parse error in EXPLAIN query: %v" , err ) ,
@ -155,12 +473,12 @@ func (e *SQLEngine) executeExplain(ctx context.Context, actualSQL string, startT
// Route to appropriate handler based on statement type (with plan tracking)
// Route to appropriate handler based on statement type (with plan tracking)
switch stmt := stmt . ( type ) {
switch stmt := stmt . ( type ) {
case * sqlparser . Selec t:
case * SelectStatemen t:
result , err = e . executeSelectStatementWithPlan ( ctx , stmt , plan )
result , err = e . executeSelectStatementWithPlan ( ctx , stmt , plan )
if err != nil {
if err != nil {
plan . Details [ "error" ] = err . Error ( )
plan . Details [ "error" ] = err . Error ( )
}
}
case * sqlparser . Show :
case * ShowStatement :
plan . QueryType = "SHOW"
plan . QueryType = "SHOW"
plan . ExecutionStrategy = "metadata_only"
plan . ExecutionStrategy = "metadata_only"
result , err = e . executeShowStatementWithDescribe ( ctx , stmt )
result , err = e . executeShowStatementWithDescribe ( ctx , stmt )
@ -413,14 +731,14 @@ func (e *SQLEngine) formatOptimization(opt string) string {
// executeDDLStatement handles CREATE operations only
// executeDDLStatement handles CREATE operations only
// Note: ALTER TABLE and DROP TABLE are not supported to protect topic data
// Note: ALTER TABLE and DROP TABLE are not supported to protect topic data
func ( e * SQLEngine ) executeDDLStatement ( ctx context . Context , stmt * sqlparser . DDL ) ( * QueryResult , error ) {
func ( e * SQLEngine ) executeDDLStatement ( ctx context . Context , stmt * DDLStatement ) ( * QueryResult , error ) {
switch stmt . Action {
switch stmt . Action {
case sqlparser . CreateStr :
case CreateStr :
return e . createTable ( ctx , stmt )
return e . createTable ( ctx , stmt )
case sqlparser . AlterStr :
case AlterStr :
err := fmt . Errorf ( "ALTER TABLE is not supported" )
err := fmt . Errorf ( "ALTER TABLE is not supported" )
return & QueryResult { Error : err } , err
return & QueryResult { Error : err } , err
case sqlparser . DropStr :
case DropStr :
err := fmt . Errorf ( "DROP TABLE is not supported" )
err := fmt . Errorf ( "DROP TABLE is not supported" )
return & QueryResult { Error : err } , err
return & QueryResult { Error : err } , err
default :
default :
@ -430,7 +748,7 @@ func (e *SQLEngine) executeDDLStatement(ctx context.Context, stmt *sqlparser.DDL
}
}
// executeSelectStatementWithPlan handles SELECT queries with execution plan tracking
// executeSelectStatementWithPlan handles SELECT queries with execution plan tracking
func ( e * SQLEngine ) executeSelectStatementWithPlan ( ctx context . Context , stmt * sqlparser . Selec t, plan * QueryExecutionPlan ) ( * QueryResult , error ) {
func ( e * SQLEngine ) executeSelectStatementWithPlan ( ctx context . Context , stmt * SelectStatemen t, plan * QueryExecutionPlan ) ( * QueryResult , error ) {
// Parse aggregations to populate plan
// Parse aggregations to populate plan
var aggregations [ ] AggregationSpec
var aggregations [ ] AggregationSpec
hasAggregations := false
hasAggregations := false
@ -438,11 +756,11 @@ func (e *SQLEngine) executeSelectStatementWithPlan(ctx context.Context, stmt *sq
for _ , selectExpr := range stmt . SelectExprs {
for _ , selectExpr := range stmt . SelectExprs {
switch expr := selectExpr . ( type ) {
switch expr := selectExpr . ( type ) {
case * sqlparser . StarExpr :
case * StarExpr :
selectAll = true
selectAll = true
case * sqlparser . AliasedExpr :
case * AliasedExpr :
switch col := expr . Expr . ( type ) {
switch col := expr . Expr . ( type ) {
case * sqlparser . FuncExpr :
case * FuncExpr :
// This is an aggregation function
// This is an aggregation function
aggSpec , err := e . parseAggregationFunction ( col , expr )
aggSpec , err := e . parseAggregationFunction ( col , expr )
if err != nil {
if err != nil {
@ -465,8 +783,8 @@ func (e *SQLEngine) executeSelectStatementWithPlan(ctx context.Context, stmt *sq
// Extract table information for aggregation execution
// Extract table information for aggregation execution
var database , tableName string
var database , tableName string
if len ( stmt . From ) == 1 {
if len ( stmt . From ) == 1 {
if table , ok := stmt . From [ 0 ] . ( * sqlparser . AliasedTableExpr ) ; ok {
if tableExpr , ok := table . Expr . ( sqlparser . TableName ) ; ok {
if table , ok := stmt . From [ 0 ] . ( * AliasedTableExpr ) ; ok {
if tableExpr , ok := table . Expr . ( TableName ) ; ok {
tableName = tableExpr . Name . String ( )
tableName = tableExpr . Name . String ( )
if tableExpr . Qualifier . String ( ) != "" {
if tableExpr . Qualifier . String ( ) != "" {
database = tableExpr . Qualifier . String ( )
database = tableExpr . Qualifier . String ( )
@ -508,8 +826,8 @@ func (e *SQLEngine) executeSelectStatementWithPlan(ctx context.Context, stmt *sq
// Extract table name for use in execution strategy determination
// Extract table name for use in execution strategy determination
var tableName string
var tableName string
if len ( stmt . From ) == 1 {
if len ( stmt . From ) == 1 {
if table , ok := stmt . From [ 0 ] . ( * sqlparser . AliasedTableExpr ) ; ok {
if tableExpr , ok := table . Expr . ( sqlparser . TableName ) ; ok {
if table , ok := stmt . From [ 0 ] . ( * AliasedTableExpr ) ; ok {
if tableExpr , ok := table . Expr . ( TableName ) ; ok {
tableName = tableExpr . Name . String ( )
tableName = tableExpr . Name . String ( )
}
}
}
}
@ -662,7 +980,7 @@ func (e *SQLEngine) executeSelectStatementWithPlan(ctx context.Context, stmt *sq
if stmt . Limit != nil {
if stmt . Limit != nil {
plan . OptimizationsUsed = append ( plan . OptimizationsUsed , "limit_pushdown" )
plan . OptimizationsUsed = append ( plan . OptimizationsUsed , "limit_pushdown" )
if stmt . Limit . Rowcount != nil {
if stmt . Limit . Rowcount != nil {
if limitExpr , ok := stmt . Limit . Rowcount . ( * sqlparser . SQLVal ) ; ok && limitExpr . Type == sqlparser . IntVal {
if limitExpr , ok := stmt . Limit . Rowcount . ( * SQLVal ) ; ok && limitExpr . Type == IntVal {
plan . Details [ "limit" ] = string ( limitExpr . Val )
plan . Details [ "limit" ] = string ( limitExpr . Val )
}
}
}
}
@ -677,7 +995,7 @@ func (e *SQLEngine) executeSelectStatementWithPlan(ctx context.Context, stmt *sq
// 1. Queries run against Parquet files in MQ topics
// 1. Queries run against Parquet files in MQ topics
// 2. Predicate pushdown is used for efficiency
// 2. Predicate pushdown is used for efficiency
// 3. Cross-topic joins are supported via partition-aware execution
// 3. Cross-topic joins are supported via partition-aware execution
func ( e * SQLEngine ) executeSelectStatement ( ctx context . Context , stmt * sqlparser . Selec t) ( * QueryResult , error ) {
func ( e * SQLEngine ) executeSelectStatement ( ctx context . Context , stmt * SelectStatemen t) ( * QueryResult , error ) {
// Parse FROM clause to get table (topic) information
// Parse FROM clause to get table (topic) information
if len ( stmt . From ) != 1 {
if len ( stmt . From ) != 1 {
err := fmt . Errorf ( "SELECT supports single table queries only" )
err := fmt . Errorf ( "SELECT supports single table queries only" )
@ -687,9 +1005,9 @@ func (e *SQLEngine) executeSelectStatement(ctx context.Context, stmt *sqlparser.
// Extract table reference
// Extract table reference
var database , tableName string
var database , tableName string
switch table := stmt . From [ 0 ] . ( type ) {
switch table := stmt . From [ 0 ] . ( type ) {
case * sqlparser . AliasedTableExpr :
case * AliasedTableExpr :
switch tableExpr := table . Expr . ( type ) {
switch tableExpr := table . Expr . ( type ) {
case sqlparser . TableName :
case TableName :
tableName = tableExpr . Name . String ( )
tableName = tableExpr . Name . String ( )
if tableExpr . Qualifier . String ( ) != "" {
if tableExpr . Qualifier . String ( ) != "" {
database = tableExpr . Qualifier . String ( )
database = tableExpr . Qualifier . String ( )
@ -755,13 +1073,13 @@ func (e *SQLEngine) executeSelectStatement(ctx context.Context, stmt *sqlparser.
for _ , selectExpr := range stmt . SelectExprs {
for _ , selectExpr := range stmt . SelectExprs {
switch expr := selectExpr . ( type ) {
switch expr := selectExpr . ( type ) {
case * sqlparser . StarExpr :
case * StarExpr :
selectAll = true
selectAll = true
case * sqlparser . AliasedExpr :
case * AliasedExpr :
switch col := expr . Expr . ( type ) {
switch col := expr . Expr . ( type ) {
case * sqlparser . ColName :
case * ColName :
columns = append ( columns , col . Name . String ( ) )
columns = append ( columns , col . Name . String ( ) )
case * sqlparser . FuncExpr :
case * FuncExpr :
// Handle aggregation functions
// Handle aggregation functions
aggSpec , err := e . parseAggregationFunction ( col , expr )
aggSpec , err := e . parseAggregationFunction ( col , expr )
if err != nil {
if err != nil {
@ -797,8 +1115,8 @@ func (e *SQLEngine) executeSelectStatement(ctx context.Context, stmt *sqlparser.
limit := 0
limit := 0
if stmt . Limit != nil && stmt . Limit . Rowcount != nil {
if stmt . Limit != nil && stmt . Limit . Rowcount != nil {
switch limitExpr := stmt . Limit . Rowcount . ( type ) {
switch limitExpr := stmt . Limit . Rowcount . ( type ) {
case * sqlparser . SQLVal :
if limitExpr . Type == sqlparser . IntVal {
case * SQLVal :
if limitExpr . Type == IntVal {
var parseErr error
var parseErr error
limit64 , parseErr := strconv . ParseInt ( string ( limitExpr . Val ) , 10 , 64 )
limit64 , parseErr := strconv . ParseInt ( string ( limitExpr . Val ) , 10 , 64 )
if parseErr != nil {
if parseErr != nil {
@ -846,7 +1164,7 @@ func (e *SQLEngine) executeSelectStatement(ctx context.Context, stmt *sqlparser.
// executeSelectStatementWithBrokerStats handles SELECT queries with broker buffer statistics capture
// executeSelectStatementWithBrokerStats handles SELECT queries with broker buffer statistics capture
// This is used by EXPLAIN queries to capture complete data source information including broker memory
// This is used by EXPLAIN queries to capture complete data source information including broker memory
func ( e * SQLEngine ) executeSelectStatementWithBrokerStats ( ctx context . Context , stmt * sqlparser . Selec t, plan * QueryExecutionPlan ) ( * QueryResult , error ) {
func ( e * SQLEngine ) executeSelectStatementWithBrokerStats ( ctx context . Context , stmt * SelectStatemen t, plan * QueryExecutionPlan ) ( * QueryResult , error ) {
// Parse FROM clause to get table (topic) information
// Parse FROM clause to get table (topic) information
if len ( stmt . From ) != 1 {
if len ( stmt . From ) != 1 {
err := fmt . Errorf ( "SELECT supports single table queries only" )
err := fmt . Errorf ( "SELECT supports single table queries only" )
@ -856,9 +1174,9 @@ func (e *SQLEngine) executeSelectStatementWithBrokerStats(ctx context.Context, s
// Extract table reference
// Extract table reference
var database , tableName string
var database , tableName string
switch table := stmt . From [ 0 ] . ( type ) {
switch table := stmt . From [ 0 ] . ( type ) {
case * sqlparser . AliasedTableExpr :
case * AliasedTableExpr :
switch tableExpr := table . Expr . ( type ) {
switch tableExpr := table . Expr . ( type ) {
case sqlparser . TableName :
case TableName :
tableName = tableExpr . Name . String ( )
tableName = tableExpr . Name . String ( )
if tableExpr . Qualifier . String ( ) != "" {
if tableExpr . Qualifier . String ( ) != "" {
database = tableExpr . Qualifier . String ( )
database = tableExpr . Qualifier . String ( )
@ -924,13 +1242,13 @@ func (e *SQLEngine) executeSelectStatementWithBrokerStats(ctx context.Context, s
for _ , selectExpr := range stmt . SelectExprs {
for _ , selectExpr := range stmt . SelectExprs {
switch expr := selectExpr . ( type ) {
switch expr := selectExpr . ( type ) {
case * sqlparser . StarExpr :
case * StarExpr :
selectAll = true
selectAll = true
case * sqlparser . AliasedExpr :
case * AliasedExpr :
switch col := expr . Expr . ( type ) {
switch col := expr . Expr . ( type ) {
case * sqlparser . ColName :
case * ColName :
columns = append ( columns , col . Name . String ( ) )
columns = append ( columns , col . Name . String ( ) )
case * sqlparser . FuncExpr :
case * FuncExpr :
// Handle aggregation functions
// Handle aggregation functions
aggSpec , err := e . parseAggregationFunction ( col , expr )
aggSpec , err := e . parseAggregationFunction ( col , expr )
if err != nil {
if err != nil {
@ -966,8 +1284,8 @@ func (e *SQLEngine) executeSelectStatementWithBrokerStats(ctx context.Context, s
limit := 0
limit := 0
if stmt . Limit != nil && stmt . Limit . Rowcount != nil {
if stmt . Limit != nil && stmt . Limit . Rowcount != nil {
switch limitExpr := stmt . Limit . Rowcount . ( type ) {
switch limitExpr := stmt . Limit . Rowcount . ( type ) {
case * sqlparser . SQLVal :
if limitExpr . Type == sqlparser . IntVal {
case * SQLVal :
if limitExpr . Type == IntVal {
var parseErr error
var parseErr error
limit64 , parseErr := strconv . ParseInt ( string ( limitExpr . Val ) , 10 , 64 )
limit64 , parseErr := strconv . ParseInt ( string ( limitExpr . Val ) , 10 , 64 )
if parseErr != nil {
if parseErr != nil {
@ -1049,7 +1367,7 @@ func (e *SQLEngine) executeSelectStatementWithBrokerStats(ctx context.Context, s
// extractTimeFilters extracts time range filters from WHERE clause for optimization
// extractTimeFilters extracts time range filters from WHERE clause for optimization
// This allows push-down of time-based queries to improve scan performance
// This allows push-down of time-based queries to improve scan performance
// Returns (startTimeNs, stopTimeNs) where 0 means unbounded
// Returns (startTimeNs, stopTimeNs) where 0 means unbounded
func ( e * SQLEngine ) extractTimeFilters ( expr sqlparser . Expr ) ( int64 , int64 ) {
func ( e * SQLEngine ) extractTimeFilters ( expr ExprNode ) ( int64 , int64 ) {
startTimeNs , stopTimeNs := int64 ( 0 ) , int64 ( 0 )
startTimeNs , stopTimeNs := int64 ( 0 ) , int64 ( 0 )
// Recursively extract time filters from expression tree
// Recursively extract time filters from expression tree
@ -1059,19 +1377,19 @@ func (e *SQLEngine) extractTimeFilters(expr sqlparser.Expr) (int64, int64) {
}
}
// extractTimeFiltersRecursive recursively processes WHERE expressions to find time comparisons
// extractTimeFiltersRecursive recursively processes WHERE expressions to find time comparisons
func ( e * SQLEngine ) extractTimeFiltersRecursive ( expr sqlparser . Expr , startTimeNs , stopTimeNs * int64 ) {
func ( e * SQLEngine ) extractTimeFiltersRecursive ( expr ExprNode , startTimeNs , stopTimeNs * int64 ) {
switch exprType := expr . ( type ) {
switch exprType := expr . ( type ) {
case * sqlparser . ComparisonExpr :
case * ComparisonExpr :
e . extractTimeFromComparison ( exprType , startTimeNs , stopTimeNs )
e . extractTimeFromComparison ( exprType , startTimeNs , stopTimeNs )
case * sqlparser . AndExpr :
case * AndExpr :
// For AND expressions, combine time filters (intersection)
// For AND expressions, combine time filters (intersection)
e . extractTimeFiltersRecursive ( exprType . Left , startTimeNs , stopTimeNs )
e . extractTimeFiltersRecursive ( exprType . Left , startTimeNs , stopTimeNs )
e . extractTimeFiltersRecursive ( exprType . Right , startTimeNs , stopTimeNs )
e . extractTimeFiltersRecursive ( exprType . Right , startTimeNs , stopTimeNs )
case * sqlparser . OrExpr :
case * OrExpr :
// For OR expressions, we can't easily optimize time ranges
// For OR expressions, we can't easily optimize time ranges
// Skip time filter extraction for OR clauses to avoid incorrect results
// Skip time filter extraction for OR clauses to avoid incorrect results
return
return
case * sqlparser . ParenExpr :
case * ParenExpr :
// Unwrap parentheses and continue
// Unwrap parentheses and continue
e . extractTimeFiltersRecursive ( exprType . Expr , startTimeNs , stopTimeNs )
e . extractTimeFiltersRecursive ( exprType . Expr , startTimeNs , stopTimeNs )
}
}
@ -1079,12 +1397,12 @@ func (e *SQLEngine) extractTimeFiltersRecursive(expr sqlparser.Expr, startTimeNs
// extractTimeFromComparison extracts time bounds from comparison expressions
// extractTimeFromComparison extracts time bounds from comparison expressions
// Handles comparisons against timestamp columns (_timestamp_ns, timestamp, created_at, etc.)
// Handles comparisons against timestamp columns (_timestamp_ns, timestamp, created_at, etc.)
func ( e * SQLEngine ) extractTimeFromComparison ( comp * sqlparser . ComparisonExpr , startTimeNs , stopTimeNs * int64 ) {
func ( e * SQLEngine ) extractTimeFromComparison ( comp * ComparisonExpr , startTimeNs , stopTimeNs * int64 ) {
// Check if this is a time-related column comparison
// Check if this is a time-related column comparison
leftCol := e . getColumnName ( comp . Left )
leftCol := e . getColumnName ( comp . Left )
rightCol := e . getColumnName ( comp . Right )
rightCol := e . getColumnName ( comp . Right )
var valueExpr sqlparser . Expr
var valueExpr ExprNode
var reversed bool
var reversed bool
// Determine which side is the time column
// Determine which side is the time column
@ -1114,23 +1432,23 @@ func (e *SQLEngine) extractTimeFromComparison(comp *sqlparser.ComparisonExpr, st
}
}
switch operator {
switch operator {
case sqlparser . GreaterThanStr : // timestamp > value
case GreaterThanStr : // timestamp > value
if * startTimeNs == 0 || timeValue > * startTimeNs {
if * startTimeNs == 0 || timeValue > * startTimeNs {
* startTimeNs = timeValue
* startTimeNs = timeValue
}
}
case sqlparser . GreaterEqualStr : // timestamp >= value
case GreaterEqualStr : // timestamp >= value
if * startTimeNs == 0 || timeValue >= * startTimeNs {
if * startTimeNs == 0 || timeValue >= * startTimeNs {
* startTimeNs = timeValue
* startTimeNs = timeValue
}
}
case sqlparser . LessThanStr : // timestamp < value
case LessThanStr : // timestamp < value
if * stopTimeNs == 0 || timeValue < * stopTimeNs {
if * stopTimeNs == 0 || timeValue < * stopTimeNs {
* stopTimeNs = timeValue
* stopTimeNs = timeValue
}
}
case sqlparser . LessEqualStr : // timestamp <= value
case LessEqualStr : // timestamp <= value
if * stopTimeNs == 0 || timeValue <= * stopTimeNs {
if * stopTimeNs == 0 || timeValue <= * stopTimeNs {
* stopTimeNs = timeValue
* stopTimeNs = timeValue
}
}
case sqlparser . EqualStr : // timestamp = value (point query)
case EqualStr : // timestamp = value (point query)
// For exact matches, set both bounds to the same value
// For exact matches, set both bounds to the same value
* startTimeNs = timeValue
* startTimeNs = timeValue
* stopTimeNs = timeValue
* stopTimeNs = timeValue
@ -1165,9 +1483,9 @@ func (e *SQLEngine) isTimeColumn(columnName string) bool {
}
}
// getColumnName extracts column name from expression (handles ColName types)
// getColumnName extracts column name from expression (handles ColName types)
func ( e * SQLEngine ) getColumnName ( expr sqlparser . Expr ) string {
func ( e * SQLEngine ) getColumnName ( expr ExprNode ) string {
switch exprType := expr . ( type ) {
switch exprType := expr . ( type ) {
case * sqlparser . ColName :
case * ColName :
return exprType . Name . String ( )
return exprType . Name . String ( )
}
}
return ""
return ""
@ -1175,16 +1493,16 @@ func (e *SQLEngine) getColumnName(expr sqlparser.Expr) string {
// extractTimeValue parses time values from SQL expressions
// extractTimeValue parses time values from SQL expressions
// Supports nanosecond timestamps, ISO dates, and relative times
// Supports nanosecond timestamps, ISO dates, and relative times
func ( e * SQLEngine ) extractTimeValue ( expr sqlparser . Expr ) int64 {
func ( e * SQLEngine ) extractTimeValue ( expr ExprNode ) int64 {
switch exprType := expr . ( type ) {
switch exprType := expr . ( type ) {
case * sqlparser . SQLVal :
case * SQLVal :
switch exprType . Type {
switch exprType . Type {
case sqlparser . IntVal :
case IntVal :
// Parse as nanosecond timestamp
// Parse as nanosecond timestamp
if val , err := strconv . ParseInt ( string ( exprType . Val ) , 10 , 64 ) ; err == nil {
if val , err := strconv . ParseInt ( string ( exprType . Val ) , 10 , 64 ) ; err == nil {
return val
return val
}
}
case sqlparser . StrVal :
case StrVal :
// Parse as ISO date or other string formats
// Parse as ISO date or other string formats
timeStr := string ( exprType . Val )
timeStr := string ( exprType . Val )
@ -1216,18 +1534,18 @@ func (e *SQLEngine) extractTimeValue(expr sqlparser.Expr) int64 {
// reverseOperator reverses comparison operators when column and value are swapped
// reverseOperator reverses comparison operators when column and value are swapped
func ( e * SQLEngine ) reverseOperator ( op string ) string {
func ( e * SQLEngine ) reverseOperator ( op string ) string {
switch op {
switch op {
case sqlparser . GreaterThanStr :
return sqlparser . LessThanStr
case sqlparser . GreaterEqualStr :
return sqlparser . LessEqualStr
case sqlparser . LessThanStr :
return sqlparser . GreaterThanStr
case sqlparser . LessEqualStr :
return sqlparser . GreaterEqualStr
case sqlparser . EqualStr :
return sqlparser . EqualStr
case sqlparser . NotEqualStr :
return sqlparser . NotEqualStr
case GreaterThanStr :
return LessThanStr
case GreaterEqualStr :
return LessEqualStr
case LessThanStr :
return GreaterThanStr
case LessEqualStr :
return GreaterEqualStr
case EqualStr :
return EqualStr
case NotEqualStr :
return NotEqualStr
default :
default :
return op
return op
}
}
@ -1235,11 +1553,11 @@ func (e *SQLEngine) reverseOperator(op string) string {
// buildPredicate creates a predicate function from a WHERE clause expression
// buildPredicate creates a predicate function from a WHERE clause expression
// This is a simplified implementation - a full implementation would be much more complex
// This is a simplified implementation - a full implementation would be much more complex
func ( e * SQLEngine ) buildPredicate ( expr sqlparser . Expr ) ( func ( * schema_pb . RecordValue ) bool , error ) {
func ( e * SQLEngine ) buildPredicate ( expr ExprNode ) ( func ( * schema_pb . RecordValue ) bool , error ) {
switch exprType := expr . ( type ) {
switch exprType := expr . ( type ) {
case * sqlparser . ComparisonExpr :
case * ComparisonExpr :
return e . buildComparisonPredicate ( exprType )
return e . buildComparisonPredicate ( exprType )
case * sqlparser . AndExpr :
case * AndExpr :
leftPred , err := e . buildPredicate ( exprType . Left )
leftPred , err := e . buildPredicate ( exprType . Left )
if err != nil {
if err != nil {
return nil , err
return nil , err
@ -1251,7 +1569,7 @@ func (e *SQLEngine) buildPredicate(expr sqlparser.Expr) (func(*schema_pb.RecordV
return func ( record * schema_pb . RecordValue ) bool {
return func ( record * schema_pb . RecordValue ) bool {
return leftPred ( record ) && rightPred ( record )
return leftPred ( record ) && rightPred ( record )
} , nil
} , nil
case * sqlparser . OrExpr :
case * OrExpr :
leftPred , err := e . buildPredicate ( exprType . Left )
leftPred , err := e . buildPredicate ( exprType . Left )
if err != nil {
if err != nil {
return nil , err
return nil , err
@ -1270,13 +1588,13 @@ func (e *SQLEngine) buildPredicate(expr sqlparser.Expr) (func(*schema_pb.RecordV
// buildComparisonPredicate creates a predicate for comparison operations (=, <, >, etc.)
// buildComparisonPredicate creates a predicate for comparison operations (=, <, >, etc.)
// Handles column names on both left and right sides of the comparison
// Handles column names on both left and right sides of the comparison
func ( e * SQLEngine ) buildComparisonPredicate ( expr * sqlparser . ComparisonExpr ) ( func ( * schema_pb . RecordValue ) bool , error ) {
func ( e * SQLEngine ) buildComparisonPredicate ( expr * ComparisonExpr ) ( func ( * schema_pb . RecordValue ) bool , error ) {
var columnName string
var columnName string
var compareValue interface { }
var compareValue interface { }
var operator string
var operator string
// Check if column is on the left side (normal case: column > value)
// Check if column is on the left side (normal case: column > value)
if colName , ok := expr . Left . ( * sqlparser . ColName ) ; ok {
if colName , ok := expr . Left . ( * ColName ) ; ok {
columnName = colName . Name . String ( )
columnName = colName . Name . String ( )
operator = expr . Operator
operator = expr . Operator
@ -1287,7 +1605,7 @@ func (e *SQLEngine) buildComparisonPredicate(expr *sqlparser.ComparisonExpr) (fu
}
}
compareValue = val
compareValue = val
} else if colName , ok := expr . Right . ( * sqlparser . ColName ) ; ok {
} else if colName , ok := expr . Right . ( * ColName ) ; ok {
// Column is on the right side (reversed case: value < column)
// Column is on the right side (reversed case: value < column)
columnName = colName . Name . String ( )
columnName = colName . Name . String ( )
@ -1317,35 +1635,35 @@ func (e *SQLEngine) buildComparisonPredicate(expr *sqlparser.ComparisonExpr) (fu
}
}
// extractComparisonValue extracts the comparison value from a SQL expression
// extractComparisonValue extracts the comparison value from a SQL expression
func ( e * SQLEngine ) extractComparisonValue ( expr sqlparser . Expr ) ( interface { } , error ) {
func ( e * SQLEngine ) extractComparisonValue ( expr ExprNode ) ( interface { } , error ) {
switch val := expr . ( type ) {
switch val := expr . ( type ) {
case * sqlparser . SQLVal :
case * SQLVal :
switch val . Type {
switch val . Type {
case sqlparser . IntVal :
case IntVal :
intVal , err := strconv . ParseInt ( string ( val . Val ) , 10 , 64 )
intVal , err := strconv . ParseInt ( string ( val . Val ) , 10 , 64 )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
return intVal , nil
return intVal , nil
case sqlparser . StrVal :
case StrVal :
return string ( val . Val ) , nil
return string ( val . Val ) , nil
default :
default :
return nil , fmt . Errorf ( "unsupported SQL value type: %v" , val . Type )
return nil , fmt . Errorf ( "unsupported SQL value type: %v" , val . Type )
}
}
case sqlparser . ValTuple :
case ValTuple :
// Handle IN expressions with multiple values: column IN (value1, value2, value3)
// Handle IN expressions with multiple values: column IN (value1, value2, value3)
var inValues [ ] interface { }
var inValues [ ] interface { }
for _ , tupleVal := range val {
for _ , tupleVal := range val {
switch v := tupleVal . ( type ) {
switch v := tupleVal . ( type ) {
case * sqlparser . SQLVal :
case * SQLVal :
switch v . Type {
switch v . Type {
case sqlparser . IntVal :
case IntVal :
intVal , err := strconv . ParseInt ( string ( v . Val ) , 10 , 64 )
intVal , err := strconv . ParseInt ( string ( v . Val ) , 10 , 64 )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
inValues = append ( inValues , intVal )
inValues = append ( inValues , intVal )
case sqlparser . StrVal :
case StrVal :
inValues = append ( inValues , string ( v . Val ) )
inValues = append ( inValues , string ( v . Val ) )
}
}
}
}
@ -1572,7 +1890,7 @@ func (e *SQLEngine) showTables(ctx context.Context, dbName string) (*QueryResult
return result , nil
return result , nil
}
}
func ( e * SQLEngine ) createTable ( ctx context . Context , stmt * sqlparser . DDL ) ( * QueryResult , error ) {
func ( e * SQLEngine ) createTable ( ctx context . Context , stmt * DDLStatement ) ( * QueryResult , error ) {
// Parse CREATE TABLE statement
// Parse CREATE TABLE statement
// Assumption: Table name format is [database.]table_name
// Assumption: Table name format is [database.]table_name
tableName := stmt . NewName . Name . String ( )
tableName := stmt . NewName . Name . String ( )
@ -1658,7 +1976,7 @@ func NewExecutionPlanBuilder(engine *SQLEngine) *ExecutionPlanBuilder {
// BuildAggregationPlan builds an execution plan for aggregation queries
// BuildAggregationPlan builds an execution plan for aggregation queries
func ( builder * ExecutionPlanBuilder ) BuildAggregationPlan (
func ( builder * ExecutionPlanBuilder ) BuildAggregationPlan (
stmt * sqlparser . Selec t,
stmt * SelectStatemen t,
aggregations [ ] AggregationSpec ,
aggregations [ ] AggregationSpec ,
strategy AggregationStrategy ,
strategy AggregationStrategy ,
dataSources * TopicDataSources ,
dataSources * TopicDataSources ,
@ -1689,7 +2007,7 @@ func (builder *ExecutionPlanBuilder) BuildAggregationPlan(
}
}
// determineExecutionStrategy determines the execution strategy based on query characteristics
// determineExecutionStrategy determines the execution strategy based on query characteristics
func ( builder * ExecutionPlanBuilder ) determineExecutionStrategy ( stmt * sqlparser . Selec t, strategy AggregationStrategy ) string {
func ( builder * ExecutionPlanBuilder ) determineExecutionStrategy ( stmt * SelectStatemen t, strategy AggregationStrategy ) string {
if stmt . Where != nil {
if stmt . Where != nil {
return "full_scan"
return "full_scan"
}
}
@ -1735,7 +2053,7 @@ func (builder *ExecutionPlanBuilder) countLiveLogFiles(dataSources *TopicDataSou
}
}
// buildOptimizationsList builds the list of optimizations used
// buildOptimizationsList builds the list of optimizations used
func ( builder * ExecutionPlanBuilder ) buildOptimizationsList ( stmt * sqlparser . Selec t, strategy AggregationStrategy ) [ ] string {
func ( builder * ExecutionPlanBuilder ) buildOptimizationsList ( stmt * SelectStatemen t, strategy AggregationStrategy ) [ ] string {
optimizations := [ ] string { }
optimizations := [ ] string { }
if strategy . CanUseFastPath {
if strategy . CanUseFastPath {
@ -1769,7 +2087,7 @@ func (builder *ExecutionPlanBuilder) buildAggregationsList(aggregations []Aggreg
}
}
// parseAggregationFunction parses an aggregation function expression
// parseAggregationFunction parses an aggregation function expression
func ( e * SQLEngine ) parseAggregationFunction ( funcExpr * sqlparser . FuncExpr , aliasExpr * sqlparser . AliasedExpr ) ( * AggregationSpec , error ) {
func ( e * SQLEngine ) parseAggregationFunction ( funcExpr * FuncExpr , aliasExpr * AliasedExpr ) ( * AggregationSpec , error ) {
funcName := strings . ToUpper ( funcExpr . Name . String ( ) )
funcName := strings . ToUpper ( funcExpr . Name . String ( ) )
spec := & AggregationSpec {
spec := & AggregationSpec {
@ -1784,11 +2102,11 @@ func (e *SQLEngine) parseAggregationFunction(funcExpr *sqlparser.FuncExpr, alias
}
}
switch arg := funcExpr . Exprs [ 0 ] . ( type ) {
switch arg := funcExpr . Exprs [ 0 ] . ( type ) {
case * sqlparser . StarExpr :
case * StarExpr :
spec . Column = "*"
spec . Column = "*"
spec . Alias = "COUNT(*)"
spec . Alias = "COUNT(*)"
case * sqlparser . AliasedExpr :
if colName , ok := arg . Expr . ( * sqlparser . ColName ) ; ok {
case * AliasedExpr :
if colName , ok := arg . Expr . ( * ColName ) ; ok {
spec . Column = colName . Name . String ( )
spec . Column = colName . Name . String ( )
spec . Alias = fmt . Sprintf ( "COUNT(%s)" , spec . Column )
spec . Alias = fmt . Sprintf ( "COUNT(%s)" , spec . Column )
} else {
} else {
@ -1804,8 +2122,8 @@ func (e *SQLEngine) parseAggregationFunction(funcExpr *sqlparser.FuncExpr, alias
}
}
switch arg := funcExpr . Exprs [ 0 ] . ( type ) {
switch arg := funcExpr . Exprs [ 0 ] . ( type ) {
case * sqlparser . AliasedExpr :
if colName , ok := arg . Expr . ( * sqlparser . ColName ) ; ok {
case * AliasedExpr :
if colName , ok := arg . Expr . ( * ColName ) ; ok {
spec . Column = colName . Name . String ( )
spec . Column = colName . Name . String ( )
spec . Alias = fmt . Sprintf ( "%s(%s)" , funcName , spec . Column )
spec . Alias = fmt . Sprintf ( "%s(%s)" , funcName , spec . Column )
} else {
} else {