Skip to content

Commit

Permalink
[SPARK-37867][SQL] Compile aggregate functions of build-in JDBC dialect
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?
DS V2 translate a lot of standard aggregate functions.
Currently, only H2Dialect compile these standard aggregate functions. This PR compile these standard aggregate functions for other build-in JDBC dialect.

### Why are the changes needed?
Make build-in JDBC dialect support complete aggregate push-down.

### Does this PR introduce _any_ user-facing change?
'Yes'.
Users could use complete aggregate push-down with build-in JDBC dialect.

### How was this patch tested?
New tests.

Closes #35166 from beliefer/SPARK-37867.

Authored-by: Jiaan Geng <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
  • Loading branch information
beliefer authored and cloud-fan committed Jan 25, 2022
1 parent ac2b0df commit 7148980
Show file tree
Hide file tree
Showing 14 changed files with 493 additions and 41 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,14 @@
package org.apache.spark.sql.jdbc.v2

import java.sql.Connection
import java.util.Locale

import org.scalatest.time.SpanSugar._

import org.apache.spark.SparkConf
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite}
import org.apache.spark.sql.jdbc.DatabaseOnDocker
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

Expand All @@ -36,8 +37,9 @@ import org.apache.spark.tags.DockerTest
* }}}
*/
@DockerTest
class DB2IntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
class DB2IntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest {
override val catalogName: String = "db2"
override val namespaceOpt: Option[String] = Some("DB2INST1")
override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.6.0a")
override val env = Map(
Expand All @@ -59,8 +61,13 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
override def sparkConf: SparkConf = super.sparkConf
.set("spark.sql.catalog.db2", classOf[JDBCTableCatalog].getName)
.set("spark.sql.catalog.db2.url", db.getJdbcUrl(dockerIp, externalPort))
.set("spark.sql.catalog.db2.pushDownAggregate", "true")

override def dataPreparation(conn: Connection): Unit = {}
override def tablePreparation(connection: Connection): Unit = {
connection.prepareStatement(
"CREATE TABLE employee (dept INTEGER, name VARCHAR(10), salary DECIMAL(20, 2), bonus DOUBLE)")
.executeUpdate()
}

override def testUpdateColumnType(tbl: String): Unit = {
sql(s"CREATE TABLE $tbl (ID INTEGER)")
Expand All @@ -86,4 +93,8 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
val expectedSchema = new StructType().add("ID", IntegerType, true, defaultMetadata)
assert(t.schema === expectedSchema)
}

override def caseConvert(tableName: String): String = tableName.toUpperCase(Locale.ROOT)

testVarPop()
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.jdbc.v2

import java.sql.Connection

import org.apache.spark.sql.jdbc.DockerJDBCIntegrationSuite

abstract class DockerJDBCIntegrationV2Suite extends DockerJDBCIntegrationSuite {

/**
* Prepare databases and tables for testing.
*/
override def dataPreparation(connection: Connection): Unit = {
tablePreparation(connection)
connection.prepareStatement("INSERT INTO employee VALUES (1, 'amy', 10000, 1000)")
.executeUpdate()
connection.prepareStatement("INSERT INTO employee VALUES (2, 'alex', 12000, 1200)")
.executeUpdate()
connection.prepareStatement("INSERT INTO employee VALUES (1, 'cathy', 9000, 1200)")
.executeUpdate()
connection.prepareStatement("INSERT INTO employee VALUES (2, 'david', 10000, 1300)")
.executeUpdate()
connection.prepareStatement("INSERT INTO employee VALUES (6, 'jen', 12000, 1200)")
.executeUpdate()
}

def tablePreparation(connection: Connection): Unit
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.scalatest.time.SpanSugar._
import org.apache.spark.SparkConf
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite}
import org.apache.spark.sql.jdbc.DatabaseOnDocker
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

Expand All @@ -37,7 +37,7 @@ import org.apache.spark.tags.DockerTest
* }}}
*/
@DockerTest
class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest {

override val catalogName: String = "mssql"

Expand All @@ -58,10 +58,15 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBC
override def sparkConf: SparkConf = super.sparkConf
.set("spark.sql.catalog.mssql", classOf[JDBCTableCatalog].getName)
.set("spark.sql.catalog.mssql.url", db.getJdbcUrl(dockerIp, externalPort))
.set("spark.sql.catalog.mssql.pushDownAggregate", "true")

override val connectionTimeout = timeout(7.minutes)

override def dataPreparation(conn: Connection): Unit = {}
override def tablePreparation(connection: Connection): Unit = {
connection.prepareStatement(
"CREATE TABLE employee (dept INT, name VARCHAR(32), salary NUMERIC(20, 2), bonus FLOAT)")
.executeUpdate()
}

override def notSupportsTableComment: Boolean = true

Expand Down Expand Up @@ -91,4 +96,9 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBC

assert(msg.contains("UpdateColumnNullability is not supported"))
}

testVarPop()
testVarSamp()
testStddevPop()
testStddevSamp()
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.scalatest.time.SpanSugar._
import org.apache.spark.SparkConf
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite}
import org.apache.spark.sql.jdbc.DatabaseOnDocker
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

