hook up seaweed sql engine

This commit is contained in:
chrislu
2025-09-02 08:42:36 -07:00
parent 5a8c31fcda
commit 8b4914be55
4 changed files with 100 additions and 379 deletions

View File

@@ -66,15 +66,37 @@ func (s *PostgreSQLServer) handleMessage(session *PostgreSQLSession) error {
func (s *PostgreSQLServer) handleSimpleQuery(session *PostgreSQLSession, query string) error {
glog.V(2).Infof("PostgreSQL Query (ID: %d): %s", session.processID, query)
// Translate PostgreSQL SQL to SeaweedFS SQL
translatedQuery, err := s.translator.TranslateQuery(query)
if err != nil {
return s.sendError(session, "42601", err.Error())
// Handle USE database commands for session context
queryUpper := strings.ToUpper(strings.TrimSpace(query))
if strings.HasPrefix(queryUpper, "USE ") {
parts := strings.Fields(query)
if len(parts) >= 2 {
newDatabase := strings.TrimSpace(parts[1])
session.database = newDatabase
s.sqlEngine.GetCatalog().SetCurrentDatabase(newDatabase)
// Send command complete for USE
err := s.sendCommandComplete(session, "USE")
if err != nil {
return err
}
return s.sendReadyForQuery(session)
}
}
// Execute using SQL engine
// Set database context in SQL engine if session database is different from current
if session.database != "" && session.database != s.sqlEngine.GetCatalog().GetCurrentDatabase() {
s.sqlEngine.GetCatalog().SetCurrentDatabase(session.database)
}
// Handle PostgreSQL-specific system queries
if postgresQuery := s.translatePostgreSQLSystemQuery(query); postgresQuery != "" {
query = postgresQuery
}
// Execute using SQL engine directly
ctx := context.Background()
result, err := s.sqlEngine.ExecuteSQL(ctx, translatedQuery)
result, err := s.sqlEngine.ExecuteSQL(ctx, query)
if err != nil {
return s.sendError(session, "42000", err.Error())
}
@@ -111,6 +133,57 @@ func (s *PostgreSQLServer) handleSimpleQuery(session *PostgreSQLSession, query s
return s.sendReadyForQuery(session)
}
// translatePostgreSQLSystemQuery translates essential PostgreSQL system queries
// Only handles queries that PostgreSQL clients expect but SeaweedFS SQL engine doesn't natively support
func (s *PostgreSQLServer) translatePostgreSQLSystemQuery(query string) string {
// Trim and normalize query
query = strings.TrimSpace(query)
query = strings.TrimSuffix(query, ";")
queryLower := strings.ToLower(query)
// Handle essential PostgreSQL system queries
switch queryLower {
case "select version()":
return "SELECT 'SeaweedFS 1.0 (PostgreSQL 14.0 compatible)' as version"
case "select current_database()":
return "SELECT '" + s.config.Database + "' as current_database"
case "select current_user":
return "SELECT 'seaweedfs' as current_user"
case "select current_setting('server_version')":
return "SELECT '14.0' as server_version"
case "select current_setting('server_encoding')":
return "SELECT 'UTF8' as server_encoding"
case "select current_setting('client_encoding')":
return "SELECT 'UTF8' as client_encoding"
}
// Handle pg_* catalog queries by mapping to equivalent SHOW commands
if strings.Contains(queryLower, "pg_tables") || strings.Contains(queryLower, "information_schema.tables") {
return "SHOW TABLES"
}
if strings.Contains(queryLower, "pg_database") || strings.Contains(queryLower, "information_schema.schemata") {
return "SHOW DATABASES"
}
// Handle transaction commands (no-op for read-only)
switch queryLower {
case "begin", "start transaction":
return "SELECT 'BEGIN' as status"
case "commit":
return "SELECT 'COMMIT' as status"
case "rollback":
return "SELECT 'ROLLBACK' as status"
}
// If starts with SET, return a no-op
if strings.HasPrefix(queryLower, "set ") {
return "SELECT 'SET' as status"
}
// Return empty string to use original query (let SQL engine handle it)
return ""
}
// handleParse processes a Parse message (prepared statement)
func (s *PostgreSQLServer) handleParse(session *PostgreSQLSession, msgBody []byte) error {
// Parse message format: statement_name\0query\0param_count(int16)[param_type(int32)...]