From 871c30358003f03c9db78ce7f60c59cf8b353e8d Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 11 Jun 2017 10:09:33 +0800 Subject: [PATCH 01/17] Read JDBC table use custom schema --- .../sql/jdbc/OracleIntegrationSuite.scala | 48 +++++++++++++++++-- .../org/apache/spark/sql/types/Metadata.scala | 3 ++ .../apache/spark/sql/DataFrameReader.scala | 5 +- .../datasources/jdbc/JDBCRelation.scala | 6 ++- 4 files changed, 56 insertions(+), 6 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index f7b1ec34ced76..29ba38e58a61b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -70,10 +70,17 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo """.stripMargin.replaceAll("\n", " ")).executeUpdate() conn.commit() - conn.prepareStatement("CREATE TABLE ts_with_timezone (id NUMBER(10), t TIMESTAMP WITH TIME ZONE)") - .executeUpdate() - conn.prepareStatement("INSERT INTO ts_with_timezone VALUES (1, to_timestamp_tz('1999-12-01 11:00:00 UTC','YYYY-MM-DD HH:MI:SS TZR'))") - .executeUpdate() + conn.prepareStatement( + "CREATE TABLE ts_with_timezone (id NUMBER(10), t TIMESTAMP WITH TIME ZONE)").executeUpdate() + conn.prepareStatement( + "INSERT INTO ts_with_timezone VALUES " + + "(1, to_timestamp_tz('1999-12-01 11:00:00 UTC','YYYY-MM-DD HH:MI:SS TZR'))").executeUpdate() + conn.commit() + + conn.prepareStatement( + "CREATE TABLE custom_column_types (id NUMBER, n1 number(1), n2 number(1))").executeUpdate() + conn.prepareStatement( + "INSERT INTO custom_column_types values(12312321321321312312312312123, 1, 0)").executeUpdate() conn.commit() sql( @@ -198,4 +205,37 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types(1).equals("class java.sql.Timestamp")) } + + test("SPARK-20427/SPARK-20921: read table use custom schema") { + + // default will throw IllegalArgumentException + val e = intercept[org.apache.spark.SparkException] { + spark.read.jdbc(jdbcUrl, "custom_column_types", new Properties()).collect() + } + assert(e.getMessage.contains( + "requirement failed: Decimal precision 39 exceeds max precision 38")) + + // custom schema can read data + val schema = StructType(Seq( + StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0), true, + new MetadataBuilder().putName("ID").build()), + StructField("N1", IntegerType, true, new MetadataBuilder().putName("N1").build()), + StructField("N2", BooleanType, true, new MetadataBuilder().putName("N2").build()))) + + val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "custom_column_types", new Properties()) + val rows = dfRead.collect() + + // verify the data type inserted + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types(0).equals("class java.math.BigDecimal")) + assert(types(1).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Boolean")) + + // verify the value inserted + val values = rows(0) + assert(values.getDecimal(0).equals(new java.math.BigDecimal("12312321321321312312312312123"))) + assert(values.getInt(1).equals(1)) + assert(values.getBoolean(2).equals(false)) + } + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 3aa4bf619f274..8ea526e583dc8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -273,6 +273,9 @@ class MetadataBuilder { /** Puts a [[Metadata]] array. */ def putMetadataArray(key: String, value: Array[Metadata]): this.type = put(key, value) + /** Puts a name. */ + def putName(name: String): this.type = put("name", name) + /** Builds the [[Metadata]] instance. */ def build(): Metadata = { new Metadata(map.toMap) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 0f96e82cedf4e..1ad91293a8906 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -197,11 +197,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def jdbc(url: String, table: String, properties: Properties): DataFrame = { - assertNoSpecifiedSchema("jdbc") // properties should override settings in extraOptions. this.extraOptions ++= properties.asScala // explicit url and dbtable should override all this.extraOptions += (JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) + if (!userSpecifiedSchema.isEmpty) { + this.extraOptions += + (JDBCOptions.JDBC_CREATE_TABLE_COLUMN_TYPES -> userSpecifiedSchema.get.json) + } format("jdbc").load() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index a06f1ce3287e6..aa90e7ae83042 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -110,7 +110,11 @@ private[sql] case class JDBCRelation( override val needConversion: Boolean = false - override val schema: StructType = JDBCRDD.resolveTable(jdbcOptions) + override val schema: StructType = if (!jdbcOptions.createTableColumnTypes.isEmpty) { + StructType.fromString(jdbcOptions.createTableColumnTypes.get) + } else { + JDBCRDD.resolveTable(jdbcOptions) + } // Check if JDBCRDD.compileFilter can accept input filters override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { From 0444c4d25d5943408a3ea11f84395dd38246e2f8 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 11 Jun 2017 20:58:50 +0800 Subject: [PATCH 02/17] Fix test error. --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 3 +-- .../spark/sql/execution/datasources/jdbc/JDBCOptions.scala | 2 ++ .../spark/sql/execution/datasources/jdbc/JDBCRelation.scala | 4 ++-- .../src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 1ad91293a8906..96452e154c24b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -202,8 +202,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { // explicit url and dbtable should override all this.extraOptions += (JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) if (!userSpecifiedSchema.isEmpty) { - this.extraOptions += - (JDBCOptions.JDBC_CREATE_TABLE_COLUMN_TYPES -> userSpecifiedSchema.get.json) + this.extraOptions += (JDBCOptions.JDBC_CUSTOM_SCHEMA -> userSpecifiedSchema.get.json) } format("jdbc").load() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 591096d5efd22..6bfc0404b2451 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -120,6 +120,7 @@ class JDBCOptions( // TODO: to reuse the existing partition parameters for those partition specific options val createTableOptions = parameters.getOrElse(JDBC_CREATE_TABLE_OPTIONS, "") val createTableColumnTypes = parameters.get(JDBC_CREATE_TABLE_COLUMN_TYPES) + val customSchema = parameters.get(JDBC_CUSTOM_SCHEMA) val batchSize = { val size = parameters.getOrElse(JDBC_BATCH_INSERT_SIZE, "1000").toInt require(size >= 1, @@ -156,6 +157,7 @@ object JDBCOptions { val JDBC_TRUNCATE = newOption("truncate") val JDBC_CREATE_TABLE_OPTIONS = newOption("createTableOptions") val JDBC_CREATE_TABLE_COLUMN_TYPES = newOption("createTableColumnTypes") + val JDBC_CUSTOM_SCHEMA = newOption("customSchema") val JDBC_BATCH_INSERT_SIZE = newOption("batchsize") val JDBC_TXN_ISOLATION_LEVEL = newOption("isolationLevel") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index aa90e7ae83042..e1e588ade35ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -110,8 +110,8 @@ private[sql] case class JDBCRelation( override val needConversion: Boolean = false - override val schema: StructType = if (!jdbcOptions.createTableColumnTypes.isEmpty) { - StructType.fromString(jdbcOptions.createTableColumnTypes.get) + override val schema: StructType = if (!jdbcOptions.customSchema.isEmpty) { + StructType.fromString(jdbcOptions.customSchema.get) } else { JDBCRDD.resolveTable(jdbcOptions) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 70bee929b31da..4ec8bb9b5841e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -907,7 +907,7 @@ class JDBCSuite extends SparkFunSuite assert(new JDBCOptions(CaseInsensitiveMap(parameters)).asConnectionProperties.isEmpty) } - test("SPARK-16848: jdbc API throws an exception for user specified schema") { + ignore("SPARK-16848: jdbc API throws an exception for user specified schema") { val schema = StructType(Seq( StructField("name", StringType, false), StructField("theid", IntegerType, false))) val parts = Array[String]("THEID < 2", "THEID >= 2") From 06881e800c7ddd1e72069ae1c055d1dbd1f38bdc Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 15 Jun 2017 16:17:57 +0800 Subject: [PATCH 03/17] Revert api. --- .../org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala | 6 +++--- .../main/scala/org/apache/spark/sql/types/Metadata.scala | 3 --- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 29ba38e58a61b..0342bd628d33d 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -218,9 +218,9 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo // custom schema can read data val schema = StructType(Seq( StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0), true, - new MetadataBuilder().putName("ID").build()), - StructField("N1", IntegerType, true, new MetadataBuilder().putName("N1").build()), - StructField("N2", BooleanType, true, new MetadataBuilder().putName("N2").build()))) + new MetadataBuilder().putString("name", "ID").build()), + StructField("N1", IntegerType, true, new MetadataBuilder().putString("name", "N1").build()), + StructField("N2", BooleanType, true, new MetadataBuilder().putString("name", "N2").build()))) val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "custom_column_types", new Properties()) val rows = dfRead.collect() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 8ea526e583dc8..3aa4bf619f274 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -273,9 +273,6 @@ class MetadataBuilder { /** Puts a [[Metadata]] array. */ def putMetadataArray(key: String, value: Array[Metadata]): this.type = put(key, value) - /** Puts a name. */ - def putName(name: String): this.type = put("name", name) - /** Builds the [[Metadata]] instance. */ def build(): Metadata = { new Metadata(map.toMap) From ffaee4280a9fb84d3aa5a6657423eb588c1c9609 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 23 Jun 2017 20:29:15 +0800 Subject: [PATCH 04/17] Improve custom schema. --- .../org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala | 7 +++---- .../spark/sql/execution/datasources/jdbc/JDBCRDD.scala | 4 ++-- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 0342bd628d33d..b0eb2960f0db0 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -217,10 +217,9 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo // custom schema can read data val schema = StructType(Seq( - StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0), true, - new MetadataBuilder().putString("name", "ID").build()), - StructField("N1", IntegerType, true, new MetadataBuilder().putString("name", "N1").build()), - StructField("N2", BooleanType, true, new MetadataBuilder().putString("name", "N2").build()))) + StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0)), + StructField("N1", IntegerType, true), + StructField("N2", BooleanType, true))) val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "custom_column_types", new Properties()) val rows = dfRead.collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 2bdc43254133e..81d2fcb5c6960 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -82,8 +82,8 @@ object JDBCRDD extends Logging { * @return A Catalyst schema corresponding to columns in the given order. */ private def pruneSchema(schema: StructType, columns: Array[String]): StructType = { - val fieldMap = Map(schema.fields.map(x => x.metadata.getString("name") -> x): _*) - new StructType(columns.map(name => fieldMap(name))) + val fieldMap = Map(schema.fields.map(x => x.name -> x): _*) + new StructType(columns.map(name => fieldMap(name))) } /** From a984f3b02f17a43c9f0333351a462f8722ece7fb Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 23 Jun 2017 20:30:40 +0800 Subject: [PATCH 05/17] Improve custom schema. --- .../apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 81d2fcb5c6960..c08e4762e10a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -82,8 +82,8 @@ object JDBCRDD extends Logging { * @return A Catalyst schema corresponding to columns in the given order. */ private def pruneSchema(schema: StructType, columns: Array[String]): StructType = { - val fieldMap = Map(schema.fields.map(x => x.name -> x): _*) - new StructType(columns.map(name => fieldMap(name))) + val fieldMap = Map(schema.fields.map(x => x.name -> x): _*) + new StructType(columns.map(name => fieldMap(name))) } /** From e0fc6b45a46532e02986dd3870bab4519f9f1d7e Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 4 Jul 2017 15:36:40 +0800 Subject: [PATCH 06/17] Throw exception if custom schema field names does not match table column names --- .../sql/jdbc/OracleIntegrationSuite.scala | 9 +++++++++ .../apache/spark/sql/DataFrameReader.scala | 2 +- .../datasources/jdbc/JDBCOptions.scala | 7 ++++++- .../datasources/jdbc/JDBCRelation.scala | 19 +++++++++++++++---- 4 files changed, 31 insertions(+), 6 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index b0eb2960f0db0..75eace2d287d0 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -235,6 +235,15 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo assert(values.getDecimal(0).equals(new java.math.BigDecimal("12312321321321312312312312123"))) assert(values.getInt(1).equals(1)) assert(values.getBoolean(2).equals(false)) + + // throw exception if custom schema field names does not match table column names + val wrongSchema = StructType(Seq( + StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0)), + StructField("N2", BooleanType, true))) + + intercept[IllegalArgumentException] { + spark.read.schema(wrongSchema).jdbc(jdbcUrl, "CUSTOM_COLUMN_TYPES", new Properties()).count() + }.getMessage.contains("Field ID,N2 does not match ID,N1,N2.") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 96452e154c24b..14f11394d5755 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -201,7 +201,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { this.extraOptions ++= properties.asScala // explicit url and dbtable should override all this.extraOptions += (JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) - if (!userSpecifiedSchema.isEmpty) { + if (userSpecifiedSchema.isDefined) { this.extraOptions += (JDBCOptions.JDBC_CUSTOM_SCHEMA -> userSpecifiedSchema.get.json) } format("jdbc").load() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 6bfc0404b2451..c8e87b3705ca0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -21,6 +21,7 @@ import java.sql.{Connection, DriverManager} import java.util.{Locale, Properties} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.types.StructType /** * Options for the JDBC data source. @@ -120,7 +121,11 @@ class JDBCOptions( // TODO: to reuse the existing partition parameters for those partition specific options val createTableOptions = parameters.getOrElse(JDBC_CREATE_TABLE_OPTIONS, "") val createTableColumnTypes = parameters.get(JDBC_CREATE_TABLE_COLUMN_TYPES) - val customSchema = parameters.get(JDBC_CUSTOM_SCHEMA) + val customSchema = parameters.get(JDBC_CUSTOM_SCHEMA) match { + case Some(value) => Some(StructType.fromString(value)) + case _ => None + } + val batchSize = { val size = parameters.getOrElse(JDBC_BATCH_INSERT_SIZE, "1000").toInt require(size >= 1, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index e1e588ade35ce..c4c3fc6d5809f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -110,10 +110,21 @@ private[sql] case class JDBCRelation( override val needConversion: Boolean = false - override val schema: StructType = if (!jdbcOptions.customSchema.isEmpty) { - StructType.fromString(jdbcOptions.customSchema.get) - } else { - JDBCRDD.resolveTable(jdbcOptions) + override val schema: StructType = { + val schema = JDBCRDD.resolveTable(jdbcOptions) + val customSchema = jdbcOptions.customSchema + if (customSchema.isDefined) { + val schemaFieldNames = schema.fieldNames.mkString(",") + val customSchemaFieldNames = customSchema.get.fieldNames.mkString(",") + if(schemaFieldNames.equals(customSchemaFieldNames)) { + customSchema.get + } else { + throw new IllegalArgumentException( + s"Field ${customSchemaFieldNames} does not match ${schemaFieldNames}.") + } + } else { + schema + } } // Check if JDBCRDD.compileFilter can accept input filters From 5fdd2bbdb2005f1de6cebf33126045cab4793d84 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 4 Jul 2017 15:58:24 +0800 Subject: [PATCH 07/17] Remove SPARK-16848: jdbc API throws an exception for user specified schema --- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 5f57526801a31..0066fc2d7cd4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -916,21 +916,6 @@ class JDBCSuite extends SparkFunSuite assert(new JDBCOptions(CaseInsensitiveMap(parameters)).asConnectionProperties.isEmpty) } - ignore("SPARK-16848: jdbc API throws an exception for user specified schema") { - val schema = StructType(Seq( - StructField("name", StringType, false), StructField("theid", IntegerType, false))) - val parts = Array[String]("THEID < 2", "THEID >= 2") - val e1 = intercept[AnalysisException] { - spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) - }.getMessage - assert(e1.contains("User specified schema not supported with `jdbc`")) - - val e2 = intercept[AnalysisException] { - spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) - }.getMessage - assert(e2.contains("User specified schema not supported with `jdbc`")) - } - test("SPARK-15648: teradataDialect StringType data mapping") { val teradataDialect = JdbcDialects.get("jdbc:teradata://127.0.0.1/db") assert(teradataDialect.getJDBCType(StringType). From 87df01499db9a4a5e81146ba54d3be97dd615628 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 16 Aug 2017 18:13:16 +0800 Subject: [PATCH 08/17] Support DDL-like JDBC custom schema. --- .../sql/jdbc/OracleIntegrationSuite.scala | 52 +++++++++++++------ .../apache/spark/sql/DataFrameReader.scala | 4 +- .../execution/datasources/DataSource.scala | 8 ++- .../datasources/jdbc/JDBCRelation.scala | 12 ++--- 4 files changed, 51 insertions(+), 25 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index ca497a6e6fe79..68533e7e6d745 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -21,7 +21,7 @@ import java.sql.{Connection, Date, Timestamp} import java.util.Properties import java.math.BigDecimal -import org.apache.spark.sql.Row +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest @@ -79,9 +79,9 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo conn.commit() conn.prepareStatement( - "CREATE TABLE custom_column_types (id NUMBER, n1 number(1), n2 number(1))").executeUpdate() + "CREATE TABLE tableWithCustomSchema (id NUMBER, n1 number(1), n2 number(1))").executeUpdate() conn.prepareStatement( - "INSERT INTO custom_column_types values(12312321321321312312312312123, 1, 0)").executeUpdate() + "INSERT INTO tableWithCustomSchema values(12312321321321312312312312123, 1, 0)").executeUpdate() conn.commit() sql( @@ -276,11 +276,12 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo assert(row.getTimestamp(1).equals(timestampVal)) } - test("SPARK-20427/SPARK-20921: read table use custom schema") { + test("SPARK-20427/SPARK-20921: read table use custom schema by jdbc api") { + val props = new Properties() // default will throw IllegalArgumentException val e = intercept[org.apache.spark.SparkException] { - spark.read.jdbc(jdbcUrl, "custom_column_types", new Properties()).collect() + spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", props).collect() } assert(e.getMessage.contains( "requirement failed: Decimal precision 39 exceeds max precision 38")) @@ -291,9 +292,36 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo StructField("N1", IntegerType, true), StructField("N2", BooleanType, true))) - val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "custom_column_types", new Properties()) - val rows = dfRead.collect() + val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "tableWithCustomSchema", props) + verify(dfRead) + + // throw exception if custom schema field names does not match table column names + val wrongSchema = StructType(Seq( + StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0)), + StructField("N2", BooleanType, true))) + intercept[IllegalArgumentException] { + spark.read.schema(wrongSchema).jdbc(jdbcUrl, "tableWithCustomSchema", props).count() + }.getMessage.contains("Field ID,N2 does not match ID,N1,N2.") + } + + test("SPARK-20427/SPARK-20921: read table use custom schema by DDL-like") { + sql( + s""" + |CREATE TEMPORARY VIEW tableWithCustomSchema ( + | ID decimal(38, 0), + | N1 int, + | N2 boolean + |) + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$jdbcUrl', dbTable 'tableWithCustomSchema') + """.stripMargin.replaceAll("\n", " ")) + + verify(sql("select * from tableWithCustomSchema")) + } + + def verify(df: DataFrame): Unit = { + val rows = df.collect() // verify the data type inserted val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types(0).equals("class java.math.BigDecimal")) @@ -305,14 +333,6 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo assert(values.getDecimal(0).equals(new java.math.BigDecimal("12312321321321312312312312123"))) assert(values.getInt(1).equals(1)) assert(values.getBoolean(2).equals(false)) - - // throw exception if custom schema field names does not match table column names - val wrongSchema = StructType(Seq( - StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0)), - StructField("N2", BooleanType, true))) - - intercept[IllegalArgumentException] { - spark.read.schema(wrongSchema).jdbc(jdbcUrl, "CUSTOM_COLUMN_TYPES", new Properties()).count() - }.getMessage.contains("Field ID,N2 does not match ID,N1,N2.") } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 63e8803a4f713..0cfc349af3a81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -270,7 +270,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { table: String, predicates: Array[String], connectionProperties: Properties): DataFrame = { - assertNoSpecifiedSchema("jdbc") + if (userSpecifiedSchema.isDefined) { + this.extraOptions += (JDBCOptions.JDBC_CUSTOM_SCHEMA -> userSpecifiedSchema.get.json) + } // connectionProperties should override settings in extraOptions. val params = extraOptions.toMap ++ connectionProperties.asScala.toMap val options = new JDBCOptions(url, table, params) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 75e530607570f..0b4d17865767b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat -import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcRelationProvider} import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming._ @@ -86,7 +86,11 @@ case class DataSource( lazy val providingClass: Class[_] = DataSource.lookupDataSource(className) lazy val sourceInfo: SourceInfo = sourceSchema() - private val caseInsensitiveOptions = CaseInsensitiveMap(options) + private val caseInsensitiveOptions = if (userSpecifiedSchema.isDefined) { + CaseInsensitiveMap(options + (JDBCOptions.JDBC_CUSTOM_SCHEMA -> userSpecifiedSchema.get.json)) + } else { + CaseInsensitiveMap(options) + } /** * Get the schema of the given FileFormat, if provided by `userSpecifiedSchema`, or try to infer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index d8998b413b2e2..7bb7b9daa7854 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -23,7 +23,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.Partition import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession, SQLContext} -import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.jdbc.JdbcDialects import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -111,17 +110,18 @@ private[sql] case class JDBCRelation( override val needConversion: Boolean = false + // This is resolved by names, only check the column names. override val schema: StructType = { val schema = JDBCRDD.resolveTable(jdbcOptions) val customSchema = jdbcOptions.customSchema if (customSchema.isDefined) { - val schemaFieldNames = schema.fieldNames.mkString(",") - val customSchemaFieldNames = customSchema.get.fieldNames.mkString(",") - if(schemaFieldNames.equals(customSchemaFieldNames)) { + val fieldNames = schema.fieldNames.mkString(",") + val customFieldNames = customSchema.get.fieldNames.mkString(",") + if ((sqlContext.conf.caseSensitiveAnalysis && fieldNames.equals(customFieldNames)) || + fieldNames.toLowerCase.equals(customFieldNames.toLowerCase)) { customSchema.get } else { - throw new IllegalArgumentException( - s"Field ${customSchemaFieldNames} does not match ${schemaFieldNames}.") + throw new IllegalArgumentException(s"Field $customFieldNames does not match $fieldNames.") } } else { schema From 247fc78a13c10f1def3840a7d79e9589993862c7 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 23 Aug 2017 12:11:05 +0800 Subject: [PATCH 09/17] Refactor code. --- docs/sql-programming-guide.md | 49 ++++++------ examples/src/main/python/sql/datasource.py | 6 ++ .../examples/sql/SQLDataSourceExample.scala | 4 + .../sql/jdbc/OracleIntegrationSuite.scala | 50 +++---------- .../apache/spark/sql/DataFrameReader.scala | 18 +++-- .../execution/datasources/DataSource.scala | 8 +- .../datasources/jdbc/JDBCOptions.scala | 7 +- .../datasources/jdbc/JDBCRelation.scala | 15 +--- .../datasources/jdbc/JdbcUtils.scala | 20 +++++ .../datasources/jdbc/JdbcUtilsSuite.scala | 74 +++++++++++++++++++ .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 54 ++++++++++++++ 11 files changed, 215 insertions(+), 90 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index ee231a934a3af..022e2554014d6 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -741,24 +741,24 @@ columns, `gender` and `country` as partitioning columns: {% highlight text %} path -└── to - └── table - ├── gender=male - │   ├── ... - │   │ - │   ├── country=US - │   │   └── data.parquet - │   ├── country=CN - │   │   └── data.parquet - │   └── ... - └── gender=female -    ├── ... -    │ -    ├── country=US -    │   └── data.parquet -    ├── country=CN -    │   └── data.parquet -    └── ... +????????? to + ????????? table + ????????? gender=male + ??????? ????????? ... + ??????? ??? + ??????? ????????? country=US + ??????? ??????? ????????? data.parquet + ??????? ????????? country=CN + ??????? ??????? ????????? data.parquet + ??????? ????????? ... + ????????? gender=female + ???? ????????? ... + ???? ??? + ???? ????????? country=US + ???? ??????? ????????? data.parquet + ???? ????????? country=CN + ???? ??????? ????????? data.parquet + ???? ????????? ... {% endhighlight %} @@ -1334,7 +1334,14 @@ the following case-insensitive options: The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. - + + + + customDataFrameColumnTypes + + The DataFrame column data types to use instead of the defaults when reading data from jdbc API. (e.g: "id decimal(38, 0), name string"). The specified types should be valid spark sql data types. This option applies only to reading. + +
@@ -1571,7 +1578,7 @@ options. `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in - Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + Python and R is not a language feature, the concept of Dataset does not apply to these languages??? APIs. Instead, `DataFrame` remains the primary programing abstraction, which is analogous to the single-node data frame notion in these languages. @@ -1828,7 +1835,7 @@ Spark SQL supports the vast majority of Hive features, such as: * `CLUSTER BY` * `SORT BY` * All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Relational operators (`=`, `???`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) * Logical operators (`AND`, `&&`, `OR`, `||`, etc) * Complex type constructors diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 8777cca66bfe9..04606dcd4a417 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -177,6 +177,12 @@ def jdbc_dataset_example(spark): .jdbc("jdbc:postgresql:dbserver", "schema.tablename", properties={"user": "username", "password": "password"}) + # Specifying dataframe column data types on read + jdbcDF3 = spark.read \ + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", + properties={"user": "username", "password": "password", + "customDataFrameColumnTypes": "id decimal(38, 0), name string"}) + # Saving data to a JDBC source jdbcDF.write \ .format("jdbc") \ diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 6ff03bdb22129..923711886a5b0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -185,6 +185,10 @@ object SQLDataSourceExample { connectionProperties.put("password", "password") val jdbcDF2 = spark.read .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) + // Specifying dataframe column data types on read + connectionProperties.put("customDataFrameColumnTypes", "id decimal(38, 0), name string") + val jdbcDF3 = spark.read + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) // Saving data to a JDBC source jdbcDF.write diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index aff027f8b6e2d..8da544a16925b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -104,7 +104,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo } - test("SPARK-16625 : Importing Oracle numeric types") { + test("SPARK-16625 : Importing Oracle numeric types") { val df = sqlContext.read.jdbc(jdbcUrl, "numerics", new Properties); val rows = df.collect() assert(rows.size == 1) @@ -274,62 +274,30 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo } test("SPARK-20427/SPARK-20921: read table use custom schema by jdbc api") { - - val props = new Properties() // default will throw IllegalArgumentException val e = intercept[org.apache.spark.SparkException] { - spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", props).collect() + spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", new Properties()).collect() } assert(e.getMessage.contains( "requirement failed: Decimal precision 39 exceeds max precision 38")) // custom schema can read data - val schema = StructType(Seq( - StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0)), - StructField("N1", IntegerType, true), - StructField("N2", BooleanType, true))) - - val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "tableWithCustomSchema", props) - verify(dfRead) - - // throw exception if custom schema field names does not match table column names - val wrongSchema = StructType(Seq( - StructField("ID", DecimalType(DecimalType.MAX_PRECISION, 0)), - StructField("N2", BooleanType, true))) - - intercept[IllegalArgumentException] { - spark.read.schema(wrongSchema).jdbc(jdbcUrl, "tableWithCustomSchema", props).count() - }.getMessage.contains("Field ID,N2 does not match ID,N1,N2.") - } - - test("SPARK-20427/SPARK-20921: read table use custom schema by DDL-like") { - sql( - s""" - |CREATE TEMPORARY VIEW tableWithCustomSchema ( - | ID decimal(38, 0), - | N1 int, - | N2 boolean - |) - |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$jdbcUrl', dbTable 'tableWithCustomSchema') - """.stripMargin.replaceAll("\n", " ")) - - verify(sql("select * from tableWithCustomSchema")) - } + val props = new Properties() + props.put("customDataFrameColumnTypes", + s"ID decimal(${DecimalType.MAX_PRECISION}, 0), N1 int, N2 boolean") + val dfRead = spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", props) - def verify(df: DataFrame): Unit = { - val rows = df.collect() - // verify the data type inserted + val rows = dfRead.collect() + // verify the data type val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types(0).equals("class java.math.BigDecimal")) assert(types(1).equals("class java.lang.Integer")) assert(types(2).equals("class java.lang.Boolean")) - // verify the value inserted + // verify the value val values = rows(0) assert(values.getDecimal(0).equals(new java.math.BigDecimal("12312321321321312312312312123"))) assert(values.getInt(1).equals(1)) assert(values.getBoolean(2).equals(false)) } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index d919e004a7d57..b50902c198067 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -197,13 +197,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def jdbc(url: String, table: String, properties: Properties): DataFrame = { + assertJdbcAPISpecifiedDataFrameSchema() // properties should override settings in extraOptions. this.extraOptions ++= properties.asScala // explicit url and dbtable should override all this.extraOptions += (JDBCOptions.JDBC_URL -> url, JDBCOptions.JDBC_TABLE_NAME -> table) - if (userSpecifiedSchema.isDefined) { - this.extraOptions += (JDBCOptions.JDBC_CUSTOM_SCHEMA -> userSpecifiedSchema.get.json) - } format("jdbc").load() } @@ -270,9 +268,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { table: String, predicates: Array[String], connectionProperties: Properties): DataFrame = { - if (userSpecifiedSchema.isDefined) { - this.extraOptions += (JDBCOptions.JDBC_CUSTOM_SCHEMA -> userSpecifiedSchema.get.json) - } + assertJdbcAPISpecifiedDataFrameSchema() // connectionProperties should override settings in extraOptions. val params = extraOptions.toMap ++ connectionProperties.asScala.toMap val options = new JDBCOptions(url, table, params) @@ -682,6 +678,16 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } } + /** + * A convenient function for validate specified column types schema in jdbc API. + */ + private def assertJdbcAPISpecifiedDataFrameSchema(): Unit = { + if (userSpecifiedSchema.nonEmpty) { + throw new AnalysisException("Please use customDataFrameColumnTypes option to " + + "specified column types.") + } + } + /** * A convenient function for schema validation in datasources supporting * `columnNameOfCorruptRecord` as an option. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 04e990ce2d539..567ff49773f9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat -import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcRelationProvider} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming._ @@ -87,11 +87,7 @@ case class DataSource( lazy val providingClass: Class[_] = DataSource.lookupDataSource(className) lazy val sourceInfo: SourceInfo = sourceSchema() - private val caseInsensitiveOptions = if (userSpecifiedSchema.isDefined) { - CaseInsensitiveMap(options + (JDBCOptions.JDBC_CUSTOM_SCHEMA -> userSpecifiedSchema.get.json)) - } else { - CaseInsensitiveMap(options) - } + private val caseInsensitiveOptions = CaseInsensitiveMap(options) private val equality = sparkSession.sessionState.conf.resolver bucketSpec.map { bucket => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 4f48f5cd14f7c..f777307aaa3b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -124,10 +124,7 @@ class JDBCOptions( // TODO: to reuse the existing partition parameters for those partition specific options val createTableOptions = parameters.getOrElse(JDBC_CREATE_TABLE_OPTIONS, "") val createTableColumnTypes = parameters.get(JDBC_CREATE_TABLE_COLUMN_TYPES) - val customSchema = parameters.get(JDBC_CUSTOM_SCHEMA) match { - case Some(value) => Some(StructType.fromString(value)) - case _ => None - } + val customDataFrameColumnTypes = parameters.get(JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES) val batchSize = { val size = parameters.getOrElse(JDBC_BATCH_INSERT_SIZE, "1000").toInt @@ -167,7 +164,7 @@ object JDBCOptions { val JDBC_TRUNCATE = newOption("truncate") val JDBC_CREATE_TABLE_OPTIONS = newOption("createTableOptions") val JDBC_CREATE_TABLE_COLUMN_TYPES = newOption("createTableColumnTypes") - val JDBC_CUSTOM_SCHEMA = newOption("customSchema") + val JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES = newOption("customDataFrameColumnTypes") val JDBC_BATCH_INSERT_SIZE = newOption("batchsize") val JDBC_TXN_ISOLATION_LEVEL = newOption("isolationLevel") val JDBC_SESSION_INIT_STATEMENT = newOption("sessionInitStatement") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index a14a8d5fbc282..673658b117884 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -111,19 +111,12 @@ private[sql] case class JDBCRelation( override val needConversion: Boolean = false - // This is resolved by names, only check the column names. override val schema: StructType = { val schema = JDBCRDD.resolveTable(jdbcOptions) - val customSchema = jdbcOptions.customSchema - if (customSchema.isDefined) { - val fieldNames = schema.fieldNames.mkString(",") - val customFieldNames = customSchema.get.fieldNames.mkString(",") - if ((sqlContext.conf.caseSensitiveAnalysis && fieldNames.equals(customFieldNames)) || - fieldNames.toLowerCase.equals(customFieldNames.toLowerCase)) { - customSchema.get - } else { - throw new IllegalArgumentException(s"Field $customFieldNames does not match $fieldNames.") - } + val customDataFrameColumnTypes = jdbcOptions.customDataFrameColumnTypes + if (customDataFrameColumnTypes.isDefined) { + JdbcUtils.parseUserSpecifiedColumnTypes(schema, customDataFrameColumnTypes.get, + sqlContext.sessionState.conf.resolver) } else { schema } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index bbe9024f13a44..f5a0d6623a1df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -29,6 +29,7 @@ import org.apache.spark.executor.InputMetrics import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -767,6 +768,25 @@ object JdbcUtils extends Logging { if (isCaseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) } + /** + * Parses the user specified customDataFrameColumnTypes option value string, and returns + */ + def parseUserSpecifiedColumnTypes( + schema: StructType, + columnTypes: String, + nameEquality: Resolver): StructType = { + val userSchema = CatalystSqlParser.parseTableSchema(columnTypes) + // This is resolved by names, only check the column names. + userSchema.fieldNames.foreach { col => + schema.find(f => nameEquality(f.name, col)).getOrElse { + throw new AnalysisException( + s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column $col not found in " + + s"schema ${schema.catalogString}") + } + } + userSchema + } + /** * Saves the RDD to the database in a single transaction. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala new file mode 100644 index 0000000000000..dec1e53f4ca5a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala @@ -0,0 +1,74 @@ +/* + * 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.execution.datasources.jdbc + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.types._ + +class JdbcUtilsSuite extends SparkFunSuite { + + val schema = StructType(Seq( + StructField("C1", StringType, false), StructField("C2", IntegerType, false))) + val caseSensitive = org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + val caseInsensitive = org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution + + test("Parse user specified column types") { + assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 string", caseInsensitive) === + StructType(Seq(StructField("C1", StringType, true)))) + assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C2 string", caseInsensitive) === + StructType(Seq(StructField("C2", StringType, true)))) + assert( + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 date, C2 string", caseInsensitive) === + StructType(Seq(StructField("C1", DateType, true), StructField("C2", StringType, true)))) + assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 date, C2 string", caseSensitive) === + StructType(Seq(StructField("C1", DateType, true), StructField("C2", StringType, true)))) + assert( + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, C2 string", caseInsensitive) === + StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true)))) + assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 decimal(38, 0)", caseInsensitive) === + StructType(Seq(StructField("c1", DecimalType(38, 0), true)))) + + // Throw AnalysisException + val exception1 = intercept[AnalysisException]{ + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, C2 string", caseSensitive) === + StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true))) + } + assert(exception1.getMessage.contains( + s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c1 not found in schema")) + val exception2 = intercept[AnalysisException]{ + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 date, C2 string", caseInsensitive) === + StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) + } + assert(exception2.getMessage.contains( + s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c3 not found in schema")) + + // Throw ParseException + val exception3 = intercept[ParseException]{ + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 datee, C2 string", caseInsensitive) === + StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) + } + assert(exception3.getMessage.contains("DataType datee is not supported")) + val exception4 = intercept[ParseException]{ + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 date. C2 string", caseInsensitive) === + StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) + } + assert(exception4.getMessage.contains("mismatched input '.' expecting")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 6dea85b60f1b1..cc88950daa87c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -953,6 +953,60 @@ class JDBCSuite extends SparkFunSuite assert(new JDBCOptions(CaseInsensitiveMap(parameters)).asConnectionProperties.isEmpty) } + test("SPARK-16848: jdbc API throws an exception for user specified schema") { + val schema = StructType(Seq( + StructField("name", StringType, false), StructField("theid", IntegerType, false))) + val parts = Array[String]("THEID < 2", "THEID >= 2") + val e1 = intercept[AnalysisException] { + spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) + }.getMessage + assert(e1.contains("Please use options to specified schema")) + + val e2 = intercept[AnalysisException] { + spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) + }.getMessage + assert(e2.contains("Please use options to specified schema")) + } + + test("jdbc API support custom schema") { + val parts = Array[String]("THEID < 2", "THEID >= 2") + val props = new Properties() + props.put("customDataFrameColumnTypes", "NAME string, THEID bigint") + val schema = StructType(Seq( + StructField("NAME", StringType, true), StructField("THEID", LongType, true))) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, props) + assert(df.schema.size === 2) + assert(df.schema === schema) + assert(df.count() === 3) + } + + test("jdbc API can reduce column by custom schema") { + val props = new Properties() + props.put("customDataFrameColumnTypes", "NAME string") + val schema = StructType(Seq(StructField("NAME", StringType, true))) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", props) + assert(df.schema.size === 1) + assert(df.schema === schema) + assert(df.count() === 3) + } + + test("jdbc API custom schema DDL-like strings.") { + withTempView("people_view") { + sql( + s""" + |CREATE TEMPORARY VIEW people_view + |USING org.apache.spark.sql.jdbc + |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass', + |customDataFrameColumnTypes 'NAME STRING') + """.stripMargin.replaceAll("\n", " ")) + val schema = StructType(Seq(StructField("NAME", StringType, true))) + val df = sql("select * from people_view") + assert(df.schema.size === 1) + assert(df.schema === schema) + assert(df.count() === 3) + } + } + test("SPARK-15648: teradataDialect StringType data mapping") { val teradataDialect = JdbcDialects.get("jdbc:teradata://127.0.0.1/db") assert(teradataDialect.getJDBCType(StringType). From b8b03e2e022c9440a4778a7bed2c88d2ac6aaa11 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 23 Aug 2017 12:46:32 +0800 Subject: [PATCH 10/17] Revert wrong change --- docs/sql-programming-guide.md | 40 +++++++++++++++++------------------ 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 022e2554014d6..deaefca4fa6b2 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -741,24 +741,24 @@ columns, `gender` and `country` as partitioning columns: {% highlight text %} path -????????? to - ????????? table - ????????? gender=male - ??????? ????????? ... - ??????? ??? - ??????? ????????? country=US - ??????? ??????? ????????? data.parquet - ??????? ????????? country=CN - ??????? ??????? ????????? data.parquet - ??????? ????????? ... - ????????? gender=female - ???? ????????? ... - ???? ??? - ???? ????????? country=US - ???? ??????? ????????? data.parquet - ???? ????????? country=CN - ???? ??????? ????????? data.parquet - ???? ????????? ... +└── to + └── table + ├── gender=male + │   ├── ... + │   │ + │   ├── country=US + │   │   └── data.parquet + │   ├── country=CN + │   │   └── data.parquet + │   └── ... + └── gender=female +    ├── ... +    │ +    ├── country=US +    │   └── data.parquet +    ├── country=CN +    │   └── data.parquet +    └── ... {% endhighlight %} @@ -1578,7 +1578,7 @@ options. `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in - Python and R is not a language feature, the concept of Dataset does not apply to these languages??? + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ APIs. Instead, `DataFrame` remains the primary programing abstraction, which is analogous to the single-node data frame notion in these languages. @@ -1835,7 +1835,7 @@ Spark SQL supports the vast majority of Hive features, such as: * `CLUSTER BY` * `SORT BY` * All Hive operators, including: - * Relational operators (`=`, `???`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) * Logical operators (`AND`, `&&`, `OR`, `||`, etc) * Complex type constructors From b040c72413de5689a61488312fce04315852374a Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 23 Aug 2017 13:49:34 +0800 Subject: [PATCH 11/17] Fix python style --- examples/src/main/python/sql/datasource.py | 10 +++++++--- .../spark/sql/jdbc/OracleIntegrationSuite.scala | 15 +++++++++++---- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 04606dcd4a417..b7fdf50ec03de 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -179,9 +179,13 @@ def jdbc_dataset_example(spark): # Specifying dataframe column data types on read jdbcDF3 = spark.read \ - .jdbc("jdbc:postgresql:dbserver", "schema.tablename", - properties={"user": "username", "password": "password", - "customDataFrameColumnTypes": "id decimal(38, 0), name string"}) + .format("jdbc") \ + .option("url", "jdbc:postgresql:dbserver") \ + .option("dbtable", "schema.tablename") \ + .option("user", "username") \ + .option("password", "password") \ + .option("customDataFrameColumnTypes", "id decimal(38, 0), name string") \ + .load() # Saving data to a JDBC source jdbcDF.write \ diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 8da544a16925b..252cf730886e4 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -72,10 +72,17 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo """.stripMargin.replaceAll("\n", " ")).executeUpdate() conn.commit() - conn.prepareStatement("CREATE TABLE ts_with_timezone (id NUMBER(10), t TIMESTAMP WITH TIME ZONE)") - .executeUpdate() - conn.prepareStatement("INSERT INTO ts_with_timezone VALUES (1, to_timestamp_tz('1999-12-01 11:00:00 UTC','YYYY-MM-DD HH:MI:SS TZR'))") - .executeUpdate() + conn.prepareStatement( + "CREATE TABLE ts_with_timezone (id NUMBER(10), t TIMESTAMP WITH TIME ZONE)").executeUpdate() + conn.prepareStatement( + "INSERT INTO ts_with_timezone VALUES " + + "(1, to_timestamp_tz('1999-12-01 11:00:00 UTC','YYYY-MM-DD HH:MI:SS TZR'))").executeUpdate() + conn.commit() + + conn.prepareStatement( + "CREATE TABLE tableWithCustomSchema (id NUMBER, n1 number(1), n2 number(1))").executeUpdate() + conn.prepareStatement( + "INSERT INTO tableWithCustomSchema values(12312321321321312312312312123, 1, 0)").executeUpdate() conn.commit() sql( From 2ea56fc4e145de462f767a1c347b24d4e85f87d5 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 23 Aug 2017 17:42:30 +0800 Subject: [PATCH 12/17] Fix test error --- .../src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index cc88950daa87c..38b1c3fb85869 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -960,12 +960,12 @@ class JDBCSuite extends SparkFunSuite val e1 = intercept[AnalysisException] { spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) }.getMessage - assert(e1.contains("Please use options to specified schema")) + assert(e1.contains("Please use customDataFrameColumnTypes option to specified column types.")) val e2 = intercept[AnalysisException] { spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) }.getMessage - assert(e2.contains("Please use options to specified schema")) + assert(e2.contains("Please use customDataFrameColumnTypes option to specified column types.")) } test("jdbc API support custom schema") { From 1e2c1d9f891217342679942cd93769ccac70af02 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 24 Aug 2017 22:17:49 +0800 Subject: [PATCH 13/17] Let user specified all the columns. --- examples/src/main/python/sql/datasource.py | 2 +- .../examples/sql/SQLDataSourceExample.scala | 2 +- .../datasources/jdbc/JdbcUtils.scala | 12 +++++- .../datasources/jdbc/JdbcUtilsSuite.scala | 41 ++++++++++++------- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 17 ++------ 5 files changed, 44 insertions(+), 30 deletions(-) diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index b7fdf50ec03de..36c91fa33f5f8 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -184,7 +184,7 @@ def jdbc_dataset_example(spark): .option("dbtable", "schema.tablename") \ .option("user", "username") \ .option("password", "password") \ - .option("customDataFrameColumnTypes", "id decimal(38, 0), name string") \ + .option("customDataFrameColumnTypes", "id DECIMAL(38, 0), name STRING") \ .load() # Saving data to a JDBC source diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 923711886a5b0..04921b74f8e19 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -186,7 +186,7 @@ object SQLDataSourceExample { val jdbcDF2 = spark.read .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) // Specifying dataframe column data types on read - connectionProperties.put("customDataFrameColumnTypes", "id decimal(38, 0), name string") + connectionProperties.put("customDataFrameColumnTypes", "id DECIMAL(38, 0), name STRING") val jdbcDF3 = spark.read .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index f5a0d6623a1df..3d1feb24d83d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -769,13 +769,23 @@ object JdbcUtils extends Logging { } /** - * Parses the user specified customDataFrameColumnTypes option value string, and returns + * Parses the user specified customDataFrameColumnTypes option value to DataFrame schema, + * and returns it if it's all columns are equals to default schema's. */ def parseUserSpecifiedColumnTypes( schema: StructType, columnTypes: String, nameEquality: Resolver): StructType = { val userSchema = CatalystSqlParser.parseTableSchema(columnTypes) + + SchemaUtils.checkColumnNameDuplication( + userSchema.map(_.name), "in the createTableColumnTypes option value", nameEquality) + + if (userSchema.size != schema.size) { + throw new AnalysisException("Please provide all the columns, " + + s"all columns are: ${schema.fields.map(_.name).mkString(",")}") + } + // This is resolved by names, only check the column names. userSchema.fieldNames.foreach { col => schema.find(f => nameEquality(f.name, col)).getOrElse { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala index dec1e53f4ca5a..89419127b00da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala @@ -30,10 +30,6 @@ class JdbcUtilsSuite extends SparkFunSuite { val caseInsensitive = org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution test("Parse user specified column types") { - assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 string", caseInsensitive) === - StructType(Seq(StructField("C1", StringType, true)))) - assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C2 string", caseInsensitive) === - StructType(Seq(StructField("C2", StringType, true)))) assert( JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 date, C2 string", caseInsensitive) === StructType(Seq(StructField("C1", DateType, true), StructField("C2", StringType, true)))) @@ -42,33 +38,50 @@ class JdbcUtilsSuite extends SparkFunSuite { assert( JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, C2 string", caseInsensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true)))) - assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 decimal(38, 0)", caseInsensitive) === - StructType(Seq(StructField("c1", DecimalType(38, 0), true)))) + assert(JdbcUtils.parseUserSpecifiedColumnTypes( + schema, "c1 decimal(38, 0), C2 string", caseInsensitive) === + StructType(Seq(StructField("c1", DecimalType(38, 0), true), + StructField("C2", StringType, true)))) // Throw AnalysisException - val exception1 = intercept[AnalysisException]{ + val duplicate = intercept[AnalysisException]{ + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, c1 string", caseInsensitive) === + StructType(Seq(StructField("c1", DateType, true), StructField("c1", StringType, true))) + } + assert(duplicate.getMessage.contains( + "Found duplicate column(s) in the createTableColumnTypes option value")) + + val allColumns = intercept[AnalysisException]{ + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 string", caseSensitive) === + StructType(Seq(StructField("C1", DateType, true))) + } + assert(allColumns.getMessage.contains("Please provide all the columns,")) + + val caseSensitiveColumnNotFound = intercept[AnalysisException]{ JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, C2 string", caseSensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true))) } - assert(exception1.getMessage.contains( + assert(caseSensitiveColumnNotFound.getMessage.contains( s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c1 not found in schema")) - val exception2 = intercept[AnalysisException]{ + + val caseInsensitiveColumnNotFound = intercept[AnalysisException]{ JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 date, C2 string", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } - assert(exception2.getMessage.contains( + assert(caseInsensitiveColumnNotFound.getMessage.contains( s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c3 not found in schema")) // Throw ParseException - val exception3 = intercept[ParseException]{ + val DataTypeNotSupported = intercept[ParseException]{ JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 datee, C2 string", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } - assert(exception3.getMessage.contains("DataType datee is not supported")) - val exception4 = intercept[ParseException]{ + assert(DataTypeNotSupported.getMessage.contains("DataType datee is not supported")) + + val mismatchedInput = intercept[ParseException]{ JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 date. C2 string", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } - assert(exception4.getMessage.contains("mismatched input '.' expecting")) + assert(mismatchedInput.getMessage.contains("mismatched input '.' expecting")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 38b1c3fb85869..24135e31df39f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -980,16 +980,6 @@ class JDBCSuite extends SparkFunSuite assert(df.count() === 3) } - test("jdbc API can reduce column by custom schema") { - val props = new Properties() - props.put("customDataFrameColumnTypes", "NAME string") - val schema = StructType(Seq(StructField("NAME", StringType, true))) - val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", props) - assert(df.schema.size === 1) - assert(df.schema === schema) - assert(df.count() === 3) - } - test("jdbc API custom schema DDL-like strings.") { withTempView("people_view") { sql( @@ -997,11 +987,12 @@ class JDBCSuite extends SparkFunSuite |CREATE TEMPORARY VIEW people_view |USING org.apache.spark.sql.jdbc |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass', - |customDataFrameColumnTypes 'NAME STRING') + |customDataFrameColumnTypes 'NAME string, THEID int') """.stripMargin.replaceAll("\n", " ")) - val schema = StructType(Seq(StructField("NAME", StringType, true))) + val schema = StructType( + Seq(StructField("NAME", StringType, true), StructField("THEID", IntegerType, true))) val df = sql("select * from people_view") - assert(df.schema.size === 1) + assert(df.schema.size === 2) assert(df.schema === schema) assert(df.count() === 3) } From b38a1a8b2d9ffee250b9e8637dc579f2a8f9182d Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 8 Sep 2017 11:58:38 +0800 Subject: [PATCH 14/17] Change all SQL type to upper case. --- docs/sql-programming-guide.md | 2 +- .../sql/jdbc/OracleIntegrationSuite.scala | 4 ++-- .../datasources/jdbc/JdbcUtilsSuite.scala | 24 +++++++++---------- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 ++-- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index deaefca4fa6b2..e3bbd7c3c41a3 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1339,7 +1339,7 @@ the following case-insensitive options: customDataFrameColumnTypes - The DataFrame column data types to use instead of the defaults when reading data from jdbc API. (e.g: "id decimal(38, 0), name string"). The specified types should be valid spark sql data types. This option applies only to reading. + The DataFrame column data types to use instead of the defaults when reading data from jdbc API. (e.g: "id DECIMAL(38, 0), name STRING"). The specified types should be valid spark sql data types. This option applies only to reading. diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 252cf730886e4..6c65da8698308 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -80,7 +80,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo conn.commit() conn.prepareStatement( - "CREATE TABLE tableWithCustomSchema (id NUMBER, n1 number(1), n2 number(1))").executeUpdate() + "CREATE TABLE tableWithCustomSchema (id NUMBER, n1 NUMBER(1), n2 NUMBER(1))").executeUpdate() conn.prepareStatement( "INSERT INTO tableWithCustomSchema values(12312321321321312312312312123, 1, 0)").executeUpdate() conn.commit() @@ -291,7 +291,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo // custom schema can read data val props = new Properties() props.put("customDataFrameColumnTypes", - s"ID decimal(${DecimalType.MAX_PRECISION}, 0), N1 int, N2 boolean") + s"ID DECIMAL(${DecimalType.MAX_PRECISION}, 0), N1 INT, N2 BOOLEAN") val dfRead = spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", props) val rows = dfRead.collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala index 89419127b00da..8ecf3038c9e1f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala @@ -31,55 +31,55 @@ class JdbcUtilsSuite extends SparkFunSuite { test("Parse user specified column types") { assert( - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 date, C2 string", caseInsensitive) === + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 DATE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("C1", DateType, true), StructField("C2", StringType, true)))) - assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 date, C2 string", caseSensitive) === + assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 DATE, C2 STRING", caseSensitive) === StructType(Seq(StructField("C1", DateType, true), StructField("C2", StringType, true)))) assert( - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, C2 string", caseInsensitive) === + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 DATE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true)))) assert(JdbcUtils.parseUserSpecifiedColumnTypes( - schema, "c1 decimal(38, 0), C2 string", caseInsensitive) === + schema, "c1 DECIMAL(38, 0), C2 STRING", caseInsensitive) === StructType(Seq(StructField("c1", DecimalType(38, 0), true), StructField("C2", StringType, true)))) // Throw AnalysisException val duplicate = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, c1 string", caseInsensitive) === + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 DATE, c1 STRING", caseInsensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("c1", StringType, true))) } assert(duplicate.getMessage.contains( "Found duplicate column(s) in the createTableColumnTypes option value")) val allColumns = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 string", caseSensitive) === + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 STRING", caseSensitive) === StructType(Seq(StructField("C1", DateType, true))) } assert(allColumns.getMessage.contains("Please provide all the columns,")) val caseSensitiveColumnNotFound = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 date, C2 string", caseSensitive) === + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 DATE, C2 STRING", caseSensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true))) } assert(caseSensitiveColumnNotFound.getMessage.contains( s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c1 not found in schema")) val caseInsensitiveColumnNotFound = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 date, C2 string", caseInsensitive) === + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 DATE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } assert(caseInsensitiveColumnNotFound.getMessage.contains( s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c3 not found in schema")) // Throw ParseException - val DataTypeNotSupported = intercept[ParseException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 datee, C2 string", caseInsensitive) === + val dataTypeNotSupported = intercept[ParseException]{ + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 DATEE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } - assert(DataTypeNotSupported.getMessage.contains("DataType datee is not supported")) + assert(dataTypeNotSupported.getMessage.contains("DataType datee is not supported")) val mismatchedInput = intercept[ParseException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 date. C2 string", caseInsensitive) === + JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 DATE. C2 STRING", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } assert(mismatchedInput.getMessage.contains("mismatched input '.' expecting")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 24135e31df39f..219354bd696be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -971,7 +971,7 @@ class JDBCSuite extends SparkFunSuite test("jdbc API support custom schema") { val parts = Array[String]("THEID < 2", "THEID >= 2") val props = new Properties() - props.put("customDataFrameColumnTypes", "NAME string, THEID bigint") + props.put("customDataFrameColumnTypes", "NAME STRING, THEID BIGINT") val schema = StructType(Seq( StructField("NAME", StringType, true), StructField("THEID", LongType, true))) val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, props) @@ -987,7 +987,7 @@ class JDBCSuite extends SparkFunSuite |CREATE TEMPORARY VIEW people_view |USING org.apache.spark.sql.jdbc |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass', - |customDataFrameColumnTypes 'NAME string, THEID int') + |customDataFrameColumnTypes 'NAME STRING, THEID INT') """.stripMargin.replaceAll("\n", " ")) val schema = StructType( Seq(StructField("NAME", StringType, true), StructField("THEID", IntegerType, true))) From 0b67f0f40b531d86d44e978140f85f68af1de65a Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 10 Sep 2017 16:02:26 +0800 Subject: [PATCH 15/17] customDataFrameColumnTypes -> customSchema --- docs/sql-programming-guide.md | 4 ++-- .../spark/examples/sql/SQLDataSourceExample.scala | 4 ++-- .../spark/sql/jdbc/OracleIntegrationSuite.scala | 2 +- .../org/apache/spark/sql/DataFrameReader.scala | 14 ++------------ .../execution/datasources/jdbc/JDBCOptions.scala | 4 ++-- .../execution/datasources/jdbc/JDBCRelation.scala | 6 +++--- .../sql/execution/datasources/jdbc/JdbcUtils.scala | 4 ++-- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 8 ++++---- 8 files changed, 18 insertions(+), 28 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e3bbd7c3c41a3..8713ff4df6185 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1337,9 +1337,9 @@ the following case-insensitive options: - customDataFrameColumnTypes + customSchema - The DataFrame column data types to use instead of the defaults when reading data from jdbc API. (e.g: "id DECIMAL(38, 0), name STRING"). The specified types should be valid spark sql data types. This option applies only to reading. + The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING"). The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 04921b74f8e19..86b3dc4a84f58 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -185,8 +185,8 @@ object SQLDataSourceExample { connectionProperties.put("password", "password") val jdbcDF2 = spark.read .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) - // Specifying dataframe column data types on read - connectionProperties.put("customDataFrameColumnTypes", "id DECIMAL(38, 0), name STRING") + // Specifying the custom data types of the read schema + connectionProperties.put("customSchema", "id DECIMAL(38, 0), name STRING") val jdbcDF3 = spark.read .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 6c65da8698308..7680ae3835132 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -290,7 +290,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo // custom schema can read data val props = new Properties() - props.put("customDataFrameColumnTypes", + props.put("customSchema", s"ID DECIMAL(${DecimalType.MAX_PRECISION}, 0), N1 INT, N2 BOOLEAN") val dfRead = spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", props) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b50902c198067..10b28ce812afc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -197,7 +197,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def jdbc(url: String, table: String, properties: Properties): DataFrame = { - assertJdbcAPISpecifiedDataFrameSchema() + assertNoSpecifiedSchema("jdbc") // properties should override settings in extraOptions. this.extraOptions ++= properties.asScala // explicit url and dbtable should override all @@ -268,7 +268,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { table: String, predicates: Array[String], connectionProperties: Properties): DataFrame = { - assertJdbcAPISpecifiedDataFrameSchema() + assertNoSpecifiedSchema("jdbc") // connectionProperties should override settings in extraOptions. val params = extraOptions.toMap ++ connectionProperties.asScala.toMap val options = new JDBCOptions(url, table, params) @@ -678,16 +678,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } } - /** - * A convenient function for validate specified column types schema in jdbc API. - */ - private def assertJdbcAPISpecifiedDataFrameSchema(): Unit = { - if (userSpecifiedSchema.nonEmpty) { - throw new AnalysisException("Please use customDataFrameColumnTypes option to " + - "specified column types.") - } - } - /** * A convenient function for schema validation in datasources supporting * `columnNameOfCorruptRecord` as an option. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index f777307aaa3b8..b4e5d169066d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -124,7 +124,7 @@ class JDBCOptions( // TODO: to reuse the existing partition parameters for those partition specific options val createTableOptions = parameters.getOrElse(JDBC_CREATE_TABLE_OPTIONS, "") val createTableColumnTypes = parameters.get(JDBC_CREATE_TABLE_COLUMN_TYPES) - val customDataFrameColumnTypes = parameters.get(JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES) + val customSchema = parameters.get(JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES) val batchSize = { val size = parameters.getOrElse(JDBC_BATCH_INSERT_SIZE, "1000").toInt @@ -164,7 +164,7 @@ object JDBCOptions { val JDBC_TRUNCATE = newOption("truncate") val JDBC_CREATE_TABLE_OPTIONS = newOption("createTableOptions") val JDBC_CREATE_TABLE_COLUMN_TYPES = newOption("createTableColumnTypes") - val JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES = newOption("customDataFrameColumnTypes") + val JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES = newOption("customSchema") val JDBC_BATCH_INSERT_SIZE = newOption("batchsize") val JDBC_TXN_ISOLATION_LEVEL = newOption("isolationLevel") val JDBC_SESSION_INIT_STATEMENT = newOption("sessionInitStatement") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 673658b117884..8cb018a2af09b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -113,9 +113,9 @@ private[sql] case class JDBCRelation( override val schema: StructType = { val schema = JDBCRDD.resolveTable(jdbcOptions) - val customDataFrameColumnTypes = jdbcOptions.customDataFrameColumnTypes - if (customDataFrameColumnTypes.isDefined) { - JdbcUtils.parseUserSpecifiedColumnTypes(schema, customDataFrameColumnTypes.get, + val customSchema = jdbcOptions.customSchema + if (customSchema.isDefined) { + JdbcUtils.parseUserSpecifiedColumnTypes(schema, customSchema.get, sqlContext.sessionState.conf.resolver) } else { schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 3d1feb24d83d0..57937d8ec44f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -769,7 +769,7 @@ object JdbcUtils extends Logging { } /** - * Parses the user specified customDataFrameColumnTypes option value to DataFrame schema, + * Parses the user specified customSchema option value to DataFrame schema, * and returns it if it's all columns are equals to default schema's. */ def parseUserSpecifiedColumnTypes( @@ -779,7 +779,7 @@ object JdbcUtils extends Logging { val userSchema = CatalystSqlParser.parseTableSchema(columnTypes) SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "in the createTableColumnTypes option value", nameEquality) + userSchema.map(_.name), "in the customSchema option value", nameEquality) if (userSchema.size != schema.size) { throw new AnalysisException("Please provide all the columns, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 219354bd696be..94383133022d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -960,18 +960,18 @@ class JDBCSuite extends SparkFunSuite val e1 = intercept[AnalysisException] { spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) }.getMessage - assert(e1.contains("Please use customDataFrameColumnTypes option to specified column types.")) + assert(e1.contains("Please use customSchema option to specified column types.")) val e2 = intercept[AnalysisException] { spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) }.getMessage - assert(e2.contains("Please use customDataFrameColumnTypes option to specified column types.")) + assert(e2.contains("Please use customSchema option to specified column types.")) } test("jdbc API support custom schema") { val parts = Array[String]("THEID < 2", "THEID >= 2") val props = new Properties() - props.put("customDataFrameColumnTypes", "NAME STRING, THEID BIGINT") + props.put("customSchema", "NAME STRING, THEID BIGINT") val schema = StructType(Seq( StructField("NAME", StringType, true), StructField("THEID", LongType, true))) val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, props) @@ -987,7 +987,7 @@ class JDBCSuite extends SparkFunSuite |CREATE TEMPORARY VIEW people_view |USING org.apache.spark.sql.jdbc |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass', - |customDataFrameColumnTypes 'NAME STRING, THEID INT') + |customSchema 'NAME STRING, THEID INT') """.stripMargin.replaceAll("\n", " ")) val schema = StructType( Seq(StructField("NAME", StringType, true), StructField("THEID", IntegerType, true))) From 7fc97b49d21fa05bca819ee93b72be084b3e67c7 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 10 Sep 2017 17:09:33 +0800 Subject: [PATCH 16/17] revert assertNoSpecifiedSchema --- .../spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala index 8ecf3038c9e1f..3dd2f7276217b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala @@ -49,7 +49,7 @@ class JdbcUtilsSuite extends SparkFunSuite { StructType(Seq(StructField("c1", DateType, true), StructField("c1", StringType, true))) } assert(duplicate.getMessage.contains( - "Found duplicate column(s) in the createTableColumnTypes option value")) + "Found duplicate column(s) in the customSchema option value")) val allColumns = intercept[AnalysisException]{ JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 STRING", caseSensitive) === diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 94383133022d7..5f3148d74339d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -960,12 +960,12 @@ class JDBCSuite extends SparkFunSuite val e1 = intercept[AnalysisException] { spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) }.getMessage - assert(e1.contains("Please use customSchema option to specified column types.")) + assert(e1.contains("User specified schema not supported with `jdbc`")) val e2 = intercept[AnalysisException] { spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) }.getMessage - assert(e2.contains("Please use customSchema option to specified column types.")) + assert(e2.contains("User specified schema not supported with `jdbc`")) } test("jdbc API support custom schema") { From 1fdf002b64ed381b31b6a4ba721357c647b11772 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 13 Sep 2017 21:33:55 +0800 Subject: [PATCH 17/17] parseUserSpecifiedColumnTypes -> getCustomSchema --- examples/src/main/python/sql/datasource.py | 2 +- .../datasources/jdbc/JDBCRelation.scala | 12 ++++---- .../datasources/jdbc/JdbcUtils.scala | 24 ++++++++-------- .../datasources/jdbc/JdbcUtilsSuite.scala | 28 +++++++++---------- 4 files changed, 31 insertions(+), 35 deletions(-) diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 36c91fa33f5f8..f86012ea382e8 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -184,7 +184,7 @@ def jdbc_dataset_example(spark): .option("dbtable", "schema.tablename") \ .option("user", "username") \ .option("password", "password") \ - .option("customDataFrameColumnTypes", "id DECIMAL(38, 0), name STRING") \ + .option("customSchema", "id DECIMAL(38, 0), name STRING") \ .load() # Saving data to a JDBC source diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 8cb018a2af09b..b23e5a7722004 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -112,13 +112,11 @@ private[sql] case class JDBCRelation( override val needConversion: Boolean = false override val schema: StructType = { - val schema = JDBCRDD.resolveTable(jdbcOptions) - val customSchema = jdbcOptions.customSchema - if (customSchema.isDefined) { - JdbcUtils.parseUserSpecifiedColumnTypes(schema, customSchema.get, - sqlContext.sessionState.conf.resolver) - } else { - schema + val tableSchema = JDBCRDD.resolveTable(jdbcOptions) + jdbcOptions.customSchema match { + case Some(customSchema) => JdbcUtils.getCustomSchema( + tableSchema, customSchema, sparkSession.sessionState.conf.resolver) + case None => tableSchema } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 57937d8ec44f5..75327f0d38c2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -302,7 +302,6 @@ object JdbcUtils extends Logging { rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls } val metadata = new MetadataBuilder() - .putString("name", columnName) .putLong("scale", fieldScale) val columnType = dialect.getCatalystType(dataType, typeName, fieldSize, metadata).getOrElse( @@ -772,26 +771,25 @@ object JdbcUtils extends Logging { * Parses the user specified customSchema option value to DataFrame schema, * and returns it if it's all columns are equals to default schema's. */ - def parseUserSpecifiedColumnTypes( - schema: StructType, - columnTypes: String, - nameEquality: Resolver): StructType = { - val userSchema = CatalystSqlParser.parseTableSchema(columnTypes) + def getCustomSchema( + tableSchema: StructType, + customSchema: String, + nameEquality: Resolver): StructType = { + val userSchema = CatalystSqlParser.parseTableSchema(customSchema) SchemaUtils.checkColumnNameDuplication( userSchema.map(_.name), "in the customSchema option value", nameEquality) - if (userSchema.size != schema.size) { - throw new AnalysisException("Please provide all the columns, " + - s"all columns are: ${schema.fields.map(_.name).mkString(",")}") + val colNames = tableSchema.fieldNames.mkString(",") + val errorMsg = s"Please provide all the columns, all columns are: $colNames" + if (userSchema.size != tableSchema.size) { + throw new AnalysisException(errorMsg) } // This is resolved by names, only check the column names. userSchema.fieldNames.foreach { col => - schema.find(f => nameEquality(f.name, col)).getOrElse { - throw new AnalysisException( - s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column $col not found in " + - s"schema ${schema.catalogString}") + tableSchema.find(f => nameEquality(f.name, col)).getOrElse { + throw new AnalysisException(errorMsg) } } userSchema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala index 3dd2f7276217b..1255f262bce94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala @@ -24,62 +24,62 @@ import org.apache.spark.sql.types._ class JdbcUtilsSuite extends SparkFunSuite { - val schema = StructType(Seq( + val tableSchema = StructType(Seq( StructField("C1", StringType, false), StructField("C2", IntegerType, false))) val caseSensitive = org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution val caseInsensitive = org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution test("Parse user specified column types") { assert( - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 DATE, C2 STRING", caseInsensitive) === + JdbcUtils.getCustomSchema(tableSchema, "C1 DATE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("C1", DateType, true), StructField("C2", StringType, true)))) - assert(JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 DATE, C2 STRING", caseSensitive) === + assert(JdbcUtils.getCustomSchema(tableSchema, "C1 DATE, C2 STRING", caseSensitive) === StructType(Seq(StructField("C1", DateType, true), StructField("C2", StringType, true)))) assert( - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 DATE, C2 STRING", caseInsensitive) === + JdbcUtils.getCustomSchema(tableSchema, "c1 DATE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true)))) - assert(JdbcUtils.parseUserSpecifiedColumnTypes( - schema, "c1 DECIMAL(38, 0), C2 STRING", caseInsensitive) === + assert(JdbcUtils.getCustomSchema( + tableSchema, "c1 DECIMAL(38, 0), C2 STRING", caseInsensitive) === StructType(Seq(StructField("c1", DecimalType(38, 0), true), StructField("C2", StringType, true)))) // Throw AnalysisException val duplicate = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 DATE, c1 STRING", caseInsensitive) === + JdbcUtils.getCustomSchema(tableSchema, "c1 DATE, c1 STRING", caseInsensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("c1", StringType, true))) } assert(duplicate.getMessage.contains( "Found duplicate column(s) in the customSchema option value")) val allColumns = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "C1 STRING", caseSensitive) === + JdbcUtils.getCustomSchema(tableSchema, "C1 STRING", caseSensitive) === StructType(Seq(StructField("C1", DateType, true))) } assert(allColumns.getMessage.contains("Please provide all the columns,")) val caseSensitiveColumnNotFound = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c1 DATE, C2 STRING", caseSensitive) === + JdbcUtils.getCustomSchema(tableSchema, "c1 DATE, C2 STRING", caseSensitive) === StructType(Seq(StructField("c1", DateType, true), StructField("C2", StringType, true))) } assert(caseSensitiveColumnNotFound.getMessage.contains( - s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c1 not found in schema")) + "Please provide all the columns, all columns are: C1,C2;")) val caseInsensitiveColumnNotFound = intercept[AnalysisException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 DATE, C2 STRING", caseInsensitive) === + JdbcUtils.getCustomSchema(tableSchema, "c3 DATE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } assert(caseInsensitiveColumnNotFound.getMessage.contains( - s"${JDBCOptions.JDBC_CUSTOM_DATAFRAME_COLUMN_TYPES} option column c3 not found in schema")) + "Please provide all the columns, all columns are: C1,C2;")) // Throw ParseException val dataTypeNotSupported = intercept[ParseException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 DATEE, C2 STRING", caseInsensitive) === + JdbcUtils.getCustomSchema(tableSchema, "c3 DATEE, C2 STRING", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } assert(dataTypeNotSupported.getMessage.contains("DataType datee is not supported")) val mismatchedInput = intercept[ParseException]{ - JdbcUtils.parseUserSpecifiedColumnTypes(schema, "c3 DATE. C2 STRING", caseInsensitive) === + JdbcUtils.getCustomSchema(tableSchema, "c3 DATE. C2 STRING", caseInsensitive) === StructType(Seq(StructField("c3", DateType, true), StructField("C2", StringType, true))) } assert(mismatchedInput.getMessage.contains("mismatched input '.' expecting"))