Expand All @@ -39,7 +39,7 @@ import org.apache.spark.tags.DockerTest
*
*/
@DockerTest
class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest {
override val catalogName: String = "mysql"
override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("MYSQL_DOCKER_IMAGE_NAME", "mysql:5.7.36")
Expand All @@ -57,13 +57,17 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
override def sparkConf: SparkConf = super.sparkConf
.set("spark.sql.catalog.mysql", classOf[JDBCTableCatalog].getName)
.set("spark.sql.catalog.mysql.url", db.getJdbcUrl(dockerIp, externalPort))
.set("spark.sql.catalog.mysql.pushDownAggregate", "true")

override val connectionTimeout = timeout(7.minutes)

private var mySQLVersion = -1

override def dataPreparation(conn: Connection): Unit = {
mySQLVersion = conn.getMetaData.getDatabaseMajorVersion
override def tablePreparation(connection: Connection): Unit = {
mySQLVersion = connection.getMetaData.getDatabaseMajorVersion
connection.prepareStatement(
"CREATE TABLE employee (dept INT, name VARCHAR(32), salary DECIMAL(20, 2)," +
" bonus DOUBLE)").executeUpdate()
}

override def testUpdateColumnType(tbl: String): Unit = {
Expand Down Expand Up @@ -119,4 +123,9 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
override def supportsIndex: Boolean = true

override def indexOptions: String = "KEY_BLOCK_SIZE=10"

testVarPop()
testVarSamp()
testStddevPop()
testStddevSamp()
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,14 @@
package org.apache.spark.sql.jdbc.v2

import java.sql.Connection
import java.util.Locale

import org.scalatest.time.SpanSugar._

import org.apache.spark.SparkConf
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite}
import org.apache.spark.sql.jdbc.DatabaseOnDocker
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

Expand Down Expand Up @@ -54,8 +55,9 @@ import org.apache.spark.tags.DockerTest
* This procedure has been validated with Oracle 18.4.0 Express Edition.
*/
@DockerTest
class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest {
override val catalogName: String = "oracle"
override val namespaceOpt: Option[String] = Some("SYSTEM")
override val db = new DatabaseOnDocker {
lazy override val imageName =
sys.env.getOrElse("ORACLE_DOCKER_IMAGE_NAME", "gvenzl/oracle-xe:18.4.0")
Expand All @@ -73,9 +75,15 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest
override def sparkConf: SparkConf = super.sparkConf
.set("spark.sql.catalog.oracle", classOf[JDBCTableCatalog].getName)
.set("spark.sql.catalog.oracle.url", db.getJdbcUrl(dockerIp, externalPort))
.set("spark.sql.catalog.oracle.pushDownAggregate", "true")

override val connectionTimeout = timeout(7.minutes)
override def dataPreparation(conn: Connection): Unit = {}

override def tablePreparation(connection: Connection): Unit = {
connection.prepareStatement(
"CREATE TABLE employee (dept NUMBER(32), name VARCHAR2(32), salary NUMBER(20, 2)," +
" bonus BINARY_DOUBLE)").executeUpdate()
}

override def testUpdateColumnType(tbl: String): Unit = {
sql(s"CREATE TABLE $tbl (ID INTEGER)")
Expand All @@ -93,4 +101,14 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest
assert(msg1.contains(
s"Cannot update $catalogName.alt_table field ID: string cannot be cast to int"))
}

override def caseConvert(tableName: String): String = tableName.toUpperCase(Locale.ROOT)

testVarPop()
testVarSamp()
testStddevPop()
testStddevSamp()
testCovarPop()
testCovarSamp()
testCorr()
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.sql.Connection
import org.apache.spark.SparkConf
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite}
import org.apache.spark.sql.jdbc.DatabaseOnDocker
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

Expand All @@ -34,7 +34,7 @@ import org.apache.spark.tags.DockerTest
* }}}
*/
@DockerTest
class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTest {
class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest {
override val catalogName: String = "postgresql"
override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:14.0-alpine")
Expand All @@ -51,8 +51,13 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTes
.set("spark.sql.catalog.postgresql.url", db.getJdbcUrl(dockerIp, externalPort))
.set("spark.sql.catalog.postgresql.pushDownTableSample", "true")
.set("spark.sql.catalog.postgresql.pushDownLimit", "true")
.set("spark.sql.catalog.postgresql.pushDownAggregate", "true")

override def dataPreparation(conn: Connection): Unit = {}
override def tablePreparation(connection: Connection): Unit = {
connection.prepareStatement(
"CREATE TABLE employee (dept INTEGER, name VARCHAR(32), salary NUMERIC(20, 2)," +
" bonus double precision)").executeUpdate()
}

override def testUpdateColumnType(tbl: String): Unit = {
sql(s"CREATE TABLE $tbl (ID INTEGER)")
Expand Down Expand Up @@ -84,4 +89,12 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite with V2JDBCTes
override def supportsIndex: Boolean = true

override def indexOptions: String = "FILLFACTOR=70"

testVarPop()
testVarSamp()
testStddevPop()
testStddevSamp()
testCovarPop()
testCovarSamp()
testCorr()
}
Loading

0 comments on commit 7148980

Please sign in to comment.