Skip to content
Closed
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.
*/

package org.apache.spark.sql.connector.expressions;

import java.io.Serializable;

import org.apache.spark.annotation.Evolving;

/**
* The general SQL string corresponding to expression.
*
* @since 3.3.0
*/
@Evolving
public class GeneralSQLExpression implements Expression, Serializable {
Copy link
Member

Choose a reason for hiding this comment

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

I would also come up with another name instead of GeneralSQLExpression (e.g., PushedExpression)

private String sql;

public GeneralSQLExpression(String sql) {
Copy link
Member

Choose a reason for hiding this comment

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

Hm, shouldn't we keep the expression tree here, and let the JDBC source side to translate into SQL string? I think we can't assume that all downstream source implementation can understand SQL expression in string

Copy link
Contributor Author

@beliefer beliefer Jan 30, 2022

Choose a reason for hiding this comment

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

The V2 expression holds the V1 expression, which looks strange.
cc @cloud-fan

this.sql = sql;
}

public String sql() { return sql; }

@Override
public String toString() { return sql; }
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.connector.expressions.aggregate;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.expressions.Expression;

/**
* An aggregate function that returns the mean of all the values in a group.
Expand All @@ -27,23 +27,23 @@
*/
@Evolving
public final class Avg implements AggregateFunc {
private final NamedReference column;
private final Expression input;
private final boolean isDistinct;

public Avg(NamedReference column, boolean isDistinct) {
this.column = column;
public Avg(Expression column, boolean isDistinct) {
this.input = column;
this.isDistinct = isDistinct;
}

public NamedReference column() { return column; }
public Expression column() { return input; }
public boolean isDistinct() { return isDistinct; }

@Override
public String toString() {
if (isDistinct) {
return "AVG(DISTINCT " + column.describe() + ")";
return "AVG(DISTINCT " + input.describe() + ")";
} else {
return "AVG(" + column.describe() + ")";
return "AVG(" + input.describe() + ")";
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.connector.expressions.aggregate;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.expressions.Expression;

/**
* An aggregate function that returns the number of the specific row in a group.
Expand All @@ -27,23 +27,23 @@
*/
@Evolving
public final class Count implements AggregateFunc {
private final NamedReference column;
private final Expression input;
private final boolean isDistinct;

public Count(NamedReference column, boolean isDistinct) {
this.column = column;
public Count(Expression column, boolean isDistinct) {
this.input = column;
this.isDistinct = isDistinct;
}

public NamedReference column() { return column; }
public Expression column() { return input; }
public boolean isDistinct() { return isDistinct; }

@Override
public String toString() {
if (isDistinct) {
return "COUNT(DISTINCT " + column.describe() + ")";
return "COUNT(DISTINCT " + input.describe() + ")";
} else {
return "COUNT(" + column.describe() + ")";
return "COUNT(" + input.describe() + ")";
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.connector.expressions.aggregate;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.expressions.Expression;

/**
* An aggregate function that returns the maximum value in a group.
Expand All @@ -27,12 +27,12 @@
*/
@Evolving
public final class Max implements AggregateFunc {
private final NamedReference column;
private final Expression input;

public Max(NamedReference column) { this.column = column; }
public Max(Expression column) { this.input = column; }

public NamedReference column() { return column; }
public Expression column() { return input; }

@Override
public String toString() { return "MAX(" + column.describe() + ")"; }
public String toString() { return "MAX(" + input.describe() + ")"; }
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.connector.expressions.aggregate;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.expressions.Expression;

/**
* An aggregate function that returns the minimum value in a group.
Expand All @@ -27,12 +27,12 @@
*/
@Evolving
public final class Min implements AggregateFunc {
private final NamedReference column;
private final Expression input;

public Min(NamedReference column) { this.column = column; }
public Min(Expression column) { this.input = column; }

public NamedReference column() { return column; }
public Expression column() { return input; }

@Override
public String toString() { return "MIN(" + column.describe() + ")"; }
public String toString() { return "MIN(" + input.describe() + ")"; }
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.connector.expressions.aggregate;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.expressions.Expression;

/**
* An aggregate function that returns the summation of all the values in a group.
Expand All @@ -27,23 +27,23 @@
*/
@Evolving
public final class Sum implements AggregateFunc {
private final NamedReference column;
private final Expression input;
private final boolean isDistinct;

public Sum(NamedReference column, boolean isDistinct) {
this.column = column;
public Sum(Expression column, boolean isDistinct) {
this.input = column;
this.isDistinct = isDistinct;
}

public NamedReference column() { return column; }
public Expression column() { return input; }
public boolean isDistinct() { return isDistinct; }

@Override
public String toString() {
if (isDistinct) {
return "SUM(DISTINCT " + column.describe() + ")";
return "SUM(DISTINCT " + input.describe() + ")";
} else {
return "SUM(" + column.describe() + ")";
return "SUM(" + input.describe() + ")";
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.
*/

package org.apache.spark.sql.catalyst.util

import org.apache.spark.sql.catalyst.expressions.{Attribute, BinaryOperator, CaseWhen, EqualTo, Expression, IsNotNull, IsNull, Literal, Not}
import org.apache.spark.sql.connector.expressions.LiteralValue

/**
* The builder to generate SQL string from catalyst expressions.
*/
class ExpressionSQLBuilder(e: Expression) {

def build(): Option[String] = generateSQL(e)

private def generateSQL(expr: Expression): Option[String] = expr match {
case Literal(value, dataType) => Some(LiteralValue(value, dataType).toString)
case a: Attribute => Some(quoteIfNeeded(a.name))
case IsNull(col) => generateSQL(col).map(c => s"$c IS NULL")
case IsNotNull(col) => generateSQL(col).map(c => s"$c IS NOT NULL")
case b: BinaryOperator =>
val l = generateSQL(b.left)
val r = generateSQL(b.right)
if (l.isDefined && r.isDefined) {
Some(s"(${l.get}) ${b.sqlOperator} (${r.get})")
} else {
None
}
case Not(EqualTo(left, right)) =>
val l = generateSQL(left)
val r = generateSQL(right)
if (l.isDefined && r.isDefined) {
Some(s"${l.get} != ${r.get}")
} else {
None
}
case Not(child) => generateSQL(child).map(v => s"NOT ($v)")
case CaseWhen(branches, elseValue) =>
val conditionsSQL = branches.map(_._1).flatMap(generateSQL)
val valuesSQL = branches.map(_._2).flatMap(generateSQL)
if (conditionsSQL.length == branches.length && valuesSQL.length == branches.length) {
val branchSQL =
conditionsSQL.zip(valuesSQL).map { case (c, v) => s" WHEN $c THEN $v" }.mkString
if (elseValue.isDefined) {
elseValue.flatMap(generateSQL).map(v => s"CASE$branchSQL ELSE $v END")
} else {
Some(s"CASE$branchSQL END")
}
} else {
None
}
// TODO supports other expressions
case _ => None
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow}
import org.apache.spark.sql.connector.expressions.NamedReference
import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Aggregation, Count, CountStar, Max, Min}
import org.apache.spark.sql.execution.RowToColumnConverter
import org.apache.spark.sql.execution.datasources.v2.V2ColumnUtils
import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector}
import org.apache.spark.sql.types.{BooleanType, ByteType, DateType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructField, StructType}
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
Expand All @@ -42,27 +42,28 @@ object AggregatePushDownUtils {

var finalSchema = new StructType()

def getStructFieldForCol(col: NamedReference): StructField = {
schema.apply(col.fieldNames.head)
def getStructFieldForCol(colName: String): StructField = {
schema.apply(colName)
}

def isPartitionCol(col: NamedReference) = {
partitionNames.contains(col.fieldNames.head)
def isPartitionCol(colName: String) = {
partitionNames.contains(colName)
}

def processMinOrMax(agg: AggregateFunc): Boolean = {
val (column, aggType) = agg match {
case max: Max => (max.column, "max")
case min: Min => (min.column, "min")
case _ =>
throw new IllegalArgumentException(s"Unexpected type of AggregateFunc ${agg.describe}")
val (columnName, aggType) = agg match {
case max: Max if V2ColumnUtils.extractV2Column(max.column).isDefined =>
(V2ColumnUtils.extractV2Column(max.column).get, "max")
case min: Min if V2ColumnUtils.extractV2Column(min.column).isDefined =>
(V2ColumnUtils.extractV2Column(min.column).get, "min")
case _ => return false
}

if (isPartitionCol(column)) {
if (isPartitionCol(columnName)) {
// don't push down partition column, footer doesn't have max/min for partition column
return false
}
val structField = getStructFieldForCol(column)
val structField = getStructFieldForCol(columnName)

structField.dataType match {
// not push down complex type
Expand Down Expand Up @@ -108,19 +109,19 @@ object AggregatePushDownUtils {
aggregation.groupByColumns.foreach { col =>
// don't push down if the group by columns are not the same as the partition columns (orders
// doesn't matter because reorder can be done at data source layer)
if (col.fieldNames.length != 1 || !isPartitionCol(col)) return None
finalSchema = finalSchema.add(getStructFieldForCol(col))
if (col.fieldNames.length != 1 || !isPartitionCol(col.fieldNames.head)) return None
finalSchema = finalSchema.add(getStructFieldForCol(col.fieldNames.head))
}

aggregation.aggregateExpressions.foreach {
case max: Max =>
if (!processMinOrMax(max)) return None
case min: Min =>
if (!processMinOrMax(min)) return None
case count: Count =>
if (count.column.fieldNames.length != 1 || count.isDistinct) return None
finalSchema =
finalSchema.add(StructField(s"count(" + count.column.fieldNames.head + ")", LongType))
case count: Count
if V2ColumnUtils.extractV2Column(count.column).isDefined && !count.isDistinct =>
val columnName = V2ColumnUtils.extractV2Column(count.column).get
finalSchema = finalSchema.add(StructField(s"count($columnName)", LongType))
case _: CountStar =>
finalSchema = finalSchema.add(StructField("count(*)", LongType))
case _ =>
Expand Down
Loading