Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions .github/workflows/pr_build_linux.yml
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,7 @@ jobs:
org.apache.comet.CometStringExpressionSuite
org.apache.comet.CometBitwiseExpressionSuite
org.apache.comet.CometMapExpressionSuite
org.apache.comet.CometCsvExpressionSuite
org.apache.comet.CometJsonExpressionSuite
org.apache.comet.expressions.conditional.CometIfSuite
org.apache.comet.expressions.conditional.CometCoalesceSuite
Expand Down
1 change: 1 addition & 0 deletions .github/workflows/pr_build_macos.yml
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,7 @@ jobs:
org.apache.comet.CometBitwiseExpressionSuite
org.apache.comet.CometMapExpressionSuite
org.apache.comet.CometJsonExpressionSuite
org.apache.comet.CometCsvExpressionSuite
org.apache.comet.expressions.conditional.CometIfSuite
org.apache.comet.expressions.conditional.CometCoalesceSuite
org.apache.comet.expressions.conditional.CometCaseWhenSuite
Expand Down
1 change: 1 addition & 0 deletions docs/source/user-guide/latest/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -324,6 +324,7 @@ These settings can be used to determine which parts of the plan are accelerated
| `spark.comet.expression.StringTrimBoth.enabled` | Enable Comet acceleration for `StringTrimBoth` | true |
| `spark.comet.expression.StringTrimLeft.enabled` | Enable Comet acceleration for `StringTrimLeft` | true |
| `spark.comet.expression.StringTrimRight.enabled` | Enable Comet acceleration for `StringTrimRight` | true |
| `spark.comet.expression.StructsToCsv.enabled` | Enable Comet acceleration for `StructsToCsv` | true |
| `spark.comet.expression.StructsToJson.enabled` | Enable Comet acceleration for `StructsToJson` | true |
| `spark.comet.expression.Substring.enabled` | Enable Comet acceleration for `Substring` | true |
| `spark.comet.expression.Subtract.enabled` | Enable Comet acceleration for `Subtract` | true |
Expand Down
13 changes: 12 additions & 1 deletion native/core/src/execution/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ use datafusion::{
use datafusion_comet_spark_expr::{
create_comet_physical_fun, create_comet_physical_fun_with_eval_mode, BinaryOutputStyle,
BloomFilterAgg, BloomFilterMightContain, EvalMode, SparkHour, SparkMinute, SparkSecond,
SumInteger,
SumInteger, ToCsv,
};
use iceberg::expr::Bind;

Expand Down Expand Up @@ -644,6 +644,17 @@ impl PhysicalPlanner {
ExprStruct::MonotonicallyIncreasingId(_) => Ok(Arc::new(
MonotonicallyIncreasingId::from_partition_id(self.partition),
)),
ExprStruct::ToCsv(expr) => {
let csv_struct_expr =
self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&input_schema))?;
Ok(Arc::new(ToCsv::new(
csv_struct_expr,
&expr.delimiter,
&expr.quote,
&expr.escape,
&expr.null_value,
)))
}
expr => Err(GeneralError(format!("Not implemented: {expr:?}"))),
}
}
Expand Down
9 changes: 9 additions & 0 deletions native/proto/src/proto/expr.proto
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ message Expr {
EmptyExpr spark_partition_id = 63;
EmptyExpr monotonically_increasing_id = 64;
FromJson from_json = 89;
ToCsv to_csv = 90;
}
}

Expand Down Expand Up @@ -275,6 +276,14 @@ message FromJson {
string timezone = 3;
}

message ToCsv {
Expr child = 1;
string delimiter = 2;
string quote = 3;
string escape = 4;
string null_value = 5;
}

enum BinaryOutputStyle {
UTF8 = 0;
BASIC = 1;
Expand Down
20 changes: 20 additions & 0 deletions native/spark-expr/src/csv_funcs/mod.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

mod to_csv;

pub use to_csv::ToCsv;
229 changes: 229 additions & 0 deletions native/spark-expr/src/csv_funcs/to_csv.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,229 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

use arrow::array::{
as_boolean_array, as_largestring_array, as_string_array, Array, ArrayRef, StringBuilder,
};
use arrow::array::{RecordBatch, StructArray};
use arrow::datatypes::{DataType, Schema};
use datafusion::common::cast::{as_int16_array, as_int32_array, as_int64_array, as_int8_array};
use datafusion::common::{exec_err, Result};
use datafusion::logical_expr::ColumnarValue;
use datafusion::physical_expr::PhysicalExpr;
use std::any::Any;
use std::fmt::{Display, Formatter};
use std::hash::Hash;
use std::sync::Arc;

/// to_csv spark function
#[derive(Debug, Eq)]
pub struct ToCsv {
expr: Arc<dyn PhysicalExpr>,
delimiter: String,
quote: String,
escape: String,
null_value: String,
}

impl Hash for ToCsv {
fn hash<H: std::hash::Hasher>(&self, state: &mut H) {
self.expr.hash(state);
self.delimiter.hash(state);
self.quote.hash(state);
self.escape.hash(state);
self.null_value.hash(state);
}
}

impl PartialEq for ToCsv {
fn eq(&self, other: &Self) -> bool {
self.expr.eq(&other.expr)
&& self.delimiter.eq(&other.delimiter)
&& self.quote.eq(&other.quote)
&& self.escape.eq(&other.escape)
&& self.null_value.eq(&other.null_value)
}
}

impl ToCsv {
pub fn new(
expr: Arc<dyn PhysicalExpr>,
delimiter: &str,
quote: &str,
escape: &str,
null_value: &str,
) -> Self {
Self {
expr,
delimiter: delimiter.to_owned(),
quote: quote.to_owned(),
escape: escape.to_owned(),
null_value: null_value.to_owned(),
}
}
}

impl Display for ToCsv {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!(
f,
"to_csv({}, delimiter={}, quote={}, escape={}, null_value={})",
self.expr, self.delimiter, self.quote, self.escape, self.null_value
)
}
}

impl PhysicalExpr for ToCsv {
fn as_any(&self) -> &dyn Any {
self
}

fn data_type(&self, _: &Schema) -> Result<DataType> {
Ok(DataType::Utf8)
}

fn nullable(&self, input_schema: &Schema) -> Result<bool> {
self.expr.nullable(input_schema)
}

fn evaluate(&self, batch: &RecordBatch) -> Result<ColumnarValue> {
let input_value = self.expr.evaluate(batch)?.into_array(batch.num_rows())?;

let struct_array = input_value
.as_any()
.downcast_ref::<StructArray>()
.expect("A StructType is expected");

let result = struct_to_csv(struct_array, &self.delimiter, &self.null_value)?;

Ok(ColumnarValue::Array(result))
}

fn children(&self) -> Vec<&Arc<dyn PhysicalExpr>> {
vec![&self.expr]
}

fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn PhysicalExpr>>,
) -> Result<Arc<dyn PhysicalExpr>> {
Ok(Arc::new(Self::new(
Arc::clone(&children[0]),
&self.delimiter,
&self.quote,
&self.escape,
&self.null_value,
)))
}

fn fmt_sql(&self, _: &mut Formatter<'_>) -> std::fmt::Result {
unimplemented!()
}
}

fn struct_to_csv(array: &StructArray, delimiter: &str, null_value: &str) -> Result<ArrayRef> {
let mut builder = StringBuilder::with_capacity(array.len(), array.len() * 16);
let mut csv_string = String::with_capacity(array.len() * 16);

for row_idx in 0..array.len() {
if array.is_null(row_idx) {
builder.append_null();
} else {
csv_string.clear();
for (col_idx, column) in array.columns().iter().enumerate() {
if col_idx > 0 {
csv_string.push_str(delimiter);
}
if column.is_null(row_idx) {
csv_string.push_str(null_value);
} else {
convert_to_string(column, &mut csv_string, row_idx)?;
}
}
}
builder.append_value(&csv_string);
}
Ok(Arc::new(builder.finish()))
}

#[inline]
fn convert_to_string(array: &ArrayRef, csv_string: &mut String, row_idx: usize) -> Result<()> {
match array.data_type() {
DataType::Boolean => {
let array = as_boolean_array(array);
csv_string.push_str(&array.value(row_idx).to_string())
}
DataType::Int8 => {
let array = as_int8_array(array)?;
csv_string.push_str(&array.value(row_idx).to_string())
}
DataType::Int16 => {
let array = as_int16_array(array)?;
csv_string.push_str(&array.value(row_idx).to_string())
}
DataType::Int32 => {
let array = as_int32_array(array)?;
csv_string.push_str(&array.value(row_idx).to_string())
}
DataType::Int64 => {
let array = as_int64_array(array)?;
csv_string.push_str(&array.value(row_idx).to_string())
}
DataType::Utf8 => {
let array = as_string_array(array);
csv_string.push_str(&array.value(row_idx).to_string())
}
DataType::LargeUtf8 => {
let array = as_largestring_array(array);
csv_string.push_str(&array.value(row_idx).to_string())
}
_ => return exec_err!("to_csv not implemented for type: {:?}", array.data_type()),
}
Ok(())
}

