[SPARK-10216][SQL] Avoid creating empty files during overwriting with group by query

## What changes were proposed in this pull request?

Currently, `INSERT INTO` with `GROUP BY` query tries to make at least 200 files (default value of `spark.sql.shuffle.partition`), which results in lots of empty files.

This PR makes it avoid creating empty files during overwriting into Hive table and in internal data sources  with group by query.

This checks whether the given partition has data in it or not and creates/writes file only when it actually has data.

## How was this patch tested?

Unittests in `InsertIntoHiveTableSuite` and `HadoopFsRelationTest`.

Closes #8411

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Keuntae Park <sirpkt@apache.org>

Closes #12855 from HyukjinKwon/pr/8411.
This commit is contained in:
hyukjinkwon 2016-05-17 11:18:51 -07:00 committed by Michael Armbrust
parent 20a89478e1
commit 8d05a7a98b
4 changed files with 179 additions and 123 deletions

View file

@ -239,6 +239,7 @@ private[sql] class DefaultWriterContainer(
extends BaseWriterContainer(relation, job, isAppend) { extends BaseWriterContainer(relation, job, isAppend) {
def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = {
if (iterator.hasNext) {
executorSideSetup(taskContext) executorSideSetup(taskContext)
val configuration = taskAttemptContext.getConfiguration val configuration = taskAttemptContext.getConfiguration
configuration.set("spark.sql.sources.output.path", outputPath) configuration.set("spark.sql.sources.output.path", outputPath)
@ -285,6 +286,7 @@ private[sql] class DefaultWriterContainer(
} }
} }
} }
}
/** /**
* A writer that dynamically opens files based on the given partition columns. Internally this is * A writer that dynamically opens files based on the given partition columns. Internally this is
@ -363,10 +365,12 @@ private[sql] class DynamicPartitionWriterContainer(
} }
def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = {
if (iterator.hasNext) {
executorSideSetup(taskContext) executorSideSetup(taskContext)
// We should first sort by partition columns, then bucket id, and finally sorting columns. // We should first sort by partition columns, then bucket id, and finally sorting columns.
val sortingExpressions: Seq[Expression] = partitionColumns ++ bucketIdExpression ++ sortColumns val sortingExpressions: Seq[Expression] =
partitionColumns ++ bucketIdExpression ++ sortColumns
val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema)
val sortingKeySchema = StructType(sortingExpressions.map { val sortingKeySchema = StructType(sortingExpressions.map {
@ -444,3 +448,4 @@ private[sql] class DynamicPartitionWriterContainer(
} }
} }
} }
}

View file

@ -178,6 +178,7 @@ private[hive] class SparkHiveWriterContainer(
// this function is executed on executor side // this function is executed on executor side
def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = {
if (iterator.hasNext) {
val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite()
executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) executorSideSetup(context.stageId, context.partitionId, context.attemptNumber)
@ -193,6 +194,7 @@ private[hive] class SparkHiveWriterContainer(
close() close()
} }
} }
}
private[hive] object SparkHiveWriterContainer { private[hive] object SparkHiveWriterContainer {
def createPathFromString(path: String, conf: JobConf): Path = { def createPathFromString(path: String, conf: JobConf): Path = {

View file

@ -19,13 +19,13 @@ package org.apache.spark.sql.hive
import java.io.File import java.io.File
import org.apache.hadoop.hive.conf.HiveConf
import org.scalatest.BeforeAndAfter import org.scalatest.BeforeAndAfter
import org.apache.spark.SparkException import org.apache.spark.SparkException
import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable
import org.apache.spark.sql.hive.test.TestHiveSingleton 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.test.SQLTestUtils
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.apache.spark.util.Utils import org.apache.spark.util.Utils
@ -216,6 +216,35 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef
sql("DROP TABLE hiveTableWithStructValue") sql("DROP TABLE hiveTableWithStructValue")
} }
test("SPARK-10216: Avoid empty files during overwrite into Hive table with group by query") {
withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") {
val testDataset = hiveContext.sparkContext.parallelize(
(1 to 2).map(i => TestData(i, i.toString))).toDF()
testDataset.createOrReplaceTempView("testDataset")
val tmpDir = Utils.createTempDir()
sql(
s"""
|CREATE TABLE table1(key int,value string)
|location '${tmpDir.toURI.toString}'
""".stripMargin)
sql(
"""
|INSERT OVERWRITE TABLE table1
|SELECT count(key), value FROM testDataset GROUP BY value
""".stripMargin)
val overwrittenFiles = tmpDir.listFiles()
.filter(f => f.isFile && !f.getName.endsWith(".crc"))
.sortBy(_.getName)
val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0)
assert(overwrittenFiles === overwrittenFilesWithoutEmpty)
sql("DROP TABLE table1")
}
}
test("Reject partitioning that does not match table") { test("Reject partitioning that does not match table") {
withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) {
sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)")

View file

@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter
import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.sql._ import org.apache.spark.sql._
import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.DataSourceScanExec
import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} import org.apache.spark.sql.execution.datasources.{FileScanRDD, LocalityTestFileSystem}
import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.test.SQLTestUtils
@ -879,6 +879,26 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes
} }
} }
} }
test("SPARK-10216: Avoid empty files during overwriting with group by query") {
withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") {
withTempPath { path =>
val df = spark.range(0, 5)
val groupedDF = df.groupBy("id").count()
groupedDF.write
.format(dataSourceName)
.mode(SaveMode.Overwrite)
.save(path.getCanonicalPath)
val overwrittenFiles = path.listFiles()
.filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_"))
.sortBy(_.getName)
val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0)
assert(overwrittenFiles === overwrittenFilesWithoutEmpty)
}
}
}
} }
// This class is used to test SPARK-8578. We should not use any custom output committer when // This class is used to test SPARK-8578. We should not use any custom output committer when