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,24 @@
/*
* 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.read.sqlpushdown;

import org.apache.spark.annotation.Evolving;

@Evolving
public interface SQLStatement {

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.read.sqlpushdown;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.catalog.TableCatalog;

/**
* A mix-in interface for {@link TableCatalog} to indicate that Data sources whether support SQL or
* not
*
* @since 3.x.x
*/

@Evolving
public interface SupportsSQL extends TableCatalog {

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.read.sqlpushdown;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.read.ScanBuilder;
import org.apache.spark.sql.connector.read.SupportsPushDownFilters;
import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns;
import org.apache.spark.sql.sources.Filter;
import org.apache.spark.sql.types.StructType;

/**
* A mix-in interface for {@link ScanBuilder}. Data sources which support SQL can implement this
* interface to push down SQL to backend and reduce the size of the data to be read.
*
* @since 3.x.x
*/

@Evolving
public interface SupportsSQLPushDown extends ScanBuilder,
SupportsPushDownRequiredColumns,
SupportsPushDownFilters {

/**
* Return true if executing a query on them would result in a query issued to multiple partitions.
* Returns false if it would result in a query to a single partition and therefore provides global
* results.
*/
boolean isMultiplePartitionExecution();

/**
* Pushes down {@link SQLStatement} to datasource and returns filters that need to be evaluated
* after scanning.
* <p>
* Rows should be returned from the data source if and only if all of the filters match. That is,
* filters must be interpreted as ANDed together.
*/
Filter[] pushStatement(SQLStatement statement, StructType outputSchema);

/**
* Returns the statement that are pushed to the data source via
* {@link #pushStatement(SQLStatement statement, StructType outputSchema)}
*/
SQLStatement pushedStatement();
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskCon
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.v2.pushdown.sql.SingleSQLStatement
import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects}
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -152,19 +153,19 @@ object JDBCRDD extends Logging {
requiredColumns: Array[String],
filters: Array[Filter],
parts: Array[Partition],
options: JDBCOptions): RDD[InternalRow] = {
val url = options.url
val dialect = JdbcDialects.get(url)
val quotedColumns = requiredColumns.map(colName => dialect.quoteIdentifier(colName))
options: JDBCOptions,
statement: Option[SingleSQLStatement] = None)
: RDD[InternalRow] = {
val pushedStatement =
statement.getOrElse(SingleSQLStatement(requiredColumns, filters, options))
new JDBCRDD(
sc,
JdbcUtils.createConnectionFactory(options),
pruneSchema(schema, requiredColumns),
quotedColumns,
filters,
parts,
url,
options)
options.url,
options,
pushedStatement)
}
}

