Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/10498#discussion_r48820679 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala --- @@ -189,13 +220,43 @@ 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 normalizedBucketCols: Option[Seq[String]] = bucketingColumns.map { cols => + cols.map(normalize(_, "Bucketing")) + } + + private def normalizedSortCols: Option[Seq[String]] = sortingColumns.map { cols => + cols.map(normalize(_, "Sorting")) + } + + private def getBucketSpec: Option[BucketSpec] = { + 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) + } + } + + private def normalize(columnName: String, columnType: String): String = { + 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 || sortingColumns.isDefined) { --- End diff -- If users just wanna sort the data, they can call `DataFrame.sort` before write. In this context, the `sortingColumns` is part of the bucketing information and should be used together with `bucketingColumns`.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org