#[cfg(test)]
mod tests {
use crate::csv_funcs::to_csv::struct_to_csv;
use arrow::array::{as_string_array, ArrayRef, Int32Array, StringArray, StructArray};
use arrow::datatypes::{DataType, Field};
use datafusion::common::Result;
use std::sync::Arc;

#[test]
fn test_to_csv_basic() -> Result<()> {
let struct_array = StructArray::from(vec![
(
Arc::new(Field::new("a", DataType::Int32, false)),
Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
),
(
Arc::new(Field::new("b", DataType::Utf8, true)),
Arc::new(StringArray::from(vec![Some("foo"), None, Some("baz")])) as ArrayRef,
),
]);

let expected = &StringArray::from(vec!["1,foo", "2,", "3,baz"]);

let result = struct_to_csv(&Arc::new(struct_array), ",", "")?;
let result = as_string_array(&result);

assert_eq!(result, expected);

Ok(())
}
}
2 changes: 2 additions & 0 deletions native/spark-expr/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ pub use bloom_filter::{BloomFilterAgg, BloomFilterMightContain};

mod conditional_funcs;
mod conversion_funcs;
mod csv_funcs;
mod math_funcs;
mod nondetermenistic_funcs;

Expand All @@ -68,6 +69,7 @@ pub use comet_scalar_funcs::{
create_comet_physical_fun, create_comet_physical_fun_with_eval_mode,
register_all_comet_functions,
};
pub use csv_funcs::*;
pub use datetime_funcs::{SparkDateTrunc, SparkHour, SparkMinute, SparkSecond, TimestampTruncExpr};
pub use error::{SparkError, SparkResult};
pub use hash_funcs::*;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,8 @@ object QueryPlanSerde extends Logging with CometExprShim {
classOf[GetArrayStructFields] -> CometGetArrayStructFields,
classOf[GetStructField] -> CometGetStructField,
classOf[JsonToStructs] -> CometJsonToStructs,
classOf[StructsToJson] -> CometStructsToJson)
classOf[StructsToJson] -> CometStructsToJson,
classOf[StructsToCsv] -> CometStructsToCsv)

private val hashExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map(
classOf[Md5] -> CometScalarFunction("md5"),
Expand Down
35 changes: 33 additions & 2 deletions spark/src/main/scala/org/apache/comet/serde/structs.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@ package org.apache.comet.serde

import scala.jdk.CollectionConverters._

import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, GetArrayStructFields, GetStructField, JsonToStructs, StructsToJson}
import org.apache.spark.sql.types.{ArrayType, DataType, DataTypes, MapType, StructType}
import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, GetArrayStructFields, GetStructField, JsonToStructs, StructsToCsv, StructsToJson}
import org.apache.spark.sql.types._

import org.apache.comet.CometSparkSessionExtensions.withInfo
import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, serializeDataType}
Expand Down Expand Up @@ -231,3 +231,34 @@ object CometJsonToStructs extends CometExpressionSerde[JsonToStructs] {
}
}
}

object CometStructsToCsv extends CometExpressionSerde[StructsToCsv] {

override def getSupportLevel(expr: StructsToCsv): SupportLevel = {
val isSupportedSchema = expr.inputSchema.fields
.forall(sf => QueryPlanSerde.supportedDataType(sf.dataType, allowComplex = false))
if (!isSupportedSchema) {
return Unsupported(Some(s"Unsupported data type: ${expr.inputSchema}"))
}
Incompatible()
}

override def convert(
expr: StructsToCsv,
inputs: Seq[Attribute],
binding: Boolean): Option[ExprOuterClass.Expr] = {
for {
childProto <- exprToProtoInternal(expr.child, inputs, binding)
} yield {
val toCsv = ExprOuterClass.ToCsv
.newBuilder()
.setChild(childProto)
.setDelimiter(expr.options.getOrElse("delimiter", ","))
.setQuote(expr.options.getOrElse("quote", "\""))
.setEscape(expr.options.getOrElse("escape", "\\"))
.setEscape(expr.options.getOrElse("nullValue", ""))
.build()
ExprOuterClass.Expr.newBuilder().setToCsv(toCsv).build()
}
}
}
Loading