Expand All @@ -177,50 +178,17 @@ private[jdbc] class JDBCRDD(
sc: SparkContext,
getConnection: () => Connection,
schema: StructType,
columns: Array[String],
filters: Array[Filter],
partitions: Array[Partition],
url: String,
options: JDBCOptions)
options: JDBCOptions,
statement: SingleSQLStatement)
extends RDD[InternalRow](sc, Nil) {

/**
* Retrieve the list of partitions corresponding to this RDD.
*/
override def getPartitions: Array[Partition] = partitions

/**
* `columns`, but as a String suitable for injection into a SQL query.
*/
private val columnList: String = {
val sb = new StringBuilder()
columns.foreach(x => sb.append(",").append(x))
if (sb.isEmpty) "1" else sb.substring(1)
}

/**
* `filters`, but as a WHERE clause suitable for injection into a SQL query.
*/
private val filterWhereClause: String =
filters
.flatMap(JDBCRDD.compileFilter(_, JdbcDialects.get(url)))
.map(p => s"($p)").mkString(" AND ")

/**
* A WHERE clause representing both `filters`, if any, and the current partition.
*/
private def getWhereClause(part: JDBCPartition): String = {
if (part.whereClause != null && filterWhereClause.length > 0) {
"WHERE " + s"($filterWhereClause)" + " AND " + s"(${part.whereClause})"
} else if (part.whereClause != null) {
"WHERE " + part.whereClause
} else if (filterWhereClause.length > 0) {
"WHERE " + filterWhereClause
} else {
""
}
}

/**
* Runs the SQL query against the JDBC driver.
*
Expand Down Expand Up @@ -294,9 +262,8 @@ private[jdbc] class JDBCRDD(
// fully-qualified table name in the SELECT statement. I don't know how to
// talk about a table in a completely portable way.

val myWhereClause = getWhereClause(part)
val sqlText = statement.toSQL(part.whereClause)

val sqlText = s"SELECT $columnList FROM ${options.tableOrQuery} $myWhereClause"
stmt = conn.prepareStatement(sqlText,
ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)
stmt.setFetchSize(options.fetchSize)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode, SparkS
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp}
import org.apache.spark.sql.execution.datasources.v2.pushdown.sql.SingleSQLStatement
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.jdbc.JdbcDialects
import org.apache.spark.sql.sources._
Expand Down Expand Up @@ -291,6 +292,21 @@ private[sql] case class JDBCRelation(
jdbcOptions).asInstanceOf[RDD[Row]]
}

def buildScan(
requiredColumns: Array[String],
filters: Array[Filter],
statement: SingleSQLStatement): RDD[Row] = {
// Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row]
JDBCRDD.scanTable(
sparkSession.sparkContext,
schema,
requiredColumns,
filters,
parts,
jdbcOptions,
Some(statement)).asInstanceOf[RDD[Row]]
}

override def insert(data: DataFrame, overwrite: Boolean): Unit = {
data.write
.mode(if (overwrite) SaveMode.Overwrite else SaveMode.Append)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2

import scala.collection.mutable

import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, NamedExpression, PredicateHelper, SchemaPruning}
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, NamedExpression, PredicateHelper, SchemaPruning, SubqueryExpression}
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
Expand Down Expand Up @@ -81,33 +81,58 @@ object PushDownUtils extends PredicateHelper {
relation: DataSourceV2Relation,
projects: Seq[NamedExpression],
filters: Seq[Expression]): (Scan, Seq[AttributeReference]) = {
prunedColumns(scanBuilder, relation, projects, filters)
.map { prunedSchema =>
scanBuilder.asInstanceOf[SupportsPushDownRequiredColumns]
.pruneColumns(prunedSchema)
val scan = scanBuilder.build()
scan -> toOutputAttrs(scan.readSchema(), relation)}
.getOrElse(scanBuilder.build() -> relation.output)
}

def prunedColumns(
scanBuilder: ScanBuilder,
relation: DataSourceV2Relation,
projects: Seq[NamedExpression],
filters: Seq[Expression]): Option[StructType] = {
scanBuilder match {
case r: SupportsPushDownRequiredColumns if SQLConf.get.nestedSchemaPruningEnabled =>
case _: SupportsPushDownRequiredColumns if SQLConf.get.nestedSchemaPruningEnabled =>
val rootFields = SchemaPruning.identifyRootFields(projects, filters)
val prunedSchema = if (rootFields.nonEmpty) {
SchemaPruning.pruneDataSchema(relation.schema, rootFields)
} else {
new StructType()
}
r.pruneColumns(prunedSchema)
val scan = r.build()
scan -> toOutputAttrs(scan.readSchema(), relation)
Some(prunedSchema)

case r: SupportsPushDownRequiredColumns =>
case _: SupportsPushDownRequiredColumns =>
val exprs = projects ++ filters
val requiredColumns = AttributeSet(exprs.flatMap(_.references))
val neededOutput = relation.output.filter(requiredColumns.contains)
r.pruneColumns(neededOutput.toStructType)
val scan = r.build()
// always project, in case the relation's output has been updated and doesn't match
// the underlying table schema
scan -> toOutputAttrs(scan.readSchema(), relation)

case _ => scanBuilder.build() -> relation.output
Some(neededOutput.toStructType)
case _ => None
}
}
def pushDownFilter(
scanBuilder: ScanBuilder,
filters: Seq[Expression],
relation: DataSourceV2Relation): (Seq[sources.Filter], Seq[Expression]) = {
val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output)
val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) =
normalizedFilters.partition(SubqueryExpression.hasSubquery)

// `pushedFilters` will be pushed down and evaluated in the underlying data sources.
// `postScanFilters` need to be evaluated after the scan.
// `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter.
val (pushedFilters, postScanFiltersWithoutSubquery) = PushDownUtils.pushFilters(
scanBuilder, normalizedFiltersWithoutSubquery)
val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery
(pushedFilters, postScanFilters)
}

private def toOutputAttrs(
def toOutputAttrs(
schema: StructType,
relation: DataSourceV2Relation): Seq[AttributeReference] = {
val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap
Expand Down
Loading