|  |  | @ -500,8 +500,8 @@ func (e *SQLEngine) executeSelectStatementWithPlan(ctx context.Context, stmt *sq | 
			
		
	
		
			
				
					|  |  |  | 		// Execute aggregation query with plan tracking
 | 
			
		
	
		
			
				
					|  |  |  | 		result, err = e.executeAggregationQueryWithPlan(ctx, hybridScanner, aggregations, stmt, plan) | 
			
		
	
		
			
				
					|  |  |  | 	} else { | 
			
		
	
		
			
				
					|  |  |  | 		// Regular SELECT query
 | 
			
		
	
		
			
				
					|  |  |  | 		result, err = e.executeSelectStatement(ctx, stmt) | 
			
		
	
		
			
				
					|  |  |  | 		// Regular SELECT query with plan tracking
 | 
			
		
	
		
			
				
					|  |  |  | 		result, err = e.executeSelectStatementWithBrokerStats(ctx, stmt, plan) | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	if err == nil && result != nil { | 
			
		
	
	
		
			
				
					|  |  | @ -834,6 +834,198 @@ func (e *SQLEngine) executeSelectStatement(ctx context.Context, stmt *sqlparser. | 
			
		
	
		
			
				
					|  |  |  | 	return hybridScanner.ConvertToSQLResult(results, columns), nil | 
			
		
	
		
			
				
					|  |  |  | } | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | // executeSelectStatementWithBrokerStats handles SELECT queries with broker buffer statistics capture
 | 
			
		
	
		
			
				
					|  |  |  | // This is used by EXPLAIN queries to capture complete data source information including broker memory
 | 
			
		
	
		
			
				
					|  |  |  | func (e *SQLEngine) executeSelectStatementWithBrokerStats(ctx context.Context, stmt *sqlparser.Select, plan *QueryExecutionPlan) (*QueryResult, error) { | 
			
		
	
		
			
				
					|  |  |  | 	// Parse FROM clause to get table (topic) information
 | 
			
		
	
		
			
				
					|  |  |  | 	if len(stmt.From) != 1 { | 
			
		
	
		
			
				
					|  |  |  | 		err := fmt.Errorf("SELECT supports single table queries only") | 
			
		
	
		
			
				
					|  |  |  | 		return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Extract table reference
 | 
			
		
	
		
			
				
					|  |  |  | 	var database, tableName string | 
			
		
	
		
			
				
					|  |  |  | 	switch table := stmt.From[0].(type) { | 
			
		
	
		
			
				
					|  |  |  | 	case *sqlparser.AliasedTableExpr: | 
			
		
	
		
			
				
					|  |  |  | 		switch tableExpr := table.Expr.(type) { | 
			
		
	
		
			
				
					|  |  |  | 		case sqlparser.TableName: | 
			
		
	
		
			
				
					|  |  |  | 			tableName = tableExpr.Name.String() | 
			
		
	
		
			
				
					|  |  |  | 			if tableExpr.Qualifier.String() != "" { | 
			
		
	
		
			
				
					|  |  |  | 				database = tableExpr.Qualifier.String() | 
			
		
	
		
			
				
					|  |  |  | 			} | 
			
		
	
		
			
				
					|  |  |  | 		default: | 
			
		
	
		
			
				
					|  |  |  | 			err := fmt.Errorf("unsupported table expression: %T", tableExpr) | 
			
		
	
		
			
				
					|  |  |  | 			return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	default: | 
			
		
	
		
			
				
					|  |  |  | 		err := fmt.Errorf("unsupported FROM clause: %T", table) | 
			
		
	
		
			
				
					|  |  |  | 		return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Use current database context if not specified
 | 
			
		
	
		
			
				
					|  |  |  | 	if database == "" { | 
			
		
	
		
			
				
					|  |  |  | 		database = e.catalog.GetCurrentDatabase() | 
			
		
	
		
			
				
					|  |  |  | 		if database == "" { | 
			
		
	
		
			
				
					|  |  |  | 			database = "default" | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Auto-discover and register topic if not already in catalog
 | 
			
		
	
		
			
				
					|  |  |  | 	if _, err := e.catalog.GetTableInfo(database, tableName); err != nil { | 
			
		
	
		
			
				
					|  |  |  | 		// Topic not in catalog, try to discover and register it
 | 
			
		
	
		
			
				
					|  |  |  | 		if regErr := e.discoverAndRegisterTopic(ctx, database, tableName); regErr != nil { | 
			
		
	
		
			
				
					|  |  |  | 			// Return error immediately for non-existent topics instead of falling back to sample data
 | 
			
		
	
		
			
				
					|  |  |  | 			return &QueryResult{Error: regErr}, regErr | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Create HybridMessageScanner for the topic (reads both live logs + Parquet files)
 | 
			
		
	
		
			
				
					|  |  |  | 	// Get filerClient from broker connection (works with both real and mock brokers)
 | 
			
		
	
		
			
				
					|  |  |  | 	var filerClient filer_pb.FilerClient | 
			
		
	
		
			
				
					|  |  |  | 	var filerClientErr error | 
			
		
	
		
			
				
					|  |  |  | 	filerClient, filerClientErr = e.catalog.brokerClient.GetFilerClient() | 
			
		
	
		
			
				
					|  |  |  | 	if filerClientErr != nil { | 
			
		
	
		
			
				
					|  |  |  | 		// Return error if filer client is not available for topic access
 | 
			
		
	
		
			
				
					|  |  |  | 		return &QueryResult{Error: filerClientErr}, filerClientErr | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	hybridScanner, err := NewHybridMessageScanner(filerClient, e.catalog.brokerClient, database, tableName) | 
			
		
	
		
			
				
					|  |  |  | 	if err != nil { | 
			
		
	
		
			
				
					|  |  |  | 		// Return error for topic access issues instead of misleading sample data
 | 
			
		
	
		
			
				
					|  |  |  | 		topicErr := fmt.Errorf("failed to access topic %s.%s: %v", database, tableName, err) | 
			
		
	
		
			
				
					|  |  |  | 		return &QueryResult{Error: topicErr}, topicErr | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Parse SELECT columns and detect aggregation functions
 | 
			
		
	
		
			
				
					|  |  |  | 	var columns []string | 
			
		
	
		
			
				
					|  |  |  | 	var aggregations []AggregationSpec | 
			
		
	
		
			
				
					|  |  |  | 	selectAll := false | 
			
		
	
		
			
				
					|  |  |  | 	hasAggregations := false | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	for _, selectExpr := range stmt.SelectExprs { | 
			
		
	
		
			
				
					|  |  |  | 		switch expr := selectExpr.(type) { | 
			
		
	
		
			
				
					|  |  |  | 		case *sqlparser.StarExpr: | 
			
		
	
		
			
				
					|  |  |  | 			selectAll = true | 
			
		
	
		
			
				
					|  |  |  | 		case *sqlparser.AliasedExpr: | 
			
		
	
		
			
				
					|  |  |  | 			switch col := expr.Expr.(type) { | 
			
		
	
		
			
				
					|  |  |  | 			case *sqlparser.ColName: | 
			
		
	
		
			
				
					|  |  |  | 				columns = append(columns, col.Name.String()) | 
			
		
	
		
			
				
					|  |  |  | 			case *sqlparser.FuncExpr: | 
			
		
	
		
			
				
					|  |  |  | 				// Handle aggregation functions
 | 
			
		
	
		
			
				
					|  |  |  | 				aggSpec, err := e.parseAggregationFunction(col, expr) | 
			
		
	
		
			
				
					|  |  |  | 				if err != nil { | 
			
		
	
		
			
				
					|  |  |  | 					return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 				} | 
			
		
	
		
			
				
					|  |  |  | 				aggregations = append(aggregations, *aggSpec) | 
			
		
	
		
			
				
					|  |  |  | 				hasAggregations = true | 
			
		
	
		
			
				
					|  |  |  | 			default: | 
			
		
	
		
			
				
					|  |  |  | 				err := fmt.Errorf("unsupported SELECT expression: %T", col) | 
			
		
	
		
			
				
					|  |  |  | 				return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 			} | 
			
		
	
		
			
				
					|  |  |  | 		default: | 
			
		
	
		
			
				
					|  |  |  | 			err := fmt.Errorf("unsupported SELECT expression: %T", expr) | 
			
		
	
		
			
				
					|  |  |  | 			return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// If we have aggregations, use aggregation query path
 | 
			
		
	
		
			
				
					|  |  |  | 	if hasAggregations { | 
			
		
	
		
			
				
					|  |  |  | 		return e.executeAggregationQuery(ctx, hybridScanner, aggregations, stmt) | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Parse WHERE clause for predicate pushdown
 | 
			
		
	
		
			
				
					|  |  |  | 	var predicate func(*schema_pb.RecordValue) bool | 
			
		
	
		
			
				
					|  |  |  | 	if stmt.Where != nil { | 
			
		
	
		
			
				
					|  |  |  | 		predicate, err = e.buildPredicate(stmt.Where.Expr) | 
			
		
	
		
			
				
					|  |  |  | 		if err != nil { | 
			
		
	
		
			
				
					|  |  |  | 			return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Parse LIMIT clause
 | 
			
		
	
		
			
				
					|  |  |  | 	limit := 0 | 
			
		
	
		
			
				
					|  |  |  | 	if stmt.Limit != nil && stmt.Limit.Rowcount != nil { | 
			
		
	
		
			
				
					|  |  |  | 		switch limitExpr := stmt.Limit.Rowcount.(type) { | 
			
		
	
		
			
				
					|  |  |  | 		case *sqlparser.SQLVal: | 
			
		
	
		
			
				
					|  |  |  | 			if limitExpr.Type == sqlparser.IntVal { | 
			
		
	
		
			
				
					|  |  |  | 				var parseErr error | 
			
		
	
		
			
				
					|  |  |  | 				limit64, parseErr := strconv.ParseInt(string(limitExpr.Val), 10, 64) | 
			
		
	
		
			
				
					|  |  |  | 				if parseErr != nil { | 
			
		
	
		
			
				
					|  |  |  | 					return &QueryResult{Error: parseErr}, parseErr | 
			
		
	
		
			
				
					|  |  |  | 				} | 
			
		
	
		
			
				
					|  |  |  | 				if limit64 > math.MaxInt32 || limit64 < 0 { | 
			
		
	
		
			
				
					|  |  |  | 					return &QueryResult{Error: fmt.Errorf("LIMIT value %d is out of valid range", limit64)}, fmt.Errorf("LIMIT value %d is out of valid range", limit64) | 
			
		
	
		
			
				
					|  |  |  | 				} | 
			
		
	
		
			
				
					|  |  |  | 				limit = int(limit64) | 
			
		
	
		
			
				
					|  |  |  | 			} | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Build hybrid scan options
 | 
			
		
	
		
			
				
					|  |  |  | 	// RESOLVED TODO: Extract from WHERE clause time filters
 | 
			
		
	
		
			
				
					|  |  |  | 	startTimeNs, stopTimeNs := int64(0), int64(0) | 
			
		
	
		
			
				
					|  |  |  | 	if stmt.Where != nil { | 
			
		
	
		
			
				
					|  |  |  | 		startTimeNs, stopTimeNs = e.extractTimeFilters(stmt.Where.Expr) | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	hybridScanOptions := HybridScanOptions{ | 
			
		
	
		
			
				
					|  |  |  | 		StartTimeNs: startTimeNs, // Extracted from WHERE clause time comparisons
 | 
			
		
	
		
			
				
					|  |  |  | 		StopTimeNs:  stopTimeNs,  // Extracted from WHERE clause time comparisons
 | 
			
		
	
		
			
				
					|  |  |  | 		Limit:       limit, | 
			
		
	
		
			
				
					|  |  |  | 		Predicate:   predicate, | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	if !selectAll { | 
			
		
	
		
			
				
					|  |  |  | 		hybridScanOptions.Columns = columns | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Execute the hybrid scan with stats capture for EXPLAIN
 | 
			
		
	
		
			
				
					|  |  |  | 	var results []HybridScanResult | 
			
		
	
		
			
				
					|  |  |  | 	if plan != nil { | 
			
		
	
		
			
				
					|  |  |  | 		// EXPLAIN mode - capture broker buffer stats
 | 
			
		
	
		
			
				
					|  |  |  | 		var stats *HybridScanStats | 
			
		
	
		
			
				
					|  |  |  | 		results, stats, err = hybridScanner.ScanWithStats(ctx, hybridScanOptions) | 
			
		
	
		
			
				
					|  |  |  | 		if err != nil { | 
			
		
	
		
			
				
					|  |  |  | 			return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 		// Populate plan with broker buffer information
 | 
			
		
	
		
			
				
					|  |  |  | 		if stats != nil { | 
			
		
	
		
			
				
					|  |  |  | 			plan.BrokerBufferQueried = stats.BrokerBufferQueried | 
			
		
	
		
			
				
					|  |  |  | 			plan.BrokerBufferMessages = stats.BrokerBufferMessages | 
			
		
	
		
			
				
					|  |  |  | 			plan.BufferStartIndex = stats.BufferStartIndex | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 			// Add broker_buffer to data sources if buffer was queried
 | 
			
		
	
		
			
				
					|  |  |  | 			if stats.BrokerBufferQueried { | 
			
		
	
		
			
				
					|  |  |  | 				// Check if broker_buffer is already in data sources
 | 
			
		
	
		
			
				
					|  |  |  | 				hasBrokerBuffer := false | 
			
		
	
		
			
				
					|  |  |  | 				for _, source := range plan.DataSources { | 
			
		
	
		
			
				
					|  |  |  | 					if source == "broker_buffer" { | 
			
		
	
		
			
				
					|  |  |  | 						hasBrokerBuffer = true | 
			
		
	
		
			
				
					|  |  |  | 						break | 
			
		
	
		
			
				
					|  |  |  | 					} | 
			
		
	
		
			
				
					|  |  |  | 				} | 
			
		
	
		
			
				
					|  |  |  | 				if !hasBrokerBuffer { | 
			
		
	
		
			
				
					|  |  |  | 					plan.DataSources = append(plan.DataSources, "broker_buffer") | 
			
		
	
		
			
				
					|  |  |  | 				} | 
			
		
	
		
			
				
					|  |  |  | 			} | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	} else { | 
			
		
	
		
			
				
					|  |  |  | 		// Normal mode - just get results
 | 
			
		
	
		
			
				
					|  |  |  | 		results, err = hybridScanner.Scan(ctx, hybridScanOptions) | 
			
		
	
		
			
				
					|  |  |  | 		if err != nil { | 
			
		
	
		
			
				
					|  |  |  | 			return &QueryResult{Error: err}, err | 
			
		
	
		
			
				
					|  |  |  | 		} | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Convert to SQL result format
 | 
			
		
	
		
			
				
					|  |  |  | 	if selectAll { | 
			
		
	
		
			
				
					|  |  |  | 		columns = nil // Let converter determine all columns
 | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	return hybridScanner.ConvertToSQLResult(results, columns), nil | 
			
		
	
		
			
				
					|  |  |  | } | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | // extractTimeFilters extracts time range filters from WHERE clause for optimization
 | 
			
		
	
		
			
				
					|  |  |  | // This allows push-down of time-based queries to improve scan performance
 | 
			
		
	
		
			
				
					|  |  |  | // Returns (startTimeNs, stopTimeNs) where 0 means unbounded
 | 
			
		
	
	
		
			
				
					|  |  | @ -1469,6 +1661,9 @@ func (builder *ExecutionPlanBuilder) buildDataSourcesList(strategy AggregationSt | 
			
		
	
		
			
				
					|  |  |  | 		sources = append(sources, "live_logs", "parquet_files") | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Note: broker_buffer is added dynamically during execution when broker is queried
 | 
			
		
	
		
			
				
					|  |  |  | 	// See aggregations.go lines 397-409 for the broker buffer data source addition logic
 | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	return sources | 
			
		
	
		
			
				
					|  |  |  | } | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
	
		
			
				
					|  |  | @ -1803,9 +1998,7 @@ func (e *SQLEngine) convertLogEntryToRecordValue(logEntry *filer_pb.LogEntry) (* | 
			
		
	
		
			
				
					|  |  |  | // Format: YYYY-MM-DD-HH-MM-SS.parquet
 | 
			
		
	
		
			
				
					|  |  |  | func (e *SQLEngine) extractTimestampFromFilename(filename string) int64 { | 
			
		
	
		
			
				
					|  |  |  | 	// Remove .parquet extension
 | 
			
		
	
		
			
				
					|  |  |  | 	if strings.HasSuffix(filename, ".parquet") { | 
			
		
	
		
			
				
					|  |  |  | 		filename = filename[:len(filename)-8] | 
			
		
	
		
			
				
					|  |  |  | 	} | 
			
		
	
		
			
				
					|  |  |  | 	filename = strings.TrimSuffix(filename, ".parquet") | 
			
		
	
		
			
				
					|  |  |  | 
 | 
			
		
	
		
			
				
					|  |  |  | 	// Parse timestamp format: 2006-01-02-15-04-05
 | 
			
		
	
		
			
				
					|  |  |  | 	t, err := time.Parse("2006-01-02-15-04-05", filename) | 
			
		
	
	
		
			
				
					|  |  | 
 |