-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-12539][SQL] support writing bucketed table #10498
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
8cb2494
a9dc997
4c99698
d2dc9b3
b6d0a0b
ba23292
21e0c48
e3c3728
70ebd69
d9ad70c
6e3c1c0
d7f3000
3df61dc
d5f390d
3ff968b
74bd524
7bb02e1
b4985d9
d3200cf
1afd3ee
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,9 +23,9 @@ import scala.collection.JavaConverters._ | |
|
|
||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier} | ||
| import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation} | ||
| import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation | ||
| import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} | ||
| import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, ResolvedDataSource} | ||
| import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, ResolvedDataSource} | ||
| import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils | ||
| import org.apache.spark.sql.sources.HadoopFsRelation | ||
|
|
||
|
|
@@ -128,6 +128,34 @@ final class DataFrameWriter private[sql](df: DataFrame) { | |
| this | ||
| } | ||
|
|
||
| /** | ||
| * Buckets the output by the given columns. If specified, the output is laid out on the file | ||
| * system similar to Hive's bucketing scheme. | ||
| * | ||
| * This is applicable for Parquet, JSON and ORC. | ||
| * | ||
| * @since 2.0 | ||
| */ | ||
| @scala.annotation.varargs | ||
| def bucketBy(numBuckets: Int, colName: String, colNames: String*): DataFrameWriter = { | ||
| this.numBuckets = Option(numBuckets) | ||
| this.bucketColumnNames = Option(colName +: colNames) | ||
| this | ||
| } | ||
|
|
||
| /** | ||
| * Sorts the output in each bucket by the given columns. | ||
| * | ||
| * This is applicable for Parquet, JSON and ORC. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why the limitation? The comment makes this seem bucketBy must be called first. If that is true, make that more clear. |
||
| * | ||
| * @since 2.0 | ||
| */ | ||
| @scala.annotation.varargs | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Did you intentionally not provide a way for sort order?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. is that needed? |
||
| def sortBy(colName: String, colNames: String*): DataFrameWriter = { | ||
| this.sortColumnNames = Option(colName +: colNames) | ||
| this | ||
| } | ||
|
|
||
| /** | ||
| * Saves the content of the [[DataFrame]] at the specified path. | ||
| * | ||
|
|
@@ -144,10 +172,12 @@ final class DataFrameWriter private[sql](df: DataFrame) { | |
| * @since 1.4.0 | ||
| */ | ||
| def save(): Unit = { | ||
| assertNotBucketed() | ||
| ResolvedDataSource( | ||
| df.sqlContext, | ||
| source, | ||
| partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), | ||
| getBucketSpec, | ||
| mode, | ||
| extraOptions.toMap, | ||
| df) | ||
|
|
@@ -166,6 +196,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { | |
| } | ||
|
|
||
| private def insertInto(tableIdent: TableIdentifier): Unit = { | ||
| assertNotBucketed() | ||
| val partitions = normalizedParCols.map(_.map(col => col -> (None: Option[String])).toMap) | ||
| val overwrite = mode == SaveMode.Overwrite | ||
|
|
||
|
|
@@ -188,13 +219,47 @@ final class DataFrameWriter private[sql](df: DataFrame) { | |
| ifNotExists = false)).toRdd | ||
| } | ||
|
|
||
| private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { parCols => | ||
| parCols.map { col => | ||
| df.logicalPlan.output | ||
| .map(_.name) | ||
| .find(df.sqlContext.analyzer.resolver(_, col)) | ||
| .getOrElse(throw new AnalysisException(s"Partition column $col not found in existing " + | ||
| s"columns (${df.logicalPlan.output.map(_.name).mkString(", ")})")) | ||
| private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { cols => | ||
| cols.map(normalize(_, "Partition")) | ||
| } | ||
|
|
||
| private def normalizedBucketColNames: Option[Seq[String]] = bucketColumnNames.map { cols => | ||
| cols.map(normalize(_, "Bucketing")) | ||
| } | ||
|
|
||
| private def normalizedSortColNames: Option[Seq[String]] = sortColumnNames.map { cols => | ||
| cols.map(normalize(_, "Sorting")) | ||
| } | ||
|
|
||
| private def getBucketSpec: Option[BucketSpec] = { | ||
| if (sortColumnNames.isDefined) { | ||
| require(numBuckets.isDefined, "sortBy must be used together with bucketBy") | ||
| } | ||
|
|
||
| for { | ||
| n <- numBuckets | ||
| } yield { | ||
| require(n > 0 && n < 100000, "Bucket number must be greater than 0 and less than 100000.") | ||
| BucketSpec(n, normalizedBucketColNames.get, normalizedSortColNames.getOrElse(Nil)) | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This method can be simplified to: if (sortingColumns.isDefined) {
require(numBuckets.isDefined, "sortBy must be used together with bucketBy")
}
for {
n <- numBuckets
cols <- normalizedBucketCols
} yield {
require(n > 0, "Bucket number must be greater than 0.")
BucketSpec(n, cols, normalizedSortCols)
}( |
||
| } | ||
|
|
||
| /** | ||
| * The given column name may not be equal to any of the existing column names if we were in | ||
| * case-insensitive context. Normalize the given column name to the real one so that we don't | ||
| * need to care about case sensitivity afterwards. | ||
| */ | ||
| private def normalize(columnName: String, columnType: String): String = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you add comment explaining what "normalize" does. |
||
| val validColumnNames = df.logicalPlan.output.map(_.name) | ||
| validColumnNames.find(df.sqlContext.analyzer.resolver(_, columnName)) | ||
| .getOrElse(throw new AnalysisException(s"$columnType column $columnName not found in " + | ||
| s"existing columns (${validColumnNames.mkString(", ")})")) | ||
| } | ||
|
|
||
| private def assertNotBucketed(): Unit = { | ||
| if (numBuckets.isDefined || sortColumnNames.isDefined) { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. previous discussion: #10498 (comment) |
||
| throw new IllegalArgumentException( | ||
| "Currently we don't support writing bucketed data to this data source.") | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -244,6 +309,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { | |
| source, | ||
| temporary = false, | ||
| partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), | ||
| getBucketSpec, | ||
| mode, | ||
| extraOptions.toMap, | ||
| df.logicalPlan) | ||
|
|
@@ -372,4 +438,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { | |
|
|
||
| private var partitioningColumns: Option[Seq[String]] = None | ||
|
|
||
| private var bucketColumnNames: Option[Seq[String]] = None | ||
|
|
||
| private var numBuckets: Option[Int] = None | ||
|
|
||
| private var sortColumnNames: Option[Seq[String]] = None | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -125,7 +125,7 @@ private[sql] case class InsertIntoHadoopFsRelation( | |
| |Actual: ${partitionColumns.mkString(", ")} | ||
| """.stripMargin) | ||
|
|
||
| val writerContainer = if (partitionColumns.isEmpty) { | ||
| val writerContainer = if (partitionColumns.isEmpty && relation.bucketSpec.isEmpty) { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. cc @davies, if we bucket without partition, we will go to the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see, thanks |
||
| new DefaultWriterContainer(relation, job, isAppend) | ||
| } else { | ||
| val output = df.queryExecution.executedPlan.output | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
and the @SInCE annotations. and comments. These are public APIs