Skip to content

Commit

Permalink
[FEAT] connect: with_columns_renamed
Browse files Browse the repository at this point in the history
  • Loading branch information
andrewgazelka committed Nov 21, 2024
1 parent e4dadeb commit e9a0b0c
Show file tree
Hide file tree
Showing 3 changed files with 70 additions and 1 deletion.
5 changes: 4 additions & 1 deletion src/daft-connect/src/translation/logical_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5,14 +5,15 @@ use tracing::warn;

use crate::translation::logical_plan::{
aggregate::aggregate, project::project, range::range, set_op::set_op,
with_columns::with_columns,
with_columns::with_columns, with_columns_renamed::with_columns_renamed,
};

mod aggregate;
mod project;
mod range;
mod set_op;
mod with_columns;
mod with_columns_renamed;

pub fn to_logical_plan(relation: Relation) -> eyre::Result<LogicalPlanBuilder> {
if let Some(common) = relation.common {
Expand All @@ -32,6 +33,8 @@ pub fn to_logical_plan(relation: Relation) -> eyre::Result<LogicalPlanBuilder> {
RelType::WithColumns(w) => {
with_columns(*w).wrap_err("Failed to apply with_columns to logical plan")
}
RelType::WithColumnsRenamed(w) => with_columns_renamed(*w)
.wrap_err("Failed to apply with_columns_renamed to logical plan"),
RelType::SetOp(s) => set_op(*s).wrap_err("Failed to apply set_op to logical plan"),
plan => bail!("Unsupported relation type: {plan:?}"),
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
use daft_dsl::col;
use eyre::{bail, Context};

pub fn with_columns_renamed(
with_columns_renamed: spark_connect::WithColumnsRenamed,
) -> eyre::Result<daft_logical_plan::LogicalPlanBuilder> {
let spark_connect::WithColumnsRenamed {
input,
rename_columns_map,
renames,
} = with_columns_renamed;

let Some(input) = input else {
bail!("Input is required");
};

let plan = crate::translation::to_logical_plan(*input)?;

// todo: do we want to implement this directly into daft?

// Convert the rename mappings into expressions
let rename_exprs = if !rename_columns_map.is_empty() {
// Use rename_columns_map if provided (legacy format)
rename_columns_map
.into_iter()
.map(|(old_name, new_name)| col(old_name.as_str()).alias(new_name.as_str()))
.collect()
} else {
// Use renames if provided (new format)
renames
.into_iter()
.map(|rename| col(rename.col_name.as_str()).alias(rename.new_col_name.as_str()))
.collect()
};

// Apply the rename expressions to the plan
let plan = plan
.select(rename_exprs)
.wrap_err("Failed to apply rename expressions to logical plan")?;

Ok(plan)
}
24 changes: 24 additions & 0 deletions tests/connect/test_with_columns_renamed.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
from __future__ import annotations


def test_with_columns_renamed(spark_session):
# Test withColumnRenamed
df = spark_session.range(5)
renamed_df = df.withColumnRenamed("id", "number")

collected = renamed_df.collect()
assert len(collected) == 5
assert "number" in renamed_df.columns
assert "id" not in renamed_df.columns
assert [row["number"] for row in collected] == list(range(5))

# todo: this fails but is this expected or no?
# # Test withColumnsRenamed
# df = spark_session.range(2)
# renamed_df = df.withColumnsRenamed({"id": "number", "id": "character"})

# collected = renamed_df.collect()
# assert len(collected) == 2
# assert set(renamed_df.columns) == {"number", "character"}
# assert "id" not in renamed_df.columns
# assert [(row["number"], row["character"]) for row in collected] == [(0, 0), (1, 1)]

0 comments on commit e9a0b0c

Please sign in to comment.