Skip to content

Conversation

@caican00
Copy link
Contributor

@caican00 caican00 commented Jan 21, 2026

Changes Made

For example, when using a function in a select expression and no column alias is specified, the displayed result directly shows the original column names, which can lead to semantic misunderstandings.

from daft import col
import daft
df = daft.from_pydict({"a": [1, 2, 3], "b": ["s", "y", "z"]})
daft.sql("select sum(a) from df").show()

╭────────╮
│  a     │
│ ---    │
│ Int64  │
╞════════╡
│ 6      │
╰────────╯

And i directly use the expression string as the final column name for display, distinguishing it from the original column names and making the result more clear,like this:

daft.sql("select sum(a) from df").show()

╭────────╮
│ sum(a) │
│ ---    │
│ Int64  │
╞════════╡
│ 6      │
╰────────╯

Related Issues

#6070

@github-actions github-actions bot added the feat label Jan 21, 2026
@greptile-apps
Copy link
Contributor

greptile-apps bot commented Jan 21, 2026

Greptile Summary

This PR adds support for default column aliases in SQL SELECT statements and enables global aggregations in DataFrame select() calls.

Key Changes:

  • Created normalized_sql_expr_name() function to generate stable, user-facing column names from SQL expressions (e.g., sum(a), count(*), (a + 1))
  • SQL planner now automatically aliases unnamed expressions using the normalized name
  • DataFrame select() now supports global aggregations (aggregations without GROUP BY) by creating an Aggregate node followed by a Project node
  • Non-aggregation expressions mixed with aggregations must be foldable (literals only)

Issues Found:

  • Test error message pattern doesn't match actual error thrown in the code
  • Minor performance optimization: redundant aggregation check can use cached variable

Confidence Score: 4/5

  • Safe to merge after fixing the test error message matcher
  • Implementation is solid and well-tested, but one test has incorrect error message expectation that will cause it to fail. The performance optimization is minor and optional.
  • tests/dataframe/test_select_global_agg.py needs the error message pattern fixed

Important Files Changed

Filename Overview
src/daft-sql/src/expr_name.rs New module for generating stable, user-facing column names from SQL expressions - implementation looks correct
src/daft-sql/src/planner.rs Modified to use normalized_sql_expr_name for unnamed SELECT items - clean integration
src/daft-logical-plan/src/builder/mod.rs Added global aggregation support to select() - complex logic with potential edge cases in error handling and alias preservation
tests/dataframe/test_select_global_agg.py Tests for DataFrame global aggregation in select() - one test has incorrect error message expectation

Sequence Diagram

sequenceDiagram
    participant User
    participant SQLPlanner
    participant ExprName
    participant LogicalPlanBuilder
    participant ExprResolver
    participant Aggregate
    participant Project

    User->>SQLPlanner: sql("select sum(a) from df")
    SQLPlanner->>SQLPlanner: parse SQL AST
    SQLPlanner->>ExprName: normalized_sql_expr_name(sum(a))
    ExprName-->>SQLPlanner: "sum(a)"
    SQLPlanner->>SQLPlanner: plan_expr() to create Daft Expr
    SQLPlanner->>SQLPlanner: alias expr as "sum(a)"
    SQLPlanner->>LogicalPlanBuilder: select([sum(a).alias("sum(a)")])
    
    LogicalPlanBuilder->>LogicalPlanBuilder: Check if to_select has aggregations
    alt Contains aggregations
        LogicalPlanBuilder->>ExprResolver: resolve with in_agg_context=true
        ExprResolver-->>LogicalPlanBuilder: resolved expressions
        LogicalPlanBuilder->>LogicalPlanBuilder: Split into agg_exprs and foldable_exprs
        LogicalPlanBuilder->>LogicalPlanBuilder: Alias aggs with semantic_id
        LogicalPlanBuilder->>Aggregate: Create Aggregate node
        Aggregate-->>LogicalPlanBuilder: agg_plan
        LogicalPlanBuilder->>Project: Create Project with user-visible aliases
        Project-->>LogicalPlanBuilder: final plan
    else No aggregations
        LogicalPlanBuilder->>ExprResolver: resolve normally
        ExprResolver-->>LogicalPlanBuilder: resolved expressions
        LogicalPlanBuilder->>Project: Create Project node
        Project-->>LogicalPlanBuilder: final plan
    end
    
    LogicalPlanBuilder-->>User: DataFrame with column "sum(a)"
Loading

Copy link
Contributor

@greptile-apps greptile-apps bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

7 files reviewed, 2 comments

Edit Code Review Agent Settings | Greptile

def test_select_global_agg_rejects_non_agg_column_reference() -> None:
df = daft.from_pydict({"a": [1, 2, 3]})

with pytest.raises(ValueError, match="Expressions in aggregations"):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

logic: test expects "Expressions in aggregations" error message, but the actual error message from mod.rs:302 is "Expected aggregation (or a foldable literal) in select() only when aggregation expressions are present without groupby."

Suggested change
with pytest.raises(ValueError, match="Expressions in aggregations"):
with pytest.raises(ValueError, match="Expected aggregation"):
Prompt To Fix With AI
This is a comment left during a code review.
Path: tests/dataframe/test_select_global_agg.py
Line: 58:58

Comment:
**logic:** test expects "Expressions in aggregations" error message, but the actual error message from `mod.rs:302` is "Expected aggregation (or a foldable literal) in select() only when aggregation expressions are present without groupby."

```suggestion
    with pytest.raises(ValueError, match="Expected aggregation"):
```

How can I resolve this? If you propose a fix, please make it concise.

let logical_plan: LogicalPlan = ops::Project::try_new(self.plan.clone(), to_select)?.into();
Ok(self.with_new_plan(logical_plan))
// if a SELECT contains aggregations and there is no GROUP BY, treat it as a global aggregation.
if to_select.iter().any(has_agg) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

style: use the cached to_select_has_aggs variable instead of rechecking

Suggested change
if to_select.iter().any(has_agg) {
if to_select_has_aggs {

Note: If this suggestion doesn't match your team's coding style, reply to this and let me know. I'll remember it for next time!

Prompt To Fix With AI
This is a comment left during a code review.
Path: src/daft-logical-plan/src/builder/mod.rs
Line: 259:259

Comment:
**style:** use the cached `to_select_has_aggs` variable instead of rechecking

```suggestion
        if to_select_has_aggs {
```

<sub>Note: If this suggestion doesn't match your team's coding style, reply to this and let me know. I'll remember it for next time!</sub>

How can I resolve this? If you propose a fix, please make it concise.

@caican00 caican00 force-pushed the to-up-select-agg-default-alias branch from 1de4e26 to aaea901 Compare January 21, 2026 06:38
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant