Skip to content

Commit

Permalink
[FEAT] connect: add df.filter
Browse files Browse the repository at this point in the history
  • Loading branch information
andrewgazelka committed Nov 20, 2024
1 parent eb1c9af commit af9bcd1
Show file tree
Hide file tree
Showing 3 changed files with 49 additions and 1 deletion.
6 changes: 5 additions & 1 deletion src/daft-connect/src/translation/logical_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,9 +3,12 @@ use eyre::{bail, Context};
use spark_connect::{relation::RelType, Relation};
use tracing::warn;

use crate::translation::logical_plan::{aggregate::aggregate, project::project, range::range};
use crate::translation::logical_plan::{
aggregate::aggregate, filter::filter, project::project, range::range,
};

mod aggregate;
mod filter;
mod project;
mod range;

Expand All @@ -21,6 +24,7 @@ pub fn to_logical_plan(relation: Relation) -> eyre::Result<LogicalPlanBuilder> {
match rel_type {
RelType::Range(r) => range(r).wrap_err("Failed to apply range to logical plan"),
RelType::Project(p) => project(*p).wrap_err("Failed to apply project to logical plan"),
RelType::Filter(f) => filter(*f).wrap_err("Failed to apply filter to logical plan"),
RelType::Aggregate(a) => {
aggregate(*a).wrap_err("Failed to apply aggregate to logical plan")
}
Expand Down
25 changes: 25 additions & 0 deletions src/daft-connect/src/translation/logical_plan/filter.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
use eyre::bail;

use crate::translation::{to_daft_expr, to_logical_plan};

pub fn filter(
filter: spark_connect::Filter,
) -> eyre::Result<daft_logical_plan::LogicalPlanBuilder> {
let spark_connect::Filter { input, condition } = filter;

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

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

let condition = to_daft_expr(&condition)?;

let plan = to_logical_plan(*input)?;

let plan = plan.filter(condition)?;

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

from pyspark.sql.functions import col


def test_filter(spark_session):
# Create DataFrame from range(10)
df = spark_session.range(10)

# Filter for values less than 5
df_filtered = df.filter(col("id") < 5)

# Verify the schema is unchanged after filter
assert df_filtered.schema == df.schema, "Schema should be unchanged after filter"

# Verify the filtered data is correct
df_filtered_pandas = df_filtered.toPandas()
assert len(df_filtered_pandas) == 5, "Should have 5 rows after filtering < 5"
assert all(df_filtered_pandas["id"] < 5), "All values should be less than 5"

0 comments on commit af9bcd1

Please sign in to comment.