Skip to content

Commit

Permalink
[SPARK-50448][SQL][TESTS] Extract postgres image common conf as Postg…
Browse files Browse the repository at this point in the history
…resDatabaseOnDocker

### What changes were proposed in this pull request?
The  pr aims to extract `postgres` image `common` conf as `PostgresDatabaseOnDocker`.

### Why are the changes needed?
Reduce the cost of the next `postgres` image upgrade.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Pass existed UT & GA.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #49004 from panbingkun/SPARK-50448.

Authored-by: panbingkun <[email protected]>
Signed-off-by: panbingkun <[email protected]>
  • Loading branch information
panbingkun committed Nov 29, 2024
1 parent 2c2c0e0 commit aaf8590
Show file tree
Hide file tree
Showing 7 changed files with 45 additions and 62 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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

import org.apache.spark.internal.Logging

class PostgresDatabaseOnDocker extends DatabaseOnDocker with Logging {
lazy override val imageName: String =
sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine")
private val postgres_user = "postgres"
private val postgres_password = "rootpass"
override val env: Map[String, String] = Map(
"POSTGRES_PASSWORD" -> postgres_password
)
override val usesIpc = false
override val jdbcPort: Int = 5432

override def getJdbcUrl(ip: String, port: Int): String = {
s"jdbc:postgresql://$ip:$port/postgres?user=$postgres_user&password=$postgres_password"
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,16 +41,7 @@ import org.apache.spark.tags.DockerTest
*/
@DockerTest
class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite {
override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine")
override val env = Map(
"POSTGRES_PASSWORD" -> "rootpass"
)
override val usesIpc = false
override val jdbcPort = 5432
override def getJdbcUrl(ip: String, port: Int): String =
s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass"
}
override val db = new PostgresDatabaseOnDocker

override def dataPreparation(conn: Connection): Unit = {
conn.prepareStatement("CREATE DATABASE foo").executeUpdate()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,14 +37,7 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite {
override protected val userName = s"postgres/$dockerIp"
override protected val keytabFileName = "postgres.keytab"

override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine")
override val env = Map(
"POSTGRES_PASSWORD" -> "rootpass"
)
override val usesIpc = false
override val jdbcPort = 5432

override val db = new PostgresDatabaseOnDocker {
override def getJdbcUrl(ip: String, port: Int): String =
s"jdbc:postgresql://$ip:$port/postgres?user=$principal&gsslib=gssapi"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,16 +38,7 @@ import org.apache.spark.tags.DockerTest
@DockerTest
class GeneratedSubquerySuite extends DockerJDBCIntegrationSuite with QueryGeneratorHelper {

override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine")
override val env = Map(
"POSTGRES_PASSWORD" -> "rootpass"
)
override val usesIpc = false
override val jdbcPort = 5432
override def getJdbcUrl(ip: String, port: Int): String =
s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass"
}
override val db = new PostgresDatabaseOnDocker

private val FIRST_COLUMN = "a"
private val SECOND_COLUMN = "b"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,23 +38,13 @@ import org.apache.spark.tags.DockerTest
* }}}
*/
@DockerTest
class PostgreSQLQueryTestSuite extends CrossDbmsQueryTestSuite {
class PostgresSQLQueryTestSuite extends CrossDbmsQueryTestSuite {

val DATABASE_NAME = CrossDbmsQueryTestSuite.POSTGRES
// Scope to only subquery directory for now.
protected val customInputFilePath: String = new File(inputFilePath, "subquery").getAbsolutePath

override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine")
override val env = Map(
"POSTGRES_PASSWORD" -> "rootpass"
)
override val usesIpc = false
override val jdbcPort = 5432

override def getJdbcUrl(ip: String, port: Int): String =
s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass"
}
override val db = new PostgresDatabaseOnDocker

override def dataPreparation(conn: Connection): Unit = {
conn.prepareStatement(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.spark.{SparkConf, SparkSQLException}
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
import org.apache.spark.sql.jdbc.DatabaseOnDocker
import org.apache.spark.sql.jdbc.PostgresDatabaseOnDocker
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

Expand All @@ -37,16 +37,7 @@ import org.apache.spark.tags.DockerTest
@DockerTest
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:17.2-alpine")
override val env = Map(
"POSTGRES_PASSWORD" -> "rootpass"
)
override val usesIpc = false
override val jdbcPort = 5432
override def getJdbcUrl(ip: String, port: Int): String =
s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass"
}
override val db = new PostgresDatabaseOnDocker
override def sparkConf: SparkConf = super.sparkConf
.set("spark.sql.catalog.postgresql", classOf[JDBCTableCatalog].getName)
.set("spark.sql.catalog.postgresql.url", db.getJdbcUrl(dockerIp, externalPort))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.sql.Connection

import scala.jdk.CollectionConverters._

import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite}
import org.apache.spark.sql.jdbc.{DockerJDBCIntegrationSuite, PostgresDatabaseOnDocker}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.tags.DockerTest

Expand All @@ -34,16 +34,7 @@ import org.apache.spark.tags.DockerTest
*/
@DockerTest
class PostgresNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespaceTest {
override val db = new DatabaseOnDocker {
override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.2-alpine")
override val env = Map(
"POSTGRES_PASSWORD" -> "rootpass"
)
override val usesIpc = false
override val jdbcPort = 5432
override def getJdbcUrl(ip: String, port: Int): String =
s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass"
}
override val db = new PostgresDatabaseOnDocker

val map = new CaseInsensitiveStringMap(
Map("url" -> db.getJdbcUrl(dockerIp, externalPort),
Expand Down

0 comments on commit aaf8590

Please sign in to comment.