From 677541b47f27fd85f44aa2e46ec44861579475a8 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 13 Sep 2017 17:24:15 +0800 Subject: [PATCH 01/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) --- .../spark/sql/hive/execution/InsertIntoHiveTable.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 5bdc97a2982df..1472e3caf3324 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -101,6 +101,13 @@ case class InsertIntoHiveTable( val tmpLocation = getExternalTmpPath(sparkSession, hadoopConf, tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) + tableDesc.getOutputFileFormatClassName match { + case "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat" => + val parquetCompression = sparkSession.sessionState.conf.parquetCompressionCodec + hadoopConf.set("parquet.compression", parquetCompression) + case _ => + } + val numDynamicPartitions = partition.values.count(_.isEmpty) val numStaticPartitions = partition.values.count(_.nonEmpty) val partitionSpec = partition.map { From 4e70fff764b0814cfcef777bb1537efb9535f66a Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Thu, 14 Sep 2017 20:32:47 +0800 Subject: [PATCH 02/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) Add test. --- .../apache/spark/sql/hive/InsertSuite.scala | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index aa5cae33f5cd9..bd57fa1a18c2c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -728,4 +728,70 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter assert(e.contains("mismatched input 'ROW'")) } } + + test("[SPARK-21786] The 'spark.sql.parquet.compression.codec' " + + "configuration doesn't take effect on tables with partition field(s)") { + withTempDir { tmpDir => + withTempView("table_source") { + (0 until 10000).toDF("a").createOrReplaceTempView("table_source") + + val tableWithPartition = "table_with_partition" + val tableNoPartition = "table_no_partition" + withTable(tableWithPartition, tableNoPartition) { + sql( + s""" + |CREATE TABLE $tableNoPartition(a int) + |STORED AS PARQUET + |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableNoPartition' + """.stripMargin) + sql( + s""" + |CREATE TABLE $tableWithPartition(a int) + |PARTITIONED BY (p int) + |STORED AS PARQUET + |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableWithPartition' + """.stripMargin) + + def insertOverwriteTable(tableName: String, codec: String, isPartitioned: Boolean): Unit = { + withSQLConf("spark.sql.parquet.compression.codec" -> codec) { + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |${if (isPartitioned) "partition (p=10000)" else "" } + |SELECT * from table_source + """.stripMargin) + } + } + + def getDirFiles(file: File): List[File] = { + if (!file.exists()) Nil + else if (file.isFile) List(file) + else { + file.listFiles().filterNot(_.getName.startsWith(".hive-staging")) + .groupBy(_.isFile).flatMap { + case (isFile, files) if isFile => files.toList + case (_, dirs) => dirs.flatMap(getDirFiles) + }.toList + } + } + + def getTableSize(tableName: String, codec: String, isPartitioned: Boolean = false): Long = { + insertOverwriteTable(tableName, codec, isPartitioned) + val path = s"${tmpDir.toURI.toString.stripSuffix("/")}/$tableName" + val dir = new File(path) + val files = getDirFiles(dir).filter(_.getName.startsWith("part-")) + files.map(_.length()).sum + } + + //In fact, partitioned and unpartitioned table meta information is slightly different, + //and partitioned tables are slightly larger, but the differences are not very large. + //Think less than 1024Byte + val maxDiff = 1024 + assert(getTableSize(tableWithPartition, "uncompressed", true) - getTableSize(tableNoPartition, "uncompressed") < maxDiff) + assert(getTableSize(tableWithPartition, "gzip", true) - getTableSize(tableNoPartition, "gzip") < maxDiff) + assert(getTableSize(tableWithPartition, "uncompressed", true) - getTableSize(tableWithPartition, "gzip", true) > maxDiff) + } + } + } + } } From 3f022f954126c3ead74296abdd2fd32f01567f05 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Fri, 15 Sep 2017 16:33:02 +0800 Subject: [PATCH 03/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) Fix scala style. --- .../apache/spark/sql/hive/InsertSuite.scala | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index bd57fa1a18c2c..a5c2c656743ea 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -728,7 +728,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter assert(e.contains("mismatched input 'ROW'")) } } - + test("[SPARK-21786] The 'spark.sql.parquet.compression.codec' " + "configuration doesn't take effect on tables with partition field(s)") { withTempDir { tmpDir => @@ -752,7 +752,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableWithPartition' """.stripMargin) - def insertOverwriteTable(tableName: String, codec: String, isPartitioned: Boolean): Unit = { + def insertOverwriteTable(tableName: String, codec: String, + isPartitioned: Boolean): Unit = { withSQLConf("spark.sql.parquet.compression.codec" -> codec) { sql( s""" @@ -775,7 +776,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } - def getTableSize(tableName: String, codec: String, isPartitioned: Boolean = false): Long = { + def getTableSize(tableName: String, codec: String, + isPartitioned: Boolean = false): Long = { insertOverwriteTable(tableName, codec, isPartitioned) val path = s"${tmpDir.toURI.toString.stripSuffix("/")}/$tableName" val dir = new File(path) @@ -783,13 +785,16 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter files.map(_.length()).sum } - //In fact, partitioned and unpartitioned table meta information is slightly different, - //and partitioned tables are slightly larger, but the differences are not very large. - //Think less than 1024Byte + // In fact, partitioned and unpartitioned table meta information is slightly different, + // and partitioned tables are slightly larger, but the differences are not very large. + // Think less than 1024Byte val maxDiff = 1024 - assert(getTableSize(tableWithPartition, "uncompressed", true) - getTableSize(tableNoPartition, "uncompressed") < maxDiff) - assert(getTableSize(tableWithPartition, "gzip", true) - getTableSize(tableNoPartition, "gzip") < maxDiff) - assert(getTableSize(tableWithPartition, "uncompressed", true) - getTableSize(tableWithPartition, "gzip", true) > maxDiff) + assert(getTableSize(tableWithPartition, "uncompressed", true) + - getTableSize(tableNoPartition, "uncompressed") < maxDiff) + assert(getTableSize(tableWithPartition, "gzip", true) + - getTableSize(tableNoPartition, "gzip") < maxDiff) + assert(getTableSize(tableWithPartition, "uncompressed", true) + - getTableSize(tableWithPartition, "gzip", true) > maxDiff) } } } From 6d77bf930f02888b1ba3097e75a293e1da6a7944 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Fri, 15 Sep 2017 16:39:56 +0800 Subject: [PATCH 04/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) Fix scala style. --- .../src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index a5c2c656743ea..90969d7c87b64 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -752,7 +752,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableWithPartition' """.stripMargin) - def insertOverwriteTable(tableName: String, codec: String, + def insertOverwriteTable(tableName: String, codec: String, isPartitioned: Boolean): Unit = { withSQLConf("spark.sql.parquet.compression.codec" -> codec) { sql( From 42aca3d2c02cccf168fcac97a5d3f594f21b34ab Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Fri, 15 Sep 2017 18:30:20 +0800 Subject: [PATCH 05/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) Fix test problem --- .../src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 90969d7c87b64..715a018ed5569 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -779,7 +779,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter def getTableSize(tableName: String, codec: String, isPartitioned: Boolean = false): Long = { insertOverwriteTable(tableName, codec, isPartitioned) - val path = s"${tmpDir.toURI.toString.stripSuffix("/")}/$tableName" + val path = s"${tmpDir.getPath.stripSuffix("/")}/$tableName" val dir = new File(path) val files = getDirFiles(dir).filter(_.getName.startsWith("part-")) files.map(_.length()).sum From 5cbe999951df3a7ae981470e8dc1049d72aeaa57 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Sat, 16 Sep 2017 16:04:45 +0800 Subject: [PATCH 06/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) Fix the similar issue of orc compression --- .../hive/execution/InsertIntoHiveTable.scala | 8 +- .../apache/spark/sql/hive/InsertSuite.scala | 129 ++++++++++-------- 2 files changed, 82 insertions(+), 55 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 1472e3caf3324..834a250f3faf1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -102,9 +102,15 @@ case class InsertIntoHiveTable( val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) tableDesc.getOutputFileFormatClassName match { - case "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat" => + case formatName if formatName.endsWith("ParquetOutputFormat") => val parquetCompression = sparkSession.sessionState.conf.parquetCompressionCodec hadoopConf.set("parquet.compression", parquetCompression) + case formatName if formatName.endsWith("OrcOutputFormat") => + val orcCompression = sparkSession.sessionState.conf.orcCompressionCodec.toUpperCase match { + case "UNCOMPRESSED" => "NONE" + case _@x => x + } + hadoopConf.set("orc.compress", orcCompression) case _ => } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 715a018ed5569..b7f2b7ad2a4a2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -731,72 +731,93 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter test("[SPARK-21786] The 'spark.sql.parquet.compression.codec' " + "configuration doesn't take effect on tables with partition field(s)") { - withTempDir { tmpDir => - withTempView("table_source") { - (0 until 10000).toDF("a").createOrReplaceTempView("table_source") + val tableWithPartition = "table_with_partition" + val tableNoPartition = "table_no_partition" - val tableWithPartition = "table_with_partition" - val tableNoPartition = "table_no_partition" - withTable(tableWithPartition, tableNoPartition) { - sql( - s""" + def insertOverwriteTable(tableName: String, paramName: String, codec: String, + isPartitioned: Boolean): Unit = { + withSQLConf(paramName -> codec) { + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |${if (isPartitioned) "partition (p=10000)" else "" } + |SELECT * from table_source + """.stripMargin) + } + } + + def getDirFiles(file: File): List[File] = { + if (!file.exists()) Nil + else if (file.isFile) List(file) + else { + file.listFiles().filterNot(_.getName.startsWith(".hive-staging")) + .groupBy(_.isFile).flatMap { + case (isFile, files) if isFile => files.toList + case (_, dirs) => dirs.flatMap(getDirFiles) + }.toList + } + } + + def getTableSize(tmpDir: File, tableName: String, paramName: String, codec: String, + isPartitioned: Boolean = false): Long = { + insertOverwriteTable(tableName, paramName, codec, isPartitioned) + val path = s"${tmpDir.getPath.stripSuffix("/")}/$tableName" + val dir = new File(path) + val files = getDirFiles(dir).filter(_.getName.startsWith("part-")) + files.map(_.length()).sum + } + + def checkCompressionCodec(format: String)(f: File => Unit): Unit = { + withTempDir { tmpDir => + withTempView("table_source") { + (0 until 10000).toDF("a").createOrReplaceTempView("table_source") + + withTable(tableWithPartition, tableNoPartition) { + sql( + s""" |CREATE TABLE $tableNoPartition(a int) - |STORED AS PARQUET + |STORED AS $format |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableNoPartition' """.stripMargin) - sql( - s""" + sql( + s""" |CREATE TABLE $tableWithPartition(a int) |PARTITIONED BY (p int) - |STORED AS PARQUET + |STORED AS $format |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableWithPartition' """.stripMargin) - def insertOverwriteTable(tableName: String, codec: String, - isPartitioned: Boolean): Unit = { - withSQLConf("spark.sql.parquet.compression.codec" -> codec) { - sql( - s""" - |INSERT OVERWRITE TABLE $tableName - |${if (isPartitioned) "partition (p=10000)" else "" } - |SELECT * from table_source - """.stripMargin) - } - } - - def getDirFiles(file: File): List[File] = { - if (!file.exists()) Nil - else if (file.isFile) List(file) - else { - file.listFiles().filterNot(_.getName.startsWith(".hive-staging")) - .groupBy(_.isFile).flatMap { - case (isFile, files) if isFile => files.toList - case (_, dirs) => dirs.flatMap(getDirFiles) - }.toList - } - } - - def getTableSize(tableName: String, codec: String, - isPartitioned: Boolean = false): Long = { - insertOverwriteTable(tableName, codec, isPartitioned) - val path = s"${tmpDir.getPath.stripSuffix("/")}/$tableName" - val dir = new File(path) - val files = getDirFiles(dir).filter(_.getName.startsWith("part-")) - files.map(_.length()).sum + f(tmpDir) } - - // In fact, partitioned and unpartitioned table meta information is slightly different, - // and partitioned tables are slightly larger, but the differences are not very large. - // Think less than 1024Byte - val maxDiff = 1024 - assert(getTableSize(tableWithPartition, "uncompressed", true) - - getTableSize(tableNoPartition, "uncompressed") < maxDiff) - assert(getTableSize(tableWithPartition, "gzip", true) - - getTableSize(tableNoPartition, "gzip") < maxDiff) - assert(getTableSize(tableWithPartition, "uncompressed", true) - - getTableSize(tableWithPartition, "gzip", true) > maxDiff) } } } + + val parquetCompression = "spark.sql.parquet.compression.codec" + checkCompressionCodec("PARQUET") { tmpDir => + // In fact, partitioned and unpartitioned table meta information is slightly different, + // and partitioned tables are slightly larger, but the differences are not very large. + // Think less than 1024Byte + val maxDiff = 1024 + assert(getTableSize(tmpDir, tableWithPartition, parquetCompression, "uncompressed", true) + - getTableSize(tmpDir, tableNoPartition, parquetCompression, "uncompressed") < maxDiff) + assert(getTableSize(tmpDir, tableWithPartition, parquetCompression, "gzip", true) + - getTableSize(tmpDir, tableNoPartition, parquetCompression, "gzip") < maxDiff) + assert(getTableSize(tmpDir, tableWithPartition, parquetCompression, "uncompressed", true) + - getTableSize(tmpDir, tableWithPartition, parquetCompression, "gzip", true) > maxDiff) + } + + val orcCompression = "spark.sql.orc.compression.codec" + checkCompressionCodec("ORC") { tmpDir => + val maxDiff = 1024 + assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "none", true) + - getTableSize(tmpDir, tableNoPartition, orcCompression, "none") < maxDiff) + assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "uncompressed", true) + == getTableSize(tmpDir, tableNoPartition, orcCompression, "none")) + assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "zlib", true) + - getTableSize(tmpDir, tableNoPartition, orcCompression, "zlib") < maxDiff) + assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "none", true) + - getTableSize(tmpDir, tableWithPartition, orcCompression, "zlib", true) > maxDiff) + } } } From 732266cc77eb370442dce0f125a70d634b3ebc6d Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Sat, 16 Sep 2017 17:23:23 +0800 Subject: [PATCH 07/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) Fix test problem --- .../scala/org/apache/spark/sql/hive/InsertSuite.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index b7f2b7ad2a4a2..efa29c564c9c6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -770,7 +770,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter def checkCompressionCodec(format: String)(f: File => Unit): Unit = { withTempDir { tmpDir => withTempView("table_source") { - (0 until 10000).toDF("a").createOrReplaceTempView("table_source") + (0 until 100000).toDF("a").createOrReplaceTempView("table_source") withTable(tableWithPartition, tableNoPartition) { sql( @@ -809,15 +809,14 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter val orcCompression = "spark.sql.orc.compression.codec" checkCompressionCodec("ORC") { tmpDir => - val maxDiff = 1024 assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "none", true) - - getTableSize(tmpDir, tableNoPartition, orcCompression, "none") < maxDiff) + == getTableSize(tmpDir, tableNoPartition, orcCompression, "none")) assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "uncompressed", true) == getTableSize(tmpDir, tableNoPartition, orcCompression, "none")) assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "zlib", true) - - getTableSize(tmpDir, tableNoPartition, orcCompression, "zlib") < maxDiff) + == getTableSize(tmpDir, tableNoPartition, orcCompression, "zlib")) assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "none", true) - - getTableSize(tmpDir, tableWithPartition, orcCompression, "zlib", true) > maxDiff) + > getTableSize(tmpDir, tableWithPartition, orcCompression, "zlib", true)) } } } From c7ff62cc1d4e35f7ffd2f711068a2ed5dc47e406 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Sat, 16 Sep 2017 18:52:35 +0800 Subject: [PATCH 08/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' configuration doesn't take effect on tables with partition field(s) Fix test problem --- .../sql/hive/execution/HiveDDLSuite.scala | 68 ++++++++++--------- 1 file changed, 35 insertions(+), 33 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index ee64bc9f9ee04..6083e147146ef 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1438,39 +1438,41 @@ class HiveDDLSuite } test("create hive serde table with new syntax") { - withTable("t", "t2", "t3") { - withTempPath { path => - sql( - s""" - |CREATE TABLE t(id int) USING hive - |OPTIONS(fileFormat 'orc', compression 'Zlib') - |LOCATION '${path.toURI}' - """.stripMargin) - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) - assert(DDLUtils.isHiveTable(table)) - assert(table.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) - assert(table.storage.properties.get("compression") == Some("Zlib")) - assert(spark.table("t").collect().isEmpty) - - sql("INSERT INTO t SELECT 1") - checkAnswer(spark.table("t"), Row(1)) - // Check if this is compressed as ZLIB. - val maybeOrcFile = path.listFiles().find(!_.getName.endsWith(".crc")) - assert(maybeOrcFile.isDefined) - val orcFilePath = maybeOrcFile.get.toPath.toString - val expectedCompressionKind = - OrcFileOperator.getFileReader(orcFilePath).get.getCompression - assert("ZLIB" === expectedCompressionKind.name()) - - sql("CREATE TABLE t2 USING HIVE AS SELECT 1 AS c1, 'a' AS c2") - val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t2")) - assert(DDLUtils.isHiveTable(table2)) - assert(table2.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) - checkAnswer(spark.table("t2"), Row(1, "a")) - - sql("CREATE TABLE t3(a int, p int) USING hive PARTITIONED BY (p)") - sql("INSERT INTO t3 PARTITION(p=1) SELECT 0") - checkAnswer(spark.table("t3"), Row(0, 1)) + withSQLConf("spark.sql.orc.compression.codec" -> "zlib") { + withTable("t", "t2", "t3") { + withTempPath { path => + sql( + s""" + |CREATE TABLE t(id int) USING hive + |OPTIONS(fileFormat 'orc', compression 'Zlib') + |LOCATION '${path.toURI}' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.isHiveTable(table)) + assert(table.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + assert(table.storage.properties.get("compression") == Some("Zlib")) + assert(spark.table("t").collect().isEmpty) + + sql("INSERT INTO t SELECT 1") + checkAnswer(spark.table("t"), Row(1)) + // Check if this is compressed as ZLIB. + val maybeOrcFile = path.listFiles().find(!_.getName.endsWith(".crc")) + assert(maybeOrcFile.isDefined) + val orcFilePath = maybeOrcFile.get.toPath.toString + val expectedCompressionKind = + OrcFileOperator.getFileReader(orcFilePath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + + sql("CREATE TABLE t2 USING HIVE AS SELECT 1 AS c1, 'a' AS c2") + val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t2")) + assert(DDLUtils.isHiveTable(table2)) + assert(table2.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + checkAnswer(spark.table("t2"), Row(1, "a")) + + sql("CREATE TABLE t3(a int, p int) USING hive PARTITIONED BY (p)") + sql("INSERT INTO t3 PARTITION(p=1) SELECT 0") + checkAnswer(spark.table("t3"), Row(0, 1)) + } } } } From 384ee044c4c5884c09ca432fe680caad5a6b8441 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 20 Sep 2017 16:45:30 +0800 Subject: [PATCH 09/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix the priority issue --- .../hive/execution/InsertIntoHiveTable.scala | 13 -- .../sql/hive/execution/SaveAsHiveFile.scala | 30 +++ .../apache/spark/sql/hive/InsertSuite.scala | 173 ++++++++++-------- .../sql/hive/execution/HiveDDLSuite.scala | 68 ++++--- 4 files changed, 162 insertions(+), 122 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 834a250f3faf1..5bdc97a2982df 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -101,19 +101,6 @@ case class InsertIntoHiveTable( val tmpLocation = getExternalTmpPath(sparkSession, hadoopConf, tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - tableDesc.getOutputFileFormatClassName match { - case formatName if formatName.endsWith("ParquetOutputFormat") => - val parquetCompression = sparkSession.sessionState.conf.parquetCompressionCodec - hadoopConf.set("parquet.compression", parquetCompression) - case formatName if formatName.endsWith("OrcOutputFormat") => - val orcCompression = sparkSession.sessionState.conf.orcCompressionCodec.toUpperCase match { - case "UNCOMPRESSED" => "NONE" - case _@x => x - } - hadoopConf.set("orc.compress", orcCompression) - case _ => - } - val numDynamicPartitions = partition.values.count(_.isEmpty) val numStaticPartitions = partition.values.count(_.nonEmpty) val partitionSpec = partition.map { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 7de9b421245f0..f6dda5e1e0646 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import java.util.Locale + import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.io.FileCommitProtocol @@ -52,6 +54,26 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { .get("mapreduce.output.fileoutputformat.compress.type")) } + fileSinkConf.tableInfo.getOutputFileFormatClassName match { + case formatName if formatName.endsWith("ParquetOutputFormat") => + val compressionConf = "parquet.compression" + val compressionCodec = getCompressionByPriority(fileSinkConf, compressionConf, + sparkSession.sessionState.conf.parquetCompressionCodec) match { + case "NONE" => "UNCOMPRESSED" + case _@x => x + } + hadoopConf.set(compressionConf, compressionCodec) + case formatName if formatName.endsWith("OrcOutputFormat") => + val compressionConf = "orc.compress" + val compressionCodec = getCompressionByPriority(fileSinkConf, compressionConf, + sparkSession.sessionState.conf.orcCompressionCodec) match { + case "UNCOMPRESSED" => "NONE" + case _@x => x + } + hadoopConf.set(compressionConf, compressionCodec) + case _ => + } + val committer = FileCommitProtocol.instantiate( sparkSession.sessionState.conf.fileCommitProtocolClass, jobId = java.util.UUID.randomUUID().toString, @@ -69,5 +91,13 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), options = Map.empty) } + + private def getCompressionByPriority(fileSinkConf: FileSinkDesc, + compressionConf: String, default: String): String = { + val props = fileSinkConf.tableInfo.getProperties + val priorities = List("compression", compressionConf) + priorities.find(props.getProperty(_ , null) != null) + .map(props.getProperty).getOrElse(default).toUpperCase(Locale.ROOT) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index efa29c564c9c6..3bd9517fc475d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -729,94 +729,119 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } - test("[SPARK-21786] The 'spark.sql.parquet.compression.codec' " + - "configuration doesn't take effect on tables with partition field(s)") { - val tableWithPartition = "table_with_partition" - val tableNoPartition = "table_no_partition" - - def insertOverwriteTable(tableName: String, paramName: String, codec: String, - isPartitioned: Boolean): Unit = { - withSQLConf(paramName -> codec) { + test("[SPARK-21786] Check 'spark.sql.parquet.compression.codec' " + + "and 'spark.sql.parquet.compression.codec' taking effect on hive table writing") { + case class CompressionConf(name: String, codeC: String) + + case class TableDefine(tableName: String, isPartitioned: Boolean, format: String, + compressionConf: Option[CompressionConf]) { + def createTable(rootDir: File): Unit = { + val compression = compressionConf.map(cf => s"'${cf.name}'='${cf.codeC}'") sql( s""" - |INSERT OVERWRITE TABLE $tableName - |${if (isPartitioned) "partition (p=10000)" else "" } - |SELECT * from table_source - """.stripMargin) + |CREATE TABLE $tableName(a int) + |${ if (isPartitioned) "PARTITIONED BY (p int)" else "" } + |STORED AS $format + |LOCATION '${rootDir.toURI.toString.stripSuffix("/")}/$tableName' + |${ if (compressionConf.nonEmpty) s"TBLPROPERTIES(${compression.get})" else "" } + """.stripMargin) } - } - def getDirFiles(file: File): List[File] = { - if (!file.exists()) Nil - else if (file.isFile) List(file) - else { - file.listFiles().filterNot(_.getName.startsWith(".hive-staging")) - .groupBy(_.isFile).flatMap { - case (isFile, files) if isFile => files.toList - case (_, dirs) => dirs.flatMap(getDirFiles) - }.toList + def insertOverwriteTable(): Unit = { + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |${ if (isPartitioned) "partition (p=10000)" else "" } + |SELECT * from table_source + """.stripMargin) } - } - - def getTableSize(tmpDir: File, tableName: String, paramName: String, codec: String, - isPartitioned: Boolean = false): Long = { - insertOverwriteTable(tableName, paramName, codec, isPartitioned) - val path = s"${tmpDir.getPath.stripSuffix("/")}/$tableName" - val dir = new File(path) - val files = getDirFiles(dir).filter(_.getName.startsWith("part-")) - files.map(_.length()).sum - } - def checkCompressionCodec(format: String)(f: File => Unit): Unit = { - withTempDir { tmpDir => - withTempView("table_source") { - (0 until 100000).toDF("a").createOrReplaceTempView("table_source") - - withTable(tableWithPartition, tableNoPartition) { - sql( - s""" - |CREATE TABLE $tableNoPartition(a int) - |STORED AS $format - |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableNoPartition' - """.stripMargin) - sql( - s""" - |CREATE TABLE $tableWithPartition(a int) - |PARTITIONED BY (p int) - |STORED AS $format - |LOCATION '${tmpDir.toURI.toString.stripSuffix("/")}/$tableWithPartition' - """.stripMargin) + def getDirFiles(file: File): List[File] = { + if (!file.exists()) Nil + else if (file.isFile) List(file) + else { + file.listFiles().filterNot(_.getName.startsWith(".hive-staging")) + .groupBy(_.isFile).flatMap { + case (isFile, files) if isFile => files.toList + case (_, dirs) => dirs.flatMap(getDirFiles) + }.toList + } + } - f(tmpDir) + def getTableSize: Long = { + var totalSize = 0L + withTempDir { tmpDir => + withTable(tableName) { + createTable(tmpDir) + insertOverwriteTable() + val path = s"${tmpDir.getPath.stripSuffix("/")}/$tableName" + val dir = new File(path) + val files = getDirFiles(dir).filter(_.getName.startsWith("part-")) + totalSize = files.map(_.length()).sum } } + totalSize + } + } + + def checkParquetCompressionCodec(isPartitioned: Boolean, tableCodec: String, + sessionCodec: String, f: (Long, Long) => Boolean = _ == _) = { + val tableOrg = TableDefine(s"tbl_parquet$tableCodec", isPartitioned, "parquet", + Some(CompressionConf("parquet.compression", tableCodec))) + val tableOrgSize = tableOrg.getTableSize + + withSQLConf("spark.sql.parquet.compression.codec" -> sessionCodec) { + // priority check, when table-level compression conf was set, expecting + // table-level compression conf is not affected by the session conf, and table-level + // compression conf takes precedence even the two conf of codec is different + val tableOrgSessionConfSize = tableOrg.getTableSize + assert(tableOrgSize == tableOrgSessionConfSize) + + // check session conf of compression codec taking effect + val table = TableDefine(s"tbl_parquet", isPartitioned, "parquet", None) + assert(f(tableOrg.getTableSize, table.getTableSize)) } } - val parquetCompression = "spark.sql.parquet.compression.codec" - checkCompressionCodec("PARQUET") { tmpDir => - // In fact, partitioned and unpartitioned table meta information is slightly different, - // and partitioned tables are slightly larger, but the differences are not very large. - // Think less than 1024Byte - val maxDiff = 1024 - assert(getTableSize(tmpDir, tableWithPartition, parquetCompression, "uncompressed", true) - - getTableSize(tmpDir, tableNoPartition, parquetCompression, "uncompressed") < maxDiff) - assert(getTableSize(tmpDir, tableWithPartition, parquetCompression, "gzip", true) - - getTableSize(tmpDir, tableNoPartition, parquetCompression, "gzip") < maxDiff) - assert(getTableSize(tmpDir, tableWithPartition, parquetCompression, "uncompressed", true) - - getTableSize(tmpDir, tableWithPartition, parquetCompression, "gzip", true) > maxDiff) + def checkOrcCompressionCodec(isPartitioned: Boolean, tableCodec: String, + sessionCodec: String, f: (Long, Long) => Boolean = _ == _) = { + val tableOrg = TableDefine(s"tbl_orc$tableCodec", isPartitioned, "orc", + Some(CompressionConf("orc.compress", tableCodec))) + val tableOrgSize = tableOrg.getTableSize + + withSQLConf("spark.sql.orc.compression.codec" -> sessionCodec) { + // priority check, when table-level compression conf was set, expecting + // table-level compression conf is not affected by the session conf, and table-level + // compression conf takes precedence even the two conf of codec is different + val tableOrgSessionConfSize = tableOrg.getTableSize + assert(tableOrgSize == tableOrgSessionConfSize) + + // check session conf of compression codec taking effect + val table = TableDefine(s"tbl_orc", isPartitioned, "orc", None) + assert(f(tableOrg.getTableSize, table.getTableSize)) + } } - val orcCompression = "spark.sql.orc.compression.codec" - checkCompressionCodec("ORC") { tmpDir => - assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "none", true) - == getTableSize(tmpDir, tableNoPartition, orcCompression, "none")) - assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "uncompressed", true) - == getTableSize(tmpDir, tableNoPartition, orcCompression, "none")) - assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "zlib", true) - == getTableSize(tmpDir, tableNoPartition, orcCompression, "zlib")) - assert(getTableSize(tmpDir, tableWithPartition, orcCompression, "none", true) - > getTableSize(tmpDir, tableWithPartition, orcCompression, "zlib", true)) + withTempView("table_source") { + (0 until 100000).toDF("a").createOrReplaceTempView("table_source") + + checkParquetCompressionCodec(true, "uncompressed", "uncompressed") + checkParquetCompressionCodec(true, "gzip", "gzip") + checkParquetCompressionCodec(true, "gzip", "uncompressed", _ < _) + + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { + checkParquetCompressionCodec(false, "uncompressed", "uncompressed") + checkParquetCompressionCodec(false, "gzip", "gzip") + checkParquetCompressionCodec(false, "gzip", "uncompressed", _ < _) + } + + checkOrcCompressionCodec(true, "none", "none") + checkOrcCompressionCodec(true, "zlib", "zlib") + checkOrcCompressionCodec(true, "zlib", "none", _ < _) + + checkOrcCompressionCodec(false, "none", "none") + checkOrcCompressionCodec(false, "zlib", "zlib") + checkOrcCompressionCodec(false, "zlib", "none", _ < _) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 6083e147146ef..ee64bc9f9ee04 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1438,41 +1438,39 @@ class HiveDDLSuite } test("create hive serde table with new syntax") { - withSQLConf("spark.sql.orc.compression.codec" -> "zlib") { - withTable("t", "t2", "t3") { - withTempPath { path => - sql( - s""" - |CREATE TABLE t(id int) USING hive - |OPTIONS(fileFormat 'orc', compression 'Zlib') - |LOCATION '${path.toURI}' - """.stripMargin) - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) - assert(DDLUtils.isHiveTable(table)) - assert(table.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) - assert(table.storage.properties.get("compression") == Some("Zlib")) - assert(spark.table("t").collect().isEmpty) - - sql("INSERT INTO t SELECT 1") - checkAnswer(spark.table("t"), Row(1)) - // Check if this is compressed as ZLIB. - val maybeOrcFile = path.listFiles().find(!_.getName.endsWith(".crc")) - assert(maybeOrcFile.isDefined) - val orcFilePath = maybeOrcFile.get.toPath.toString - val expectedCompressionKind = - OrcFileOperator.getFileReader(orcFilePath).get.getCompression - assert("ZLIB" === expectedCompressionKind.name()) - - sql("CREATE TABLE t2 USING HIVE AS SELECT 1 AS c1, 'a' AS c2") - val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t2")) - assert(DDLUtils.isHiveTable(table2)) - assert(table2.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) - checkAnswer(spark.table("t2"), Row(1, "a")) - - sql("CREATE TABLE t3(a int, p int) USING hive PARTITIONED BY (p)") - sql("INSERT INTO t3 PARTITION(p=1) SELECT 0") - checkAnswer(spark.table("t3"), Row(0, 1)) - } + withTable("t", "t2", "t3") { + withTempPath { path => + sql( + s""" + |CREATE TABLE t(id int) USING hive + |OPTIONS(fileFormat 'orc', compression 'Zlib') + |LOCATION '${path.toURI}' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.isHiveTable(table)) + assert(table.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + assert(table.storage.properties.get("compression") == Some("Zlib")) + assert(spark.table("t").collect().isEmpty) + + sql("INSERT INTO t SELECT 1") + checkAnswer(spark.table("t"), Row(1)) + // Check if this is compressed as ZLIB. + val maybeOrcFile = path.listFiles().find(!_.getName.endsWith(".crc")) + assert(maybeOrcFile.isDefined) + val orcFilePath = maybeOrcFile.get.toPath.toString + val expectedCompressionKind = + OrcFileOperator.getFileReader(orcFilePath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + + sql("CREATE TABLE t2 USING HIVE AS SELECT 1 AS c1, 'a' AS c2") + val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t2")) + assert(DDLUtils.isHiveTable(table2)) + assert(table2.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + checkAnswer(spark.table("t2"), Row(1, "a")) + + sql("CREATE TABLE t3(a int, p int) USING hive PARTITIONED BY (p)") + sql("INSERT INTO t3 PARTITION(p=1) SELECT 0") + checkAnswer(spark.table("t3"), Row(0, 1)) } } } From 8c9207414c99388bae5c9e4d31d2964902beed88 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 20 Sep 2017 18:46:56 +0800 Subject: [PATCH 10/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix scala style --- .../org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index f6dda5e1e0646..6dd53fe6224e3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -96,7 +96,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { compressionConf: String, default: String): String = { val props = fileSinkConf.tableInfo.getProperties val priorities = List("compression", compressionConf) - priorities.find(props.getProperty(_ , null) != null) + priorities.find(props.getProperty(_, null) != null) .map(props.getProperty).getOrElse(default).toUpperCase(Locale.ROOT) } } From d427df5fbbeddebd2cc2f50d94b024b89d0fca64 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 20 Sep 2017 19:29:51 +0800 Subject: [PATCH 11/28] Update InsertSuite.scala Fix scala style --- .../test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 3bd9517fc475d..e08361af3cf76 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -785,7 +785,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } def checkParquetCompressionCodec(isPartitioned: Boolean, tableCodec: String, - sessionCodec: String, f: (Long, Long) => Boolean = _ == _) = { + sessionCodec: String, f: (Long, Long) => Boolean = _ == _):Unit = { val tableOrg = TableDefine(s"tbl_parquet$tableCodec", isPartitioned, "parquet", Some(CompressionConf("parquet.compression", tableCodec))) val tableOrgSize = tableOrg.getTableSize @@ -804,7 +804,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } def checkOrcCompressionCodec(isPartitioned: Boolean, tableCodec: String, - sessionCodec: String, f: (Long, Long) => Boolean = _ == _) = { + sessionCodec: String, f: (Long, Long) => Boolean = _ == _):Unit = { val tableOrg = TableDefine(s"tbl_orc$tableCodec", isPartitioned, "orc", Some(CompressionConf("orc.compress", tableCodec))) val tableOrgSize = tableOrg.getTableSize From 35cfa0134e64b0a3350d8c6302ed31612beaf2eb Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 20 Sep 2017 19:36:53 +0800 Subject: [PATCH 12/28] Update InsertSuite.scala --- .../test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index e08361af3cf76..115f2f2fe41e7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -785,7 +785,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } def checkParquetCompressionCodec(isPartitioned: Boolean, tableCodec: String, - sessionCodec: String, f: (Long, Long) => Boolean = _ == _):Unit = { + sessionCodec: String, f: (Long, Long) => Boolean = _ == _): Unit = { val tableOrg = TableDefine(s"tbl_parquet$tableCodec", isPartitioned, "parquet", Some(CompressionConf("parquet.compression", tableCodec))) val tableOrgSize = tableOrg.getTableSize @@ -804,7 +804,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } def checkOrcCompressionCodec(isPartitioned: Boolean, tableCodec: String, - sessionCodec: String, f: (Long, Long) => Boolean = _ == _):Unit = { + sessionCodec: String, f: (Long, Long) => Boolean = _ == _): Unit = { val tableOrg = TableDefine(s"tbl_orc$tableCodec", isPartitioned, "orc", Some(CompressionConf("orc.compress", tableCodec))) val tableOrgSize = tableOrg.getTableSize From 5387497c69dcee314634f8b4b86fc7f64f9c3a43 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 20 Sep 2017 21:47:08 +0800 Subject: [PATCH 13/28] Fix test problems --- .../apache/spark/sql/hive/InsertSuite.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 115f2f2fe41e7..04413931e19be 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -825,23 +825,23 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter withTempView("table_source") { (0 until 100000).toDF("a").createOrReplaceTempView("table_source") - checkParquetCompressionCodec(true, "uncompressed", "uncompressed") - checkParquetCompressionCodec(true, "gzip", "gzip") - checkParquetCompressionCodec(true, "gzip", "uncompressed", _ < _) + checkParquetCompressionCodec(true, "UNCOMPRESSED", "UNCOMPRESSED") + checkParquetCompressionCodec(true, "GZIP", "GZIP") + checkParquetCompressionCodec(true, "GZIP", "UNCOMPRESSED", _ < _) withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { - checkParquetCompressionCodec(false, "uncompressed", "uncompressed") - checkParquetCompressionCodec(false, "gzip", "gzip") - checkParquetCompressionCodec(false, "gzip", "uncompressed", _ < _) + checkParquetCompressionCodec(false, "UNCOMPRESSED", "UNCOMPRESSED") + checkParquetCompressionCodec(false, "GZIP", "GZIP") + checkParquetCompressionCodec(false, "GZIP", "UNCOMPRESSED", _ < _) } - checkOrcCompressionCodec(true, "none", "none") - checkOrcCompressionCodec(true, "zlib", "zlib") - checkOrcCompressionCodec(true, "zlib", "none", _ < _) + checkOrcCompressionCodec(true, "NONE", "NONE") + checkOrcCompressionCodec(true, "ZLIB", "ZLIB") + checkOrcCompressionCodec(true, "ZLIB", "NONE", _ < _) - checkOrcCompressionCodec(false, "none", "none") - checkOrcCompressionCodec(false, "zlib", "zlib") - checkOrcCompressionCodec(false, "zlib", "none", _ < _) + checkOrcCompressionCodec(false, "NONE", "NONE") + checkOrcCompressionCodec(false, "ZLIB", "ZLIB") + checkOrcCompressionCodec(false, "ZLIB", "NONE", _ < _) } } } From 676d6a7a64e137ca05981fae6cebea27b2d9d401 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 27 Sep 2017 21:07:37 +0800 Subject: [PATCH 14/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix the test case issue --- .../apache/spark/sql/hive/InsertSuite.scala | 217 ++++++++++++------ 1 file changed, 144 insertions(+), 73 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 04413931e19be..a6d3a6eeddad2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -21,12 +21,16 @@ import java.io.File import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable +import org.apache.spark.sql.execution.datasources.parquet.ParquetTest +import org.apache.spark.sql.hive.orc.OrcFileOperator import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -35,7 +39,7 @@ case class TestData(key: Int, value: String) case class ThreeCloumntable(key: Int, value: String, key1: String) class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter - with SQLTestUtils { + with ParquetTest { import spark.implicits._ override lazy val testData = spark.sparkContext.parallelize( @@ -730,20 +734,25 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } test("[SPARK-21786] Check 'spark.sql.parquet.compression.codec' " + - "and 'spark.sql.parquet.compression.codec' taking effect on hive table writing") { - case class CompressionConf(name: String, codeC: String) + "and 'spark.sql.orc.compression.codec' taking effect on hive table writing") { + + val hadoopConf = spark.sessionState.newHadoopConf() + + val partitionStr = "p=10000" + + case class TableCompressionConf(name: String, codeC: String) case class TableDefine(tableName: String, isPartitioned: Boolean, format: String, - compressionConf: Option[CompressionConf]) { + compressionConf: Option[TableCompressionConf]) { def createTable(rootDir: File): Unit = { val compression = compressionConf.map(cf => s"'${cf.name}'='${cf.codeC}'") sql( s""" |CREATE TABLE $tableName(a int) - |${ if (isPartitioned) "PARTITIONED BY (p int)" else "" } + |${if (isPartitioned) "PARTITIONED BY (p int)" else ""} |STORED AS $format |LOCATION '${rootDir.toURI.toString.stripSuffix("/")}/$tableName' - |${ if (compressionConf.nonEmpty) s"TBLPROPERTIES(${compression.get})" else "" } + |${if (compressionConf.nonEmpty) s"TBLPROPERTIES(${compression.get})" else ""} """.stripMargin) } @@ -751,97 +760,159 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter sql( s""" |INSERT OVERWRITE TABLE $tableName - |${ if (isPartitioned) "partition (p=10000)" else "" } + |${if (isPartitioned) s"partition ($partitionStr)" else ""} |SELECT * from table_source """.stripMargin) } + } + + def getTableCompressionCodec(path: String, format: String): String = { + val codecs = format match { + case "parquet" => for { + footer <- readAllFootersWithoutSummaryFiles(new Path(path), hadoopConf) + block <- footer.getParquetMetadata.getBlocks.asScala + column <- block.getColumns.asScala + } yield column.getCodec.name() + case "orc" => new File(path).listFiles() + .filter(file => file.isFile && !file.getName.endsWith(".crc")).map { + orcFile => + OrcFileOperator.getFileReader(orcFile.toPath.toString).get.getCompression.toString + }.toSeq + } - def getDirFiles(file: File): List[File] = { - if (!file.exists()) Nil - else if (file.isFile) List(file) - else { - file.listFiles().filterNot(_.getName.startsWith(".hive-staging")) - .groupBy(_.isFile).flatMap { - case (isFile, files) if isFile => files.toList - case (_, dirs) => dirs.flatMap(getDirFiles) - }.toList + assert(codecs.distinct.length == 1) + codecs.head + } + + def checkCompressionCodecForTable(format:String, isPartitioned: Boolean, + compressionConf: Option[TableCompressionConf])(assertion: String => Boolean): Unit = { + val table = TableDefine(s"tbl_$format${isPartitioned}", + isPartitioned, format, compressionConf) + withTempDir { tmpDir => + withTable(table.tableName) { + table.createTable(tmpDir) + table.insertOverwriteTable() + val partition = if (table.isPartitioned) partitionStr else "" + val path = s"${tmpDir.getPath.stripSuffix("/")}/${table.tableName}/$partition" + assertion(getTableCompressionCodec(path, table.format)) } } + } + + def getConvertMetastoreConfName(format: String): String = format match { + case "parquet" => "spark.sql.hive.convertMetastoreParquet" + case "orc" => "spark.sql.hive.convertMetastoreOrc" + } - def getTableSize: Long = { - var totalSize = 0L - withTempDir { tmpDir => - withTable(tableName) { - createTable(tmpDir) - insertOverwriteTable() - val path = s"${tmpDir.getPath.stripSuffix("/")}/$tableName" - val dir = new File(path) - val files = getDirFiles(dir).filter(_.getName.startsWith("part-")) - totalSize = files.map(_.length()).sum + def getSparkCompressionConfName(format: String): String = format match { + case "parquet" => "spark.sql.parquet.compression.codec" + case "orc" => "spark.sql.orc.compression.codec" + } + + def checkTableCompressionCodecForCodecs(format: String, isPartitioned: Boolean, + convertMetastore: Boolean, compressionCodecs: List[String], + tableCompressionConf: List[TableCompressionConf]) + (assertion: (Option[TableCompressionConf], String, String) => Boolean): Unit = { + withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { + tableCompressionConf.foreach { tableCompression => + compressionCodecs.foreach { sessionCompressionCodec => + withSQLConf(getSparkCompressionConfName(format) -> sessionCompressionCodec) { + val compression = if (tableCompression == null) None else Some(tableCompression) + checkCompressionCodecForTable(format, isPartitioned, compression) { + case realCompressionCodec => assertion(compression, + sessionCompressionCodec, realCompressionCodec) + } + } } } - totalSize } } - def checkParquetCompressionCodec(isPartitioned: Boolean, tableCodec: String, - sessionCodec: String, f: (Long, Long) => Boolean = _ == _): Unit = { - val tableOrg = TableDefine(s"tbl_parquet$tableCodec", isPartitioned, "parquet", - Some(CompressionConf("parquet.compression", tableCodec))) - val tableOrgSize = tableOrg.getTableSize + def checkTableCompressionCodec(format: String, compressionCodecs: List[String], + tableCompressionConf: List[TableCompressionConf]): Unit = { + // For tables with table-level compression property, when + // 'spark.sql.hive.convertMetastoreParquet' was set to 'false', partitioned parquet tables + // and non-partitioned parquet tables will always take the table-level compression + // configuration first and ignore session compression configuration. + checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, + convertMetastore = false, compressionCodecs, tableCompressionConf) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // table-level take effect + tableCompressionCodec.get.codeC == realCompressionCodec + } + + checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, + convertMetastore = false, compressionCodecs, tableCompressionConf) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // table-level take effect + tableCompressionCodec.get.codeC == realCompressionCodec + } - withSQLConf("spark.sql.parquet.compression.codec" -> sessionCodec) { - // priority check, when table-level compression conf was set, expecting - // table-level compression conf is not affected by the session conf, and table-level - // compression conf takes precedence even the two conf of codec is different - val tableOrgSessionConfSize = tableOrg.getTableSize - assert(tableOrgSize == tableOrgSessionConfSize) + // For tables with table-level compression property, when + // 'spark.sql.hive.convertMetastoreParquet' was set to 'true', partitioned parquet tables + // will always take the table-level compression configuration first, but non-partitioned tables + // will take the session-level compression configuration. + checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, + convertMetastore = true, compressionCodecs, tableCompressionConf) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // table-level take effect + tableCompressionCodec.get.codeC == realCompressionCodec + } - // check session conf of compression codec taking effect - val table = TableDefine(s"tbl_parquet", isPartitioned, "parquet", None) - assert(f(tableOrg.getTableSize, table.getTableSize)) + checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, + convertMetastore = true, compressionCodecs, tableCompressionConf) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // session-level take effect + sessionCompressionCodec == realCompressionCodec } - } - def checkOrcCompressionCodec(isPartitioned: Boolean, tableCodec: String, - sessionCodec: String, f: (Long, Long) => Boolean = _ == _): Unit = { - val tableOrg = TableDefine(s"tbl_orc$tableCodec", isPartitioned, "orc", - Some(CompressionConf("orc.compress", tableCodec))) - val tableOrgSize = tableOrg.getTableSize + // For tables without table-level compression property, session-level compression configuration + // will take effect. + checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, + convertMetastore = true, compressionCodecs, List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // session-level take effect + sessionCompressionCodec == realCompressionCodec + } + + checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, + convertMetastore = true, compressionCodecs, List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // session-level take effect + sessionCompressionCodec == realCompressionCodec + } - withSQLConf("spark.sql.orc.compression.codec" -> sessionCodec) { - // priority check, when table-level compression conf was set, expecting - // table-level compression conf is not affected by the session conf, and table-level - // compression conf takes precedence even the two conf of codec is different - val tableOrgSessionConfSize = tableOrg.getTableSize - assert(tableOrgSize == tableOrgSessionConfSize) + checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, + convertMetastore = false, compressionCodecs, List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // session-level take effect + sessionCompressionCodec == realCompressionCodec + } - // check session conf of compression codec taking effect - val table = TableDefine(s"tbl_orc", isPartitioned, "orc", None) - assert(f(tableOrg.getTableSize, table.getTableSize)) + checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, + convertMetastore = false, compressionCodecs, List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // session-level take effect + sessionCompressionCodec == realCompressionCodec } } withTempView("table_source") { (0 until 100000).toDF("a").createOrReplaceTempView("table_source") - - checkParquetCompressionCodec(true, "UNCOMPRESSED", "UNCOMPRESSED") - checkParquetCompressionCodec(true, "GZIP", "GZIP") - checkParquetCompressionCodec(true, "GZIP", "UNCOMPRESSED", _ < _) - - withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { - checkParquetCompressionCodec(false, "UNCOMPRESSED", "UNCOMPRESSED") - checkParquetCompressionCodec(false, "GZIP", "GZIP") - checkParquetCompressionCodec(false, "GZIP", "UNCOMPRESSED", _ < _) + val parquetCompressionCodec = List("UNCOMPRESSED", "SNAPPY", "GZIP") + val tableCompressionConf = parquetCompressionCodec.map { tableCodec => + TableCompressionConf("parquet.compression", tableCodec) } + checkTableCompressionCodec("parquet", parquetCompressionCodec, tableCompressionConf) + } - checkOrcCompressionCodec(true, "NONE", "NONE") - checkOrcCompressionCodec(true, "ZLIB", "ZLIB") - checkOrcCompressionCodec(true, "ZLIB", "NONE", _ < _) - - checkOrcCompressionCodec(false, "NONE", "NONE") - checkOrcCompressionCodec(false, "ZLIB", "ZLIB") - checkOrcCompressionCodec(false, "ZLIB", "NONE", _ < _) + withTempView("table_source") { + (0 until 100000).toDF("a").createOrReplaceTempView("table_source") + val orcCompressionCodec = List("NONE", "SNAPPY", "ZLIB") + val tableCompressionConf = orcCompressionCodec.map { tableCodec => + TableCompressionConf("parquet.compression", tableCodec) + } + checkTableCompressionCodec("orc", orcCompressionCodec, tableCompressionConf) } } } From ae1da8f8df8ad52694882fcd3a953f2f19f29974 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 27 Sep 2017 22:05:24 +0800 Subject: [PATCH 15/28] Fix scala style issue --- .../org/apache/spark/sql/hive/InsertSuite.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index a6d3a6eeddad2..043b741e03f42 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.hive import java.io.File -import org.scalatest.BeforeAndAfter - import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path +import org.scalatest.BeforeAndAfter + import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.parser.ParseException @@ -784,7 +784,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter codecs.head } - def checkCompressionCodecForTable(format:String, isPartitioned: Boolean, + def checkCompressionCodecForTable(format: String, isPartitioned: Boolean, compressionConf: Option[TableCompressionConf])(assertion: String => Boolean): Unit = { val table = TableDefine(s"tbl_$format${isPartitioned}", isPartitioned, format, compressionConf) @@ -850,8 +850,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter // For tables with table-level compression property, when // 'spark.sql.hive.convertMetastoreParquet' was set to 'true', partitioned parquet tables - // will always take the table-level compression configuration first, but non-partitioned tables - // will take the session-level compression configuration. + // will always take the table-level compression configuration first, but non-partitioned + // tables will take the session-level compression configuration. checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, convertMetastore = true, compressionCodecs, tableCompressionConf) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => @@ -866,8 +866,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter sessionCompressionCodec == realCompressionCodec } - // For tables without table-level compression property, session-level compression configuration - // will take effect. + // For tables without table-level compression property, session-level compression + // configuration will take effect. checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, convertMetastore = true, compressionCodecs, List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => From fd731457e774895a4e28c3d5118ab5e6113151d6 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Thu, 28 Sep 2017 10:09:54 +0800 Subject: [PATCH 16/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix the test case issue --- .../apache/spark/sql/hive/InsertSuite.scala | 68 +++++++++++-------- 1 file changed, 38 insertions(+), 30 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 043b741e03f42..624335fb08e31 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -773,9 +773,9 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter block <- footer.getParquetMetadata.getBlocks.asScala column <- block.getColumns.asScala } yield column.getCodec.name() - case "orc" => new File(path).listFiles() - .filter(file => file.isFile && !file.getName.endsWith(".crc")).map { - orcFile => + case "orc" => new File(path).listFiles().filter{ file => + file.isFile && !file.getName.endsWith(".crc") && file.getName != "_SUCCESS" + }.map { orcFile => OrcFileOperator.getFileReader(orcFile.toPath.toString).get.getCompression.toString }.toSeq } @@ -784,8 +784,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter codecs.head } - def checkCompressionCodecForTable(format: String, isPartitioned: Boolean, - compressionConf: Option[TableCompressionConf])(assertion: String => Boolean): Unit = { + def checkCompressionCodecForTable(format:String, isPartitioned: Boolean, + compressionConf: Option[TableCompressionConf])(assertion: String => Unit): Unit = { val table = TableDefine(s"tbl_$format${isPartitioned}", isPartitioned, format, compressionConf) withTempDir { tmpDir => @@ -812,7 +812,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter def checkTableCompressionCodecForCodecs(format: String, isPartitioned: Boolean, convertMetastore: Boolean, compressionCodecs: List[String], tableCompressionConf: List[TableCompressionConf]) - (assertion: (Option[TableCompressionConf], String, String) => Boolean): Unit = { + (assertion: (Option[TableCompressionConf], String, String) => Unit): Unit = { withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { tableCompressionConf.foreach { tableCompression => compressionCodecs.foreach { sessionCompressionCodec => @@ -831,69 +831,77 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter def checkTableCompressionCodec(format: String, compressionCodecs: List[String], tableCompressionConf: List[TableCompressionConf]): Unit = { // For tables with table-level compression property, when - // 'spark.sql.hive.convertMetastoreParquet' was set to 'false', partitioned parquet tables - // and non-partitioned parquet tables will always take the table-level compression + // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'false', partitioned tables + // and non-partitioned tables will always take the table-level compression // configuration first and ignore session compression configuration. + // Check for partitioned table, when convertMetastore is false checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, convertMetastore = false, compressionCodecs, tableCompressionConf) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // table-level take effect - tableCompressionCodec.get.codeC == realCompressionCodec + // expect table-level take effect + assert(tableCompressionCodec.get.codeC == realCompressionCodec) } + // Check for non-partitioned table, when convertMetastore is false checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, convertMetastore = false, compressionCodecs, tableCompressionConf) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // table-level take effect - tableCompressionCodec.get.codeC == realCompressionCodec + // expect table-level take effect + assert(tableCompressionCodec.get.codeC == realCompressionCodec) } // For tables with table-level compression property, when - // 'spark.sql.hive.convertMetastoreParquet' was set to 'true', partitioned parquet tables + // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'true', partitioned tables // will always take the table-level compression configuration first, but non-partitioned // tables will take the session-level compression configuration. + // Check for partitioned table, when convertMetastore is true checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, convertMetastore = true, compressionCodecs, tableCompressionConf) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // table-level take effect - tableCompressionCodec.get.codeC == realCompressionCodec + // expect table-level take effect + assert(tableCompressionCodec.get.codeC == realCompressionCodec) } + // Check for non-partitioned table, when convertMetastore is true checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, convertMetastore = true, compressionCodecs, tableCompressionConf) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // session-level take effect - sessionCompressionCodec == realCompressionCodec + // expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) } // For tables without table-level compression property, session-level compression // configuration will take effect. + // Check for partitioned table, when convertMetastore is false checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, - convertMetastore = true, compressionCodecs, List(null)) { + convertMetastore = false, compressionCodecs, List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // session-level take effect - sessionCompressionCodec == realCompressionCodec + // expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) } + // Check for non-partitioned table, when convertMetastore is false checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, - convertMetastore = true, compressionCodecs, List(null)) { + convertMetastore = false, compressionCodecs, List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // session-level take effect - sessionCompressionCodec == realCompressionCodec + // expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) } + // Check for partitioned table, when convertMetastore is true checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, - convertMetastore = false, compressionCodecs, List(null)) { + convertMetastore = true, compressionCodecs, List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // session-level take effect - sessionCompressionCodec == realCompressionCodec + // expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) } + // Check for non-partitioned table, when convertMetastore is true checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, - convertMetastore = false, compressionCodecs, List(null)) { + convertMetastore = true, compressionCodecs, List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // session-level take effect - sessionCompressionCodec == realCompressionCodec + // expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) } } @@ -910,7 +918,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter (0 until 100000).toDF("a").createOrReplaceTempView("table_source") val orcCompressionCodec = List("NONE", "SNAPPY", "ZLIB") val tableCompressionConf = orcCompressionCodec.map { tableCodec => - TableCompressionConf("parquet.compression", tableCodec) + TableCompressionConf("orc.compress", tableCodec) } checkTableCompressionCodec("orc", orcCompressionCodec, tableCompressionConf) } From 7615939268e1bea213e6511dcf6e5346ec21fb23 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Thu, 28 Sep 2017 10:18:34 +0800 Subject: [PATCH 17/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix scala style issue --- .../src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 624335fb08e31..fb53ae0ef52d8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -784,7 +784,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter codecs.head } - def checkCompressionCodecForTable(format:String, isPartitioned: Boolean, + def checkCompressionCodecForTable(format: String, isPartitioned: Boolean, compressionConf: Option[TableCompressionConf])(assertion: String => Unit): Unit = { val table = TableDefine(s"tbl_$format${isPartitioned}", isPartitioned, format, compressionConf) From 90cbcb3c58e115995eaa58f61a9cc818d2f17cdf Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Tue, 10 Oct 2017 17:35:28 +0800 Subject: [PATCH 18/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix some issue --- .../sql/hive/execution/SaveAsHiveFile.scala | 19 +- .../apache/spark/sql/hive/InsertSuite.scala | 319 +++++++++++------- 2 files changed, 203 insertions(+), 135 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 467406d35fccb..4099afec14ac8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -69,18 +69,22 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { } fileSinkConf.tableInfo.getOutputFileFormatClassName match { - case formatName if formatName.endsWith("ParquetOutputFormat") => + case formatName if formatName.toLowerCase.endsWith("parquetoutputformat") => val compressionConf = "parquet.compression" - val compressionCodec = getCompressionByPriority(fileSinkConf, compressionConf, - sparkSession.sessionState.conf.parquetCompressionCodec) match { + val compressionCodec = getCompressionByPriority( + fileSinkConf, + compressionConf, + default = sparkSession.sessionState.conf.parquetCompressionCodec) match { case "NONE" => "UNCOMPRESSED" case _@x => x } hadoopConf.set(compressionConf, compressionCodec) case formatName if formatName.endsWith("OrcOutputFormat") => val compressionConf = "orc.compress" - val compressionCodec = getCompressionByPriority(fileSinkConf, compressionConf, - sparkSession.sessionState.conf.orcCompressionCodec) match { + val compressionCodec = getCompressionByPriority( + fileSinkConf, + compressionConf, + default = sparkSession.sessionState.conf.orcCompressionCodec) match { case "UNCOMPRESSED" => "NONE" case _@x => x } @@ -106,8 +110,13 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { options = Map.empty) } + // Because compression configurations can come in a variety of ways, + // we choose the compression configuration in this order: + // For parquet: `compression` > `parquet.compression` > `spark.sql.parquet.compression.codec` + // For orc: `compression` > `orc.compress` > `spark.sql.orc.compression.codec` private def getCompressionByPriority(fileSinkConf: FileSinkDesc, compressionConf: String, default: String): String = { + // The variable `default` was set to spark sql conf. val props = fileSinkConf.tableInfo.getProperties val priorities = List("compression", compressionConf) priorities.find(props.getProperty(_, null) != null) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index fb53ae0ef52d8..69619cbec3c9b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -733,194 +733,253 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } - test("[SPARK-21786] Check 'spark.sql.parquet.compression.codec' " + - "and 'spark.sql.orc.compression.codec' taking effect on hive table writing") { - - val hadoopConf = spark.sessionState.newHadoopConf() - - val partitionStr = "p=10000" - - case class TableCompressionConf(name: String, codeC: String) + private def getConvertMetastoreConfName(format: String): String = format match { + case "parquet" => "spark.sql.hive.convertMetastoreParquet" + case "orc" => "spark.sql.hive.convertMetastoreOrc" + } - case class TableDefine(tableName: String, isPartitioned: Boolean, format: String, - compressionConf: Option[TableCompressionConf]) { - def createTable(rootDir: File): Unit = { - val compression = compressionConf.map(cf => s"'${cf.name}'='${cf.codeC}'") - sql( - s""" - |CREATE TABLE $tableName(a int) - |${if (isPartitioned) "PARTITIONED BY (p int)" else ""} - |STORED AS $format - |LOCATION '${rootDir.toURI.toString.stripSuffix("/")}/$tableName' - |${if (compressionConf.nonEmpty) s"TBLPROPERTIES(${compression.get})" else ""} - """.stripMargin) - } + private def getSparkCompressionConfName(format: String): String = format match { + case "parquet" => "spark.sql.parquet.compression.codec" + case "orc" => "spark.sql.orc.compression.codec" + } - def insertOverwriteTable(): Unit = { - sql( - s""" - |INSERT OVERWRITE TABLE $tableName - |${if (isPartitioned) s"partition ($partitionStr)" else ""} - |SELECT * from table_source - """.stripMargin) - } + private def getTableCompressPropName(format: String): String = { + format.toLowerCase match { + case "parquet" => "parquet.compression" + case "orc" => "orc.compress" } + } - def getTableCompressionCodec(path: String, format: String): String = { - val codecs = format match { - case "parquet" => for { - footer <- readAllFootersWithoutSummaryFiles(new Path(path), hadoopConf) - block <- footer.getParquetMetadata.getBlocks.asScala - column <- block.getColumns.asScala - } yield column.getCodec.name() - case "orc" => new File(path).listFiles().filter{ file => - file.isFile && !file.getName.endsWith(".crc") && file.getName != "_SUCCESS" - }.map { orcFile => - OrcFileOperator.getFileReader(orcFile.toPath.toString).get.getCompression.toString - }.toSeq - } - - assert(codecs.distinct.length == 1) - codecs.head + private def getTableCompressionCodec(path: String, format: String): String = { + val hadoopConf = spark.sessionState.newHadoopConf() + val codecs = format match { + case "parquet" => for { + footer <- readAllFootersWithoutSummaryFiles(new Path(path), hadoopConf) + block <- footer.getParquetMetadata.getBlocks.asScala + column <- block.getColumns.asScala + } yield column.getCodec.name() + case "orc" => new File(path).listFiles().filter{ file => + file.isFile && !file.getName.endsWith(".crc") && file.getName != "_SUCCESS" + }.map { orcFile => + OrcFileOperator.getFileReader(orcFile.toPath.toString).get.getCompression.toString + }.toSeq } - def checkCompressionCodecForTable(format: String, isPartitioned: Boolean, - compressionConf: Option[TableCompressionConf])(assertion: String => Unit): Unit = { - val table = TableDefine(s"tbl_$format${isPartitioned}", - isPartitioned, format, compressionConf) - withTempDir { tmpDir => - withTable(table.tableName) { - table.createTable(tmpDir) - table.insertOverwriteTable() - val partition = if (table.isPartitioned) partitionStr else "" - val path = s"${tmpDir.getPath.stripSuffix("/")}/${table.tableName}/$partition" - assertion(getTableCompressionCodec(path, table.format)) - } - } - } + assert(codecs.distinct.length == 1) + codecs.head + } - def getConvertMetastoreConfName(format: String): String = format match { - case "parquet" => "spark.sql.hive.convertMetastoreParquet" - case "orc" => "spark.sql.hive.convertMetastoreOrc" + private def writeDataToTable( + rootDir: File, + tableName: String, + isPartitioned: Boolean, + format: String, + compressionCodec: Option[String]) { + val tblProperties = compressionCodec match { + case Some(prop) => s"TBLPROPERTIES(${getTableCompressPropName(format)}=$prop)" + case _ => "" } - - def getSparkCompressionConfName(format: String): String = format match { - case "parquet" => "spark.sql.parquet.compression.codec" - case "orc" => "spark.sql.orc.compression.codec" + val partitionCreate = if (isPartitioned) "PARTITIONED BY (p int)" else "" + sql( + s""" + |CREATE TABLE $tableName(a int) + |$partitionCreate + |STORED AS $format + |LOCATION '${rootDir.toURI.toString.stripSuffix("/")}/$tableName' + |$tblProperties + """.stripMargin) + + val partitionInsert = if (isPartitioned) s"partition (p=10000)" else "" + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |$partitionInsert + |SELECT * from table_source + """.stripMargin) + } + + private def checkCompressionCodecForTable( + format: String, + isPartitioned: Boolean, + compressionCodec: Option[String])(assertion: String => Unit): Unit = { + val tableName = s"tbl_$format${isPartitioned}" + withTempDir { tmpDir => + withTable(tableName) { + writeDataToTable(tmpDir, tableName, isPartitioned, format, compressionCodec) + val partition = if (isPartitioned) "p=10000" else "" + val path = s"${tmpDir.getPath.stripSuffix("/")}/${tableName}/$partition" + assertion(getTableCompressionCodec(path, format)) + } } + } - def checkTableCompressionCodecForCodecs(format: String, isPartitioned: Boolean, - convertMetastore: Boolean, compressionCodecs: List[String], - tableCompressionConf: List[TableCompressionConf]) - (assertion: (Option[TableCompressionConf], String, String) => Unit): Unit = { - withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { - tableCompressionConf.foreach { tableCompression => - compressionCodecs.foreach { sessionCompressionCodec => - withSQLConf(getSparkCompressionConfName(format) -> sessionCompressionCodec) { - val compression = if (tableCompression == null) None else Some(tableCompression) - checkCompressionCodecForTable(format, isPartitioned, compression) { - case realCompressionCodec => assertion(compression, - sessionCompressionCodec, realCompressionCodec) - } + private def checkTableCompressionCodecForCodecs( + format: String, + isPartitioned: Boolean, + convertMetastore: Boolean, + compressionCodecs: List[String], + tableCompressionCodecs: List[String]) ( + assertion: (Option[String], String, String) => Unit): Unit = { + withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { + tableCompressionCodecs.foreach { tableCompression => + compressionCodecs.foreach { sessionCompressionCodec => + withSQLConf(getSparkCompressionConfName(format) -> sessionCompressionCodec) { + val compression = if (tableCompression == null) None else Some(tableCompression) + checkCompressionCodecForTable(format, isPartitioned, compression) { + case realCompressionCodec => assertion(compression, + sessionCompressionCodec, realCompressionCodec) } } } } } + } + + private def testCompressionCodec(testCondition: String)(f: => Unit): Unit = { + test("[SPARK-21786] - Check the priority between table-level compression and " + + s"session-level compression $testCondition") { + withTempView("table_source") { + (0 until 100000).toDF("a").createOrReplaceTempView("table_source") + f + } + } + } - def checkTableCompressionCodec(format: String, compressionCodecs: List[String], - tableCompressionConf: List[TableCompressionConf]): Unit = { + testCompressionCodec("when table-level and session-level compression are both configured and " + + "convertMetastore is false") { + def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { // For tables with table-level compression property, when // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'false', partitioned tables // and non-partitioned tables will always take the table-level compression // configuration first and ignore session compression configuration. // Check for partitioned table, when convertMetastore is false - checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, - convertMetastore = false, compressionCodecs, tableCompressionConf) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect table-level take effect - assert(tableCompressionCodec.get.codeC == realCompressionCodec) - } - - // Check for non-partitioned table, when convertMetastore is false - checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, - convertMetastore = false, compressionCodecs, tableCompressionConf) { + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => // expect table-level take effect - assert(tableCompressionCodec.get.codeC == realCompressionCodec) + assert(tableCompressionCodec.get == realCompressionCodec) } - // For tables with table-level compression property, when - // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'true', partitioned tables - // will always take the table-level compression configuration first, but non-partitioned - // tables will take the session-level compression configuration. - // Check for partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, - convertMetastore = true, compressionCodecs, tableCompressionConf) { + // Check for non-partitioned table, when convertMetastoreParquet is false + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => // expect table-level take effect - assert(tableCompressionCodec.get.codeC == realCompressionCodec) + assert(tableCompressionCodec.get == realCompressionCodec) } + } - // Check for non-partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, - convertMetastore = true, compressionCodecs, tableCompressionConf) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect session-level take effect - assert(sessionCompressionCodec == realCompressionCodec) - } + checkForTableWithCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + } + testCompressionCodec("when there's no table-level compression and convertMetastore is false") { + def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { // For tables without table-level compression property, session-level compression // configuration will take effect. // Check for partitioned table, when convertMetastore is false - checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, - convertMetastore = false, compressionCodecs, List(null)) { + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => // expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } // Check for non-partitioned table, when convertMetastore is false - checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, - convertMetastore = false, compressionCodecs, List(null)) { + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => // expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } + } + checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + } + + testCompressionCodec("when table-level and session-level compression are both configured and " + + "convertMetastore is true") { + def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { + // For tables with table-level compression property, when + // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'true', partitioned tables + // will always take the table-level compression configuration first, but non-partitioned + // tables will take the session-level compression configuration. // Check for partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs(format = format, isPartitioned = true, - convertMetastore = true, compressionCodecs, List(null)) { + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect session-level take effect - assert(sessionCompressionCodec == realCompressionCodec) + // expect table-level take effect + assert(tableCompressionCodec.get == realCompressionCodec) } // Check for non-partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs(format = format, isPartitioned = false, - convertMetastore = true, compressionCodecs, List(null)) { + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => // expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } } - withTempView("table_source") { - (0 until 100000).toDF("a").createOrReplaceTempView("table_source") - val parquetCompressionCodec = List("UNCOMPRESSED", "SNAPPY", "GZIP") - val tableCompressionConf = parquetCompressionCodec.map { tableCodec => - TableCompressionConf("parquet.compression", tableCodec) + checkForTableWithCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + } + + testCompressionCodec("when there's no table-level compression and convertMetastore is true") { + def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { + // For tables without table-level compression property, session-level compression + // configuration will take effect. + // Check for partitioned table, when convertMetastore is true + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) } - checkTableCompressionCodec("parquet", parquetCompressionCodec, tableCompressionConf) - } - withTempView("table_source") { - (0 until 100000).toDF("a").createOrReplaceTempView("table_source") - val orcCompressionCodec = List("NONE", "SNAPPY", "ZLIB") - val tableCompressionConf = orcCompressionCodec.map { tableCodec => - TableCompressionConf("orc.compress", tableCodec) + // Check for non-partitioned table, when convertMetastore is true + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => + // expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) } - checkTableCompressionCodec("orc", orcCompressionCodec, tableCompressionConf) } + + checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) } } From dd6d635eee108706296e13a6d09d0c79ff912f13 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Tue, 10 Oct 2017 18:51:47 +0800 Subject: [PATCH 19/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix test issue --- .../src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 69619cbec3c9b..5a810d90bf8cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -776,7 +776,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter format: String, compressionCodec: Option[String]) { val tblProperties = compressionCodec match { - case Some(prop) => s"TBLPROPERTIES(${getTableCompressPropName(format)}=$prop)" + case Some(prop) => s"TBLPROPERTIES('${getTableCompressPropName(format)}'='$prop')" case _ => "" } val partitionCreate = if (isPartitioned) "PARTITIONED BY (p int)" else "" From 4fe8170a915319a461b409ae284840a83e83c784 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Thu, 12 Oct 2017 14:22:56 +0800 Subject: [PATCH 20/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix test issue --- .../org/apache/spark/sql/hive/InsertSuite.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 5a810d90bf8cb..b852d7918f444 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.hive.orc.OrcFileOperator import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -734,16 +735,16 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } private def getConvertMetastoreConfName(format: String): String = format match { - case "parquet" => "spark.sql.hive.convertMetastoreParquet" - case "orc" => "spark.sql.hive.convertMetastoreOrc" + case "parquet" => HiveUtils.CONVERT_METASTORE_PARQUET.key + case "orc" => HiveUtils.CONVERT_METASTORE_ORC.key } private def getSparkCompressionConfName(format: String): String = format match { - case "parquet" => "spark.sql.parquet.compression.codec" - case "orc" => "spark.sql.orc.compression.codec" + case "parquet" => SQLConf.PARQUET_COMPRESSION.key + case "orc" => SQLConf.ORC_COMPRESSION.key } - private def getTableCompressPropName(format: String): String = { + private def getHiveCompressPropName(format: String): String = { format.toLowerCase match { case "parquet" => "parquet.compression" case "orc" => "orc.compress" @@ -776,7 +777,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter format: String, compressionCodec: Option[String]) { val tblProperties = compressionCodec match { - case Some(prop) => s"TBLPROPERTIES('${getTableCompressPropName(format)}'='$prop')" + case Some(prop) => s"TBLPROPERTIES('${getHiveCompressPropName(format)}'='$prop')" case _ => "" } val partitionCreate = if (isPartitioned) "PARTITIONED BY (p int)" else "" @@ -794,7 +795,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter s""" |INSERT OVERWRITE TABLE $tableName |$partitionInsert - |SELECT * from table_source + |SELECT * FROM table_source """.stripMargin) } From aa31261860a1042fca471e4cec801353e8a7e2ac Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Mon, 16 Oct 2017 09:12:46 +0800 Subject: [PATCH 21/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Add a comment. --- .../src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index b852d7918f444..a10597f8ca7cd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -825,6 +825,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter tableCompressionCodecs.foreach { tableCompression => compressionCodecs.foreach { sessionCompressionCodec => withSQLConf(getSparkCompressionConfName(format) -> sessionCompressionCodec) { + // 'tableCompression = null' means no table-level compression val compression = if (tableCompression == null) None else Some(tableCompression) checkCompressionCodecForTable(format, isPartitioned, compression) { case realCompressionCodec => assertion(compression, From c4801f62800b397ce1c04d218cd4b92431fc7246 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Mon, 16 Oct 2017 14:00:58 +0800 Subject: [PATCH 22/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix scala style --- .../src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 2fe66dc30052b..52ac657a01b4f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -1006,4 +1006,4 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) } -} \ No newline at end of file +} From 105e1290f734995bb1712422afc89e5e63664bef Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Mon, 16 Oct 2017 20:26:47 +0800 Subject: [PATCH 23/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Modify comment. --- .../org/apache/spark/sql/hive/InsertSuite.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 52ac657a01b4f..d4cf5c93c40c3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -884,7 +884,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect table-level take effect + // Expect table-level take effect assert(tableCompressionCodec.get == realCompressionCodec) } @@ -896,7 +896,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect table-level take effect + // Expect table-level take effect assert(tableCompressionCodec.get == realCompressionCodec) } } @@ -917,7 +917,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect session-level take effect + // Expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } @@ -929,7 +929,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect session-level take effect + // Expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } } @@ -953,7 +953,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect table-level take effect + // Expect table-level take effect assert(tableCompressionCodec.get == realCompressionCodec) } @@ -965,7 +965,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = compressCodecs) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect session-level take effect + // Expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } } @@ -986,7 +986,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect session-level take effect + // Expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } @@ -998,7 +998,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter compressionCodecs = compressCodecs, tableCompressionCodecs = List(null)) { case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // expect session-level take effect + // Expect session-level take effect assert(sessionCompressionCodec == realCompressionCodec) } } From d779ee6524b199b695c938bbb0c3436c73c64c87 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Tue, 19 Dec 2017 14:34:45 +0800 Subject: [PATCH 24/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Move the whole determination logics to HiveOptions --- .../datasources/parquet/ParquetOptions.scala | 14 +++++-- .../sql/hive/execution/HiveOptions.scala | 21 ++++++++++ .../sql/hive/execution/SaveAsHiveFile.scala | 41 +++---------------- .../apache/spark/sql/hive/InsertSuite.scala | 29 +++++++------ 4 files changed, 51 insertions(+), 54 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 772d4565de548..e5215169e3101 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.util.Locale import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.internal.SQLConf @@ -27,7 +28,7 @@ import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -private[parquet] class ParquetOptions( +class ParquetOptions( @transient private val parameters: CaseInsensitiveMap[String], @transient private val sqlConf: SQLConf) extends Serializable { @@ -42,8 +43,15 @@ private[parquet] class ParquetOptions( * Acceptable values are defined in [[shortParquetCompressionCodecNames]]. */ val compressionCodecClassName: String = { - val codecName = parameters.getOrElse("compression", - sqlConf.parquetCompressionCodec).toLowerCase(Locale.ROOT) + // `compression`, `parquet.compression`(i.e., ParquetOutputFormat.COMPRESSION), and + // `spark.sql.parquet.compression.codec` + // are in order of precedence from highest to lowest. + val parquetCompressionConf = parameters.get(ParquetOutputFormat.COMPRESSION) + val codecName = parameters + .get("compression") + .orElse(parquetCompressionConf) + .getOrElse(sqlConf.parquetCompressionCodec) + .toLowerCase(Locale.ROOT) if (!shortParquetCompressionCodecNames.contains(codecName)) { val availableCodecs = shortParquetCompressionCodecNames.keys.map(_.toLowerCase(Locale.ROOT)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala index 5c515515b9b9c..3d6e98d4252bd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala @@ -19,7 +19,14 @@ package org.apache.spark.sql.hive.execution import java.util.Locale +import scala.collection.JavaConverters._ +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.orc.OrcConf.COMPRESS +import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.datasources.orc.OrcOptions +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.internal.SQLConf /** * Options for the Hive data source. Note that rule `DetermineHiveSerde` will extract Hive @@ -102,4 +109,18 @@ object HiveOptions { "collectionDelim" -> "colelction.delim", "mapkeyDelim" -> "mapkey.delim", "lineDelim" -> "line.delim").map { case (k, v) => k.toLowerCase(Locale.ROOT) -> v } + + def getHiveWriteCompression(tableInfo: TableDesc, sqlConf: SQLConf): Option[(String, String)]= { + tableInfo.getOutputFileFormatClassName.toLowerCase match { + case formatName if formatName.endsWith("parquetoutputformat") => + val compressionCodec = new ParquetOptions(tableInfo.getProperties.asScala.toMap, + sqlConf).compressionCodecClassName + Option((ParquetOutputFormat.COMPRESSION, compressionCodec)) + case formatName if formatName.endsWith("orcoutputformat") => + val compressionCodec = new OrcOptions(tableInfo.getProperties.asScala.toMap, + sqlConf).compressionCodec + Option((COMPRESS.getAttribute, compressionCodec)) + case _ => None + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 452b0309a344e..054686bbcd200 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -68,29 +68,11 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { .get("mapreduce.output.fileoutputformat.compress.type")) } - fileSinkConf.tableInfo.getOutputFileFormatClassName match { - case formatName if formatName.toLowerCase.endsWith("parquetoutputformat") => - val compressionConf = "parquet.compression" - val compressionCodec = getCompressionByPriority( - fileSinkConf, - compressionConf, - default = sparkSession.sessionState.conf.parquetCompressionCodec) match { - case "NONE" => "UNCOMPRESSED" - case _@x => x - } - hadoopConf.set(compressionConf, compressionCodec) - case formatName if formatName.endsWith("OrcOutputFormat") => - val compressionConf = "orc.compress" - val compressionCodec = getCompressionByPriority( - fileSinkConf, - compressionConf, - default = sparkSession.sessionState.conf.orcCompressionCodec) match { - case "UNCOMPRESSED" => "NONE" - case _@x => x - } - hadoopConf.set(compressionConf, compressionCodec) - case _ => - } + // Set compression by priority + HiveOptions.getHiveWriteCompression(fileSinkConf.getTableInfo, sparkSession.sessionState.conf) + .foreach{ case (compression, codec) => + hadoopConf.set(compression, codec) + } val committer = FileCommitProtocol.instantiate( sparkSession.sessionState.conf.fileCommitProtocolClass, @@ -110,19 +92,6 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { options = Map.empty) } - // Because compression configurations can come in a variety of ways, - // we choose the compression configuration in this order: - // For parquet: `compression` > `parquet.compression` > `spark.sql.parquet.compression.codec` - // For orc: `compression` > `orc.compress` > `spark.sql.orc.compression.codec` - private def getCompressionByPriority(fileSinkConf: FileSinkDesc, - compressionConf: String, default: String): String = { - // The variable `default` was set to spark sql conf. - val props = fileSinkConf.tableInfo.getProperties - val priorities = List("compression", compressionConf) - priorities.find(props.getProperty(_, null) != null) - .map(props.getProperty).getOrElse(default).toUpperCase(Locale.ROOT) - } - protected def getExternalTmpPath( sparkSession: SparkSession, hadoopConf: Configuration, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index d4cf5c93c40c3..9a6971f8e0cef 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.hive.orc.OrcFileOperator import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -793,11 +792,11 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } private def writeDataToTable( - rootDir: File, - tableName: String, - isPartitioned: Boolean, - format: String, - compressionCodec: Option[String]) { + rootDir: File, + tableName: String, + isPartitioned: Boolean, + format: String, + compressionCodec: Option[String]) { val tblProperties = compressionCodec match { case Some(prop) => s"TBLPROPERTIES('${getHiveCompressPropName(format)}'='$prop')" case _ => "" @@ -822,9 +821,9 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } private def checkCompressionCodecForTable( - format: String, - isPartitioned: Boolean, - compressionCodec: Option[String])(assertion: String => Unit): Unit = { + format: String, + isPartitioned: Boolean, + compressionCodec: Option[String])(assertion: String => Unit): Unit = { val tableName = s"tbl_$format${isPartitioned}" withTempDir { tmpDir => withTable(tableName) { @@ -837,12 +836,12 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } private def checkTableCompressionCodecForCodecs( - format: String, - isPartitioned: Boolean, - convertMetastore: Boolean, - compressionCodecs: List[String], - tableCompressionCodecs: List[String]) ( - assertion: (Option[String], String, String) => Unit): Unit = { + format: String, + isPartitioned: Boolean, + convertMetastore: Boolean, + compressionCodecs: List[String], + tableCompressionCodecs: List[String]) + (assertion: (Option[String], String, String) => Unit): Unit = { withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { tableCompressionCodecs.foreach { tableCompression => compressionCodecs.foreach { sessionCompressionCodec => From 0cb7b7af517f37c428faa394f652bc564ecb097f Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Wed, 20 Dec 2017 17:54:46 +0800 Subject: [PATCH 25/28] [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing Fix scala style --- .../sql/execution/datasources/parquet/ParquetOptions.scala | 2 +- .../org/apache/spark/sql/hive/execution/HiveOptions.scala | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index e5215169e3101..193ce8e9d8e63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.execution.datasources.parquet import java.util.Locale -import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.parquet.hadoop.ParquetOutputFormat +import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.internal.SQLConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala index 3d6e98d4252bd..089f6d7e05bf3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala @@ -20,9 +20,11 @@ package org.apache.spark.sql.hive.execution import java.util.Locale import scala.collection.JavaConverters._ + import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} import org.apache.orc.OrcConf.COMPRESS import org.apache.parquet.hadoop.ParquetOutputFormat + import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.orc.OrcOptions import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -110,7 +112,7 @@ object HiveOptions { "mapkeyDelim" -> "mapkey.delim", "lineDelim" -> "line.delim").map { case (k, v) => k.toLowerCase(Locale.ROOT) -> v } - def getHiveWriteCompression(tableInfo: TableDesc, sqlConf: SQLConf): Option[(String, String)]= { + def getHiveWriteCompression(tableInfo: TableDesc, sqlConf: SQLConf): Option[(String, String)] = { tableInfo.getOutputFileFormatClassName.toLowerCase match { case formatName if formatName.endsWith("parquetoutputformat") => val compressionCodec = new ParquetOptions(tableInfo.getProperties.asScala.toMap, From 78e0403053b00c9b1272e700effb4af4eae34ce8 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Sat, 23 Dec 2017 16:56:54 +0800 Subject: [PATCH 26/28] Resume the changing, and change it in another pr later. --- .../datasources/parquet/ParquetOptions.scala | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 193ce8e9d8e63..772d4565de548 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.datasources.parquet import java.util.Locale -import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap @@ -28,7 +27,7 @@ import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -class ParquetOptions( +private[parquet] class ParquetOptions( @transient private val parameters: CaseInsensitiveMap[String], @transient private val sqlConf: SQLConf) extends Serializable { @@ -43,15 +42,8 @@ class ParquetOptions( * Acceptable values are defined in [[shortParquetCompressionCodecNames]]. */ val compressionCodecClassName: String = { - // `compression`, `parquet.compression`(i.e., ParquetOutputFormat.COMPRESSION), and - // `spark.sql.parquet.compression.codec` - // are in order of precedence from highest to lowest. - val parquetCompressionConf = parameters.get(ParquetOutputFormat.COMPRESSION) - val codecName = parameters - .get("compression") - .orElse(parquetCompressionConf) - .getOrElse(sqlConf.parquetCompressionCodec) - .toLowerCase(Locale.ROOT) + val codecName = parameters.getOrElse("compression", + sqlConf.parquetCompressionCodec).toLowerCase(Locale.ROOT) if (!shortParquetCompressionCodecNames.contains(codecName)) { val availableCodecs = shortParquetCompressionCodecNames.keys.map(_.toLowerCase(Locale.ROOT)) From 7804f60f9fb6095873f5b6ba6d7e975361327f62 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Sat, 23 Dec 2017 17:02:11 +0800 Subject: [PATCH 27/28] Change to public --- .../sql/execution/datasources/parquet/ParquetOptions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 772d4565de548..e8992703de240 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -private[parquet] class ParquetOptions( +class ParquetOptions( @transient private val parameters: CaseInsensitiveMap[String], @transient private val sqlConf: SQLConf) extends Serializable { From 52cdd75f7845d35aca41562f670c98384fb015b2 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Sat, 23 Dec 2017 21:34:58 +0800 Subject: [PATCH 28/28] Fix the code with gatorsmile's suggestion. --- .../datasources/orc/OrcFileFormat.scala | 2 +- .../datasources/orc/OrcOptions.scala | 2 +- .../datasources/orc/OrcSourceSuite.scala | 13 +- .../sql/hive/execution/HiveOptions.scala | 9 +- .../sql/hive/execution/SaveAsHiveFile.scala | 4 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 2 +- .../sql/hive/CompressionCodecSuite.scala | 360 ++++++++++++++++++ .../apache/spark/sql/hive/InsertSuite.scala | 261 +------------ 8 files changed, 378 insertions(+), 275 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index f7471cd7debce..b6c135bb23219 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -94,7 +94,7 @@ class OrcFileFormat conf.set(MAPRED_OUTPUT_SCHEMA.getAttribute, dataSchema.catalogString) - conf.set(COMPRESS.getAttribute, orcOptions.compressionCodec) + conf.set(COMPRESS.getAttribute, orcOptions.compressionCodecClassName) conf.asInstanceOf[JobConf] .setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala index c866dd834a525..12930c8e3b971 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala @@ -41,7 +41,7 @@ class OrcOptions( * Compression codec to use. * Acceptable values are defined in [[shortOrcCompressionCodecNames]]. */ - val compressionCodec: String = { + val compressionCodecClassName: String = { // `compression`, `orc.compress`(i.e., OrcConf.COMPRESS), and `spark.sql.orc.compression.codec` // are in order of precedence from highest to lowest. val orcCompressionConf = parameters.get(COMPRESS.getAttribute) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 6f5f2fd795f74..beef8c2f1c3b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -134,29 +134,30 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { val conf = spark.sessionState.conf val option = new OrcOptions(Map(COMPRESS.getAttribute.toUpperCase(Locale.ROOT) -> "NONE"), conf) - assert(option.compressionCodec == "NONE") + assert(option.compressionCodecClassName == "NONE") } test("SPARK-21839: Add SQL config for ORC compression") { val conf = spark.sessionState.conf // Test if the default of spark.sql.orc.compression.codec is snappy - assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "SNAPPY") + assert(new OrcOptions(Map.empty[String, String], conf).compressionCodecClassName == "SNAPPY") // OrcOptions's parameters have a higher priority than SQL configuration. // `compression` -> `orc.compression` -> `spark.sql.orc.compression.codec` withSQLConf(SQLConf.ORC_COMPRESSION.key -> "uncompressed") { - assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "NONE") + assert(new OrcOptions(Map.empty[String, String], conf).compressionCodecClassName == "NONE") val map1 = Map(COMPRESS.getAttribute -> "zlib") val map2 = Map(COMPRESS.getAttribute -> "zlib", "compression" -> "lzo") - assert(new OrcOptions(map1, conf).compressionCodec == "ZLIB") - assert(new OrcOptions(map2, conf).compressionCodec == "LZO") + assert(new OrcOptions(map1, conf).compressionCodecClassName == "ZLIB") + assert(new OrcOptions(map2, conf).compressionCodecClassName == "LZO") } // Test all the valid options of spark.sql.orc.compression.codec Seq("NONE", "UNCOMPRESSED", "SNAPPY", "ZLIB", "LZO").foreach { c => withSQLConf(SQLConf.ORC_COMPRESSION.key -> c) { val expected = if (c == "UNCOMPRESSED") "NONE" else c - assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == expected) + assert( + new OrcOptions(Map.empty[String, String], conf).compressionCodecClassName == expected) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala index 089f6d7e05bf3..c15a62811c3da 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala @@ -21,7 +21,7 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.orc.OrcConf.COMPRESS import org.apache.parquet.hadoop.ParquetOutputFormat @@ -113,14 +113,13 @@ object HiveOptions { "lineDelim" -> "line.delim").map { case (k, v) => k.toLowerCase(Locale.ROOT) -> v } def getHiveWriteCompression(tableInfo: TableDesc, sqlConf: SQLConf): Option[(String, String)] = { + val tableProps = tableInfo.getProperties.asScala.toMap tableInfo.getOutputFileFormatClassName.toLowerCase match { case formatName if formatName.endsWith("parquetoutputformat") => - val compressionCodec = new ParquetOptions(tableInfo.getProperties.asScala.toMap, - sqlConf).compressionCodecClassName + val compressionCodec = new ParquetOptions(tableProps, sqlConf).compressionCodecClassName Option((ParquetOutputFormat.COMPRESSION, compressionCodec)) case formatName if formatName.endsWith("orcoutputformat") => - val compressionCodec = new OrcOptions(tableInfo.getProperties.asScala.toMap, - sqlConf).compressionCodec + val compressionCodec = new OrcOptions(tableProps, sqlConf).compressionCodecClassName Option((COMPRESS.getAttribute, compressionCodec)) case _ => None } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 054686bbcd200..734d8350f3870 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -70,9 +70,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { // Set compression by priority HiveOptions.getHiveWriteCompression(fileSinkConf.getTableInfo, sparkSession.sessionState.conf) - .foreach{ case (compression, codec) => - hadoopConf.set(compression, codec) - } + .foreach { case (compression, codec) => hadoopConf.set(compression, codec) } val committer = FileCommitProtocol.instantiate( sparkSession.sessionState.conf.fileCommitProtocolClass, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 95741c7b30289..5715893920ac6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -74,7 +74,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable val configuration = job.getConfiguration - configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodec) + configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodecClassName) configuration match { case conf: JobConf => conf.setOutputFormat(classOf[OrcOutputFormat]) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala new file mode 100644 index 0000000000000..c7a6f254fc184 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala @@ -0,0 +1,360 @@ +/* + * 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.hive + +import java.io.File + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path +import org.apache.orc.OrcConf.COMPRESS +import org.apache.parquet.hadoop.ParquetOutputFormat + +import org.apache.spark.sql.execution.datasources.parquet.ParquetTest +import org.apache.spark.sql.hive.orc.OrcFileOperator +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf + +class CompressionCodecSuite extends TestHiveSingleton with ParquetTest { + import spark.implicits._ + + private val maxRecordNum = 100000 + + private def getConvertMetastoreConfName(format: String): String = format match { + case "parquet" => HiveUtils.CONVERT_METASTORE_PARQUET.key + case "orc" => HiveUtils.CONVERT_METASTORE_ORC.key + } + + private def getSparkCompressionConfName(format: String): String = format match { + case "parquet" => SQLConf.PARQUET_COMPRESSION.key + case "orc" => SQLConf.ORC_COMPRESSION.key + } + + private def getHiveCompressPropName(format: String): String = { + format.toLowerCase match { + case "parquet" => ParquetOutputFormat.COMPRESSION + case "orc" => COMPRESS.getAttribute + } + } + + private def getTableCompressionCodec(path: String, format: String): String = { + val hadoopConf = spark.sessionState.newHadoopConf() + val codecs = format match { + case "parquet" => for { + footer <- readAllFootersWithoutSummaryFiles(new Path(path), hadoopConf) + block <- footer.getParquetMetadata.getBlocks.asScala + column <- block.getColumns.asScala + } yield column.getCodec.name() + case "orc" => new File(path).listFiles().filter{ file => + file.isFile && !file.getName.endsWith(".crc") && file.getName != "_SUCCESS" + }.map { orcFile => + OrcFileOperator.getFileReader(orcFile.toPath.toString).get.getCompression.toString + }.toSeq + } + + assert(codecs.distinct.length == 1) + codecs.head + } + + private def writeDataToTable( + rootDir: File, + tableName: String, + isPartitioned: Boolean, + format: String, + compressionCodec: Option[String], + dataSourceName: String = "table_source"): Unit = { + val tblProperties = compressionCodec match { + case Some(prop) => s"TBLPROPERTIES('${getHiveCompressPropName(format)}'='$prop')" + case _ => "" + } + val partitionCreate = if (isPartitioned) "PARTITIONED BY (p int)" else "" + sql( + s""" + |CREATE TABLE $tableName(a int) + |$partitionCreate + |STORED AS $format + |LOCATION '${rootDir.toURI.toString.stripSuffix("/")}/$tableName' + |$tblProperties + """.stripMargin) + + val partitionInsert = if (isPartitioned) s"partition (p=10000)" else "" + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |$partitionInsert + |SELECT * FROM $dataSourceName + """.stripMargin) + } + + private def getTableSize(path: String): Long = { + val dir = new File(path) + val files = dir.listFiles().filter(_.getName.startsWith("part-")) + files.map(_.length()).sum + } + + private def getDataSizeByFormat(format: String, compressionCodec: Option[String], isPartitioned: Boolean): Long = { + var totalSize = 0L + val tableName = s"tbl_$format${compressionCodec.mkString}" + withTempView("datasource_table") { + (0 until maxRecordNum).toDF("a").createOrReplaceTempView("datasource_table") + withSQLConf(getSparkCompressionConfName(format) -> compressionCodec.getOrElse("uncompressed")) { + withTempDir { tmpDir => + withTable(tableName) { + writeDataToTable(tmpDir, tableName, isPartitioned, format, compressionCodec, "datasource_table") + val partition = if (isPartitioned) "p=10000" else "" + val path =s"${tmpDir.getPath.stripSuffix("/")}/${tableName}/$partition" + totalSize = getTableSize(path) + } + } + } + } + assert(totalSize > 0L) + totalSize + } + + private def checkCompressionCodecForTable( + format: String, + isPartitioned: Boolean, + compressionCodec: Option[String]) + (assertion: (String, Long) => Unit): Unit = { + val tableName = s"tbl_$format${isPartitioned}" + withTempDir { tmpDir => + withTable(tableName) { + writeDataToTable(tmpDir, tableName, isPartitioned, format, compressionCodec) + val partition = if (isPartitioned) "p=10000" else "" + val path = s"${tmpDir.getPath.stripSuffix("/")}/${tableName}/$partition" + val relCompressionCodec = getTableCompressionCodec(path, format) + val tableSize = getTableSize(path) + assertion(relCompressionCodec, tableSize) + } + } + } + + private def checkTableCompressionCodecForCodecs( + format: String, + isPartitioned: Boolean, + convertMetastore: Boolean, + compressionCodecs: List[String], + tableCompressionCodecs: List[String]) + (assertionCompressionCodec: (Option[String], String, String, Long) => Unit): Unit = { + withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { + tableCompressionCodecs.foreach { tableCompression => + compressionCodecs.foreach { sessionCompressionCodec => + withSQLConf(getSparkCompressionConfName(format) -> sessionCompressionCodec) { + // 'tableCompression = null' means no table-level compression + val compression = Option(tableCompression) + checkCompressionCodecForTable(format, isPartitioned, compression) { + case (realCompressionCodec, tableSize) => assertionCompressionCodec(compression, + sessionCompressionCodec, realCompressionCodec, tableSize) + } + } + } + } + } + } + + // To check if the compressionCodec takes effect, we check the data size with uncompressed size. + // and because partitioned table's schema may different with non-partitioned table's schema when + // convertMetastore is true, e.g. parquet, we save them independently. + private val partitionedParquetTableUncompressedSize = getDataSizeByFormat("parquet", None, true) + private val nonpartitionedParquetTableUncompressedSize = getDataSizeByFormat("parquet", None, false) + + // Orc data seems be consistent within partitioned table and non-partitioned table, + // but we just make the code look the same. + private val partitionedOrcTableUncompressedSize = getDataSizeByFormat("orc", None, true) + private val nonpartitionedOrcTableUncompressedSize = getDataSizeByFormat("orc", None, false) + + // When the amount of data is small, compressed data size may be smaller than uncompressed one, + // so we just check the difference when compressionCodec is not NONE or UNCOMPRESSED. + // When convertMetastore is false, the uncompressed table size should be same as + // `partitionedParquetTableUncompressedSize`, regardless of whether there is a partition. + private def checkTableSize(format: String, compressionCodec: String, + ispartitioned: Boolean, convertMetastore: Boolean, tableSize: Long): Boolean = { + format match { + case "parquet" => val uncompressedSize = + if(!convertMetastore || ispartitioned) partitionedParquetTableUncompressedSize + else nonpartitionedParquetTableUncompressedSize + + if(compressionCodec == "UNCOMPRESSED") tableSize == uncompressedSize + else tableSize != uncompressedSize + case "orc" => val uncompressedSize = + if(!convertMetastore || ispartitioned) partitionedOrcTableUncompressedSize + else nonpartitionedOrcTableUncompressedSize + + if(compressionCodec == "NONE") tableSize == uncompressedSize + else tableSize != uncompressedSize + case _ => false + } + } + + private def testCompressionCodec(testCondition: String)(f: => Unit): Unit = { + test("[SPARK-21786] - Check the priority between table-level compression and " + + s"session-level compression $testCondition") { + withTempView("table_source") { + (0 until maxRecordNum).toDF("a").createOrReplaceTempView("table_source") + f + } + } + } + + testCompressionCodec("when table-level and session-level compression are both configured and " + + "convertMetastore is false") { + def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { + // For tables with table-level compression property, when + // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'false', partitioned tables + // and non-partitioned tables will always take the table-level compression + // configuration first and ignore session compression configuration. + // Check for partitioned table, when convertMetastore is false + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect table-level take effect + assert(tableCompressionCodec.get == realCompressionCodec) + assert(checkTableSize(format, tableCompressionCodec.get, true, false, tableSize)) + } + + // Check for non-partitioned table, when convertMetastoreParquet is false + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect table-level take effect + assert(tableCompressionCodec.get == realCompressionCodec) + assert(checkTableSize(format, tableCompressionCodec.get, false, false, tableSize)) + } + } + + checkForTableWithCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + } + + testCompressionCodec("when there's no table-level compression and convertMetastore is false") { + def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { + // For tables without table-level compression property, session-level compression + // configuration will take effect. + // Check for partitioned table, when convertMetastore is false + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) + assert(checkTableSize(format, sessionCompressionCodec, true, false, tableSize)) + } + + // Check for non-partitioned table, when convertMetastore is false + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = false, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) + assert(checkTableSize(format, sessionCompressionCodec, false, false, tableSize)) + } + } + + checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + } + + testCompressionCodec("when table-level and session-level compression are both configured and " + + "convertMetastore is true") { + def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { + // For tables with table-level compression property, when + // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'true', partitioned tables + // will always take the table-level compression configuration first, but non-partitioned + // tables will take the session-level compression configuration. + // Check for partitioned table, when convertMetastore is true + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect table-level take effect + assert(tableCompressionCodec.get == realCompressionCodec) + assert(checkTableSize(format, tableCompressionCodec.get, true, true, tableSize)) + } + + // Check for non-partitioned table, when convertMetastore is true + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = compressCodecs) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) + assert(checkTableSize(format, sessionCompressionCodec, false, true, tableSize)) + } + } + + checkForTableWithCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + } + + testCompressionCodec("when there's no table-level compression and convertMetastore is true") { + def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { + // For tables without table-level compression property, session-level compression + // configuration will take effect. + // Check for partitioned table, when convertMetastore is true + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = true, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) + assert(checkTableSize(format, sessionCompressionCodec, true, true, tableSize)) + } + + // Check for non-partitioned table, when convertMetastore is true + checkTableCompressionCodecForCodecs( + format = format, + isPartitioned = false, + convertMetastore = true, + compressionCodecs = compressCodecs, + tableCompressionCodecs = List(null)) { + case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec, tableSize) => + // Expect session-level take effect + assert(sessionCompressionCodec == realCompressionCodec) + assert(checkTableSize(format, sessionCompressionCodec, false, true, tableSize)) + } + } + + checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 9a6971f8e0cef..ab91727049ff5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -19,18 +19,14 @@ package org.apache.spark.sql.hive import java.io.File -import scala.collection.JavaConverters._ - -import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.execution.datasources.parquet.ParquetTest -import org.apache.spark.sql.hive.orc.OrcFileOperator +import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -39,7 +35,7 @@ case class TestData(key: Int, value: String) case class ThreeCloumntable(key: Int, value: String, key1: String) class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter - with ParquetTest { + with SQLTestUtils { import spark.implicits._ override lazy val testData = spark.sparkContext.parallelize( @@ -754,255 +750,4 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } } - - private def getConvertMetastoreConfName(format: String): String = format match { - case "parquet" => HiveUtils.CONVERT_METASTORE_PARQUET.key - case "orc" => HiveUtils.CONVERT_METASTORE_ORC.key - } - - private def getSparkCompressionConfName(format: String): String = format match { - case "parquet" => SQLConf.PARQUET_COMPRESSION.key - case "orc" => SQLConf.ORC_COMPRESSION.key - } - - private def getHiveCompressPropName(format: String): String = { - format.toLowerCase match { - case "parquet" => "parquet.compression" - case "orc" => "orc.compress" - } - } - - private def getTableCompressionCodec(path: String, format: String): String = { - val hadoopConf = spark.sessionState.newHadoopConf() - val codecs = format match { - case "parquet" => for { - footer <- readAllFootersWithoutSummaryFiles(new Path(path), hadoopConf) - block <- footer.getParquetMetadata.getBlocks.asScala - column <- block.getColumns.asScala - } yield column.getCodec.name() - case "orc" => new File(path).listFiles().filter{ file => - file.isFile && !file.getName.endsWith(".crc") && file.getName != "_SUCCESS" - }.map { orcFile => - OrcFileOperator.getFileReader(orcFile.toPath.toString).get.getCompression.toString - }.toSeq - } - - assert(codecs.distinct.length == 1) - codecs.head - } - - private def writeDataToTable( - rootDir: File, - tableName: String, - isPartitioned: Boolean, - format: String, - compressionCodec: Option[String]) { - val tblProperties = compressionCodec match { - case Some(prop) => s"TBLPROPERTIES('${getHiveCompressPropName(format)}'='$prop')" - case _ => "" - } - val partitionCreate = if (isPartitioned) "PARTITIONED BY (p int)" else "" - sql( - s""" - |CREATE TABLE $tableName(a int) - |$partitionCreate - |STORED AS $format - |LOCATION '${rootDir.toURI.toString.stripSuffix("/")}/$tableName' - |$tblProperties - """.stripMargin) - - val partitionInsert = if (isPartitioned) s"partition (p=10000)" else "" - sql( - s""" - |INSERT OVERWRITE TABLE $tableName - |$partitionInsert - |SELECT * FROM table_source - """.stripMargin) - } - - private def checkCompressionCodecForTable( - format: String, - isPartitioned: Boolean, - compressionCodec: Option[String])(assertion: String => Unit): Unit = { - val tableName = s"tbl_$format${isPartitioned}" - withTempDir { tmpDir => - withTable(tableName) { - writeDataToTable(tmpDir, tableName, isPartitioned, format, compressionCodec) - val partition = if (isPartitioned) "p=10000" else "" - val path = s"${tmpDir.getPath.stripSuffix("/")}/${tableName}/$partition" - assertion(getTableCompressionCodec(path, format)) - } - } - } - - private def checkTableCompressionCodecForCodecs( - format: String, - isPartitioned: Boolean, - convertMetastore: Boolean, - compressionCodecs: List[String], - tableCompressionCodecs: List[String]) - (assertion: (Option[String], String, String) => Unit): Unit = { - withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { - tableCompressionCodecs.foreach { tableCompression => - compressionCodecs.foreach { sessionCompressionCodec => - withSQLConf(getSparkCompressionConfName(format) -> sessionCompressionCodec) { - // 'tableCompression = null' means no table-level compression - val compression = if (tableCompression == null) None else Some(tableCompression) - checkCompressionCodecForTable(format, isPartitioned, compression) { - case realCompressionCodec => assertion(compression, - sessionCompressionCodec, realCompressionCodec) - } - } - } - } - } - } - - private def testCompressionCodec(testCondition: String)(f: => Unit): Unit = { - test("[SPARK-21786] - Check the priority between table-level compression and " + - s"session-level compression $testCondition") { - withTempView("table_source") { - (0 until 100000).toDF("a").createOrReplaceTempView("table_source") - f - } - } - } - - testCompressionCodec("when table-level and session-level compression are both configured and " + - "convertMetastore is false") { - def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { - // For tables with table-level compression property, when - // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'false', partitioned tables - // and non-partitioned tables will always take the table-level compression - // configuration first and ignore session compression configuration. - // Check for partitioned table, when convertMetastore is false - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = true, - convertMetastore = false, - compressionCodecs = compressCodecs, - tableCompressionCodecs = compressCodecs) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect table-level take effect - assert(tableCompressionCodec.get == realCompressionCodec) - } - - // Check for non-partitioned table, when convertMetastoreParquet is false - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = false, - convertMetastore = false, - compressionCodecs = compressCodecs, - tableCompressionCodecs = compressCodecs) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect table-level take effect - assert(tableCompressionCodec.get == realCompressionCodec) - } - } - - checkForTableWithCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) - checkForTableWithCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) - } - - testCompressionCodec("when there's no table-level compression and convertMetastore is false") { - def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { - // For tables without table-level compression property, session-level compression - // configuration will take effect. - // Check for partitioned table, when convertMetastore is false - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = true, - convertMetastore = false, - compressionCodecs = compressCodecs, - tableCompressionCodecs = List(null)) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect session-level take effect - assert(sessionCompressionCodec == realCompressionCodec) - } - - // Check for non-partitioned table, when convertMetastore is false - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = false, - convertMetastore = false, - compressionCodecs = compressCodecs, - tableCompressionCodecs = List(null)) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect session-level take effect - assert(sessionCompressionCodec == realCompressionCodec) - } - } - - checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) - checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) - } - - testCompressionCodec("when table-level and session-level compression are both configured and " + - "convertMetastore is true") { - def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { - // For tables with table-level compression property, when - // 'spark.sql.hive.convertMetastore[Parquet|Orc]' was set to 'true', partitioned tables - // will always take the table-level compression configuration first, but non-partitioned - // tables will take the session-level compression configuration. - // Check for partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = true, - convertMetastore = true, - compressionCodecs = compressCodecs, - tableCompressionCodecs = compressCodecs) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect table-level take effect - assert(tableCompressionCodec.get == realCompressionCodec) - } - - // Check for non-partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = false, - convertMetastore = true, - compressionCodecs = compressCodecs, - tableCompressionCodecs = compressCodecs) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect session-level take effect - assert(sessionCompressionCodec == realCompressionCodec) - } - } - - checkForTableWithCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) - checkForTableWithCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) - } - - testCompressionCodec("when there's no table-level compression and convertMetastore is true") { - def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { - // For tables without table-level compression property, session-level compression - // configuration will take effect. - // Check for partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = true, - convertMetastore = true, - compressionCodecs = compressCodecs, - tableCompressionCodecs = List(null)) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect session-level take effect - assert(sessionCompressionCodec == realCompressionCodec) - } - - // Check for non-partitioned table, when convertMetastore is true - checkTableCompressionCodecForCodecs( - format = format, - isPartitioned = false, - convertMetastore = true, - compressionCodecs = compressCodecs, - tableCompressionCodecs = List(null)) { - case (tableCompressionCodec, sessionCompressionCodec, realCompressionCodec) => - // Expect session-level take effect - assert(sessionCompressionCodec == realCompressionCodec) - } - } - - checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) - checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) - } }