From 7dd8dd5c73f26c92fb885bb52fc2e5f97a1731a4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 15 Apr 2015 21:49:43 +0800 Subject: [PATCH 01/45] Adds new interfaces and stub methods for data sources API partitioning support --- .../org/apache/spark/sql/DataFrame.scala | 30 +++++ .../apache/spark/sql/sources/interfaces.scala | 108 +++++++++++++++++- 2 files changed, 135 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 265a61592b943..932a5e2bc5547 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1417,6 +1417,21 @@ class DataFrame private[sql]( save(source, mode, options.toMap) } + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path based on the given data source, + * [[SaveMode]] specified by mode, and partition columns specified by `partitionColumns`. + * @group output + */ + @Experimental + def save( + source: String, + mode: SaveMode, + options: java.util.Map[String, String], + partitionColumns: java.util.List[String]): Unit = { + ??? + } + /** * :: Experimental :: * (Scala-specific) @@ -1432,6 +1447,21 @@ class DataFrame private[sql]( ResolvedDataSource(sqlContext, source, mode, options, this) } + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path based on the given data source, + * [[SaveMode]] specified by mode, and partition columns specified by `partitionColumns`. + * @group output + */ + @Experimental + def save( + source: String, + mode: SaveMode, + options: Map[String, String], + partitionColumns: Seq[String]): Unit = { + ??? + } + /** * :: Experimental :: * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index ca53dcdb92c52..6ed9fbca92e40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.sources -import org.apache.spark.annotation.{Experimental, DeveloperApi} +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SaveMode, DataFrame, Row, SQLContext} -import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode} /** * ::DeveloperApi:: @@ -78,6 +80,40 @@ trait SchemaRelationProvider { schema: StructType): BaseRelation } +/** + * ::DeveloperApi:: + * Implemented by objects that produce relations for a specific kind of data source + * with a given schema and partitioned columns. When Spark SQL is given a DDL operation with a + * USING clause specified (to specify the implemented SchemaRelationProvider), a user defined + * schema, and an optional list of partition columns, this interface is used to pass in the + * parameters specified by a user. + * + * Users may specify the fully qualified class name of a given data source. When that class is + * not found Spark SQL will append the class name `DefaultSource` to the path, allowing for + * less verbose invocation. For example, 'org.apache.spark.sql.json' would resolve to the + * data source 'org.apache.spark.sql.json.DefaultSource' + * + * A new instance of this class with be instantiated each time a DDL call is made. + * + * The difference between a [[RelationProvider]] and a [[PartitionedSchemaRelationProvider]] is + * that users need to provide a schema and a (possibly empty) list of partition columns when + * using a SchemaRelationProvider. A relation provider can inherits both [[RelationProvider]], + * [[SchemaRelationProvider]], and [[PartitionedSchemaRelationProvider]] if it can support schema + * inference, user-specified schemas, and accessing partitioned relations. + */ +trait PartitionedSchemaRelationProvider { + /** + * Returns a new base relation with the given parameters, a user defined schema, and a list of + * partition columns. Note: the parameters' keywords are case insensitive and this insensitivity + * is enforced by the Map that is passed to the function. + */ + def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType, + partitionColumns: StructType): BaseRelation +} + @DeveloperApi trait CreatableRelationProvider { /** @@ -207,3 +243,69 @@ trait InsertableRelation { trait CatalystScan { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } + +/** + * ::Experimental:: + * [[OutputWriter]] is used together with [[FSBasedPrunedFilteredScan]] for persisting rows to the + * underlying file system. An [[OutputWriter]] instance is created when a new output file is + * opened. This instance is used to persist rows to this single output file. + */ +@Experimental +trait OutputWriter { + /** + * Persists a single row. Invoked on the executor side. + */ + def write(row: Row): Unit + + /** + * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before + * the task output is committed. + */ + def close(): Unit +} + +/** + * ::Experimental:: + * A [[BaseRelation]] that abstracts file system based data sources. + * + * For the read path, similar to [[PrunedFilteredScan]], it can eliminate unneeded columns and + * filter using selected predicates before producing an RDD containing all matching tuples as + * [[Row]] objects. + * + * In addition, when reading from Hive style partitioned tables stored in file systems, it's able to + * discover partitioning information from the paths of input directories, and perform partition + * pruning before start reading the data. + * + * For the write path, it provides the ability to write to both non-partitioned and partitioned + * tables. Directory layout of the partitioned tables is compatible with Hive. + */ +@Experimental +trait FSBasedPrunedFilteredScan extends BaseRelation { + /** + * Builds an `RDD[Row]` containing all rows within this relation. + * + * @param requiredColumns Required columns. + * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction + * of all `filters`. The pushed down filters are currently purely an optimization as they + * will all be evaluated again. This means it is safe to use them with methods that produce + * false positives such as filtering partitions based on a bloom filter. + * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data + * files within required partition directories are included. + */ + def buildScan( + requiredColumns: Array[String], + filters: Array[Filter], + inputPaths: Array[String]): RDD[Row] + + /** + * When writing rows to this relation, this method is invoked on the driver side before the actual + * write job is issued. It provides an opportunity to configure the write job to be performed. + */ + def prepareForWrite(conf: Configuration): Unit + + /** + * This method is responsible for producing a new [[OutputWriter]] for each newly opened output + * file on the executor side. + */ + def newOutputWriter(path: String): OutputWriter +} From 012ed2dab8f7ef067127b17e02963d18501c7613 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 15 Apr 2015 22:01:39 +0800 Subject: [PATCH 02/45] Adds PartitioningOptions --- .../sql/sources/PartitioningOptions.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java b/sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java new file mode 100644 index 0000000000000..afe565e7a81e6 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java @@ -0,0 +1,28 @@ +/* + * 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.sources; + +public interface PartitioningOptions { + /** + * When set to `false`, partition discovery is not performed. This option + * can be useful when, for example, the user only wants to load data stored + * in a single partition directory without introducing partition columns + * encoded in the directory path. + */ + String PARTITION_DISCOVERY_ENABLED = "partition.discovery.enabled"; +} From aa8ba9a50c82716685a9b6e646a5a0b81f24e2f8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 15 Apr 2015 22:05:37 +0800 Subject: [PATCH 03/45] Javadoc fix --- .../main/scala/org/apache/spark/sql/sources/interfaces.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 6ed9fbca92e40..ebd42b73a4ba3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -98,8 +98,8 @@ trait SchemaRelationProvider { * The difference between a [[RelationProvider]] and a [[PartitionedSchemaRelationProvider]] is * that users need to provide a schema and a (possibly empty) list of partition columns when * using a SchemaRelationProvider. A relation provider can inherits both [[RelationProvider]], - * [[SchemaRelationProvider]], and [[PartitionedSchemaRelationProvider]] if it can support schema - * inference, user-specified schemas, and accessing partitioned relations. + * and [[PartitionedSchemaRelationProvider]] if it can support schema inference, user-specified + * schemas, and accessing partitioned relations. */ trait PartitionedSchemaRelationProvider { /** From 1b8231ff570f1a3a769692992307907820e6051e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 15 Apr 2015 22:38:02 +0800 Subject: [PATCH 04/45] Renames FSBasedPrunedFilteredScan to FSBasedRelation --- .../main/scala/org/apache/spark/sql/sources/interfaces.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index ebd42b73a4ba3..959c472ed3afe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -246,7 +246,7 @@ trait CatalystScan { /** * ::Experimental:: - * [[OutputWriter]] is used together with [[FSBasedPrunedFilteredScan]] for persisting rows to the + * [[OutputWriter]] is used together with [[FSBasedRelation]] for persisting rows to the * underlying file system. An [[OutputWriter]] instance is created when a new output file is * opened. This instance is used to persist rows to this single output file. */ @@ -280,7 +280,7 @@ trait OutputWriter { * tables. Directory layout of the partitioned tables is compatible with Hive. */ @Experimental -trait FSBasedPrunedFilteredScan extends BaseRelation { +trait FSBasedRelation extends BaseRelation { /** * Builds an `RDD[Row]` containing all rows within this relation. * From 3ba9bbf8def8149ff44e8555f14ef54e926a7272 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 15 Apr 2015 22:48:33 +0800 Subject: [PATCH 05/45] Adds DataFrame.saveAsTable() overrides which support partitioning --- .../org/apache/spark/sql/DataFrame.scala | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 932a5e2bc5547..27496227f99d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1324,6 +1324,28 @@ class DataFrame private[sql]( saveAsTable(tableName, source, mode, options.toMap) } + /** + * :: Experimental :: + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source, [[SaveMode]] specified by mode, a set of options, and a list of + * partition columns. + * + * Note that this currently only works with DataFrames that are created from a HiveContext as + * there is no notion of a persisted catalog in a standard SQL context. Instead you can write + * an RDD out to a parquet file, and then register that file as a table. This "table" can then + * be the target of an `insertInto`. + * @group output + */ + @Experimental + def saveAsTable( + tableName: String, + source: String, + mode: SaveMode, + options: java.util.Map[String, String], + partitionColumns: java.util.List[String]): Unit = { + ??? + } + /** * :: Experimental :: * (Scala-specific) @@ -1357,6 +1379,28 @@ class DataFrame private[sql]( sqlContext.executePlan(cmd).toRdd } + /** + * :: Experimental :: + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source, [[SaveMode]] specified by mode, a set of options, and a list of + * partition columns. + * + * Note that this currently only works with DataFrames that are created from a HiveContext as + * there is no notion of a persisted catalog in a standard SQL context. Instead you can write + * an RDD out to a parquet file, and then register that file as a table. This "table" can then + * be the target of an `insertInto`. + * @group output + */ + @Experimental + def saveAsTable( + tableName: String, + source: String, + mode: SaveMode, + options: Map[String, String], + partitionColumns: Seq[String]): Unit = { + ??? + } + /** * :: Experimental :: * Saves the contents of this DataFrame to the given path, From 770b5baf3b0afaf9b7a1f6ea58031ce51bbf4ed3 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 23 Apr 2015 22:53:37 +0800 Subject: [PATCH 06/45] Adds tests for FSBasedRelation --- .../apache/spark/sql/sources/interfaces.scala | 70 ++- .../sql/sources/FSBasedRelationSuite.scala | 406 ++++++++++++++++++ 2 files changed, 458 insertions(+), 18 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 959c472ed3afe..c9c119f5b4886 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.sources -import org.apache.hadoop.conf.Configuration - import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} @@ -251,9 +249,12 @@ trait CatalystScan { * opened. This instance is used to persist rows to this single output file. */ @Experimental -trait OutputWriter { +abstract class OutputWriter { + def init(): Unit = () + /** - * Persists a single row. Invoked on the executor side. + * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned + * tables, dynamic partition columns are not included in rows to be written. */ def write(row: Row): Unit @@ -261,7 +262,7 @@ trait OutputWriter { * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before * the task output is committed. */ - def close(): Unit + def close(): Unit = () } /** @@ -270,17 +271,54 @@ trait OutputWriter { * * For the read path, similar to [[PrunedFilteredScan]], it can eliminate unneeded columns and * filter using selected predicates before producing an RDD containing all matching tuples as - * [[Row]] objects. - * - * In addition, when reading from Hive style partitioned tables stored in file systems, it's able to - * discover partitioning information from the paths of input directories, and perform partition - * pruning before start reading the data. + * [[Row]] objects. In addition, when reading from Hive style partitioned tables stored in file + * systems, it's able to discover partitioning information from the paths of input directories, and + * perform partition pruning before start reading the data.Subclasses of [[FSBasedRelation()]] must + * override one of the three `buildScan` methods to implement the read path. * * For the write path, it provides the ability to write to both non-partitioned and partitioned * tables. Directory layout of the partitioned tables is compatible with Hive. */ @Experimental -trait FSBasedRelation extends BaseRelation { +abstract class FSBasedRelation extends BaseRelation { + // Discovers partitioned columns, and merge them with `dataSchema`. All partition columns not + // existed in `dataSchema` should be appended to `dataSchema`. + override val schema: StructType = ??? + + /** + * Base path of this relation. For partitioned relations, `path` should be the root directory of + * all partition directories. + */ + def path: String + + /** + * Specifies schema of actual data files. For partitioned relations, if one or more partitioned + * columns are contained in the data files, they should also appear in `dataSchema`. + */ + def dataSchema: StructType + + /** + * Builds an `RDD[Row` containing all rows within this relation. + * + * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data + * files within required partition directories are included. + */ + def buildScan(inputPaths: Array[String]): RDD[Row] = { + throw new RuntimeException( + "At least one buildScan() method should be overridden to read the relation.") + } + + /** + * Builds an `RDD[Row` containing all rows within this relation. + * + * @param requiredColumns Required columns. + * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data + * files within required partition directories are included. + */ + def buildScan(requiredColumns: Array[String], inputPaths: Array[String]): RDD[Row] = { + buildScan(inputPaths) + } + /** * Builds an `RDD[Row]` containing all rows within this relation. * @@ -295,13 +333,9 @@ trait FSBasedRelation extends BaseRelation { def buildScan( requiredColumns: Array[String], filters: Array[Filter], - inputPaths: Array[String]): RDD[Row] - - /** - * When writing rows to this relation, this method is invoked on the driver side before the actual - * write job is issued. It provides an opportunity to configure the write job to be performed. - */ - def prepareForWrite(conf: Configuration): Unit + inputPaths: Array[String]): RDD[Row] = { + buildScan(requiredColumns, inputPaths) + } /** * This method is responsible for producing a new [[OutputWriter]] for each newly opened output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala new file mode 100644 index 0000000000000..2f05352f63ed9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -0,0 +1,406 @@ +package org.apache.spark.sql.sources + +import java.io.IOException + +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Row, SQLContext, SaveMode} +import org.apache.spark.util.Utils + +class SimpleFSBasedSource extends RelationProvider { + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + SimpleFSBasedRelation(parameters)(sqlContext) + } +} + +case class SimpleFSBasedRelation + (parameter: Map[String, String]) + (val sqlContext: SQLContext) + extends FSBasedRelation { + + override val path = parameter("path") + + override def dataSchema: StructType = + DataType.fromJson(parameter("schema")).asInstanceOf[StructType] + + override def newOutputWriter(path: String): OutputWriter = new OutputWriter { + override def write(row: Row): Unit = TestResult.writtenRows += row + } + + override def buildScan( + requiredColumns: Array[String], + filters: Array[Filter], + inputPaths: Array[String]): RDD[Row] = { + val basePath = new Path(path) + val fs = basePath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + + // Used to test queries like "INSERT OVERWRITE tbl SELECT * FROM tbl". Source directory + // shouldn't be removed before scanning the data. + assert(inputPaths.map(new Path(_)).forall(fs.exists)) + + TestResult.requiredColumns = requiredColumns + TestResult.filters = filters + TestResult.inputPaths = inputPaths + + Option(TestResult.rowsToRead).getOrElse(sqlContext.emptyResult) + } +} + +object TestResult { + var requiredColumns: Array[String] = _ + var filters: Array[Filter] = _ + var inputPaths: Array[String] = _ + var rowsToRead: RDD[Row] = _ + var writtenRows: ArrayBuffer[Row] = ArrayBuffer.empty[Row] + + def reset(): Unit = { + requiredColumns = null + filters = null + inputPaths = null + rowsToRead = null + writtenRows.clear() + } +} + +class FSBasedRelationSuite extends DataSourceTest { + import caseInsensitiveContext._ + import caseInsensitiveContext.implicits._ + + var basePath: Path = _ + + var fs: FileSystem = _ + + val dataSchema = + StructType( + Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = false))) + + val testDF = (for { + i <- 1 to 3 + p <- 1 to 2 + } yield (i, s"val_$i", p)).toDF("a", "b", "p1") + + before { + basePath = new Path(Utils.createTempDir().getCanonicalPath) + fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) + TestResult.reset() + } + + ignore("load() - partitioned table - partition column not included in data files") { + fs.mkdirs(new Path(basePath, "p1=1/p2=hello")) + fs.mkdirs(new Path(basePath, "p1=2/p2=world")) + + val df = load( + source = classOf[SimpleFSBasedSource].getCanonicalName, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json)) + + df.queryExecution.analyzed.collect { + case LogicalRelation(relation: SimpleFSBasedRelation) => + assert(relation.dataSchema === dataSchema) + case _ => + fail("Couldn't find expected SimpleFSBasedRelation instance") + } + + val expectedSchema = + StructType( + dataSchema ++ Seq( + StructField("p1", IntegerType, nullable = true), + StructField("p2", StringType, nullable = true))) + + assert(df.schema === expectedSchema) + + df.select("b").where($"a" > 0 && $"p1" === 1).collect() + + // Check for column pruning, filter push-down, and partition pruning + assert(TestResult.requiredColumns.toSet === Set("a", "b")) + assert(TestResult.filters === Seq(GreaterThan("a", 0))) + assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1").toString)) + } + + ignore("load() - partitioned table - partition column included in data files") { + val data = sparkContext.parallelize(Seq.empty[String]) + data.saveAsTextFile(new Path(basePath, "p1=1/p2=hello").toString) + data.saveAsTextFile(new Path(basePath, "p1=2/p2=world").toString) + + val dataSchema = + StructType( + Seq( + StructField("a", IntegerType, nullable = false), + StructField("p1", IntegerType, nullable = true), + StructField("b", StringType, nullable = false))) + + val df = load( + source = classOf[SimpleFSBasedSource].getCanonicalName, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json)) + + df.queryExecution.analyzed.collect { + case LogicalRelation(relation: SimpleFSBasedRelation) => + assert(relation.dataSchema === dataSchema) + case _ => + fail("Couldn't find expected SimpleFSBasedRelation instance") + } + + val expectedSchema = + StructType( + Seq( + StructField("a", IntegerType, nullable = false), + StructField("p1", IntegerType, nullable = true), + StructField("b", StringType, nullable = false), + StructField("p2", StringType, nullable = true))) + + assert(df.schema === expectedSchema) + + df.select("b").where($"a" > 0 && $"p1" === 1).collect() + + // Check for column pruning, filter push-down, and partition pruning + assert(TestResult.requiredColumns.toSet === Set("a", "b")) + assert(TestResult.filters === Seq(GreaterThan("a", 0))) + assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1").toString)) + } + + ignore("save() - partitioned table - Overwrite") { + testDF.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + assert(TestResult.writtenRows.sameElements(expectedRows)) + } + + ignore("save() - partitioned table - Overwrite - select and overwrite the same table") { + TestResult.rowsToRead = testDF.rdd + + val df = load( + source = classOf[SimpleFSBasedSource].getCanonicalName, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json)) + + df.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + assert(TestResult.writtenRows.sameElements(expectedRows)) + } + + ignore("save() - partitioned table - Append") { + testDF.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + testDF.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Append, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i") + assert(TestResult.writtenRows.sameElements(expectedRows)) + } + + ignore("save() - partitioned table - ErrorIfExists") { + fs.delete(basePath, true) + + testDF.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + assert(TestResult.writtenRows.sameElements(testDF.collect())) + + intercept[IOException] { + testDF.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + } + } + + ignore("save() - partitioned table - Ignore") { + testDF.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + assert(TestResult.writtenRows.isEmpty) + } + + ignore("saveAsTable() - partitioned table - Overwrite") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + assert(TestResult.writtenRows.sameElements(expectedRows)) + + assertResult(table("t").schema) { + StructType( + dataSchema ++ Seq( + StructField("p1", IntegerType, nullable = true))) + } + + sql("DROP TABLE t") + } + + ignore("saveAsTable() - partitioned table - Overwrite - select and overwrite the same table") { + TestResult.rowsToRead = testDF.rdd + + val df = load( + source = classOf[SimpleFSBasedSource].getCanonicalName, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json)) + + df.saveAsTable( + tableName = "t", + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + assert(TestResult.writtenRows.sameElements(expectedRows)) + + assertResult(table("t").schema) { + StructType( + dataSchema ++ Seq( + StructField("p1", IntegerType, nullable = true))) + } + + sql("DROP TABLE t") + } + + ignore("saveAsTable() - partitioned table - Append") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Append, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i") + assert(TestResult.writtenRows.sameElements(expectedRows)) + + assertResult(table("t").schema) { + StructType( + dataSchema ++ Seq( + StructField("p1", IntegerType, nullable = true))) + } + + sql("DROP TABLE t") + } + + ignore("saveAsTable() - partitioned table - ErrorIfExists") { + fs.delete(basePath, true) + + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + assert(TestResult.writtenRows.sameElements(testDF.collect())) + + assertResult(table("t").schema) { + StructType( + dataSchema ++ Seq( + StructField("p1", IntegerType, nullable = true))) + } + + intercept[IOException] { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + } + + sql("DROP TABLE t") + } + + ignore("saveAsTable() - partitioned table - Ignore") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json), + partitionColumns = Seq("p1")) + + assert(TestResult.writtenRows.isEmpty) + + assertResult(table("t").schema) { + StructType( + dataSchema ++ Seq( + StructField("p1", IntegerType, nullable = true))) + } + + sql("DROP TABLE t") + } +} From 95d0b4d8dc8cc0249f284ecdc3dee02bffe7c063 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 23 Apr 2015 22:56:10 +0800 Subject: [PATCH 07/45] Renames PartitionedSchemaRelationProvider to FSBasedRelationProvider --- .../scala/org/apache/spark/sql/sources/interfaces.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index c9c119f5b4886..d9be64b515555 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -93,13 +93,13 @@ trait SchemaRelationProvider { * * A new instance of this class with be instantiated each time a DDL call is made. * - * The difference between a [[RelationProvider]] and a [[PartitionedSchemaRelationProvider]] is + * The difference between a [[RelationProvider]] and a [[FSBasedRelationProvider]] is * that users need to provide a schema and a (possibly empty) list of partition columns when * using a SchemaRelationProvider. A relation provider can inherits both [[RelationProvider]], - * and [[PartitionedSchemaRelationProvider]] if it can support schema inference, user-specified + * and [[FSBasedRelationProvider]] if it can support schema inference, user-specified * schemas, and accessing partitioned relations. */ -trait PartitionedSchemaRelationProvider { +trait FSBasedRelationProvider { /** * Returns a new base relation with the given parameters, a user defined schema, and a list of * partition columns. Note: the parameters' keywords are case insensitive and this insensitivity From 5de194a752d4c20f6f636c577c550f3f5998babf Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 24 Apr 2015 00:02:03 +0800 Subject: [PATCH 08/45] Forgot Apache licence header --- .../sql/sources/FSBasedRelationSuite.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 2f05352f63ed9..442d18c2d084c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.sources import java.io.IOException From 9d176071b68246428901b699bb363876f8e5579a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 24 Apr 2015 00:05:40 +0800 Subject: [PATCH 09/45] Adds the contract that OutputWriter should have zero-arg constructor Also adds more arguments to OutputWriter.init() for initialization. --- .../apache/spark/sql/sources/interfaces.scala | 24 +++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index d9be64b515555..ebd7567f4f836 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.sources +import org.apache.hadoop.conf.Configuration + import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} @@ -245,12 +247,26 @@ trait CatalystScan { /** * ::Experimental:: * [[OutputWriter]] is used together with [[FSBasedRelation]] for persisting rows to the - * underlying file system. An [[OutputWriter]] instance is created when a new output file is - * opened. This instance is used to persist rows to this single output file. + * underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor. + * An [[OutputWriter]] instance is created and initialized when a new output file is opened on + * executor side. This instance is used to persist rows to this single output file. */ @Experimental abstract class OutputWriter { - def init(): Unit = () + /** + * Initializes this [[OutputWriter]] before any rows are persisted. + * + * @param path The file path to which this [[OutputWriter]] is supposed to write. + * @param dataSchema Schema of the rows to be written. Partition columns are not included in the + * schema if the corresponding relation is partitioned. + * @param options Data source options inherited from driver side. + * @param conf Hadoop configuration inherited from driver side. + */ + def init( + path: String, + dataSchema: StructType, + options: java.util.Map[String, String], + conf: Configuration): Unit = () /** * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned @@ -341,5 +357,5 @@ abstract class FSBasedRelation extends BaseRelation { * This method is responsible for producing a new [[OutputWriter]] for each newly opened output * file on the executor side. */ - def newOutputWriter(path: String): OutputWriter + def outputWriterClass: Class[_ <: OutputWriter] } From fb5a607948fc26e2d8d01d83b8963bdd4b22ef9e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 24 Apr 2015 00:30:41 +0800 Subject: [PATCH 10/45] Fixes compilation error --- .../org/apache/spark/sql/sources/interfaces.scala | 4 ++-- .../spark/sql/sources/FSBasedRelationSuite.scala | 10 ++++++---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index ebd7567f4f836..094ccefd8c832 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -314,7 +314,7 @@ abstract class FSBasedRelation extends BaseRelation { def dataSchema: StructType /** - * Builds an `RDD[Row` containing all rows within this relation. + * Builds an `RDD[Row]` containing all rows within this relation. * * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data * files within required partition directories are included. @@ -325,7 +325,7 @@ abstract class FSBasedRelation extends BaseRelation { } /** - * Builds an `RDD[Row` containing all rows within this relation. + * Builds an `RDD[Row]` containing all rows within this relation. * * @param requiredColumns Required columns. * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 442d18c2d084c..55800af8f9bc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -41,15 +41,15 @@ case class SimpleFSBasedRelation (val sqlContext: SQLContext) extends FSBasedRelation { + class SimpleOutputWriter extends OutputWriter { + override def write(row: Row): Unit = TestResult.writtenRows += row + } + override val path = parameter("path") override def dataSchema: StructType = DataType.fromJson(parameter("schema")).asInstanceOf[StructType] - override def newOutputWriter(path: String): OutputWriter = new OutputWriter { - override def write(row: Row): Unit = TestResult.writtenRows += row - } - override def buildScan( requiredColumns: Array[String], filters: Array[Filter], @@ -67,6 +67,8 @@ case class SimpleFSBasedRelation Option(TestResult.rowsToRead).getOrElse(sqlContext.emptyResult) } + + override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter] } object TestResult { From 3c5073afc3be57e65df5d62c7a154349b700a151 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 25 Apr 2015 12:02:38 +0800 Subject: [PATCH 11/45] Fixes SaveModes used in test cases --- .../apache/spark/sql/sources/FSBasedRelationSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 55800af8f9bc9..6c5bb8e425a74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -262,7 +262,7 @@ class FSBasedRelationSuite extends DataSourceTest { intercept[IOException] { testDF.save( source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = SaveMode.ErrorIfExists, options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json), @@ -273,7 +273,7 @@ class FSBasedRelationSuite extends DataSourceTest { ignore("save() - partitioned table - Ignore") { testDF.save( source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = SaveMode.Ignore, options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json), @@ -374,7 +374,7 @@ class FSBasedRelationSuite extends DataSourceTest { testDF.saveAsTable( tableName = "t", source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = SaveMode.ErrorIfExists, options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json), @@ -406,7 +406,7 @@ class FSBasedRelationSuite extends DataSourceTest { testDF.saveAsTable( tableName = "t", source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = SaveMode.Ignore, options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json), From 327bb1dead0e5cb5167ca82b09a46655227a0bf9 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 29 Apr 2015 23:02:43 +0800 Subject: [PATCH 12/45] Implements partitioning support for data sources API --- .../apache/spark/deploy/SparkHadoopUtil.scala | 47 ++- project/SparkBuild.scala | 1 + .../org/apache/spark/sql/DataFrame.scala | 21 +- .../scala/org/apache/spark/sql/SQLConf.scala | 3 + .../org/apache/spark/sql/SQLContext.scala | 4 +- .../spark/sql/execution/SparkStrategies.scala | 10 +- .../apache/spark/sql/parquet/newParquet.scala | 6 +- .../sql/sources/DataSourceStrategy.scala | 46 +++ .../spark/sql/sources/PartitioningUtils.scala | 207 ++++++++++++ .../apache/spark/sql/sources/commands.scala | 317 +++++++++++++++++- .../org/apache/spark/sql/sources/ddl.scala | 53 ++- .../apache/spark/sql/sources/interfaces.scala | 73 +++- .../org/apache/spark/sql/sources/rules.scala | 2 +- .../ParquetPartitionDiscoverySuite.scala | 1 + .../sources/CreateTableAsSelectSuite.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 13 +- .../spark/sql/hive/HiveStrategies.scala | 6 +- .../spark/sql/hive/execution/commands.scala | 12 +- .../spark/sql/hive/hiveWriterContainers.scala | 2 +- .../sql/sources/FSBasedRelationSuite.scala | 244 ++++++++++---- 20 files changed, 936 insertions(+), 134 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala rename sql/{core => hive}/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala (60%) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index b563034457a91..5def549ac411a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -22,22 +22,22 @@ import java.lang.reflect.Method import java.security.PrivilegedExceptionAction import java.util.{Arrays, Comparator} +import scala.collection.JavaConversions._ +import scala.concurrent.duration._ +import scala.language.postfixOps + import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.hadoop.fs.FileSystem.Statistics +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils - -import scala.collection.JavaConversions._ -import scala.concurrent.duration._ -import scala.language.postfixOps +import org.apache.spark.{Logging, SparkConf, SparkException} /** * :: DeveloperApi :: @@ -199,13 +199,36 @@ class SparkHadoopUtil extends Logging { * that file. */ def listLeafStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = { - def recurse(path: Path): Array[FileStatus] = { - val (directories, leaves) = fs.listStatus(path).partition(_.isDir) - leaves ++ directories.flatMap(f => listLeafStatuses(fs, f.getPath)) + listLeafStatuses(fs, fs.getFileStatus(basePath)) + } + + /** + * Get [[FileStatus]] objects for all leaf children (files) under the given base path. If the + * given path points to a file, return a single-element collection containing [[FileStatus]] of + * that file. + */ + def listLeafStatuses(fs: FileSystem, baseStatus: FileStatus): Seq[FileStatus] = { + def recurse(status: FileStatus): Seq[FileStatus] = { + val (directories, leaves) = fs.listStatus(status.getPath).partition(_.isDir) + leaves ++ directories.flatMap(f => listLeafStatuses(fs, f)) + } + + if (baseStatus.isDir) recurse(baseStatus) else Seq(baseStatus) + } + + def listLeafDirStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = { + listLeafDirStatuses(fs, fs.getFileStatus(basePath)) + } + + def listLeafDirStatuses(fs: FileSystem, baseStatus: FileStatus): Seq[FileStatus] = { + def recurse(status: FileStatus): Seq[FileStatus] = { + val (directories, files) = fs.listStatus(status.getPath).partition(_.isDir) + val leaves = if (directories.isEmpty) Seq(status) else Seq.empty[FileStatus] + leaves ++ directories.flatMap(dir => listLeafDirStatuses(fs, dir)) } - val baseStatus = fs.getFileStatus(basePath) - if (baseStatus.isDir) recurse(basePath) else Array(baseStatus) + assert(baseStatus.isDir) + recurse(baseStatus) } /** @@ -275,7 +298,7 @@ class SparkHadoopUtil extends Logging { logDebug(text + " matched " + HADOOP_CONF_PATTERN) val key = matched.substring(13, matched.length() - 1) // remove ${hadoopconf- .. } val eval = Option[String](hadoopConf.get(key)) - .map { value => + .map { value => logDebug("Substituted " + matched + " with " + value) text.replace(matched, value) } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 186345af0e60e..146c9ead80233 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -548,6 +548,7 @@ object TestSettings { javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", + javaOptions in Test += "-agentlib:jdwp=transport=dt_socket,server=n,address=127.0.0.1:5005", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, javaOptions in Test += "-ea", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 27496227f99d4..32c867f2eeabd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -27,23 +27,23 @@ import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import com.fasterxml.jackson.core.JsonFactory - import org.apache.commons.lang3.StringUtils + import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, ResolvedStar} +import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedAttribute, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} -import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, _} +import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.json.JacksonGenerator +import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, ResolvedDataSource} import org.apache.spark.sql.types._ -import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -1372,6 +1372,7 @@ class DataFrame private[sql]( tableName, source, temporary = false, + Array.empty[String], mode, options, logicalPlan) @@ -1473,7 +1474,7 @@ class DataFrame private[sql]( mode: SaveMode, options: java.util.Map[String, String], partitionColumns: java.util.List[String]): Unit = { - ??? + save(source, mode, options.toMap, partitionColumns) } /** @@ -1488,7 +1489,7 @@ class DataFrame private[sql]( source: String, mode: SaveMode, options: Map[String, String]): Unit = { - ResolvedDataSource(sqlContext, source, mode, options, this) + ResolvedDataSource(sqlContext, source, Array.empty[String], mode, options, this) } /** @@ -1503,7 +1504,7 @@ class DataFrame private[sql]( mode: SaveMode, options: Map[String, String], partitionColumns: Seq[String]): Unit = { - ??? + ResolvedDataSource(sqlContext, source, partitionColumns.toArray, mode, options, this) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index dcac97beafb04..2f9784b823173 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -66,6 +66,9 @@ private[spark] object SQLConf { // to its length exceeds the threshold. val SCHEMA_STRING_LENGTH_THRESHOLD = "spark.sql.sources.schemaStringLengthThreshold" + // Whether to perform partition discovery when loading external data sources. + val PARTITION_DISCOVERY_ENABLED = "spark.sql.sources.partitionDiscovery.enabled" + // Whether to perform eager analysis when constructing a dataframe. // Set to false when debugging requires the ability to look at invalid query plans. val DATAFRAME_EAGER_ANALYSIS = "spark.sql.eagerAnalysis" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 28fc9d04436f7..2753c913b6a0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -762,7 +762,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def load(source: String, options: Map[String, String]): DataFrame = { - val resolved = ResolvedDataSource(this, None, source, options) + val resolved = ResolvedDataSource(this, None, Array.empty[String], source, options) DataFrame(this, LogicalRelation(resolved.relation)) } @@ -792,7 +792,7 @@ class SQLContext(@transient val sparkContext: SparkContext) source: String, schema: StructType, options: Map[String, String]): DataFrame = { - val resolved = ResolvedDataSource(this, Some(schema), source, options) + val resolved = ResolvedDataSource(this, Some(schema), Array.empty[String], source, options) DataFrame(this, LogicalRelation(resolved.relation)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 56a4689eb58f0..af0029cb84f9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -343,9 +343,13 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case c: CreateTableUsing if c.temporary && c.allowExisting => sys.error("allowExisting should be set to false when creating a temporary table.") - case CreateTableUsingAsSelect(tableName, provider, true, mode, opts, query) => - val cmd = - CreateTempTableUsingAsSelect(tableName, provider, mode, opts, query) + case CreateTableUsingAsSelect(tableName, provider, true, partitionsCols, mode, opts, query) + if partitionsCols.nonEmpty => + sys.error("Cannot create temporary partitioned table.") + + case CreateTableUsingAsSelect(tableName, provider, true, _, mode, opts, query) => + val cmd = CreateTempTableUsingAsSelect( + tableName, provider, Array.empty[String], mode, opts, query) ExecutedCommand(cmd) :: Nil case c: CreateTableUsingAsSelect if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 85e60733bc57a..96a39416925b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -136,10 +136,6 @@ private[sql] class DefaultSource } } -private[sql] case class Partition(values: Row, path: String) - -private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) - /** * An alternative to [[ParquetRelation]] that plugs in using the data sources API. This class is * intended as a full replacement of the Parquet support in Spark SQL. The old implementation will @@ -805,7 +801,7 @@ private[sql] object ParquetRelation2 extends Logging { val ordinalMap = metastoreSchema.zipWithIndex.map { case (field, index) => field.name.toLowerCase -> index }.toMap - val reorderedParquetSchema = mergedParquetSchema.sortBy(f => + val reorderedParquetSchema = mergedParquetSchema.sortBy(f => ordinalMap.getOrElse(f.name.toLowerCase, metastoreSchema.size + 1)) StructType(metastoreSchema.zip(reorderedParquetSchema).map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index b3d71f687a60a..1659a7a1989dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -53,6 +53,25 @@ private[sql] object DataSourceStrategy extends Strategy { filters, (a, _) => t.buildScan(a)) :: Nil + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) => + val selectedPartitions = prunePartitions(filters, t.partitionSpec) + val inputPaths = selectedPartitions.map(_.path).toArray + + // Don't push down predicates that reference partition columns + val pushedFilters = { + val partitionColumnNames = t.partitionSpec.partitionColumns.map(_.name).toSet + filters.filter { f => + val referencedColumnNames = f.references.map(_.name).toSet + referencedColumnNames.intersect(partitionColumnNames).isEmpty + } + } + + pruneFilterProject( + l, + projectList, + pushedFilters, + (a, f) => t.buildScan(a, f, inputPaths)) :: Nil + case l @ LogicalRelation(t: TableScan) => createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil @@ -63,6 +82,33 @@ private[sql] object DataSourceStrategy extends Strategy { case _ => Nil } + protected def prunePartitions( + predicates: Seq[Expression], + partitionSpec: PartitionSpec): Seq[Partition] = { + val PartitionSpec(partitionColumns, partitions) = partitionSpec + val partitionColumnNames = partitionColumns.map(_.name).toSet + val partitionPruningPredicates = predicates.filter { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + + if (partitionPruningPredicates.nonEmpty) { + val predicate = + partitionPruningPredicates + .reduceOption(expressions.And) + .getOrElse(Literal(true)) + + val boundPredicate = InterpretedPredicate(predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }) + + partitions.filter { case Partition(values, _) => boundPredicate(values) } + } else { + partitions + } + } + // Based on Public API. protected def pruneFilterProject( relation: LogicalRelation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala new file mode 100644 index 0000000000000..d30f7f65e21c0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala @@ -0,0 +1,207 @@ +/* + * 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.sources + +import java.lang.{Double => JDouble, Float => JFloat, Long => JLong} +import java.math.{BigDecimal => JBigDecimal} + +import scala.collection.mutable.ArrayBuffer +import scala.util.Try + +import com.google.common.cache.{CacheBuilder, Cache} +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} +import org.apache.spark.sql.types._ + +private[sql] case class Partition(values: Row, path: String) + +private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) + +private[sql] object PartitioningUtils { + private[sql] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { + require(columnNames.size == literals.size) + } + + /** + * Given a group of qualified paths, tries to parse them and returns a partition specification. + * For example, given: + * {{{ + * hdfs://:/path/to/partition/a=1/b=hello/c=3.14 + * hdfs://:/path/to/partition/a=2/b=world/c=6.28 + * }}} + * it returns: + * {{{ + * PartitionSpec( + * partitionColumns = StructType( + * StructField(name = "a", dataType = IntegerType, nullable = true), + * StructField(name = "b", dataType = StringType, nullable = true), + * StructField(name = "c", dataType = DoubleType, nullable = true)), + * partitions = Seq( + * Partition( + * values = Row(1, "hello", 3.14), + * path = "hdfs://:/path/to/partition/a=1/b=hello/c=3.14"), + * Partition( + * values = Row(2, "world", 6.28), + * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28"))) + * }}} + */ + private[sql] def parsePartitions( + paths: Seq[Path], + defaultPartitionName: String): PartitionSpec = { + val partitionValues = resolvePartitions(paths.map(parsePartition(_, defaultPartitionName))) + val fields = { + val (PartitionValues(columnNames, literals)) = partitionValues.head + columnNames.zip(literals).map { case (name, Literal(_, dataType)) => + StructField(name, dataType, nullable = true) + } + } + + val partitions = partitionValues.zip(paths).map { + case (PartitionValues(_, literals), path) => + Partition(Row(literals.map(_.value): _*), path.toString) + } + + PartitionSpec(StructType(fields), partitions) + } + + /** + * Parses a single partition, returns column names and values of each partition column. For + * example, given: + * {{{ + * path = hdfs://:/path/to/partition/a=42/b=hello/c=3.14 + * }}} + * it returns: + * {{{ + * PartitionValues( + * Seq("a", "b", "c"), + * Seq( + * Literal.create(42, IntegerType), + * Literal.create("hello", StringType), + * Literal.create(3.14, FloatType))) + * }}} + */ + private[sql] def parsePartition( + path: Path, + defaultPartitionName: String): PartitionValues = { + val columns = ArrayBuffer.empty[(String, Literal)] + // Old Hadoop versions don't have `Path.isRoot` + var finished = path.getParent == null + var chopped = path + + while (!finished) { + val maybeColumn = parsePartitionColumn(chopped.getName, defaultPartitionName) + maybeColumn.foreach(columns += _) + chopped = chopped.getParent + finished = maybeColumn.isEmpty || chopped.getParent == null + } + + val (columnNames, values) = columns.reverse.unzip + PartitionValues(columnNames, values) + } + + private def parsePartitionColumn( + columnSpec: String, + defaultPartitionName: String): Option[(String, Literal)] = { + val equalSignIndex = columnSpec.indexOf('=') + if (equalSignIndex == -1) { + None + } else { + val columnName = columnSpec.take(equalSignIndex) + assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") + + val rawColumnValue = columnSpec.drop(equalSignIndex + 1) + assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") + + val literal = inferPartitionColumnValue(rawColumnValue, defaultPartitionName) + Some(columnName -> literal) + } + } + + /** + * Resolves possible type conflicts between partitions by up-casting "lower" types. The up- + * casting order is: + * {{{ + * NullType -> + * IntegerType -> LongType -> + * FloatType -> DoubleType -> DecimalType.Unlimited -> + * StringType + * }}} + */ + private[sql] def resolvePartitions(values: Seq[PartitionValues]): Seq[PartitionValues] = { + // Column names of all partitions must match + val distinctPartitionsColNames = values.map(_.columnNames).distinct + assert(distinctPartitionsColNames.size == 1, { + val list = distinctPartitionsColNames.mkString("\t", "\n", "") + s"Conflicting partition column names detected:\n$list" + }) + + // Resolves possible type conflicts for each column + val columnCount = values.head.columnNames.size + val resolvedValues = (0 until columnCount).map { i => + resolveTypeConflicts(values.map(_.literals(i))) + } + + // Fills resolved literals back to each partition + values.zipWithIndex.map { case (d, index) => + d.copy(literals = resolvedValues.map(_(index))) + } + } + + /** + * Converts a string to a `Literal` with automatic type inference. Currently only supports + * [[IntegerType]], [[LongType]], [[FloatType]], [[DoubleType]], [[DecimalType.Unlimited]], and + * [[StringType]]. + */ + private[sql] def inferPartitionColumnValue( + raw: String, + defaultPartitionName: String): Literal = { + // First tries integral types + Try(Literal.create(Integer.parseInt(raw), IntegerType)) + .orElse(Try(Literal.create(JLong.parseLong(raw), LongType))) + // Then falls back to fractional types + .orElse(Try(Literal.create(JFloat.parseFloat(raw), FloatType))) + .orElse(Try(Literal.create(JDouble.parseDouble(raw), DoubleType))) + .orElse(Try(Literal.create(new JBigDecimal(raw), DecimalType.Unlimited))) + // Then falls back to string + .getOrElse { + if (raw == defaultPartitionName) Literal.create(null, NullType) + else Literal.create(raw, StringType) + } + } + + private val upCastingOrder: Seq[DataType] = + Seq(NullType, IntegerType, LongType, FloatType, DoubleType, DecimalType.Unlimited, StringType) + + /** + * Given a collection of [[Literal]]s, resolves possible type conflicts by up-casting "lower" + * types. + */ + private def resolveTypeConflicts(literals: Seq[Literal]): Seq[Literal] = { + val desiredType = { + val topType = literals.map(_.dataType).maxBy(upCastingOrder.indexOf(_)) + // Falls back to string if all values of this column are null or empty string + if (topType == NullType) StringType else topType + } + + literals.map { case l @ Literal(_, dataType) => + Literal.create(Cast(l, desiredType).eval(), desiredType) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index dbdb0d39c26a1..3e97506ba2e25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -16,10 +16,21 @@ */ package org.apache.spark.sql.sources -import org.apache.spark.sql.{DataFrame, SQLContext} +import java.util +import java.util.Date + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred._ +import org.apache.hadoop.util.Shell + +import org.apache.spark._ +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand +import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} private[sql] case class InsertIntoDataSource( logicalRelation: LogicalRelation, @@ -41,3 +52,307 @@ private[sql] case class InsertIntoDataSource( Seq.empty[Row] } } + +private[sql] case class InsertIntoFSBasedRelation( + @transient relation: FSBasedRelation, + @transient query: LogicalPlan, + partitionColumns: Array[String], + mode: SaveMode) + extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + require( + relation.paths.length == 1, + s"Cannot write to multiple destinations: ${relation.paths.mkString(",")}") + + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val outputPath = new Path(relation.paths.head) + + val fs = outputPath.getFileSystem(hadoopConf) + val doInsertion = (mode, fs.exists(outputPath)) match { + case (SaveMode.ErrorIfExists, true) => + sys.error(s"path $outputPath already exists.") + case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => + true + case (SaveMode.Ignore, exists) => + !exists + } + + if (doInsertion) { + val jobConf = new JobConf(hadoopConf) + jobConf.setOutputKeyClass(classOf[Void]) + jobConf.setOutputValueClass(classOf[Row]) + FileOutputFormat.setOutputPath(jobConf, outputPath) + + val df = sqlContext.createDataFrame( + DataFrame(sqlContext, query).queryExecution.toRdd, + relation.schema, + needsConversion = false) + + if (partitionColumns.isEmpty) { + insert(new WriterContainer(relation, jobConf), df) + } else { + val writerContainer = new DynamicPartitionWriterContainer( + relation, jobConf, partitionColumns, "__HIVE_DEFAULT_PARTITION__") + insertWithDynamicPartitions(writerContainer, df, partitionColumns) + } + } + + Seq.empty[Row] + } + + private def insert(writerContainer: WriterContainer, df: DataFrame): Unit = { + try { + writerContainer.driverSideSetup() + df.sqlContext.sparkContext.runJob(df.rdd, writeRows _) + writerContainer.commitJob() + } catch { case cause: Throwable => + writerContainer.abortJob() + throw new SparkException("Job aborted.", cause) + } + + def writeRows(taskContext: TaskContext, iterator: Iterator[Row]): Unit = { + writerContainer.executorSideSetup(taskContext) + + try { + while (iterator.hasNext) { + val row = iterator.next() + writerContainer.outputWriterForRow(row).write(row) + } + writerContainer.commitTask() + } catch { case cause: Throwable => + writerContainer.abortTask() + throw new SparkException("Task failed while writing rows.", cause) + } + } + } + + private def insertWithDynamicPartitions( + writerContainer: WriterContainer, + df: DataFrame, + partitionColumns: Array[String]): Unit = { + + val sqlContext = df.sqlContext + + val (partitionRDD, dataRDD) = { + val fieldNames = relation.schema.fieldNames + val (partitionCols, dataCols) = fieldNames.partition(partitionColumns.contains) + val df = sqlContext.createDataFrame( + DataFrame(sqlContext, query).queryExecution.toRdd, + relation.schema, + needsConversion = false) + + assert( + partitionCols.sameElements(partitionColumns), { + val insertionPartitionCols = partitionColumns.mkString(",") + val relationPartitionCols = + relation.partitionSpec.partitionColumns.fieldNames.mkString(",") + s"""Partition columns mismatch. + |Expected: $relationPartitionCols + |Actual: $insertionPartitionCols + """.stripMargin + }) + + val partitionDF = df.select(partitionCols.head, partitionCols.tail: _*) + val dataDF = df.select(dataCols.head, dataCols.tail: _*) + + (partitionDF.rdd, dataDF.rdd) + } + + try { + writerContainer.driverSideSetup() + sqlContext.sparkContext.runJob(partitionRDD.zip(dataRDD), writeRows _) + writerContainer.commitJob() + relation.refreshPartitions() + } catch { case cause: Throwable => + writerContainer.abortJob() + throw new SparkException("Job aborted.", cause) + } + + def writeRows(taskContext: TaskContext, iterator: Iterator[(Row, Row)]): Unit = { + writerContainer.executorSideSetup(taskContext) + + try { + while (iterator.hasNext) { + val (partitionValues, data) = iterator.next() + writerContainer.outputWriterForRow(partitionValues).write(data) + } + + writerContainer.commitTask() + } catch { case cause: Throwable => + writerContainer.abortTask() + throw new SparkException("Task failed while writing rows.", cause) + } + } + } +} + +private[sql] class WriterContainer( + @transient val relation: FSBasedRelation, + @transient jobConf: JobConf) + extends SparkHadoopMapRedUtil + with Logging + with Serializable { + + protected val serializableJobConf = new SerializableWritable(jobConf) + + // This is only used on driver side. + @transient private var jobContext: JobContext = _ + + // This is only used on executor side. + @transient private var taskAttemptContext: TaskAttemptContext = _ + + // The following fields are initialized and used on both driver and executor side. + @transient private var outputCommitter: OutputCommitter = _ + @transient private var jobId: JobID = _ + @transient private var taskId: TaskID = _ + @transient private var taskAttemptId: TaskAttemptID = _ + + protected val outputPath = { + assert( + relation.paths.length == 1, + s"Cannot write to multiple destinations: ${relation.paths.mkString(",")}") + relation.paths.head + } + + protected val dataSchema = relation.dataSchema + + protected val outputWriterClass: Class[_ <: OutputWriter] = relation.outputWriterClass + + // All output writers are created on executor side. + @transient protected var outputWriters: mutable.Map[String, OutputWriter] = _ + + def driverSideSetup(): Unit = { + setupIDs(0, 0, 0) + setupJobConf() + jobContext = newJobContext(jobConf, jobId) + outputCommitter = jobConf.getOutputCommitter + outputCommitter.setupJob(jobContext) + } + + def executorSideSetup(taskContext: TaskContext): Unit = { + setupIDs(taskContext.stageId(), taskContext.partitionId(), taskContext.attemptNumber()) + setupJobConf() + taskAttemptContext = newTaskAttemptContext(serializableJobConf.value, taskAttemptId) + outputCommitter = serializableJobConf.value.getOutputCommitter + outputCommitter.setupTask(taskAttemptContext) + outputWriters = initWriters() + } + + private def setupIDs(jobId: Int, splitId: Int, attemptId: Int): Unit = { + this.jobId = SparkHadoopWriter.createJobID(new Date, jobId) + this.taskId = new TaskID(this.jobId, true, splitId) + this.taskAttemptId = new TaskAttemptID(taskId, attemptId) + } + + private def setupJobConf(): Unit = { + serializableJobConf.value.set("mapred.job.id", jobId.toString) + serializableJobConf.value.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + serializableJobConf.value.set("mapred.task.id", taskAttemptId.toString) + serializableJobConf.value.setBoolean("mapred.task.is.map", true) + serializableJobConf.value.setInt("mapred.task.partition", 0) + } + + // Called on executor side when writing rows + def outputWriterForRow(row: Row): OutputWriter = outputWriters.values.head + + protected def initWriters(): mutable.Map[String, OutputWriter] = { + val writer = outputWriterClass.newInstance() + writer.init(outputPath, dataSchema, serializableJobConf.value) + mutable.Map(outputPath -> writer) + } + + def commitTask(): Unit = { + outputWriters.values.foreach(_.close()) + SparkHadoopMapRedUtil.commitTask( + outputCommitter, taskAttemptContext, jobId.getId, taskId.getId, taskAttemptId.getId) + } + + def abortTask(): Unit = { + outputWriters.values.foreach(_.close()) + outputCommitter.abortTask(taskAttemptContext) + logError(s"Task attempt $taskAttemptId aborted.") + } + + def commitJob(): Unit = { + outputCommitter.commitJob(jobContext) + logInfo(s"Job $jobId committed.") + } + + def abortJob(): Unit = { + outputCommitter.abortJob(jobContext, JobStatus.FAILED) + logError(s"Job $jobId aborted.") + } +} + +private[sql] class DynamicPartitionWriterContainer( + @transient relation: FSBasedRelation, + @transient conf: JobConf, + partitionColumns: Array[String], + defaultPartitionName: String) + extends WriterContainer(relation, conf) { + + override protected def initWriters() = mutable.Map.empty[String, OutputWriter] + + override def outputWriterForRow(row: Row): OutputWriter = { + val partitionPath = partitionColumns.zip(row.toSeq).map { case (col, rawValue) => + val string = if (rawValue == null) null else String.valueOf(rawValue) + val valueString = if (string == null || string.isEmpty) { + defaultPartitionName + } else { + escapePathName(string) + } + s"/$col=$valueString" + }.mkString + + outputWriters.getOrElseUpdate(partitionPath, { + val path = new Path(outputPath, partitionPath.stripPrefix(Path.SEPARATOR)) + val writer = outputWriterClass.newInstance() + writer.init(path.toString, dataSchema, serializableJobConf.value) + writer + }) + } + + private def escapePathName(path: String): String = { + val builder = new StringBuilder() + path.foreach { c => + if (DynamicPartitionWriterContainer.needsEscaping(c)) { + builder.append('%') + builder.append(f"${c.asInstanceOf[Int]}%02x") + } else { + builder.append(c) + } + } + + builder.toString() + } +} + +private[sql] object DynamicPartitionWriterContainer { + val charToEscape = { + val bitSet = new util.BitSet(128) + + /** + * ASCII 01-1F are HTTP control characters that need to be escaped. + * \u000A and \u000D are \n and \r, respectively. + */ + val clist = Array( + '\u0001', '\u0002', '\u0003', '\u0004', '\u0005', '\u0006', '\u0007', '\u0008', '\u0009', + '\n', '\u000B', '\u000C', '\r', '\u000E', '\u000F', '\u0010', '\u0011', '\u0012', '\u0013', + '\u0014', '\u0015', '\u0016', '\u0017', '\u0018', '\u0019', '\u001A', '\u001B', '\u001C', + '\u001D', '\u001E', '\u001F', '"', '#', '%', '\'', '*', '/', ':', '=', '?', '\\', '\u007F', + '{', '[', ']', '^') + + clist.foreach(bitSet.set(_)) + + if (Shell.WINDOWS) { + Array(' ', '<', '>', '|').foreach(bitSet.set(_)) + } + + bitSet + } + + def needsEscaping(c: Char): Boolean = { + c >= 0 && c < charToEscape.size() && charToEscape.get(c); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 06c64f2bdd59e..39615e946a308 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -17,18 +17,17 @@ package org.apache.spark.sql.sources -import scala.language.existentials +import scala.language.{existentials, implicitConversions} import scala.util.matching.Regex -import scala.language.implicitConversions import org.apache.spark.Logging -import org.apache.spark.sql.{AnalysisException, SaveMode, DataFrame, SQLContext} -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.types._ +import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext, SaveMode} import org.apache.spark.util.Utils /** @@ -111,6 +110,7 @@ private[sql] class DDLParser( CreateTableUsingAsSelect(tableName, provider, temp.isDefined, + Array.empty[String], mode, options, queryPlan) @@ -214,6 +214,7 @@ private[sql] object ResolvedDataSource { def apply( sqlContext: SQLContext, userSpecifiedSchema: Option[StructType], + partitionColumns: Array[String], provider: String, options: Map[String, String]): ResolvedDataSource = { val clazz: Class[_] = lookupDataSource(provider) @@ -222,6 +223,12 @@ private[sql] object ResolvedDataSource { case Some(schema: StructType) => clazz.newInstance() match { case dataSource: SchemaRelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema) + case dataSource: FSBasedRelationProvider => + dataSource.createRelation( + sqlContext, + Some(schema), + Some(partitionColumnsSchema(schema, partitionColumns)), + new CaseInsensitiveMap(options)) case dataSource: org.apache.spark.sql.sources.RelationProvider => throw new AnalysisException(s"$className does not allow user-specified schemas.") case _ => @@ -231,20 +238,34 @@ private[sql] object ResolvedDataSource { case None => clazz.newInstance() match { case dataSource: RelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) + case dataSource: FSBasedRelationProvider => + dataSource.createRelation(sqlContext, None, None, new CaseInsensitiveMap(options)) case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => throw new AnalysisException( s"A schema needs to be specified when using $className.") case _ => - throw new AnalysisException(s"$className is not a RelationProvider.") + throw new AnalysisException( + s"$className is neither a RelationProvider nor a FSBasedRelationProvider.") } } new ResolvedDataSource(clazz, relation) } + private def partitionColumnsSchema( + schema: StructType, + partitionColumns: Array[String]): StructType = { + StructType(partitionColumns.map { col => + schema.find(_.name == col).getOrElse { + throw new RuntimeException(s"Partition column $col not found in schema $schema") + } + }) + } + /** Create a [[ResolvedDataSource]] for saving the content of the given [[DataFrame]]. */ def apply( sqlContext: SQLContext, provider: String, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], data: DataFrame): ResolvedDataSource = { @@ -252,6 +273,19 @@ private[sql] object ResolvedDataSource { val relation = clazz.newInstance() match { case dataSource: CreatableRelationProvider => dataSource.createRelation(sqlContext, mode, options, data) + case dataSource: FSBasedRelationProvider => + val r = dataSource.createRelation( + sqlContext, + Some(data.schema), + Some(partitionColumnsSchema(data.schema, partitionColumns)), + options) + sqlContext.executePlan( + InsertIntoFSBasedRelation( + r.asInstanceOf[FSBasedRelation], + data.logicalPlan, + partitionColumns.toArray, + mode)).toRdd + r case _ => sys.error(s"${clazz.getCanonicalName} does not allow create table as select.") } @@ -310,6 +344,7 @@ private[sql] case class CreateTableUsingAsSelect( tableName: String, provider: String, temporary: Boolean, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], child: LogicalPlan) extends UnaryNode { @@ -324,8 +359,9 @@ private[sql] case class CreateTempTableUsing( provider: String, options: Map[String, String]) extends RunnableCommand { - override def run(sqlContext: SQLContext): Seq[Row] = { - val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) + def run(sqlContext: SQLContext): Seq[Row] = { + val resolved = ResolvedDataSource( + sqlContext, userSpecifiedSchema, Array.empty[String], provider, options) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) Seq.empty @@ -335,13 +371,14 @@ private[sql] case class CreateTempTableUsing( private[sql] case class CreateTempTableUsingAsSelect( tableName: String, provider: String, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], query: LogicalPlan) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) - val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) + val resolved = ResolvedDataSource(sqlContext, provider, partitionColumns, mode, options, df) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 094ccefd8c832..d25034d900ece 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -18,11 +18,13 @@ package org.apache.spark.sql.sources import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode} /** @@ -109,9 +111,9 @@ trait FSBasedRelationProvider { */ def createRelation( sqlContext: SQLContext, - parameters: Map[String, String], - schema: StructType, - partitionColumns: StructType): BaseRelation + schema: Option[StructType], + partitionColumns: Option[StructType], + parameters: Map[String, String]): BaseRelation } @DeveloperApi @@ -259,13 +261,11 @@ abstract class OutputWriter { * @param path The file path to which this [[OutputWriter]] is supposed to write. * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the corresponding relation is partitioned. - * @param options Data source options inherited from driver side. * @param conf Hadoop configuration inherited from driver side. */ def init( path: String, dataSchema: StructType, - options: java.util.Map[String, String], conf: Configuration): Unit = () /** @@ -294,18 +294,65 @@ abstract class OutputWriter { * * For the write path, it provides the ability to write to both non-partitioned and partitioned * tables. Directory layout of the partitioned tables is compatible with Hive. + * + * @constructor This constructor is for internal uses only. The [[PartitionSpec]] argument is for + * implementing metastore table conversion. + * @param paths Base paths of this relation. For partitioned relations, it should be either root + * directories of all partition directories. + * @param maybePartitionSpec An [[FSBasedRelation]] can be created with an optional + * [[PartitionSpec]], so that partition discovery can be skipped. */ @Experimental -abstract class FSBasedRelation extends BaseRelation { - // Discovers partitioned columns, and merge them with `dataSchema`. All partition columns not - // existed in `dataSchema` should be appended to `dataSchema`. - override val schema: StructType = ??? +abstract class FSBasedRelation private[sql]( + val paths: Array[String], + maybePartitionSpec: Option[PartitionSpec]) + extends BaseRelation { /** - * Base path of this relation. For partitioned relations, `path` should be the root directory of - * all partition directories. + * Constructs an [[FSBasedRelation]]. + * + * @param paths Base paths of this relation. For partitioned relations, it should be either root + * directories of all partition directories. */ - def path: String + def this(paths: Array[String]) = this(paths, None) + + private val hadoopConf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) + + private var _partitionSpec: PartitionSpec = _ + refreshPartitions() + + private[sql] def partitionSpec: PartitionSpec = _partitionSpec + + private[sql] def refreshPartitions(): Unit = { + _partitionSpec = maybePartitionSpec.getOrElse { + val basePaths = paths.map(new Path(_)) + val leafDirs = basePaths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + if (fs.exists(path)) { + SparkHadoopUtil.get.listLeafDirStatuses(fs, path) + } else { + Seq.empty[FileStatus] + } + }.map(_.getPath) + + if (leafDirs.nonEmpty) { + PartitioningUtils.parsePartitions(leafDirs, "__HIVE_DEFAULT_PARTITION__") + } else { + PartitionSpec(StructType(Array.empty[StructField]), Array.empty[Partition]) + } + } + } + + /** + * Schema of this relation. It consists of [[dataSchema]] and all partition columns not appeared + * in [[dataSchema]]. + */ + override val schema: StructType = { + val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet + StructType(dataSchema ++ partitionSpec.partitionColumns.filterNot { column => + dataSchemaColumnNames.contains(column.name.toLowerCase) + }) + } /** * Specifies schema of actual data files. For partitioned relations, if one or more partitioned diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 6ed68d179edc9..940e5c96e2b0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -107,7 +107,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") - case CreateTableUsingAsSelect(tableName, _, _, SaveMode.Overwrite, _, query) => + case CreateTableUsingAsSelect(tableName, _, _, _, SaveMode.Overwrite, _, query) => // When the SaveMode is Overwrite, we need to check if the table is an input table of // the query. If so, we will throw an AnalysisException to let users know it is not allowed. if (catalog.tableExists(Seq(tableName))) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index b7561ce7298cb..2a9876fa899b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.parquet.ParquetRelation2._ +import org.apache.spark.sql.sources.{Partition, PartitionSpec} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, Row, SQLContext} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 54f2f3cdec298..d17391b49031d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.sources import java.io.{IOException, File} -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{SQLContext, AnalysisException} import org.scalatest.BeforeAndAfterAll import org.apache.spark.util.Utils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index bbf48efb24440..d754c8e3a8aa1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -19,25 +19,24 @@ package org.apache.spark.sql.hive import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} - import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.metastore.Warehouse +import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata._ import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.Logging -import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} -import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, Catalog, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => ParquetPartition, PartitionSpec} -import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.parquet.ParquetRelation2 +import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, LogicalRelation, Partition => ParquetPartition, PartitionSpec, ResolvedDataSource} import org.apache.spark.sql.types._ +import org.apache.spark.sql.{AnalysisException, SQLContext, SaveMode} import org.apache.spark.util.Utils /* Implicit conversions */ @@ -98,6 +97,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive ResolvedDataSource( hive, userSpecifiedSchema, + Array.empty[String], table.properties("spark.sql.sources.provider"), options) @@ -438,6 +438,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive desc.name, hive.conf.defaultDataSourceName, temporary = false, + Array.empty[String], mode, options = Map.empty[String, String], child diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index be9249a8b1f44..d46a127d47d31 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -221,14 +221,14 @@ private[hive] trait HiveStrategies { object HiveDDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case CreateTableUsing( - tableName, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => + tableName, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => ExecutedCommand( CreateMetastoreDataSource( tableName, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath)) :: Nil - case CreateTableUsingAsSelect(tableName, provider, false, mode, opts, query) => + case CreateTableUsingAsSelect(tableName, provider, false, partitionCols, mode, opts, query) => val cmd = - CreateMetastoreDataSourceAsSelect(tableName, provider, mode, opts, query) + CreateMetastoreDataSourceAsSelect(tableName, provider, partitionCols, mode, opts, query) ExecutedCommand(cmd) :: Nil case _ => Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index abab1a223a43a..70f61c5656562 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -158,6 +158,7 @@ private[hive] case class CreateMetastoreDataSourceAsSelect( tableName: String, provider: String, + partitionColumns: Array[String], mode: SaveMode, options: Map[String, String], query: LogicalPlan) extends RunnableCommand { @@ -189,12 +190,12 @@ case class CreateMetastoreDataSourceAsSelect( return Seq.empty[Row] case SaveMode.Append => // Check if the specified data source match the data source of the existing table. - val resolved = - ResolvedDataSource(sqlContext, Some(query.schema), provider, optionsWithPath) + val resolved = ResolvedDataSource( + sqlContext, Some(query.schema), partitionColumns, provider, optionsWithPath) val createdRelation = LogicalRelation(resolved.relation) EliminateSubQueries(sqlContext.table(tableName).logicalPlan) match { - case l @ LogicalRelation(i: InsertableRelation) => - if (i != createdRelation.relation) { + case l @ LogicalRelation(_: InsertableRelation | _: FSBasedRelation) => + if (l.relation != createdRelation.relation) { val errorDescription = s"Cannot append to table $tableName because the resolved relation does not " + s"match the existing relation of $tableName. " + @@ -234,7 +235,8 @@ case class CreateMetastoreDataSourceAsSelect( } // Create the relation based on the data of df. - val resolved = ResolvedDataSource(sqlContext, provider, mode, optionsWithPath, df) + val resolved = + ResolvedDataSource(sqlContext, provider, partitionColumns, mode, optionsWithPath, df) if (createMetastoreTable) { // We will use the schema of resolved.relation as the schema of the table (instead of diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 8398da268174d..cbc381cc81b59 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -204,7 +204,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( if (string == null || string.isEmpty) { defaultPartName } else { - FileUtils.escapePathName(string) + FileUtils.escapePathName(string, defaultPartName) } s"/$col=$colString" }.mkString diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala similarity index 60% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 6c5bb8e425a74..59e01b2690781 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -19,41 +19,80 @@ package org.apache.spark.sql.sources import java.io.IOException -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable +import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.types._ -import org.apache.spark.sql.{Row, SQLContext, SaveMode} +import org.apache.spark.sql._ import org.apache.spark.util.Utils -class SimpleFSBasedSource extends RelationProvider { +class SimpleFSBasedSource extends FSBasedRelationProvider { override def createRelation( sqlContext: SQLContext, + schema: Option[StructType], + partitionColumns: Option[StructType], parameters: Map[String, String]): BaseRelation = { - SimpleFSBasedRelation(parameters)(sqlContext) + val path = parameters("path") + + // Uses data sources options to simulate data schema + val dataSchema = DataType.fromJson(parameters("schema")).asInstanceOf[StructType] + + // Uses data sources options to mock partition discovery + val maybePartitionSpec = + parameters.get("partitionColumns").map { json => + new PartitionSpec( + DataType.fromJson(json).asInstanceOf[StructType], + Array.empty[Partition]) + } + + SimpleFSBasedRelation(path, dataSchema, maybePartitionSpec, parameters)(sqlContext) } } -case class SimpleFSBasedRelation - (parameter: Map[String, String]) - (val sqlContext: SQLContext) - extends FSBasedRelation { +class SimpleOutputWriter extends OutputWriter { + override def init(path: String, dataSchema: StructType, conf: Configuration): Unit = { + TestResult.synchronized { + TestResult.writerPaths += path + } + } - class SimpleOutputWriter extends OutputWriter { - override def write(row: Row): Unit = TestResult.writtenRows += row + override def write(row: Row): Unit = { + TestResult.synchronized { + TestResult.writtenRows += row + } } +} - override val path = parameter("path") +case class SimpleFSBasedRelation + (path: String, + dataSchema: StructType, + maybePartitionSpec: Option[PartitionSpec], + parameter: Map[String, String]) + (@transient val sqlContext: SQLContext) + extends FSBasedRelation(Array(path), maybePartitionSpec) { + + override def equals(obj: scala.Any): Boolean = obj match { + case that: SimpleFSBasedRelation => + this.path == that.path && + this.dataSchema == that.dataSchema && + this.maybePartitionSpec == that.maybePartitionSpec + case _ => false + } - override def dataSchema: StructType = - DataType.fromJson(parameter("schema")).asInstanceOf[StructType] + override def hashCode(): Int = + Objects.hashCode(path, dataSchema, maybePartitionSpec) override def buildScan( requiredColumns: Array[String], filters: Array[Filter], inputPaths: Array[String]): RDD[Row] = { + val sqlContext = this.sqlContext val basePath = new Path(path) val fs = basePath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) @@ -61,11 +100,12 @@ case class SimpleFSBasedRelation // shouldn't be removed before scanning the data. assert(inputPaths.map(new Path(_)).forall(fs.exists)) - TestResult.requiredColumns = requiredColumns - TestResult.filters = filters - TestResult.inputPaths = inputPaths - - Option(TestResult.rowsToRead).getOrElse(sqlContext.emptyResult) + TestResult.synchronized { + TestResult.requiredColumns = requiredColumns + TestResult.filters = filters + TestResult.inputPaths = inputPaths + Option(TestResult.rowsToRead).getOrElse(sqlContext.emptyResult) + } } override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter] @@ -76,20 +116,22 @@ object TestResult { var filters: Array[Filter] = _ var inputPaths: Array[String] = _ var rowsToRead: RDD[Row] = _ - var writtenRows: ArrayBuffer[Row] = ArrayBuffer.empty[Row] + var writerPaths: mutable.Set[String] = mutable.Set.empty[String] + var writtenRows: mutable.Set[Row] = mutable.Set.empty[Row] - def reset(): Unit = { + def reset(): Unit = this.synchronized { requiredColumns = null filters = null inputPaths = null rowsToRead = null + writerPaths.clear() writtenRows.clear() } } -class FSBasedRelationSuite extends DataSourceTest { - import caseInsensitiveContext._ - import caseInsensitiveContext.implicits._ +class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { + import TestHive._ + import TestHive.implicits._ var basePath: Path = _ @@ -101,6 +143,8 @@ class FSBasedRelationSuite extends DataSourceTest { StructField("a", IntegerType, nullable = false), StructField("b", StringType, nullable = false))) + val partitionColumns = StructType(StructField("p1", IntegerType, nullable = true) :: Nil) + val testDF = (for { i <- 1 to 3 p <- 1 to 2 @@ -109,10 +153,11 @@ class FSBasedRelationSuite extends DataSourceTest { before { basePath = new Path(Utils.createTempDir().getCanonicalPath) fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) + basePath = fs.makeQualified(basePath) TestResult.reset() } - ignore("load() - partitioned table - partition column not included in data files") { + test("load() - partitioned table - partition column not included in data files") { fs.mkdirs(new Path(basePath, "p1=1/p2=hello")) fs.mkdirs(new Path(basePath, "p1=2/p2=world")) @@ -137,15 +182,15 @@ class FSBasedRelationSuite extends DataSourceTest { assert(df.schema === expectedSchema) - df.select("b").where($"a" > 0 && $"p1" === 1).collect() + df.where($"a" > 0 && $"p1" === 1).select("b").collect() // Check for column pruning, filter push-down, and partition pruning assert(TestResult.requiredColumns.toSet === Set("a", "b")) assert(TestResult.filters === Seq(GreaterThan("a", 0))) - assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1").toString)) + assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1/p2=hello").toString)) } - ignore("load() - partitioned table - partition column included in data files") { + test("load() - partitioned table - partition column included in data files") { val data = sparkContext.parallelize(Seq.empty[String]) data.saveAsTextFile(new Path(basePath, "p1=1/p2=hello").toString) data.saveAsTextFile(new Path(basePath, "p1=2/p2=world").toString) @@ -180,26 +225,33 @@ class FSBasedRelationSuite extends DataSourceTest { assert(df.schema === expectedSchema) - df.select("b").where($"a" > 0 && $"p1" === 1).collect() + df.where($"a" > 0 && $"p1" === 1).select("b").collect() // Check for column pruning, filter push-down, and partition pruning assert(TestResult.requiredColumns.toSet === Set("a", "b")) assert(TestResult.filters === Seq(GreaterThan("a", 0))) - assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1").toString)) + assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1/p2=hello").toString)) } - ignore("save() - partitioned table - Overwrite") { + test("save() - partitioned table - Overwrite") { testDF.save( source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) + Thread.sleep(500) + // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - assert(TestResult.writtenRows.sameElements(expectedRows)) + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } } ignore("save() - partitioned table - Overwrite - select and overwrite the same table") { @@ -216,21 +268,27 @@ class FSBasedRelationSuite extends DataSourceTest { mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - assert(TestResult.writtenRows.sameElements(expectedRows)) + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } } - ignore("save() - partitioned table - Append") { + test("save() - partitioned table - Append") { testDF.save( source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) testDF.save( @@ -238,15 +296,20 @@ class FSBasedRelationSuite extends DataSourceTest { mode = SaveMode.Append, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i") - assert(TestResult.writtenRows.sameElements(expectedRows)) + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } } - ignore("save() - partitioned table - ErrorIfExists") { + test("save() - partitioned table - ErrorIfExists") { fs.delete(basePath, true) testDF.save( @@ -254,23 +317,31 @@ class FSBasedRelationSuite extends DataSourceTest { mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) - assert(TestResult.writtenRows.sameElements(testDF.collect())) + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } - intercept[IOException] { + intercept[RuntimeException] { testDF.save( source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.ErrorIfExists, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) } } - ignore("save() - partitioned table - Ignore") { + test("save() - partitioned table - Ignore") { testDF.save( source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Ignore, @@ -282,19 +353,34 @@ class FSBasedRelationSuite extends DataSourceTest { assert(TestResult.writtenRows.isEmpty) } - ignore("saveAsTable() - partitioned table - Overwrite") { + test("save() - data sources other than FSBasedRelation") { + val cause = intercept[RuntimeException] { + testDF.save( + source = classOf[FilteredScanSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> basePath.toString), + partitionColumns = Seq("p1")) + } + } + + test("saveAsTable() - partitioned table - Overwrite") { testDF.saveAsTable( tableName = "t", source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - assert(TestResult.writtenRows.sameElements(expectedRows)) + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } assertResult(table("t").schema) { StructType( @@ -312,7 +398,8 @@ class FSBasedRelationSuite extends DataSourceTest { source = classOf[SimpleFSBasedSource].getCanonicalName, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json)) + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json)) df.saveAsTable( tableName = "t", @@ -320,12 +407,17 @@ class FSBasedRelationSuite extends DataSourceTest { mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - assert(TestResult.writtenRows.sameElements(expectedRows)) + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } assertResult(table("t").schema) { StructType( @@ -336,14 +428,15 @@ class FSBasedRelationSuite extends DataSourceTest { sql("DROP TABLE t") } - ignore("saveAsTable() - partitioned table - Append") { + test("saveAsTable() - partitioned table - Append") { testDF.saveAsTable( tableName = "t", source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) testDF.saveAsTable( @@ -352,12 +445,17 @@ class FSBasedRelationSuite extends DataSourceTest { mode = SaveMode.Append, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i") - assert(TestResult.writtenRows.sameElements(expectedRows)) + val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } assertResult(table("t").schema) { StructType( @@ -368,7 +466,7 @@ class FSBasedRelationSuite extends DataSourceTest { sql("DROP TABLE t") } - ignore("saveAsTable() - partitioned table - ErrorIfExists") { + test("saveAsTable() - partitioned table - ErrorIfExists") { fs.delete(basePath, true) testDF.saveAsTable( @@ -377,10 +475,17 @@ class FSBasedRelationSuite extends DataSourceTest { mode = SaveMode.ErrorIfExists, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) - assert(TestResult.writtenRows.sameElements(testDF.collect())) + // Written rows shouldn't contain dynamic partition column + val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + + TestResult.synchronized { + assert(TestResult.writerPaths.size === 2) + assert(TestResult.writtenRows === expectedRows.toSet) + } assertResult(table("t").schema) { StructType( @@ -388,28 +493,30 @@ class FSBasedRelationSuite extends DataSourceTest { StructField("p1", IntegerType, nullable = true))) } - intercept[IOException] { + intercept[AnalysisException] { testDF.saveAsTable( tableName = "t", source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = SaveMode.ErrorIfExists, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) } sql("DROP TABLE t") } - ignore("saveAsTable() - partitioned table - Ignore") { + test("saveAsTable() - partitioned table - Ignore") { testDF.saveAsTable( tableName = "t", source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Ignore, options = Map( "path" -> basePath.toString, - "schema" -> dataSchema.json), + "schema" -> dataSchema.json, + "partitionColumns" -> partitionColumns.json), partitionColumns = Seq("p1")) assert(TestResult.writtenRows.isEmpty) @@ -422,4 +529,15 @@ class FSBasedRelationSuite extends DataSourceTest { sql("DROP TABLE t") } + + test("saveAsTable() - data sources other than FSBasedRelation") { + val cause = intercept[RuntimeException] { + testDF.saveAsTable( + tableName = "t", + source = classOf[FilteredScanSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> basePath.toString), + partitionColumns = Seq("p1")) + } + } } From ea6c8dd35df03c7fcbee2a54f84e2f01ab000444 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 29 Apr 2015 23:34:50 +0800 Subject: [PATCH 13/45] Removes remote debugging stuff --- project/SparkBuild.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 146c9ead80233..186345af0e60e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -548,7 +548,6 @@ object TestSettings { javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", - javaOptions in Test += "-agentlib:jdwp=transport=dt_socket,server=n,address=127.0.0.1:5005", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, javaOptions in Test += "-ea", From 9b487bf63f15a3f03333eace24d6ee30986d0a1e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 29 Apr 2015 23:35:03 +0800 Subject: [PATCH 14/45] Fixes compilation errors introduced while rebasing --- .../scala/org/apache/spark/sql/DataFrame.scala | 18 ++++++++++++++---- .../spark/sql/sources/DataSourceStrategy.scala | 2 +- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 32c867f2eeabd..7db5ae105c77b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, _} import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} +import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, ScalaReflection, SqlParser} import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.json.JacksonGenerator @@ -400,7 +400,9 @@ class DataFrame private[sql]( joined.left, joined.right, joinType = Inner, - Some(EqualTo(joined.left.resolve(usingColumn), joined.right.resolve(usingColumn)))) + Some(expressions.EqualTo( + joined.left.resolve(usingColumn), + joined.right.resolve(usingColumn)))) ) } @@ -1343,7 +1345,7 @@ class DataFrame private[sql]( mode: SaveMode, options: java.util.Map[String, String], partitionColumns: java.util.List[String]): Unit = { - ??? + saveAsTable(tableName, source, mode, options.toMap, partitionColumns) } /** @@ -1399,7 +1401,15 @@ class DataFrame private[sql]( mode: SaveMode, options: Map[String, String], partitionColumns: Seq[String]): Unit = { - ??? + sqlContext.executePlan( + CreateTableUsingAsSelect( + tableName, + source, + temporary = false, + partitionColumns.toArray, + mode, + options, + logicalPlan)).toRdd } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 1659a7a1989dc..52f9bd4421c53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -97,7 +97,7 @@ private[sql] object DataSourceStrategy extends Strategy { .reduceOption(expressions.And) .getOrElse(Literal(true)) - val boundPredicate = InterpretedPredicate(predicate.transform { + val boundPredicate = InterpretedPredicate.create(predicate.transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) BoundReference(index, partitionColumns(index).dataType, nullable = true) From b746ab5e112e824c1f58edd28aabed67d4122f51 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 30 Apr 2015 00:53:52 +0800 Subject: [PATCH 15/45] More tests --- .../sql/sources/FSBasedRelationSuite.scala | 177 ++++++++++-------- 1 file changed, 99 insertions(+), 78 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 59e01b2690781..5e7ef9222734f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.sources -import java.io.IOException - import scala.collection.mutable import com.google.common.base.Objects @@ -27,9 +25,9 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.types._ -import org.apache.spark.sql._ import org.apache.spark.util.Utils class SimpleFSBasedSource extends FSBasedRelationProvider { @@ -143,7 +141,14 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { StructField("a", IntegerType, nullable = false), StructField("b", StringType, nullable = false))) - val partitionColumns = StructType(StructField("p1", IntegerType, nullable = true) :: Nil) + val singlePartitionColumn = StructType( + Seq( + StructField("p1", IntegerType, nullable = true))) + + val doublePartitionColumns = StructType( + Seq( + StructField("p1", IntegerType, nullable = true), + StructField("p2", StringType, nullable = true))) val testDF = (for { i <- 1 to 3 @@ -240,7 +245,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), + "partitionColumns" -> singlePartitionColumn.json), partitionColumns = Seq("p1")) Thread.sleep(500) @@ -269,7 +274,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), + "partitionColumns" -> singlePartitionColumn.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column @@ -281,34 +286,71 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { } } - test("save() - partitioned table - Append") { + test("save() - partitioned table - Append - new partition values") { testDF.save( source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Overwrite, options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), + "partitionColumns" -> singlePartitionColumn.json), partitionColumns = Seq("p1")) - testDF.save( + val moreData = (for { + i <- 1 to 3 + p <- 3 to 4 + } yield (i, s"val_$i", p)).toDF("a", "b", "p1") + + moreData.save( source = classOf[SimpleFSBasedSource].getCanonicalName, mode = SaveMode.Append, options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), + "partitionColumns" -> singlePartitionColumn.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 4) assert(TestResult.writtenRows === expectedRows.toSet) } } + test("save() - partitioned table - Append - mismatched partition columns") { + val data = (for { + i <- 1 to 3 + p1 <- 1 to 2 + p2 <- Array("hello", "world") + } yield (i, s"val_$i", p1, p2)).toDF("a", "b", "p1", "p2") + + // Using only a subset of all partition columns + intercept[IllegalArgumentException] { + data.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Append, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json, + "partitionColumns" -> doublePartitionColumns.json), + partitionColumns = Seq("p1")) + } + + // Using different order of partition columns + intercept[IllegalArgumentException] { + data.save( + source = classOf[SimpleFSBasedSource].getCanonicalName, + mode = SaveMode.Append, + options = Map( + "path" -> basePath.toString, + "schema" -> dataSchema.json, + "partitionColumns" -> doublePartitionColumns.json), + partitionColumns = Seq("p2", "p1")) + } + } + test("save() - partitioned table - ErrorIfExists") { fs.delete(basePath, true) @@ -318,7 +360,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), + "partitionColumns" -> singlePartitionColumn.json), partitionColumns = Seq("p1")) // Written rows shouldn't contain dynamic partition column @@ -336,7 +378,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), + "partitionColumns" -> singlePartitionColumn.json), partitionColumns = Seq("p1")) } } @@ -354,7 +396,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { } test("save() - data sources other than FSBasedRelation") { - val cause = intercept[RuntimeException] { + intercept[RuntimeException] { testDF.save( source = classOf[FilteredScanSource].getCanonicalName, mode = SaveMode.Overwrite, @@ -363,16 +405,25 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { } } - test("saveAsTable() - partitioned table - Overwrite") { - testDF.saveAsTable( - tableName = "t", + def saveToPartitionedTable( + df: DataFrame, + tableName: String, + relationPartitionColumns: StructType, + partitionedBy: Seq[String], + mode: SaveMode): Unit = { + df.saveAsTable( + tableName = tableName, source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = mode, options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), - partitionColumns = Seq("p1")) + "partitionColumns" -> relationPartitionColumns.json), + partitionColumns = partitionedBy) + } + + test("saveAsTable() - partitioned table - Overwrite") { + saveToPartitionedTable(testDF, "t", singlePartitionColumn, Array("p1"), SaveMode.Overwrite) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") @@ -399,17 +450,9 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { options = Map( "path" -> basePath.toString, "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json)) + "partitionColumns" -> singlePartitionColumn.json)) - df.saveAsTable( - tableName = "t", - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), - partitionColumns = Seq("p1")) + saveToPartitionedTable(df, "t", singlePartitionColumn, Seq("p1"), SaveMode.Overwrite) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") @@ -429,25 +472,8 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { } test("saveAsTable() - partitioned table - Append") { - testDF.saveAsTable( - tableName = "t", - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), - partitionColumns = Seq("p1")) - - testDF.saveAsTable( - tableName = "t", - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Append, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), - partitionColumns = Seq("p1")) + saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.Overwrite) + saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.Append) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") @@ -466,18 +492,30 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { sql("DROP TABLE t") } + test("saveAsTable() - partitioned table - Append - mismatched partition columns") { + val data = (for { + i <- 1 to 3 + p1 <- 1 to 2 + p2 <- Array("hello", "world") + } yield (i, s"val_$i", p1, p2)).toDF("a", "b", "p1", "p2") + + // Using only a subset of all partition columns + intercept[IllegalArgumentException] { + saveToPartitionedTable(data, "t", doublePartitionColumns, Seq("p1"), SaveMode.Append) + } + + // Using different order of partition columns + intercept[IllegalArgumentException] { + saveToPartitionedTable(data, "t", doublePartitionColumns, Seq("p2", "p1"), SaveMode.Append) + } + + sql("DROP TABLE t") + } + test("saveAsTable() - partitioned table - ErrorIfExists") { fs.delete(basePath, true) - testDF.saveAsTable( - tableName = "t", - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.ErrorIfExists, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), - partitionColumns = Seq("p1")) + saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.ErrorIfExists) // Written rows shouldn't contain dynamic partition column val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") @@ -494,33 +532,16 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { } intercept[AnalysisException] { - testDF.saveAsTable( - tableName = "t", - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.ErrorIfExists, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), - partitionColumns = Seq("p1")) + saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.ErrorIfExists) } sql("DROP TABLE t") } test("saveAsTable() - partitioned table - Ignore") { - testDF.saveAsTable( - tableName = "t", - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Ignore, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> partitionColumns.json), - partitionColumns = Seq("p1")) + saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.Ignore) assert(TestResult.writtenRows.isEmpty) - assertResult(table("t").schema) { StructType( dataSchema ++ Seq( @@ -531,7 +552,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { } test("saveAsTable() - data sources other than FSBasedRelation") { - val cause = intercept[RuntimeException] { + intercept[RuntimeException] { testDF.saveAsTable( tableName = "t", source = classOf[FilteredScanSource].getCanonicalName, From f18dec2a858b6f662ca7ba3e87b1363e1e790996 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 30 Apr 2015 00:54:18 +0800 Subject: [PATCH 16/45] More strict schema checking --- .../apache/spark/sql/sources/commands.scala | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 3e97506ba2e25..ee3b9cab5e0fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -132,28 +132,32 @@ private[sql] case class InsertIntoFSBasedRelation( df: DataFrame, partitionColumns: Array[String]): Unit = { + require( + df.schema == relation.schema, + s"""DataFrame must have the same schema as the relation to which is inserted. + |DataFrame schema: ${df.schema} + |Relation schema: ${relation.schema} + """.stripMargin) + val sqlContext = df.sqlContext val (partitionRDD, dataRDD) = { val fieldNames = relation.schema.fieldNames - val (partitionCols, dataCols) = fieldNames.partition(partitionColumns.contains) + val dataCols = fieldNames.filterNot(partitionColumns.contains) val df = sqlContext.createDataFrame( DataFrame(sqlContext, query).queryExecution.toRdd, relation.schema, needsConversion = false) - assert( - partitionCols.sameElements(partitionColumns), { - val insertionPartitionCols = partitionColumns.mkString(",") - val relationPartitionCols = - relation.partitionSpec.partitionColumns.fieldNames.mkString(",") - s"""Partition columns mismatch. - |Expected: $relationPartitionCols - |Actual: $insertionPartitionCols - """.stripMargin - }) - - val partitionDF = df.select(partitionCols.head, partitionCols.tail: _*) + val partitionColumnsInSpec = relation.partitionSpec.partitionColumns.map(_.name) + require( + partitionColumnsInSpec.sameElements(partitionColumns), + s"""Partition columns mismatch. + |Expected: ${partitionColumnsInSpec.mkString(", ")} + |Actual: ${partitionColumns.mkString(", ")} + """.stripMargin) + + val partitionDF = df.select(partitionColumns.head, partitionColumns.tail: _*) val dataDF = df.select(dataCols.head, dataCols.tail: _*) (partitionDF.rdd, dataDF.rdd) From ca1805beba3612ad4c2d0d62051c4522870ca7b8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 30 Apr 2015 00:59:41 +0800 Subject: [PATCH 17/45] Removes duplicated partition discovery code in new Parquet --- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../apache/spark/sql/parquet/newParquet.scala | 176 +----------------- 2 files changed, 2 insertions(+), 176 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 5def549ac411a..31b07998d8630 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -298,7 +298,7 @@ class SparkHadoopUtil extends Logging { logDebug(text + " matched " + HADOOP_CONF_PATTERN) val key = matched.substring(13, matched.length() - 1) // remove ${hadoopconf- .. } val eval = Option[String](hadoopConf.get(key)) - .map { value => + .map { value => logDebug("Substituted " + matched + " with " + value) text.replace(matched, value) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 96a39416925b5..ee4b1c72a2148 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -303,7 +303,7 @@ private[sql] case class ParquetRelation2( if (partitionDirs.nonEmpty) { // Parses names and values of partition columns, and infer their data types. - ParquetRelation2.parsePartitions(partitionDirs, defaultPartitionName) + PartitioningUtils.parsePartitions(partitionDirs, defaultPartitionName) } else { // No partition directories found, makes an empty specification PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[Partition]) @@ -837,178 +837,4 @@ private[sql] object ParquetRelation2 extends Logging { .filter(_.nullable) StructType(parquetSchema ++ missingFields) } - - - // TODO Data source implementations shouldn't touch Catalyst types (`Literal`). - // However, we are already using Catalyst expressions for partition pruning and predicate - // push-down here... - private[parquet] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { - require(columnNames.size == literals.size) - } - - /** - * Given a group of qualified paths, tries to parse them and returns a partition specification. - * For example, given: - * {{{ - * hdfs://:/path/to/partition/a=1/b=hello/c=3.14 - * hdfs://:/path/to/partition/a=2/b=world/c=6.28 - * }}} - * it returns: - * {{{ - * PartitionSpec( - * partitionColumns = StructType( - * StructField(name = "a", dataType = IntegerType, nullable = true), - * StructField(name = "b", dataType = StringType, nullable = true), - * StructField(name = "c", dataType = DoubleType, nullable = true)), - * partitions = Seq( - * Partition( - * values = Row(1, "hello", 3.14), - * path = "hdfs://:/path/to/partition/a=1/b=hello/c=3.14"), - * Partition( - * values = Row(2, "world", 6.28), - * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28"))) - * }}} - */ - private[parquet] def parsePartitions( - paths: Seq[Path], - defaultPartitionName: String): PartitionSpec = { - val partitionValues = resolvePartitions(paths.map(parsePartition(_, defaultPartitionName))) - val fields = { - val (PartitionValues(columnNames, literals)) = partitionValues.head - columnNames.zip(literals).map { case (name, Literal(_, dataType)) => - StructField(name, dataType, nullable = true) - } - } - - val partitions = partitionValues.zip(paths).map { - case (PartitionValues(_, literals), path) => - Partition(Row(literals.map(_.value): _*), path.toString) - } - - PartitionSpec(StructType(fields), partitions) - } - - /** - * Parses a single partition, returns column names and values of each partition column. For - * example, given: - * {{{ - * path = hdfs://:/path/to/partition/a=42/b=hello/c=3.14 - * }}} - * it returns: - * {{{ - * PartitionValues( - * Seq("a", "b", "c"), - * Seq( - * Literal.create(42, IntegerType), - * Literal.create("hello", StringType), - * Literal.create(3.14, FloatType))) - * }}} - */ - private[parquet] def parsePartition( - path: Path, - defaultPartitionName: String): PartitionValues = { - val columns = ArrayBuffer.empty[(String, Literal)] - // Old Hadoop versions don't have `Path.isRoot` - var finished = path.getParent == null - var chopped = path - - while (!finished) { - val maybeColumn = parsePartitionColumn(chopped.getName, defaultPartitionName) - maybeColumn.foreach(columns += _) - chopped = chopped.getParent - finished = maybeColumn.isEmpty || chopped.getParent == null - } - - val (columnNames, values) = columns.reverse.unzip - PartitionValues(columnNames, values) - } - - private def parsePartitionColumn( - columnSpec: String, - defaultPartitionName: String): Option[(String, Literal)] = { - val equalSignIndex = columnSpec.indexOf('=') - if (equalSignIndex == -1) { - None - } else { - val columnName = columnSpec.take(equalSignIndex) - assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") - - val rawColumnValue = columnSpec.drop(equalSignIndex + 1) - assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - - val literal = inferPartitionColumnValue(rawColumnValue, defaultPartitionName) - Some(columnName -> literal) - } - } - - /** - * Resolves possible type conflicts between partitions by up-casting "lower" types. The up- - * casting order is: - * {{{ - * NullType -> - * IntegerType -> LongType -> - * FloatType -> DoubleType -> DecimalType.Unlimited -> - * StringType - * }}} - */ - private[parquet] def resolvePartitions(values: Seq[PartitionValues]): Seq[PartitionValues] = { - // Column names of all partitions must match - val distinctPartitionsColNames = values.map(_.columnNames).distinct - assert(distinctPartitionsColNames.size == 1, { - val list = distinctPartitionsColNames.mkString("\t", "\n", "") - s"Conflicting partition column names detected:\n$list" - }) - - // Resolves possible type conflicts for each column - val columnCount = values.head.columnNames.size - val resolvedValues = (0 until columnCount).map { i => - resolveTypeConflicts(values.map(_.literals(i))) - } - - // Fills resolved literals back to each partition - values.zipWithIndex.map { case (d, index) => - d.copy(literals = resolvedValues.map(_(index))) - } - } - - /** - * Converts a string to a `Literal` with automatic type inference. Currently only supports - * [[IntegerType]], [[LongType]], [[FloatType]], [[DoubleType]], [[DecimalType.Unlimited]], and - * [[StringType]]. - */ - private[parquet] def inferPartitionColumnValue( - raw: String, - defaultPartitionName: String): Literal = { - // First tries integral types - Try(Literal.create(Integer.parseInt(raw), IntegerType)) - .orElse(Try(Literal.create(JLong.parseLong(raw), LongType))) - // Then falls back to fractional types - .orElse(Try(Literal.create(JFloat.parseFloat(raw), FloatType))) - .orElse(Try(Literal.create(JDouble.parseDouble(raw), DoubleType))) - .orElse(Try(Literal.create(new JBigDecimal(raw), DecimalType.Unlimited))) - // Then falls back to string - .getOrElse { - if (raw == defaultPartitionName) Literal.create(null, NullType) - else Literal.create(raw, StringType) - } - } - - private val upCastingOrder: Seq[DataType] = - Seq(NullType, IntegerType, LongType, FloatType, DoubleType, DecimalType.Unlimited, StringType) - - /** - * Given a collection of [[Literal]]s, resolves possible type conflicts by up-casting "lower" - * types. - */ - private def resolveTypeConflicts(literals: Seq[Literal]): Seq[Literal] = { - val desiredType = { - val topType = literals.map(_.dataType).maxBy(upCastingOrder.indexOf(_)) - // Falls back to string if all values of this column are null or empty string - if (topType == NullType) StringType else topType - } - - literals.map { case l @ Literal(_, dataType) => - Literal.create(Cast(l, desiredType).eval(), desiredType) - } - } } From 8d2ff71e6fa1eaec7dcce9f0e1ccd7a90bd7cf66 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 1 May 2015 00:30:57 +0800 Subject: [PATCH 18/45] Merges partition columns when reading partitioned relations --- .../sql/sources/DataSourceStrategy.scala | 125 ++++++++++++++++-- .../apache/spark/sql/sources/interfaces.scala | 27 ++-- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../sql/sources/FSBasedRelationSuite.scala | 33 +++-- 4 files changed, 155 insertions(+), 32 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 52f9bd4421c53..0a6a6b3e78cfa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.sources +import org.apache.hadoop.fs.Path + +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ @@ -24,7 +27,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.types.{UTF8String, StringType} +import org.apache.spark.sql.types.{StructType, UTF8String, StringType} import org.apache.spark.sql.{Row, Strategy, execution, sources} /** @@ -53,11 +56,12 @@ private[sql] object DataSourceStrategy extends Strategy { filters, (a, _) => t.buildScan(a)) :: Nil - case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) => - val selectedPartitions = prunePartitions(filters, t.partitionSpec) - val inputPaths = selectedPartitions.map(_.path).toArray + // Scanning partitioned FSBasedRelation + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) + if t.partitionSpec.partitionColumns.nonEmpty => + val selectedPartition = prunePartitions(filters, t.partitionSpec).toArray - // Don't push down predicates that reference partition columns + // Only pushes down predicates that do not reference partition columns. val pushedFilters = { val partitionColumnNames = t.partitionSpec.partitionColumns.map(_.name).toSet filters.filter { f => @@ -66,10 +70,25 @@ private[sql] object DataSourceStrategy extends Strategy { } } - pruneFilterProject( + buildPartitionedTableScan( l, projectList, pushedFilters, + t.partitionSpec.partitionColumns, + selectedPartition) :: Nil + + // Scanning non-partitioned FSBasedRelation + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) => + val inputPaths = t.paths.map(new Path(_)).flatMap { path => + val fs = path.getFileSystem(t.sqlContext.sparkContext.hadoopConfiguration) + val qualifiedPath = fs.makeQualified(path) + SparkHadoopUtil.get.listLeafStatuses(fs, qualifiedPath).map(_.getPath.toString) + } + + pruneFilterProject( + l, + projectList, + filters, (a, f) => t.buildScan(a, f, inputPaths)) :: Nil case l @ LogicalRelation(t: TableScan) => @@ -82,6 +101,94 @@ private[sql] object DataSourceStrategy extends Strategy { case _ => Nil } + private def buildPartitionedTableScan( + logicalRelation: LogicalRelation, + projections: Seq[NamedExpression], + filters: Seq[Expression], + partitionColumns: StructType, + partitions: Array[Partition]) = { + val output = projections.map(_.toAttribute) + val relation = logicalRelation.relation.asInstanceOf[FSBasedRelation] + val dataSchema = relation.dataSchema + + // Builds RDD[Row]s for each selected partition. + val perPartitionRows = partitions.map { case Partition(partitionValues, dir) => + // Paths to all data files within this partition + val dataFilePaths = { + val dirPath = new Path(dir) + val fs = dirPath.getFileSystem(SparkHadoopUtil.get.conf) + fs.listStatus(dirPath) + .map(_.getPath) + .filter { path => + val name = path.getName + name.startsWith("_") || name.startsWith(".") + } + .map(fs.makeQualified(_).toString) + } + + // The table scan operator (PhysicalRDD) which retrieves required columns from data files. + // Notice that the schema of data files, represented by `relation.dataSchema`, may contain + // some partition column(s). Those partition columns that are only encoded in partition + // directory paths are not covered by this table scan operator. + val scan = + pruneFilterProject( + logicalRelation, + projections, + filters, + (requiredColumns, filters) => { + // Only columns appear in actual data, which possibly include some partition column(s) + relation.buildScan( + requiredColumns.filter(dataSchema.fieldNames.contains), + filters, + dataFilePaths) + }) + + // Merges in those partition values that are not contained in data rows. + mergePartitionValues(output, partitionValues, scan) + } + + val unionedRows = perPartitionRows.reduceOption(_ ++ _).getOrElse(relation.sqlContext.emptyResult) + createPhysicalRDD(logicalRelation.relation, output, unionedRows) + } + + private def mergePartitionValues( + output: Seq[Attribute], + partitionValues: Row, + scan: SparkPlan): RDD[Row] = { + val mergeWithPartitionValues = { + val outputColNames = output.map(_.name) + val outputDataColNames = scan.schema.fieldNames + + outputColNames.zipWithIndex.map { case (name, index) => + val i = outputDataColNames.indexOf(name) + if (i > -1) { + // Column appears in data files, retrieve it from data rows + (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => { + mutableRow(ordinal) = dataRow(i) + } + } else { + // Column doesn't appear in data file (must be a partition column), retrieve it from + // partition values of this partition. + (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => { + mutableRow(ordinal) = partitionValues(i) + } + } + } + } + + scan.execute().mapPartitions { iterator => + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + iterator.map { row => + var i = 0 + while (i < mutableRow.length) { + mergeWithPartitionValues(i)(mutableRow, row, i) + i += 1 + } + mutableRow.asInstanceOf[expressions.Row] + } + } + } + protected def prunePartitions( predicates: Seq[Expression], partitionSpec: PartitionSpec): Seq[Partition] = { @@ -225,13 +332,13 @@ private[sql] object DataSourceStrategy extends Strategy { translate(child).map(sources.Not) case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringStartsWith(a.name, v.toString)) + Some(sources.StringStartsWith(a.name, v.toString())) case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringEndsWith(a.name, v.toString)) + Some(sources.StringEndsWith(a.name, v.toString())) case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringContains(a.name, v.toString)) + Some(sources.StringContains(a.name, v.toString())) case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index d25034d900ece..b2d8a18c76e14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -361,10 +361,13 @@ abstract class FSBasedRelation private[sql]( def dataSchema: StructType /** - * Builds an `RDD[Row]` containing all rows within this relation. + * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within + * this relation. For partitioned relations, this method is called for each selected partition, + * and builds an `RDD[Row]` containg all rows within that single partition. * - * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data - * files within required partition directories are included. + * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the + * relation. For a partitioned relation, it contains paths of all data files in a single + * selected partition. */ def buildScan(inputPaths: Array[String]): RDD[Row] = { throw new RuntimeException( @@ -372,26 +375,32 @@ abstract class FSBasedRelation private[sql]( } /** - * Builds an `RDD[Row]` containing all rows within this relation. + * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within + * this relation. For partitioned relations, this method is called for each selected partition, + * and builds an `RDD[Row]` containg all rows within that single partition. * * @param requiredColumns Required columns. - * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data - * files within required partition directories are included. + * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the + * relation. For a partitioned relation, it contains paths of all data files in a single + * selected partition. */ def buildScan(requiredColumns: Array[String], inputPaths: Array[String]): RDD[Row] = { buildScan(inputPaths) } /** - * Builds an `RDD[Row]` containing all rows within this relation. + * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within + * this relation. For partitioned relations, this method is called for each selected partition, + * and builds an `RDD[Row]` containg all rows within that single partition. * * @param requiredColumns Required columns. * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction * of all `filters`. The pushed down filters are currently purely an optimization as they * will all be evaluated again. This means it is safe to use them with methods that produce * false positives such as filtering partitions based on a bloom filter. - * @param inputPaths Data files to be read. If the underlying relation is partitioned, only data - * files within required partition directories are included. + * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the + * relation. For a partitioned relation, it contains paths of all data files in a single + * selected partition. */ def buildScan( requiredColumns: Array[String], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index 2a9876fa899b0..bea568ed40049 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.parquet.ParquetRelation2._ +import org.apache.spark.sql.sources.PartitioningUtils._ import org.apache.spark.sql.sources.{Partition, PartitionSpec} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.types._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 5e7ef9222734f..4c49bbe35f861 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -49,7 +49,7 @@ class SimpleFSBasedSource extends FSBasedRelationProvider { Array.empty[Partition]) } - SimpleFSBasedRelation(path, dataSchema, maybePartitionSpec, parameters)(sqlContext) + new SimpleFSBasedRelation(path, dataSchema, maybePartitionSpec, parameters)(sqlContext) } } @@ -67,11 +67,11 @@ class SimpleOutputWriter extends OutputWriter { } } -case class SimpleFSBasedRelation - (path: String, - dataSchema: StructType, - maybePartitionSpec: Option[PartitionSpec], - parameter: Map[String, String]) +class SimpleFSBasedRelation + (val path: String, + val dataSchema: StructType, + val maybePartitionSpec: Option[PartitionSpec], + val parameter: Map[String, String]) (@transient val sqlContext: SQLContext) extends FSBasedRelation(Array(path), maybePartitionSpec) { @@ -163,8 +163,10 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { } test("load() - partitioned table - partition column not included in data files") { - fs.mkdirs(new Path(basePath, "p1=1/p2=hello")) - fs.mkdirs(new Path(basePath, "p1=2/p2=world")) + // Mocks partition directory layout. + val fakeData = sparkContext.parallelize(Seq("placeholder")) + fakeData.saveAsTextFile(new Path(basePath, "p1=1/p2=hello").toString) + fakeData.saveAsTextFile(new Path(basePath, "p1=2/p2=world").toString) val df = load( source = classOf[SimpleFSBasedSource].getCanonicalName, @@ -192,13 +194,16 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { // Check for column pruning, filter push-down, and partition pruning assert(TestResult.requiredColumns.toSet === Set("a", "b")) assert(TestResult.filters === Seq(GreaterThan("a", 0))) - assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1/p2=hello").toString)) + assertResult(Array(new Path(basePath, "p1=1/p2=hello"))) { + TestResult.inputPaths.map(new Path(_).getParent) + } } test("load() - partitioned table - partition column included in data files") { - val data = sparkContext.parallelize(Seq.empty[String]) - data.saveAsTextFile(new Path(basePath, "p1=1/p2=hello").toString) - data.saveAsTextFile(new Path(basePath, "p1=2/p2=world").toString) + // Mocks partition directory layout. + val fakeData = sparkContext.parallelize(Seq("placeholder")) + fakeData.saveAsTextFile(new Path(basePath, "p1=1/p2=hello").toString) + fakeData.saveAsTextFile(new Path(basePath, "p1=2/p2=world").toString) val dataSchema = StructType( @@ -235,7 +240,9 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { // Check for column pruning, filter push-down, and partition pruning assert(TestResult.requiredColumns.toSet === Set("a", "b")) assert(TestResult.filters === Seq(GreaterThan("a", 0))) - assert(TestResult.inputPaths === Seq(new Path(basePath, "p1=1/p2=hello").toString)) + assertResult(Array(new Path(basePath, "p1=1/p2=hello"))) { + TestResult.inputPaths.map(new Path(_).getParent) + } } test("save() - partitioned table - Overwrite") { From ce5235388fc7075201efd38150c96b9ed50af135 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 1 May 2015 02:07:05 +0800 Subject: [PATCH 19/45] Adds new SQLContext.load() overload with user defined dynamic partition columns --- .../org/apache/spark/sql/SQLContext.scala | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 2753c913b6a0a..aaa380d9ad3d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -781,6 +781,22 @@ class SQLContext(@transient val sparkContext: SparkContext) load(source, schema, options.toMap) } + /** + * :: Experimental :: + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + */ + @Experimental + def load( + source: String, + schema: StructType, + partitionColumns: Array[String], + options: java.util.Map[String, String]): DataFrame = { + load(source, schema, partitionColumns, options.toMap) + } + /** * :: Experimental :: * (Scala-specific) Returns the dataset specified by the given data source and @@ -796,6 +812,22 @@ class SQLContext(@transient val sparkContext: SparkContext) DataFrame(this, LogicalRelation(resolved.relation)) } + /** + * :: Experimental :: + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * @group genericdata + */ + @Experimental + def load( + source: String, + schema: StructType, + partitionColumns: Array[String], + options: Map[String, String]): DataFrame = { + val resolved = ResolvedDataSource(this, Some(schema), partitionColumns, source, options) + DataFrame(this, LogicalRelation(resolved.relation)) + } + /** * :: Experimental :: * Creates an external table from the given path and returns the corresponding DataFrame. From 422ff4a35ecab1f2befe7de778052516aa4fb83d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 1 May 2015 02:07:34 +0800 Subject: [PATCH 20/45] Fixes style issue --- .../org/apache/spark/sql/sources/DataSourceStrategy.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 0a6a6b3e78cfa..7f8d0101a79f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -147,7 +147,8 @@ private[sql] object DataSourceStrategy extends Strategy { mergePartitionValues(output, partitionValues, scan) } - val unionedRows = perPartitionRows.reduceOption(_ ++ _).getOrElse(relation.sqlContext.emptyResult) + val unionedRows = + perPartitionRows.reduceOption(_ ++ _).getOrElse(relation.sqlContext.emptyResult) createPhysicalRDD(logicalRelation.relation, output, unionedRows) } From f3207661a4d0a6572d11d918912b51d4b61ce65c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 2 May 2015 21:22:34 +0800 Subject: [PATCH 21/45] Adds prepareForWrite() hook, refactored writer containers --- .../apache/spark/sql/sources/commands.scala | 88 +++++++++++++------ .../apache/spark/sql/sources/interfaces.scala | 2 + .../sql/sources/FSBasedRelationSuite.scala | 2 + 3 files changed, 66 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index ee3b9cab5e0fe..92da8bfe9960d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -90,7 +90,7 @@ private[sql] case class InsertIntoFSBasedRelation( needsConversion = false) if (partitionColumns.isEmpty) { - insert(new WriterContainer(relation, jobConf), df) + insert(new DefaultWriterContainer(relation, jobConf), df) } else { val writerContainer = new DynamicPartitionWriterContainer( relation, jobConf, partitionColumns, "__HIVE_DEFAULT_PARTITION__") @@ -101,7 +101,7 @@ private[sql] case class InsertIntoFSBasedRelation( Seq.empty[Row] } - private def insert(writerContainer: WriterContainer, df: DataFrame): Unit = { + private def insert(writerContainer: BaseWriterContainer, df: DataFrame): Unit = { try { writerContainer.driverSideSetup() df.sqlContext.sparkContext.runJob(df.rdd, writeRows _) @@ -128,7 +128,7 @@ private[sql] case class InsertIntoFSBasedRelation( } private def insertWithDynamicPartitions( - writerContainer: WriterContainer, + writerContainer: BaseWriterContainer, df: DataFrame, partitionColumns: Array[String]): Unit = { @@ -191,7 +191,7 @@ private[sql] case class InsertIntoFSBasedRelation( } } -private[sql] class WriterContainer( +private[sql] abstract class BaseWriterContainer( @transient val relation: FSBasedRelation, @transient jobConf: JobConf) extends SparkHadoopMapRedUtil @@ -223,11 +223,9 @@ private[sql] class WriterContainer( protected val outputWriterClass: Class[_ <: OutputWriter] = relation.outputWriterClass - // All output writers are created on executor side. - @transient protected var outputWriters: mutable.Map[String, OutputWriter] = _ - def driverSideSetup(): Unit = { setupIDs(0, 0, 0) + relation.prepareForWrite(serializableJobConf.value) setupJobConf() jobContext = newJobContext(jobConf, jobId) outputCommitter = jobConf.getOutputCommitter @@ -240,7 +238,7 @@ private[sql] class WriterContainer( taskAttemptContext = newTaskAttemptContext(serializableJobConf.value, taskAttemptId) outputCommitter = serializableJobConf.value.getOutputCommitter outputCommitter.setupTask(taskAttemptContext) - outputWriters = initWriters() + initWriters() } private def setupIDs(jobId: Int, splitId: Int, attemptId: Int): Unit = { @@ -258,22 +256,20 @@ private[sql] class WriterContainer( } // Called on executor side when writing rows - def outputWriterForRow(row: Row): OutputWriter = outputWriters.values.head + def outputWriterForRow(row: Row): OutputWriter - protected def initWriters(): mutable.Map[String, OutputWriter] = { + protected def initWriters(): Unit = { val writer = outputWriterClass.newInstance() writer.init(outputPath, dataSchema, serializableJobConf.value) mutable.Map(outputPath -> writer) } def commitTask(): Unit = { - outputWriters.values.foreach(_.close()) SparkHadoopMapRedUtil.commitTask( outputCommitter, taskAttemptContext, jobId.getId, taskId.getId, taskAttemptId.getId) } def abortTask(): Unit = { - outputWriters.values.foreach(_.close()) outputCommitter.abortTask(taskAttemptContext) logError(s"Task attempt $taskAttemptId aborted.") } @@ -289,14 +285,44 @@ private[sql] class WriterContainer( } } +private[sql] class DefaultWriterContainer( + @transient relation: FSBasedRelation, + @transient conf: JobConf) + extends BaseWriterContainer(relation, conf) { + + @transient private var writer: OutputWriter = _ + + override protected def initWriters(): Unit = { + writer = relation.outputWriterClass.newInstance() + writer.init(outputPath, dataSchema, serializableJobConf.value) + } + + override def outputWriterForRow(row: Row): OutputWriter = writer + + override def commitTask(): Unit = { + writer.close() + super.commitTask() + } + + override def abortTask(): Unit = { + writer.close() + super.abortTask() + } +} + private[sql] class DynamicPartitionWriterContainer( @transient relation: FSBasedRelation, @transient conf: JobConf, partitionColumns: Array[String], defaultPartitionName: String) - extends WriterContainer(relation, conf) { + extends BaseWriterContainer(relation, conf) { + + // All output writers are created on executor side. + @transient protected var outputWriters: mutable.Map[String, OutputWriter] = _ - override protected def initWriters() = mutable.Map.empty[String, OutputWriter] + override protected def initWriters(): Unit = { + outputWriters = mutable.Map.empty[String, OutputWriter] + } override def outputWriterForRow(row: Row): OutputWriter = { val partitionPath = partitionColumns.zip(row.toSeq).map { case (col, rawValue) => @@ -304,7 +330,7 @@ private[sql] class DynamicPartitionWriterContainer( val valueString = if (string == null || string.isEmpty) { defaultPartitionName } else { - escapePathName(string) + DynamicPartitionWriterContainer.escapePathName(string) } s"/$col=$valueString" }.mkString @@ -317,18 +343,14 @@ private[sql] class DynamicPartitionWriterContainer( }) } - private def escapePathName(path: String): String = { - val builder = new StringBuilder() - path.foreach { c => - if (DynamicPartitionWriterContainer.needsEscaping(c)) { - builder.append('%') - builder.append(f"${c.asInstanceOf[Int]}%02x") - } else { - builder.append(c) - } - } + override def commitTask(): Unit = { + outputWriters.values.foreach(_.close()) + super.commitTask() + } - builder.toString() + override def abortTask(): Unit = { + outputWriters.values.foreach(_.close()) + super.abortTask() } } @@ -359,4 +381,18 @@ private[sql] object DynamicPartitionWriterContainer { def needsEscaping(c: Char): Boolean = { c >= 0 && c < charToEscape.size() && charToEscape.get(c); } + + def escapePathName(path: String): String = { + val builder = new StringBuilder() + path.foreach { c => + if (DynamicPartitionWriterContainer.needsEscaping(c)) { + builder.append('%') + builder.append(f"${c.asInstanceOf[Int]}%02x") + } else { + builder.append(c) + } + } + + builder.toString() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index b2d8a18c76e14..4a17d55f4acee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -409,6 +409,8 @@ abstract class FSBasedRelation private[sql]( buildScan(requiredColumns, inputPaths) } + def prepareForWrite(conf: Configuration): Unit + /** * This method is responsible for producing a new [[OutputWriter]] for each newly opened output * file on the executor side. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 4c49bbe35f861..d48a96fc65892 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -107,6 +107,8 @@ class SimpleFSBasedRelation } override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter] + + override def prepareForWrite(conf: Configuration): Unit = () } object TestResult { From 0bc6ad1ce6cab5971276db179eacb2f4a5ec1f27 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 3 May 2015 14:32:04 +0800 Subject: [PATCH 22/45] Resorts to new Hadoop API, and now FSBasedRelation can customize output format class --- .../apache/spark/sql/sources/commands.scala | 86 ++++++++++--------- .../apache/spark/sql/sources/interfaces.scala | 10 ++- .../sql/sources/FSBasedRelationSuite.scala | 7 +- 3 files changed, 59 insertions(+), 44 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 92da8bfe9960d..fc21ddc4d353e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -14,23 +14,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.sources -import java.util import java.util.Date import scala.collection.mutable +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.util.Shell +import parquet.hadoop.util.ContextUtil import org.apache.spark._ import org.apache.spark.mapred.SparkHadoopMapRedUtil +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} private[sql] case class InsertIntoDataSource( logicalRelation: LogicalRelation, @@ -79,10 +83,12 @@ private[sql] case class InsertIntoFSBasedRelation( } if (doInsertion) { - val jobConf = new JobConf(hadoopConf) - jobConf.setOutputKeyClass(classOf[Void]) - jobConf.setOutputValueClass(classOf[Row]) - FileOutputFormat.setOutputPath(jobConf, outputPath) + val job = Job.getInstance(hadoopConf) + job.setOutputKeyClass(classOf[Void]) + job.setOutputValueClass(classOf[Row]) + FileOutputFormat.setOutputPath(job, outputPath) + + val jobConf: Configuration = ContextUtil.getConfiguration(job) val df = sqlContext.createDataFrame( DataFrame(sqlContext, query).queryExecution.toRdd, @@ -90,10 +96,10 @@ private[sql] case class InsertIntoFSBasedRelation( needsConversion = false) if (partitionColumns.isEmpty) { - insert(new DefaultWriterContainer(relation, jobConf), df) + insert(new DefaultWriterContainer(relation, job), df) } else { val writerContainer = new DynamicPartitionWriterContainer( - relation, jobConf, partitionColumns, "__HIVE_DEFAULT_PARTITION__") + relation, job, partitionColumns, "__HIVE_DEFAULT_PARTITION__") insertWithDynamicPartitions(writerContainer, df, partitionColumns) } } @@ -169,6 +175,7 @@ private[sql] case class InsertIntoFSBasedRelation( writerContainer.commitJob() relation.refreshPartitions() } catch { case cause: Throwable => + logError("Aborting job.", cause) writerContainer.abortJob() throw new SparkException("Job aborted.", cause) } @@ -193,24 +200,22 @@ private[sql] case class InsertIntoFSBasedRelation( private[sql] abstract class BaseWriterContainer( @transient val relation: FSBasedRelation, - @transient jobConf: JobConf) - extends SparkHadoopMapRedUtil + @transient job: Job) + extends SparkHadoopMapReduceUtil with Logging with Serializable { - protected val serializableJobConf = new SerializableWritable(jobConf) + protected val serializableConf = new SerializableWritable(ContextUtil.getConfiguration(job)) // This is only used on driver side. - @transient private var jobContext: JobContext = _ - - // This is only used on executor side. - @transient private var taskAttemptContext: TaskAttemptContext = _ + @transient private var jobContext: JobContext = job // The following fields are initialized and used on both driver and executor side. @transient private var outputCommitter: OutputCommitter = _ @transient private var jobId: JobID = _ @transient private var taskId: TaskID = _ @transient private var taskAttemptId: TaskAttemptID = _ + @transient private var taskAttemptContext: TaskAttemptContext = _ protected val outputPath = { assert( @@ -221,22 +226,25 @@ private[sql] abstract class BaseWriterContainer( protected val dataSchema = relation.dataSchema + protected val outputFormatClass: Class[_ <: OutputFormat[Void, Row]] = relation.outputFormatClass + protected val outputWriterClass: Class[_ <: OutputWriter] = relation.outputWriterClass def driverSideSetup(): Unit = { setupIDs(0, 0, 0) - relation.prepareForWrite(serializableJobConf.value) - setupJobConf() - jobContext = newJobContext(jobConf, jobId) - outputCommitter = jobConf.getOutputCommitter + setupConf() + taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) + val outputFormat = relation.outputFormatClass.newInstance() + outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) outputCommitter.setupJob(jobContext) } def executorSideSetup(taskContext: TaskContext): Unit = { setupIDs(taskContext.stageId(), taskContext.partitionId(), taskContext.attemptNumber()) - setupJobConf() - taskAttemptContext = newTaskAttemptContext(serializableJobConf.value, taskAttemptId) - outputCommitter = serializableJobConf.value.getOutputCommitter + setupConf() + taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) + val outputFormat = outputFormatClass.newInstance() + outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) outputCommitter.setupTask(taskAttemptContext) initWriters() } @@ -247,12 +255,12 @@ private[sql] abstract class BaseWriterContainer( this.taskAttemptId = new TaskAttemptID(taskId, attemptId) } - private def setupJobConf(): Unit = { - serializableJobConf.value.set("mapred.job.id", jobId.toString) - serializableJobConf.value.set("mapred.tip.id", taskAttemptId.getTaskID.toString) - serializableJobConf.value.set("mapred.task.id", taskAttemptId.toString) - serializableJobConf.value.setBoolean("mapred.task.is.map", true) - serializableJobConf.value.setInt("mapred.task.partition", 0) + private def setupConf(): Unit = { + serializableConf.value.set("mapred.job.id", jobId.toString) + serializableConf.value.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + serializableConf.value.set("mapred.task.id", taskAttemptId.toString) + serializableConf.value.setBoolean("mapred.task.is.map", true) + serializableConf.value.setInt("mapred.task.partition", 0) } // Called on executor side when writing rows @@ -260,7 +268,7 @@ private[sql] abstract class BaseWriterContainer( protected def initWriters(): Unit = { val writer = outputWriterClass.newInstance() - writer.init(outputPath, dataSchema, serializableJobConf.value) + writer.init(outputPath, dataSchema, serializableConf.value) mutable.Map(outputPath -> writer) } @@ -280,21 +288,21 @@ private[sql] abstract class BaseWriterContainer( } def abortJob(): Unit = { - outputCommitter.abortJob(jobContext, JobStatus.FAILED) + outputCommitter.abortJob(jobContext, JobStatus.State.FAILED) logError(s"Job $jobId aborted.") } } private[sql] class DefaultWriterContainer( @transient relation: FSBasedRelation, - @transient conf: JobConf) - extends BaseWriterContainer(relation, conf) { + @transient job: Job) + extends BaseWriterContainer(relation, job) { @transient private var writer: OutputWriter = _ override protected def initWriters(): Unit = { writer = relation.outputWriterClass.newInstance() - writer.init(outputPath, dataSchema, serializableJobConf.value) + writer.init(outputPath, dataSchema, serializableConf.value) } override def outputWriterForRow(row: Row): OutputWriter = writer @@ -312,10 +320,10 @@ private[sql] class DefaultWriterContainer( private[sql] class DynamicPartitionWriterContainer( @transient relation: FSBasedRelation, - @transient conf: JobConf, + @transient job: Job, partitionColumns: Array[String], defaultPartitionName: String) - extends BaseWriterContainer(relation, conf) { + extends BaseWriterContainer(relation, job) { // All output writers are created on executor side. @transient protected var outputWriters: mutable.Map[String, OutputWriter] = _ @@ -338,7 +346,7 @@ private[sql] class DynamicPartitionWriterContainer( outputWriters.getOrElseUpdate(partitionPath, { val path = new Path(outputPath, partitionPath.stripPrefix(Path.SEPARATOR)) val writer = outputWriterClass.newInstance() - writer.init(path.toString, dataSchema, serializableJobConf.value) + writer.init(path.toString, dataSchema, serializableConf.value) writer }) } @@ -356,7 +364,7 @@ private[sql] class DynamicPartitionWriterContainer( private[sql] object DynamicPartitionWriterContainer { val charToEscape = { - val bitSet = new util.BitSet(128) + val bitSet = new java.util.BitSet(128) /** * ASCII 01-1F are HTTP control characters that need to be escaped. @@ -379,7 +387,7 @@ private[sql] object DynamicPartitionWriterContainer { } def needsEscaping(c: Char): Boolean = { - c >= 0 && c < charToEscape.size() && charToEscape.get(c); + c >= 0 && c < charToEscape.size() && charToEscape.get(c) } def escapePathName(path: String): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 4a17d55f4acee..e46d311646855 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.sources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.mapreduce.{OutputFormat, OutputCommitter} +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil @@ -363,7 +365,7 @@ abstract class FSBasedRelation private[sql]( /** * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within * this relation. For partitioned relations, this method is called for each selected partition, - * and builds an `RDD[Row]` containg all rows within that single partition. + * and builds an `RDD[Row]` containing all rows within that single partition. * * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the * relation. For a partitioned relation, it contains paths of all data files in a single @@ -377,7 +379,7 @@ abstract class FSBasedRelation private[sql]( /** * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within * this relation. For partitioned relations, this method is called for each selected partition, - * and builds an `RDD[Row]` containg all rows within that single partition. + * and builds an `RDD[Row]` containing all rows within that single partition. * * @param requiredColumns Required columns. * @param inputPaths For a non-partitioned relation, it contains paths of all data files in the @@ -391,7 +393,7 @@ abstract class FSBasedRelation private[sql]( /** * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within * this relation. For partitioned relations, this method is called for each selected partition, - * and builds an `RDD[Row]` containg all rows within that single partition. + * and builds an `RDD[Row]` containing all rows within that single partition. * * @param requiredColumns Required columns. * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction @@ -409,7 +411,7 @@ abstract class FSBasedRelation private[sql]( buildScan(requiredColumns, inputPaths) } - def prepareForWrite(conf: Configuration): Unit + def outputFormatClass: Class[_ <: OutputFormat[Void, Row]] /** * This method is responsible for producing a new [[OutputWriter]] for each newly opened output diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index d48a96fc65892..c1a0c1186990e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -22,6 +22,8 @@ import scala.collection.mutable import com.google.common.base.Objects import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapreduce.OutputFormat +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD @@ -108,7 +110,10 @@ class SimpleFSBasedRelation override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter] - override def prepareForWrite(conf: Configuration): Unit = () + override def outputFormatClass: Class[_ <: OutputFormat[Void, Row]] = { + // This is just a mock, not used within this test suite. + classOf[TextOutputFormat[Void, Row]] + } } object TestResult { From be0c268e91d9c84f706151dc0baa62dfc3ebad69 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 3 May 2015 21:20:16 +0800 Subject: [PATCH 23/45] Uses TaskAttempContext rather than Configuration in OutputWriter.init --- .../org/apache/spark/sql/sources/commands.scala | 14 +++++--------- .../org/apache/spark/sql/sources/interfaces.scala | 6 +++--- .../spark/sql/sources/FSBasedRelationSuite.scala | 4 ++-- 3 files changed, 10 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index fc21ddc4d353e..24d8920f00278 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -208,14 +208,14 @@ private[sql] abstract class BaseWriterContainer( protected val serializableConf = new SerializableWritable(ContextUtil.getConfiguration(job)) // This is only used on driver side. - @transient private var jobContext: JobContext = job + @transient private val jobContext: JobContext = job // The following fields are initialized and used on both driver and executor side. @transient private var outputCommitter: OutputCommitter = _ @transient private var jobId: JobID = _ @transient private var taskId: TaskID = _ @transient private var taskAttemptId: TaskAttemptID = _ - @transient private var taskAttemptContext: TaskAttemptContext = _ + @transient protected var taskAttemptContext: TaskAttemptContext = _ protected val outputPath = { assert( @@ -266,11 +266,7 @@ private[sql] abstract class BaseWriterContainer( // Called on executor side when writing rows def outputWriterForRow(row: Row): OutputWriter - protected def initWriters(): Unit = { - val writer = outputWriterClass.newInstance() - writer.init(outputPath, dataSchema, serializableConf.value) - mutable.Map(outputPath -> writer) - } + protected def initWriters(): Unit def commitTask(): Unit = { SparkHadoopMapRedUtil.commitTask( @@ -302,7 +298,7 @@ private[sql] class DefaultWriterContainer( override protected def initWriters(): Unit = { writer = relation.outputWriterClass.newInstance() - writer.init(outputPath, dataSchema, serializableConf.value) + writer.init(outputPath, dataSchema, taskAttemptContext) } override def outputWriterForRow(row: Row): OutputWriter = writer @@ -346,7 +342,7 @@ private[sql] class DynamicPartitionWriterContainer( outputWriters.getOrElseUpdate(partitionPath, { val path = new Path(outputPath, partitionPath.stripPrefix(Path.SEPARATOR)) val writer = outputWriterClass.newInstance() - writer.init(path.toString, dataSchema, serializableConf.value) + writer.init(path.toString, dataSchema, taskAttemptContext) writer }) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index e46d311646855..07e77f8a24100 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.sources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.mapreduce.{OutputFormat, OutputCommitter} +import org.apache.hadoop.mapreduce.{TaskAttemptContext, OutputFormat, OutputCommitter} import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.spark.annotation.{DeveloperApi, Experimental} @@ -263,12 +263,12 @@ abstract class OutputWriter { * @param path The file path to which this [[OutputWriter]] is supposed to write. * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the corresponding relation is partitioned. - * @param conf Hadoop configuration inherited from driver side. + * @param context The Hadoop MapReduce task context. */ def init( path: String, dataSchema: StructType, - conf: Configuration): Unit = () + context: TaskAttemptContext): Unit = () /** * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index c1a0c1186990e..d5948e66bfac1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import com.google.common.base.Objects import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.mapreduce.OutputFormat +import org.apache.hadoop.mapreduce.{TaskAttemptContext, OutputFormat} import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat import org.scalatest.BeforeAndAfter @@ -56,7 +56,7 @@ class SimpleFSBasedSource extends FSBasedRelationProvider { } class SimpleOutputWriter extends OutputWriter { - override def init(path: String, dataSchema: StructType, conf: Configuration): Unit = { + override def init(path: String, dataSchema: StructType, context: TaskAttemptContext): Unit = { TestResult.synchronized { TestResult.writerPaths += path } From 54c3d7b3fa8a442526d35a48033d67b65ea29f2c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 3 May 2015 23:38:19 +0800 Subject: [PATCH 24/45] Enforces that FileOutputFormat must be used --- .../apache/spark/sql/sources/commands.scala | 20 +++++++++++----- .../apache/spark/sql/sources/interfaces.scala | 6 ++--- .../sql/sources/FSBasedRelationSuite.scala | 23 +++++++++---------- 3 files changed, 28 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 24d8920f00278..842c51d17c4f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat} import org.apache.hadoop.util.Shell import parquet.hadoop.util.ContextUtil @@ -211,7 +211,7 @@ private[sql] abstract class BaseWriterContainer( @transient private val jobContext: JobContext = job // The following fields are initialized and used on both driver and executor side. - @transient private var outputCommitter: OutputCommitter = _ + @transient protected var outputCommitter: FileOutputCommitter = _ @transient private var jobId: JobID = _ @transient private var taskId: TaskID = _ @transient private var taskAttemptId: TaskAttemptID = _ @@ -235,7 +235,11 @@ private[sql] abstract class BaseWriterContainer( setupConf() taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) val outputFormat = relation.outputFormatClass.newInstance() - outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) + outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) match { + case c: FileOutputCommitter => c + case _ => sys.error( + s"Output committer must be ${classOf[FileOutputCommitter].getName} or its subclasses") + } outputCommitter.setupJob(jobContext) } @@ -244,7 +248,11 @@ private[sql] abstract class BaseWriterContainer( setupConf() taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) val outputFormat = outputFormatClass.newInstance() - outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) + outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) match { + case c: FileOutputCommitter => c + case _ => sys.error( + s"Output committer must be ${classOf[FileOutputCommitter].getName} or its subclasses") + } outputCommitter.setupTask(taskAttemptContext) initWriters() } @@ -298,7 +306,7 @@ private[sql] class DefaultWriterContainer( override protected def initWriters(): Unit = { writer = relation.outputWriterClass.newInstance() - writer.init(outputPath, dataSchema, taskAttemptContext) + writer.init(outputCommitter.getWorkPath.toString, dataSchema, taskAttemptContext) } override def outputWriterForRow(row: Row): OutputWriter = writer @@ -340,7 +348,7 @@ private[sql] class DynamicPartitionWriterContainer( }.mkString outputWriters.getOrElseUpdate(partitionPath, { - val path = new Path(outputPath, partitionPath.stripPrefix(Path.SEPARATOR)) + val path = new Path(outputCommitter.getWorkPath, partitionPath.stripPrefix(Path.SEPARATOR)) val writer = outputWriterClass.newInstance() writer.init(path.toString, dataSchema, taskAttemptContext) writer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 07e77f8a24100..6060396f59bb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.sources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.mapreduce.{TaskAttemptContext, OutputFormat, OutputCommitter} -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil @@ -411,7 +411,7 @@ abstract class FSBasedRelation private[sql]( buildScan(requiredColumns, inputPaths) } - def outputFormatClass: Class[_ <: OutputFormat[Void, Row]] + def outputFormatClass: Class[_ <: FileOutputFormat[Void, Row]] /** * This method is responsible for producing a new [[OutputWriter]] for each newly opened output diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index d5948e66bfac1..4fc12fa04523a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -20,10 +20,9 @@ package org.apache.spark.sql.sources import scala.collection.mutable import com.google.common.base.Objects -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.mapreduce.{TaskAttemptContext, OutputFormat} -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD @@ -110,7 +109,7 @@ class SimpleFSBasedRelation override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter] - override def outputFormatClass: Class[_ <: OutputFormat[Void, Row]] = { + override def outputFormatClass: Class[_ <: FileOutputFormat[Void, Row]] = { // This is just a mock, not used within this test suite. classOf[TextOutputFormat[Void, Row]] } @@ -268,7 +267,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 4) assert(TestResult.writtenRows === expectedRows.toSet) } } @@ -295,7 +294,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 4) assert(TestResult.writtenRows === expectedRows.toSet) } } @@ -328,7 +327,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 4) + assert(TestResult.writerPaths.size === 8) assert(TestResult.writtenRows === expectedRows.toSet) } } @@ -381,7 +380,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 4) assert(TestResult.writtenRows === expectedRows.toSet) } @@ -443,7 +442,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 4) assert(TestResult.writtenRows === expectedRows.toSet) } @@ -472,7 +471,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 4) assert(TestResult.writtenRows === expectedRows.toSet) } @@ -493,7 +492,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 8) assert(TestResult.writtenRows === expectedRows.toSet) } @@ -535,7 +534,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") TestResult.synchronized { - assert(TestResult.writerPaths.size === 2) + assert(TestResult.writerPaths.size === 4) assert(TestResult.writtenRows === expectedRows.toSet) } From 5f423d31b927102e56459ea87224dd879d81c4e5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 4 May 2015 14:32:45 +0800 Subject: [PATCH 25/45] Bug fixes. Lets data source to customize OutputCommitter rather than OutputFormat --- .../sql/sources/DataSourceStrategy.scala | 36 +++++++++--- .../apache/spark/sql/sources/commands.scala | 57 +++++++++++-------- .../apache/spark/sql/sources/interfaces.scala | 18 ++++-- .../org/apache/spark/sql/sources/rules.scala | 10 +++- .../sql/sources/FSBasedRelationSuite.scala | 6 -- 5 files changed, 79 insertions(+), 48 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 7f8d0101a79f5..4e60c6994c54f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.sources import org.apache.hadoop.fs.Path +import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation @@ -28,12 +30,12 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.types.{StructType, UTF8String, StringType} -import org.apache.spark.sql.{Row, Strategy, execution, sources} +import org.apache.spark.sql._ /** * A Strategy for planning scans over data sources defined using the sources API. */ -private[sql] object DataSourceStrategy extends Strategy { +private[sql] object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: CatalystScan)) => pruneFilterProjectRaw( @@ -59,7 +61,14 @@ private[sql] object DataSourceStrategy extends Strategy { // Scanning partitioned FSBasedRelation case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) if t.partitionSpec.partitionColumns.nonEmpty => - val selectedPartition = prunePartitions(filters, t.partitionSpec).toArray + val selectedPartitions = prunePartitions(filters, t.partitionSpec).toArray + + logInfo { + val total = t.partitionSpec.partitions.length + val selected = selectedPartitions.length + val percentPruned = (1 - total.toDouble / selected.toDouble) * 100 + s"Selected $selected partitions out of $total, pruned $percentPruned% partitions." + } // Only pushes down predicates that do not reference partition columns. val pushedFilters = { @@ -75,7 +84,7 @@ private[sql] object DataSourceStrategy extends Strategy { projectList, pushedFilters, t.partitionSpec.partitionColumns, - selectedPartition) :: Nil + selectedPartitions) :: Nil // Scanning non-partitioned FSBasedRelation case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) => @@ -98,6 +107,12 @@ private[sql] object DataSourceStrategy extends Strategy { l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil + case i @ logical.InsertIntoTable( + l @ LogicalRelation(t: FSBasedRelation), part, query, overwrite, false) if part.isEmpty => + val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append + execution.ExecutedCommand( + InsertIntoFSBasedRelation(t, query, Array.empty[String], mode)) :: Nil + case _ => Nil } @@ -109,7 +124,6 @@ private[sql] object DataSourceStrategy extends Strategy { partitions: Array[Partition]) = { val output = projections.map(_.toAttribute) val relation = logicalRelation.relation.asInstanceOf[FSBasedRelation] - val dataSchema = relation.dataSchema // Builds RDD[Row]s for each selected partition. val perPartitionRows = partitions.map { case Partition(partitionValues, dir) => @@ -136,9 +150,11 @@ private[sql] object DataSourceStrategy extends Strategy { projections, filters, (requiredColumns, filters) => { - // Only columns appear in actual data, which possibly include some partition column(s) + // Don't require any partition columns to save I/O. Note that here we are being + // optimistic and assuming partition columns data stored in data files are always + // consistent with those encoded in partition directory paths. relation.buildScan( - requiredColumns.filter(dataSchema.fieldNames.contains), + requiredColumns.filterNot(partitionColumns.fieldNames.contains), filters, dataFilePaths) }) @@ -147,8 +163,10 @@ private[sql] object DataSourceStrategy extends Strategy { mergePartitionValues(output, partitionValues, scan) } - val unionedRows = - perPartitionRows.reduceOption(_ ++ _).getOrElse(relation.sqlContext.emptyResult) + val unionedRows = perPartitionRows.reduceOption(_ ++ _).getOrElse { + relation.sqlContext.emptyResult + } + createPhysicalRDD(logicalRelation.relation, output, unionedRows) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 842c51d17c4f5..38ae68006e189 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -21,7 +21,6 @@ import java.util.Date import scala.collection.mutable -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat} @@ -71,11 +70,15 @@ private[sql] case class InsertIntoFSBasedRelation( val hadoopConf = sqlContext.sparkContext.hadoopConfiguration val outputPath = new Path(relation.paths.head) - val fs = outputPath.getFileSystem(hadoopConf) - val doInsertion = (mode, fs.exists(outputPath)) match { + val qualifiedOutputPath = fs.makeQualified(outputPath) + + val doInsertion = (mode, fs.exists(qualifiedOutputPath)) match { case (SaveMode.ErrorIfExists, true) => - sys.error(s"path $outputPath already exists.") + sys.error(s"path $qualifiedOutputPath already exists.") + case (SaveMode.Overwrite, true) => + fs.delete(qualifiedOutputPath, true) + true case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => true case (SaveMode.Ignore, exists) => @@ -86,9 +89,7 @@ private[sql] case class InsertIntoFSBasedRelation( val job = Job.getInstance(hadoopConf) job.setOutputKeyClass(classOf[Void]) job.setOutputValueClass(classOf[Row]) - FileOutputFormat.setOutputPath(job, outputPath) - - val jobConf: Configuration = ContextUtil.getConfiguration(job) + FileOutputFormat.setOutputPath(job, qualifiedOutputPath) val df = sqlContext.createDataFrame( DataFrame(sqlContext, query).queryExecution.toRdd, @@ -110,8 +111,9 @@ private[sql] case class InsertIntoFSBasedRelation( private def insert(writerContainer: BaseWriterContainer, df: DataFrame): Unit = { try { writerContainer.driverSideSetup() - df.sqlContext.sparkContext.runJob(df.rdd, writeRows _) + df.sqlContext.sparkContext.runJob(df.queryExecution.executedPlan.execute(), writeRows _) writerContainer.commitJob() + relation.refresh() } catch { case cause: Throwable => writerContainer.abortJob() throw new SparkException("Job aborted.", cause) @@ -166,14 +168,15 @@ private[sql] case class InsertIntoFSBasedRelation( val partitionDF = df.select(partitionColumns.head, partitionColumns.tail: _*) val dataDF = df.select(dataCols.head, dataCols.tail: _*) - (partitionDF.rdd, dataDF.rdd) + partitionDF.queryExecution.executedPlan.execute() -> + dataDF.queryExecution.executedPlan.execute() } try { writerContainer.driverSideSetup() sqlContext.sparkContext.runJob(partitionRDD.zip(dataRDD), writeRows _) writerContainer.commitJob() - relation.refreshPartitions() + relation.refresh() } catch { case cause: Throwable => logError("Aborting job.", cause) writerContainer.abortJob() @@ -217,7 +220,7 @@ private[sql] abstract class BaseWriterContainer( @transient private var taskAttemptId: TaskAttemptID = _ @transient protected var taskAttemptContext: TaskAttemptContext = _ - protected val outputPath = { + protected val outputPath: String = { assert( relation.paths.length == 1, s"Cannot write to multiple destinations: ${relation.paths.mkString(",")}") @@ -226,7 +229,8 @@ private[sql] abstract class BaseWriterContainer( protected val dataSchema = relation.dataSchema - protected val outputFormatClass: Class[_ <: OutputFormat[Void, Row]] = relation.outputFormatClass + protected val outputCommitterClass: Class[_ <: FileOutputCommitter] = + relation.outputCommitterClass protected val outputWriterClass: Class[_ <: OutputWriter] = relation.outputWriterClass @@ -234,12 +238,7 @@ private[sql] abstract class BaseWriterContainer( setupIDs(0, 0, 0) setupConf() taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) - val outputFormat = relation.outputFormatClass.newInstance() - outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) match { - case c: FileOutputCommitter => c - case _ => sys.error( - s"Output committer must be ${classOf[FileOutputCommitter].getName} or its subclasses") - } + outputCommitter = newOutputCommitter(outputCommitterClass, outputPath, taskAttemptContext) outputCommitter.setupJob(jobContext) } @@ -247,16 +246,24 @@ private[sql] abstract class BaseWriterContainer( setupIDs(taskContext.stageId(), taskContext.partitionId(), taskContext.attemptNumber()) setupConf() taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) - val outputFormat = outputFormatClass.newInstance() - outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) match { - case c: FileOutputCommitter => c - case _ => sys.error( - s"Output committer must be ${classOf[FileOutputCommitter].getName} or its subclasses") - } + outputCommitter = newOutputCommitter(outputCommitterClass, outputPath, taskAttemptContext) outputCommitter.setupTask(taskAttemptContext) initWriters() } + private def newOutputCommitter( + clazz: Class[_ <: FileOutputCommitter], + path: String, + context: TaskAttemptContext): FileOutputCommitter = { + val ctor = outputCommitterClass.getConstructor(classOf[Path], classOf[TaskAttemptContext]) + ctor.setAccessible(true) + + val hadoopPath = new Path(path) + val fs = hadoopPath.getFileSystem(serializableConf.value) + val qualified = fs.makeQualified(hadoopPath) + ctor.newInstance(qualified, context) + } + private def setupIDs(jobId: Int, splitId: Int, attemptId: Int): Unit = { this.jobId = SparkHadoopWriter.createJobID(new Date, jobId) this.taskId = new TaskID(this.jobId, true, splitId) @@ -305,7 +312,7 @@ private[sql] class DefaultWriterContainer( @transient private var writer: OutputWriter = _ override protected def initWriters(): Unit = { - writer = relation.outputWriterClass.newInstance() + writer = outputWriterClass.newInstance() writer.init(outputCommitter.getWorkPath.toString, dataSchema, taskAttemptContext) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 6060396f59bb0..3a59b62a5f496 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.sources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil @@ -325,13 +325,17 @@ abstract class FSBasedRelation private[sql]( private[sql] def partitionSpec: PartitionSpec = _partitionSpec - private[sql] def refreshPartitions(): Unit = { + private[sql] def refresh(): Unit = { + refreshPartitions() + } + + private def refreshPartitions(): Unit = { _partitionSpec = maybePartitionSpec.getOrElse { val basePaths = paths.map(new Path(_)) val leafDirs = basePaths.flatMap { path => val fs = path.getFileSystem(hadoopConf) if (fs.exists(path)) { - SparkHadoopUtil.get.listLeafDirStatuses(fs, path) + SparkHadoopUtil.get.listLeafDirStatuses(fs, fs.makeQualified(path)) } else { Seq.empty[FileStatus] } @@ -349,7 +353,7 @@ abstract class FSBasedRelation private[sql]( * Schema of this relation. It consists of [[dataSchema]] and all partition columns not appeared * in [[dataSchema]]. */ - override val schema: StructType = { + override lazy val schema: StructType = { val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet StructType(dataSchema ++ partitionSpec.partitionColumns.filterNot { column => dataSchemaColumnNames.contains(column.name.toLowerCase) @@ -411,7 +415,11 @@ abstract class FSBasedRelation private[sql]( buildScan(requiredColumns, inputPaths) } - def outputFormatClass: Class[_ <: FileOutputFormat[Void, Row]] + /** + * The output committer class to use. Default to + * [[org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter]]. + */ + def outputCommitterClass: Class[_ <: FileOutputCommitter] = classOf[FileOutputCommitter] /** * This method is responsible for producing a new [[OutputWriter]] for each newly opened output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 940e5c96e2b0e..a5183909d7440 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -101,9 +101,13 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } } - case i @ logical.InsertIntoTable( - l: LogicalRelation, partition, query, overwrite, ifNotExists) - if !l.isInstanceOf[InsertableRelation] => + case logical.InsertIntoTable(LogicalRelation(_: InsertableRelation), _, _, _, _) => + // OK + + case logical.InsertIntoTable(LogicalRelation(_: FSBasedRelation), _, _, _, _) => + // OK + + case logical.InsertIntoTable(l: LogicalRelation, _, _, _, _) => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 4fc12fa04523a..e42e0f70258f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import com.google.common.base.Objects import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD @@ -108,11 +107,6 @@ class SimpleFSBasedRelation } override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter] - - override def outputFormatClass: Class[_ <: FileOutputFormat[Void, Row]] = { - // This is just a mock, not used within this test suite. - classOf[TextOutputFormat[Void, Row]] - } } object TestResult { From a29e663fefad1dc25ed80514ca562464a9a98be9 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 5 May 2015 00:38:34 +0800 Subject: [PATCH 26/45] Bug fix: should only pass actuall data files to FSBaseRelation.buildScan --- .../spark/sql/sources/DataSourceStrategy.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 4e60c6994c54f..91caa8fba10c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -91,7 +91,10 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { val inputPaths = t.paths.map(new Path(_)).flatMap { path => val fs = path.getFileSystem(t.sqlContext.sparkContext.hadoopConfiguration) val qualifiedPath = fs.makeQualified(path) - SparkHadoopUtil.get.listLeafStatuses(fs, qualifiedPath).map(_.getPath.toString) + SparkHadoopUtil.get.listLeafStatuses(fs, qualifiedPath).map(_.getPath).filterNot { path => + val name = path.getName + name.startsWith("_") || name.startsWith(".") + }.map(fs.makeQualified(_).toString) } pruneFilterProject( @@ -131,13 +134,10 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { val dataFilePaths = { val dirPath = new Path(dir) val fs = dirPath.getFileSystem(SparkHadoopUtil.get.conf) - fs.listStatus(dirPath) - .map(_.getPath) - .filter { path => - val name = path.getName - name.startsWith("_") || name.startsWith(".") - } - .map(fs.makeQualified(_).toString) + fs.listStatus(dirPath).map(_.getPath).filterNot { path => + val name = path.getName + name.startsWith("_") || name.startsWith(".") + }.map(fs.makeQualified(_).toString) } // The table scan operator (PhysicalRDD) which retrieves required columns from data files. From c4ed4fee4d9432e9be25b7d32b9382d9bac481b7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 6 May 2015 00:32:25 +0800 Subject: [PATCH 27/45] Bug fixes and a new test suite --- .../sql/sources/DataSourceStrategy.scala | 83 +- .../apache/spark/sql/sources/interfaces.scala | 18 +- .../sql/sources/FSBasedRelationSuite.scala | 840 ++++++++---------- .../sql/sources/SimpleTextRelation.scala | 135 +++ 4 files changed, 591 insertions(+), 485 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 91caa8fba10c1..76ee4b5d7d897 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -142,25 +142,31 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { // The table scan operator (PhysicalRDD) which retrieves required columns from data files. // Notice that the schema of data files, represented by `relation.dataSchema`, may contain - // some partition column(s). Those partition columns that are only encoded in partition - // directory paths are not covered by this table scan operator. + // some partition column(s). val scan = pruneFilterProject( logicalRelation, projections, filters, (requiredColumns, filters) => { - // Don't require any partition columns to save I/O. Note that here we are being - // optimistic and assuming partition columns data stored in data files are always - // consistent with those encoded in partition directory paths. - relation.buildScan( - requiredColumns.filterNot(partitionColumns.fieldNames.contains), - filters, - dataFilePaths) + val partitionColNames = partitionColumns.fieldNames + + // Don't scan any partition columns to save I/O. Here we are being optimistic and + // assuming partition columns data stored in data files are always consistent with those + // partition values encoded in partition directory paths. + val nonPartitionColumns = requiredColumns.filterNot(partitionColNames.contains) + val dataRows = relation.buildScan(nonPartitionColumns, filters, dataFilePaths) + + // Merges data values with partition values. + mergeWithPartitionValues( + relation.schema, + requiredColumns, + partitionColNames, + partitionValues, + dataRows) }) - // Merges in those partition values that are not contained in data rows. - mergePartitionValues(output, partitionValues, scan) + scan.execute() } val unionedRows = perPartitionRows.reduceOption(_ ++ _).getOrElse { @@ -170,41 +176,48 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { createPhysicalRDD(logicalRelation.relation, output, unionedRows) } - private def mergePartitionValues( - output: Seq[Attribute], + private def mergeWithPartitionValues( + schema: StructType, + requiredColumns: Array[String], + partitionColumns: Array[String], partitionValues: Row, - scan: SparkPlan): RDD[Row] = { - val mergeWithPartitionValues = { - val outputColNames = output.map(_.name) - val outputDataColNames = scan.schema.fieldNames - - outputColNames.zipWithIndex.map { case (name, index) => - val i = outputDataColNames.indexOf(name) - if (i > -1) { - // Column appears in data files, retrieve it from data rows + dataRows: RDD[Row]): RDD[Row] = { + val nonPartitionColumns = requiredColumns.filterNot(partitionColumns.contains) + + // If output columns contain any partition column(s), we need to merge scanned data + // columns and requested partition columns to form the final result. + if (!requiredColumns.sameElements(nonPartitionColumns)) { + val mergers = requiredColumns.zipWithIndex.map { case (name, index) => + // To see whether the `index`-th column is a partition column... + val i = partitionColumns.indexOf(name) + if (i != -1) { + // If yes, gets column value from partition values. (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => { - mutableRow(ordinal) = dataRow(i) + mutableRow(ordinal) = partitionValues(i) } } else { - // Column doesn't appear in data file (must be a partition column), retrieve it from - // partition values of this partition. + // Otherwise, inherits the value from scanned data. + val i = nonPartitionColumns.indexOf(name) (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => { - mutableRow(ordinal) = partitionValues(i) + mutableRow(ordinal) = dataRow(i) } } } - } - scan.execute().mapPartitions { iterator => - val mutableRow = new SpecificMutableRow(output.map(_.dataType)) - iterator.map { row => - var i = 0 - while (i < mutableRow.length) { - mergeWithPartitionValues(i)(mutableRow, row, i) - i += 1 + dataRows.mapPartitions { iterator => + val dataTypes = requiredColumns.map(schema(_).dataType) + val mutableRow = new SpecificMutableRow(dataTypes) + iterator.map { dataRow => + var i = 0 + while (i < mutableRow.length) { + mergers(i)(mutableRow, dataRow, i) + i += 1 + } + mutableRow.asInstanceOf[expressions.Row] } - mutableRow.asInstanceOf[expressions.Row] } + } else { + dataRows } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 3a59b62a5f496..e46e9516bc203 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -260,7 +260,10 @@ abstract class OutputWriter { /** * Initializes this [[OutputWriter]] before any rows are persisted. * - * @param path The file path to which this [[OutputWriter]] is supposed to write. + * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that + * this may not point to the final output file. For example, `FileOutputFormat` writes to + * temporary directories and then merge written files back to the final destination. In + * this case, `path` points to a temporary output file under the temporary directory. * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the corresponding relation is partitioned. * @param context The Hadoop MapReduce task context. @@ -310,6 +313,19 @@ abstract class FSBasedRelation private[sql]( maybePartitionSpec: Option[PartitionSpec]) extends BaseRelation { + /** + * Constructs an [[FSBasedRelation]]. + * + * @param paths Base paths of this relation. For partitioned relations, it should be either root + * directories of all partition directories. + * @param partitionColumns Partition columns of this relation. + */ + def this(paths: Array[String], partitionColumns: StructType) = + this(paths, { + if (partitionColumns.isEmpty) None + else Some(PartitionSpec(partitionColumns, Array.empty[Partition])) + }) + /** * Constructs an [[FSBasedRelation]]. * diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index e42e0f70258f3..5ee0fc892d46e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -17,123 +17,21 @@ package org.apache.spark.sql.sources -import scala.collection.mutable +import org.apache.hadoop.fs.Path -import com.google.common.base.Objects -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.scalatest.BeforeAndAfter - -import org.apache.spark.rdd.RDD +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.parquet.ParquetTest import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils - -class SimpleFSBasedSource extends FSBasedRelationProvider { - override def createRelation( - sqlContext: SQLContext, - schema: Option[StructType], - partitionColumns: Option[StructType], - parameters: Map[String, String]): BaseRelation = { - val path = parameters("path") - - // Uses data sources options to simulate data schema - val dataSchema = DataType.fromJson(parameters("schema")).asInstanceOf[StructType] - - // Uses data sources options to mock partition discovery - val maybePartitionSpec = - parameters.get("partitionColumns").map { json => - new PartitionSpec( - DataType.fromJson(json).asInstanceOf[StructType], - Array.empty[Partition]) - } - - new SimpleFSBasedRelation(path, dataSchema, maybePartitionSpec, parameters)(sqlContext) - } -} - -class SimpleOutputWriter extends OutputWriter { - override def init(path: String, dataSchema: StructType, context: TaskAttemptContext): Unit = { - TestResult.synchronized { - TestResult.writerPaths += path - } - } - - override def write(row: Row): Unit = { - TestResult.synchronized { - TestResult.writtenRows += row - } - } -} - -class SimpleFSBasedRelation - (val path: String, - val dataSchema: StructType, - val maybePartitionSpec: Option[PartitionSpec], - val parameter: Map[String, String]) - (@transient val sqlContext: SQLContext) - extends FSBasedRelation(Array(path), maybePartitionSpec) { - - override def equals(obj: scala.Any): Boolean = obj match { - case that: SimpleFSBasedRelation => - this.path == that.path && - this.dataSchema == that.dataSchema && - this.maybePartitionSpec == that.maybePartitionSpec - case _ => false - } - - override def hashCode(): Int = - Objects.hashCode(path, dataSchema, maybePartitionSpec) - - override def buildScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputPaths: Array[String]): RDD[Row] = { - val sqlContext = this.sqlContext - val basePath = new Path(path) - val fs = basePath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) - - // Used to test queries like "INSERT OVERWRITE tbl SELECT * FROM tbl". Source directory - // shouldn't be removed before scanning the data. - assert(inputPaths.map(new Path(_)).forall(fs.exists)) - - TestResult.synchronized { - TestResult.requiredColumns = requiredColumns - TestResult.filters = filters - TestResult.inputPaths = inputPaths - Option(TestResult.rowsToRead).getOrElse(sqlContext.emptyResult) - } - } - - override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter] -} - -object TestResult { - var requiredColumns: Array[String] = _ - var filters: Array[Filter] = _ - var inputPaths: Array[String] = _ - var rowsToRead: RDD[Row] = _ - var writerPaths: mutable.Set[String] = mutable.Set.empty[String] - var writtenRows: mutable.Set[Row] = mutable.Set.empty[Row] - - def reset(): Unit = this.synchronized { - requiredColumns = null - filters = null - inputPaths = null - rowsToRead = null - writerPaths.clear() - writtenRows.clear() - } -} -class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { - import TestHive._ - import TestHive.implicits._ +// This test suite extends ParquetTest for some convenient utility methods. These methods should be +// moved to some more general places, maybe QueryTest. +class FSBasedRelationSuite extends QueryTest with ParquetTest { + override val sqlContext: SQLContext = TestHive - var basePath: Path = _ - - var fs: FileSystem = _ + import sqlContext._ + import sqlContext.implicits._ val dataSchema = StructType( @@ -141,431 +39,475 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter { StructField("a", IntegerType, nullable = false), StructField("b", StringType, nullable = false))) - val singlePartitionColumn = StructType( - Seq( - StructField("p1", IntegerType, nullable = true))) - - val doublePartitionColumns = StructType( - Seq( - StructField("p1", IntegerType, nullable = true), - StructField("p2", StringType, nullable = true))) + val testDF = (1 to 3).map(i => (i, s"val_$i")).toDF("a", "b") - val testDF = (for { + val partitionedTestDF1 = (for { i <- 1 to 3 - p <- 1 to 2 - } yield (i, s"val_$i", p)).toDF("a", "b", "p1") - - before { - basePath = new Path(Utils.createTempDir().getCanonicalPath) - fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) - basePath = fs.makeQualified(basePath) - TestResult.reset() - } + p2 <- Seq("foo", "bar") + } yield (i, s"val_$i", 1, p2)).toDF("a", "b", "p1", "p2") - test("load() - partitioned table - partition column not included in data files") { - // Mocks partition directory layout. - val fakeData = sparkContext.parallelize(Seq("placeholder")) - fakeData.saveAsTextFile(new Path(basePath, "p1=1/p2=hello").toString) - fakeData.saveAsTextFile(new Path(basePath, "p1=2/p2=world").toString) - - val df = load( - source = classOf[SimpleFSBasedSource].getCanonicalName, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json)) - - df.queryExecution.analyzed.collect { - case LogicalRelation(relation: SimpleFSBasedRelation) => - assert(relation.dataSchema === dataSchema) - case _ => - fail("Couldn't find expected SimpleFSBasedRelation instance") + val partitionedTestDF2 = (for { + i <- 1 to 3 + p2 <- Seq("foo", "bar") + } yield (i, s"val_$i", 2, p2)).toDF("a", "b", "p1", "p2") + + val partitionedTestDF = partitionedTestDF1.unionAll(partitionedTestDF2) + + def checkQueries(df: DataFrame): Unit = { + // Selects everything + checkAnswer( + df, + for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) + + // Simple filtering and partition pruning + checkAnswer( + df.filter('a > 1 && 'p1 === 2), + for (i <- 2 to 3; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", 2, p2)) + + // Simple projection and filtering + checkAnswer( + df.filter('a > 1).select('b, 'a + 1), + for (i <- 2 to 3; _ <- 1 to 2; _ <- Seq("foo", "bar")) yield Row(s"val_$i", i + 1)) + + // Simple projection and partition pruning + checkAnswer( + df.filter('a > 1 && 'p1 < 2).select('b, 'p1), + for (i <- 2 to 3; _ <- Seq("foo", "bar")) yield Row(s"val_$i", 1)) + + // Self-join + df.registerTempTable("t") + withTempTable("t") { + checkAnswer( + sql( + """SELECT l.a, r.b, l.p1, r.p2 + |FROM t l JOIN t r + |ON l.a = r.a AND l.p1 = r.p1 AND l.p2 = r.p2 + """.stripMargin), + for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) } + } - val expectedSchema = - StructType( - dataSchema ++ Seq( - StructField("p1", IntegerType, nullable = true), - StructField("p2", StringType, nullable = true))) + test("save()/load() - non-partitioned table - Overwrite") { + withTempPath { file => + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) - assert(df.schema === expectedSchema) + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + testDF.collect()) + } + } - df.where($"a" > 0 && $"p1" === 1).select("b").collect() + test("save()/load() - non-partitioned table - Append") { + withTempPath { file => + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) - // Check for column pruning, filter push-down, and partition pruning - assert(TestResult.requiredColumns.toSet === Set("a", "b")) - assert(TestResult.filters === Seq(GreaterThan("a", 0))) - assertResult(Array(new Path(basePath, "p1=1/p2=hello"))) { - TestResult.inputPaths.map(new Path(_).getParent) + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)).orderBy("a"), + testDF.unionAll(testDF).orderBy("a").collect()) } } - test("load() - partitioned table - partition column included in data files") { - // Mocks partition directory layout. - val fakeData = sparkContext.parallelize(Seq("placeholder")) - fakeData.saveAsTextFile(new Path(basePath, "p1=1/p2=hello").toString) - fakeData.saveAsTextFile(new Path(basePath, "p1=2/p2=world").toString) - - val dataSchema = - StructType( - Seq( - StructField("a", IntegerType, nullable = false), - StructField("p1", IntegerType, nullable = true), - StructField("b", StringType, nullable = false))) - - val df = load( - source = classOf[SimpleFSBasedSource].getCanonicalName, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json)) - - df.queryExecution.analyzed.collect { - case LogicalRelation(relation: SimpleFSBasedRelation) => - assert(relation.dataSchema === dataSchema) - case _ => - fail("Couldn't find expected SimpleFSBasedRelation instance") + test("save()/load() - non-partitioned table - ErrorIfExists") { + withTempDir { file => + intercept[RuntimeException] { + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists) + } } + } - val expectedSchema = - StructType( - Seq( - StructField("a", IntegerType, nullable = false), - StructField("p1", IntegerType, nullable = true), - StructField("b", StringType, nullable = false), - StructField("p2", StringType, nullable = true))) - - assert(df.schema === expectedSchema) - - df.where($"a" > 0 && $"p1" === 1).select("b").collect() + test("save()/load() - non-partitioned table - Ignore") { + withTempDir { file => + testDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore) - // Check for column pruning, filter push-down, and partition pruning - assert(TestResult.requiredColumns.toSet === Set("a", "b")) - assert(TestResult.filters === Seq(GreaterThan("a", 0))) - assertResult(Array(new Path(basePath, "p1=1/p2=hello"))) { - TestResult.inputPaths.map(new Path(_).getParent) + val path = new Path(file.getCanonicalPath) + val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + assert(fs.listStatus(path).isEmpty) } } - test("save() - partitioned table - Overwrite") { - testDF.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> singlePartitionColumn.json), - partitionColumns = Seq("p1")) - - Thread.sleep(500) - - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - - TestResult.synchronized { - assert(TestResult.writerPaths.size === 4) - assert(TestResult.writtenRows === expectedRows.toSet) + test("save()/load() - partitioned table - simple queries") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkQueries( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json))) } } - ignore("save() - partitioned table - Overwrite - select and overwrite the same table") { - TestResult.rowsToRead = testDF.rdd + test("save()/load() - partitioned table - simple queries - partition columns in data") { + withTempDir { file => + val basePath = new Path(file.getCanonicalPath) + val fs = basePath.getFileSystem(SparkHadoopUtil.get.conf) + val qualifiedBasePath = fs.makeQualified(basePath) + + for (p1 <- 1 to 2; p2 <- Seq("foo", "bar")) { + val partitionDir = new Path(qualifiedBasePath, s"p1=$p1/p2=$p2") + sparkContext + .parallelize(for (i <- 1 to 3) yield s"$i,val_$i,$p1") + .saveAsTextFile(partitionDir.toString) + } - val df = load( - source = classOf[SimpleFSBasedSource].getCanonicalName, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json)) + val dataSchemaWithPartition = + StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) - df.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> singlePartitionColumn.json), - partitionColumns = Seq("p1")) - - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - - TestResult.synchronized { - assert(TestResult.writerPaths.size === 4) - assert(TestResult.writtenRows === expectedRows.toSet) + checkQueries( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchemaWithPartition.json))) } } - test("save() - partitioned table - Append - new partition values") { - testDF.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> singlePartitionColumn.json), - partitionColumns = Seq("p1")) - - val moreData = (for { - i <- 1 to 3 - p <- 3 to 4 - } yield (i, s"val_$i", p)).toDF("a", "b", "p1") - - moreData.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Append, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> singlePartitionColumn.json), - partitionColumns = Seq("p1")) - - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i") - - TestResult.synchronized { - assert(TestResult.writerPaths.size === 8) - assert(TestResult.writtenRows === expectedRows.toSet) + test("save()/load() - partitioned table - Overwrite") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + partitionedTestDF.collect()) } } - test("save() - partitioned table - Append - mismatched partition columns") { - val data = (for { - i <- 1 to 3 - p1 <- 1 to 2 - p2 <- Array("hello", "world") - } yield (i, s"val_$i", p1, p2)).toDF("a", "b", "p1", "p2") + test("save()/load() - partitioned table - Append") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) - // Using only a subset of all partition columns - intercept[IllegalArgumentException] { - data.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, mode = SaveMode.Append, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> doublePartitionColumns.json), - partitionColumns = Seq("p1")) + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + partitionedTestDF.unionAll(partitionedTestDF).collect()) } + } - // Using different order of partition columns - intercept[IllegalArgumentException] { - data.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, + test("save()/load() - partitioned table - Append - new partition values") { + withTempPath { file => + partitionedTestDF1.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF2.save( + source = classOf[SimpleTextSource].getCanonicalName, mode = SaveMode.Append, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> doublePartitionColumns.json), - partitionColumns = Seq("p2", "p1")) + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + checkAnswer( + load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> file.getCanonicalPath, + "dataSchema" -> dataSchema.json)), + partitionedTestDF.collect()) } } - test("save() - partitioned table - ErrorIfExists") { - fs.delete(basePath, true) + ignore("save()/load() - partitioned table - Append - mismatched partition columns") { + withTempPath { file => + partitionedTestDF1.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + // Using only a subset of all partition columns + intercept[IllegalArgumentException] { + partitionedTestDF2.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1")) + } - testDF.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> singlePartitionColumn.json), - partitionColumns = Seq("p1")) - - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - - TestResult.synchronized { - assert(TestResult.writerPaths.size === 4) - assert(TestResult.writtenRows === expectedRows.toSet) + // Using different order of partition columns + intercept[IllegalArgumentException] { + partitionedTestDF2.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p2", "p1")) + } } + } - intercept[RuntimeException] { - testDF.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.ErrorIfExists, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> singlePartitionColumn.json), - partitionColumns = Seq("p1")) + test("save()/load() - partitioned table - ErrorIfExists") { + withTempDir { file => + intercept[RuntimeException] { + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + } } } - test("save() - partitioned table - Ignore") { - testDF.save( - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = SaveMode.Ignore, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json), - partitionColumns = Seq("p1")) + test("save()/load() - partitioned table - Ignore") { + withTempDir { file => + partitionedTestDF.save( + path = file.getCanonicalPath, + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore) - assert(TestResult.writtenRows.isEmpty) + val path = new Path(file.getCanonicalPath) + val fs = path.getFileSystem(SparkHadoopUtil.get.conf) + assert(fs.listStatus(path).isEmpty) + } } - test("save() - data sources other than FSBasedRelation") { - intercept[RuntimeException] { - testDF.save( - source = classOf[FilteredScanSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map("path" -> basePath.toString), - partitionColumns = Seq("p1")) - } + def withTable(tableName: String)(f: => Unit): Unit = { + try f finally sql(s"DROP TABLE $tableName") } - def saveToPartitionedTable( - df: DataFrame, - tableName: String, - relationPartitionColumns: StructType, - partitionedBy: Seq[String], - mode: SaveMode): Unit = { - df.saveAsTable( - tableName = tableName, - source = classOf[SimpleFSBasedSource].getCanonicalName, - mode = mode, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> relationPartitionColumns.json), - partitionColumns = partitionedBy) + test("saveAsTable()/load() - non-partitioned table - Overwrite") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + Map("dataSchema" -> dataSchema.json)) + + withTable("t") { + checkAnswer(table("t"), testDF.collect()) + } } - test("saveAsTable() - partitioned table - Overwrite") { - saveToPartitionedTable(testDF, "t", singlePartitionColumn, Array("p1"), SaveMode.Overwrite) + test("saveAsTable()/load() - non-partitioned table - Append") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite) - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append) - TestResult.synchronized { - assert(TestResult.writerPaths.size === 4) - assert(TestResult.writtenRows === expectedRows.toSet) + withTable("t") { + checkAnswer(table("t"), testDF.unionAll(testDF).orderBy("a").collect()) } + } - assertResult(table("t").schema) { - StructType( - dataSchema ++ Seq( - StructField("p1", IntegerType, nullable = true))) - } + test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") - sql("DROP TABLE t") + withTempTable("t") { + intercept[AnalysisException] { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists) + } + } } - ignore("saveAsTable() - partitioned table - Overwrite - select and overwrite the same table") { - TestResult.rowsToRead = testDF.rdd + test("saveAsTable()/load() - non-partitioned table - Ignore") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") - val df = load( - source = classOf[SimpleFSBasedSource].getCanonicalName, - options = Map( - "path" -> basePath.toString, - "schema" -> dataSchema.json, - "partitionColumns" -> singlePartitionColumn.json)) + withTempTable("t") { + testDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore) - saveToPartitionedTable(df, "t", singlePartitionColumn, Seq("p1"), SaveMode.Overwrite) + assert(table("t").collect().isEmpty) + } + } - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + test("saveAsTable()/load() - partitioned table - simple queries") { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + Map("dataSchema" -> dataSchema.json)) - TestResult.synchronized { - assert(TestResult.writerPaths.size === 4) - assert(TestResult.writtenRows === expectedRows.toSet) + withTable("t") { + checkQueries(table("t")) } + } - assertResult(table("t").schema) { - StructType( - dataSchema ++ Seq( - StructField("p1", IntegerType, nullable = true))) - } + test("saveAsTable()/load() - partitioned table - Overwrite") { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) - sql("DROP TABLE t") + withTable("t") { + checkAnswer(table("t"), partitionedTestDF.collect()) + } } - test("saveAsTable() - partitioned table - Append") { - saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.Overwrite) - saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.Append) + test("saveAsTable()/load() - partitioned table - Append") { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) - TestResult.synchronized { - assert(TestResult.writerPaths.size === 8) - assert(TestResult.writtenRows === expectedRows.toSet) + withTable("t") { + checkAnswer(table("t"), partitionedTestDF.unionAll(partitionedTestDF).collect()) } + } - assertResult(table("t").schema) { - StructType( - dataSchema ++ Seq( - StructField("p1", IntegerType, nullable = true))) - } + test("saveAsTable()/load() - partitioned table - Append - new partition values") { + partitionedTestDF1.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + partitionedTestDF2.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Append, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) - sql("DROP TABLE t") + withTable("t") { + checkAnswer(table("t"), partitionedTestDF.collect()) + } } - test("saveAsTable() - partitioned table - Append - mismatched partition columns") { - val data = (for { - i <- 1 to 3 - p1 <- 1 to 2 - p2 <- Array("hello", "world") - } yield (i, s"val_$i", p1, p2)).toDF("a", "b", "p1", "p2") + ignore("saveAsTable()/load() - partitioned table - Append - mismatched partition columns") { + partitionedTestDF1.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) // Using only a subset of all partition columns intercept[IllegalArgumentException] { - saveToPartitionedTable(data, "t", doublePartitionColumns, Seq("p1"), SaveMode.Append) + partitionedTestDF2.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1")) } // Using different order of partition columns intercept[IllegalArgumentException] { - saveToPartitionedTable(data, "t", doublePartitionColumns, Seq("p2", "p1"), SaveMode.Append) + partitionedTestDF2.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p2", "p1")) } - - sql("DROP TABLE t") } - test("saveAsTable() - partitioned table - ErrorIfExists") { - fs.delete(basePath, true) - - saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.ErrorIfExists) - - // Written rows shouldn't contain dynamic partition column - val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i") - - TestResult.synchronized { - assert(TestResult.writerPaths.size === 4) - assert(TestResult.writtenRows === expectedRows.toSet) - } - - assertResult(table("t").schema) { - StructType( - dataSchema ++ Seq( - StructField("p1", IntegerType, nullable = true))) - } - - intercept[AnalysisException] { - saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.ErrorIfExists) + test("saveAsTable()/load() - partitioned table - ErrorIfExists") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") + + withTempTable("t") { + intercept[AnalysisException] { + partitionedTestDF.saveAsTable( + tableName = "t", + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.ErrorIfExists, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + } } - - sql("DROP TABLE t") } - test("saveAsTable() - partitioned table - Ignore") { - saveToPartitionedTable(testDF, "t", singlePartitionColumn, Seq("p1"), SaveMode.Ignore) + test("saveAsTable()/load() - partitioned table - Ignore") { + Seq.empty[(Int, String)].toDF().registerTempTable("t") - assert(TestResult.writtenRows.isEmpty) - assertResult(table("t").schema) { - StructType( - dataSchema ++ Seq( - StructField("p1", IntegerType, nullable = true))) - } - - sql("DROP TABLE t") - } - - test("saveAsTable() - data sources other than FSBasedRelation") { - intercept[RuntimeException] { - testDF.saveAsTable( + withTempTable("t") { + partitionedTestDF.saveAsTable( tableName = "t", - source = classOf[FilteredScanSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map("path" -> basePath.toString), - partitionColumns = Seq("p1")) + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Ignore, + options = Map("dataSchema" -> dataSchema.json), + partitionColumns = Seq("p1", "p2")) + + assert(table("t").collect().isEmpty) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala new file mode 100644 index 0000000000000..8d026020feaf6 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -0,0 +1,135 @@ +/* + * 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.sources + +import java.text.NumberFormat +import java.util.UUID + +import com.google.common.base.Objects +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{NullWritable, Text} +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} +import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} +import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.{Row, SQLContext} + +/** + * A simple example [[FSBasedRelationProvider]]. + */ +class SimpleTextSource extends FSBasedRelationProvider { + override def createRelation( + sqlContext: SQLContext, + schema: Option[StructType], + partitionColumns: Option[StructType], + parameters: Map[String, String]): BaseRelation = { + new SimpleTextRelation( + Array(parameters("path")), schema, + partitionColumns.getOrElse(StructType(Array.empty[StructField])), parameters)(sqlContext) + } +} + +class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullWritable, Text] { + val numberFormat = NumberFormat.getInstance() + + numberFormat.setMinimumIntegerDigits(5) + numberFormat.setGroupingUsed(false) + + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + val split = context.getTaskAttemptID.getTaskID.getId + val name = FileOutputFormat.getOutputName(context) + new Path(outputFile, s"$name-${numberFormat.format(split)}-${UUID.randomUUID()}") + } +} + +class SimpleTextOutputWriter extends OutputWriter { + private var converter: Any => Any = _ + private var recordWriter: RecordWriter[NullWritable, Text] = _ + private var taskAttemptContext: TaskAttemptContext = _ + + override def init( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): Unit = { + converter = CatalystTypeConverters.createToScalaConverter(dataSchema) + recordWriter = new AppendingTextOutputFormat(new Path(path)).getRecordWriter(context) + taskAttemptContext = context + } + + override def write(row: Row): Unit = { + // Serializes values in `row` into a comma separated string + val convertedRow = converter(row).asInstanceOf[Row] + val serialized = convertedRow.toSeq.map(_.toString).mkString(",") + recordWriter.write(null, new Text(serialized)) + } + + override def close(): Unit = recordWriter.close(taskAttemptContext) +} + +/** + * A simple example [[FSBasedRelation]], used for testing purposes. Data are stored as comma + * separated string lines. When scanning data, schema must be explicitly provided via data source + * option `"dataSchema"`. + */ +class SimpleTextRelation( + paths: Array[String], + val maybeDataSchema: Option[StructType], + val partitionColumns: StructType, + parameters: Map[String, String])( + @transient val sqlContext: SQLContext) + extends FSBasedRelation(paths, partitionColumns) { + + import sqlContext.sparkContext + + override val dataSchema: StructType = + maybeDataSchema.getOrElse(DataType.fromJson(parameters("dataSchema")).asInstanceOf[StructType]) + + override def equals(other: Any): Boolean = other match { + case that: SimpleTextRelation => + this.paths.sameElements(that.paths) && + this.maybeDataSchema == that.maybeDataSchema && + this.dataSchema == that.dataSchema + + case _ => false + } + + override def hashCode(): Int = + Objects.hashCode(paths, maybeDataSchema, dataSchema) + + override def outputWriterClass: Class[_ <: OutputWriter] = + classOf[SimpleTextOutputWriter] + + override def buildScan( + requiredColumns: Array[String], + filters: Array[Filter], + inputPaths: Array[String]): RDD[Row] = { + val fields = dataSchema.fields + sparkContext.textFile(inputPaths.mkString(",")).map { record => + val valueMap = record.split(",").zip(fields).map { + case (value, StructField(name, dataType, _, _)) => + name -> Cast(Literal(value), dataType).eval() + }.toMap + + // This mocks a simple projection + Row.fromSeq(requiredColumns.map(valueMap).toSeq) + } + } +} From 51be44366033ea59d6171c02c2208f27b1bb2f6f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 6 May 2015 19:14:07 +0800 Subject: [PATCH 28/45] Replaces FSBasedRelation.outputCommitterClass with FSBasedRelation.prepareForWrite --- .../apache/spark/sql/sources/commands.scala | 28 ++++++++----------- .../apache/spark/sql/sources/interfaces.scala | 12 +++++--- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 38ae68006e189..e25b818ceaf53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -229,16 +229,17 @@ private[sql] abstract class BaseWriterContainer( protected val dataSchema = relation.dataSchema - protected val outputCommitterClass: Class[_ <: FileOutputCommitter] = - relation.outputCommitterClass - protected val outputWriterClass: Class[_ <: OutputWriter] = relation.outputWriterClass + private var outputFormatClass: Class[_ <: OutputFormat[_, _]] = _ + def driverSideSetup(): Unit = { setupIDs(0, 0, 0) setupConf() taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) - outputCommitter = newOutputCommitter(outputCommitterClass, outputPath, taskAttemptContext) + relation.prepareForWrite(job) + outputFormatClass = job.getOutputFormatClass + outputCommitter = newOutputCommitter(taskAttemptContext) outputCommitter.setupJob(jobContext) } @@ -246,22 +247,17 @@ private[sql] abstract class BaseWriterContainer( setupIDs(taskContext.stageId(), taskContext.partitionId(), taskContext.attemptNumber()) setupConf() taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) - outputCommitter = newOutputCommitter(outputCommitterClass, outputPath, taskAttemptContext) + outputCommitter = newOutputCommitter(taskAttemptContext) outputCommitter.setupTask(taskAttemptContext) initWriters() } - private def newOutputCommitter( - clazz: Class[_ <: FileOutputCommitter], - path: String, - context: TaskAttemptContext): FileOutputCommitter = { - val ctor = outputCommitterClass.getConstructor(classOf[Path], classOf[TaskAttemptContext]) - ctor.setAccessible(true) - - val hadoopPath = new Path(path) - val fs = hadoopPath.getFileSystem(serializableConf.value) - val qualified = fs.makeQualified(hadoopPath) - ctor.newInstance(qualified, context) + private def newOutputCommitter(context: TaskAttemptContext): FileOutputCommitter = { + outputFormatClass.newInstance().getOutputCommitter(context) match { + case f: FileOutputCommitter => f + case f => sys.error( + s"FileOutputCommitter or its subclass is expected, but got a ${f.getClass.getName}.") + } } private def setupIDs(jobId: Int, splitId: Int, attemptId: Int): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index e46e9516bc203..e42384c4cee32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.sources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.spark.annotation.{DeveloperApi, Experimental} @@ -432,10 +432,14 @@ abstract class FSBasedRelation private[sql]( } /** - * The output committer class to use. Default to - * [[org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter]]. + * Client side preparation for data writing can be put here. For example, user defined output + * committer can be configured here. + * + * Note that the only side effect expected here is mutating `job` via its setters. Especially, + * Spark SQL caches [[BaseRelation]] instances for performance, mutating relation internal states + * may cause unexpected behaviors. */ - def outputCommitterClass: Class[_ <: FileOutputCommitter] = classOf[FileOutputCommitter] + def prepareForWrite(job: Job): Unit = () /** * This method is responsible for producing a new [[OutputWriter]] for each newly opened output From 5849dd0629a7278d3b57264ac5c3943d75366819 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 May 2015 00:39:08 +0800 Subject: [PATCH 29/45] Fixes doc typos. Fixes partition discovery refresh. --- .../org/apache/spark/sql/sources/ddl.scala | 8 +++- .../apache/spark/sql/sources/interfaces.scala | 47 ++++++++++--------- .../sql/sources/FSBasedRelationSuite.scala | 38 ++------------- .../sql/sources/SimpleTextRelation.scala | 7 +-- 4 files changed, 42 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 39615e946a308..182f6122a34f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -224,10 +224,16 @@ private[sql] object ResolvedDataSource { case dataSource: SchemaRelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema) case dataSource: FSBasedRelationProvider => + val maybePartitionsSchema = if (partitionColumns.isEmpty) { + None + } else { + Some(partitionColumnsSchema(schema, partitionColumns)) + } + dataSource.createRelation( sqlContext, Some(schema), - Some(partitionColumnsSchema(schema, partitionColumns)), + maybePartitionsSchema, new CaseInsensitiveMap(options)) case dataSource: org.apache.spark.sql.sources.RelationProvider => throw new AnalysisException(s"$className does not allow user-specified schemas.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index e42384c4cee32..ecf5ad78e1f6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.sources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil @@ -294,7 +293,7 @@ abstract class OutputWriter { * filter using selected predicates before producing an RDD containing all matching tuples as * [[Row]] objects. In addition, when reading from Hive style partitioned tables stored in file * systems, it's able to discover partitioning information from the paths of input directories, and - * perform partition pruning before start reading the data.Subclasses of [[FSBasedRelation()]] must + * perform partition pruning before start reading the data. Subclasses of [[FSBasedRelation()]] must * override one of the three `buildScan` methods to implement the read path. * * For the write path, it provides the ability to write to both non-partitioned and partitioned @@ -329,39 +328,45 @@ abstract class FSBasedRelation private[sql]( /** * Constructs an [[FSBasedRelation]]. * - * @param paths Base paths of this relation. For partitioned relations, it should be either root + * @param paths Base paths of this relation. For partitioned relations, it should be root * directories of all partition directories. */ def this(paths: Array[String]) = this(paths, None) private val hadoopConf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) - private var _partitionSpec: PartitionSpec = _ - refreshPartitions() + private var _partitionSpec: PartitionSpec = maybePartitionSpec.map { spec => + spec.copy(partitionColumns = spec.partitionColumns.asNullable) + }.getOrElse { + discoverPartitions() + } private[sql] def partitionSpec: PartitionSpec = _partitionSpec + /** + * Partition columns. Note that they are always nullable. + */ + def partitionColumns: StructType = partitionSpec.partitionColumns + private[sql] def refresh(): Unit = { - refreshPartitions() + _partitionSpec = discoverPartitions() } - private def refreshPartitions(): Unit = { - _partitionSpec = maybePartitionSpec.getOrElse { - val basePaths = paths.map(new Path(_)) - val leafDirs = basePaths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - if (fs.exists(path)) { - SparkHadoopUtil.get.listLeafDirStatuses(fs, fs.makeQualified(path)) - } else { - Seq.empty[FileStatus] - } - }.map(_.getPath) - - if (leafDirs.nonEmpty) { - PartitioningUtils.parsePartitions(leafDirs, "__HIVE_DEFAULT_PARTITION__") + private def discoverPartitions(): PartitionSpec = { + val basePaths = paths.map(new Path(_)) + val leafDirs = basePaths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + if (fs.exists(path)) { + SparkHadoopUtil.get.listLeafDirStatuses(fs, fs.makeQualified(path)) } else { - PartitionSpec(StructType(Array.empty[StructField]), Array.empty[Partition]) + Seq.empty[FileStatus] } + }.map(_.getPath) + + if (leafDirs.nonEmpty) { + PartitioningUtils.parsePartitions(leafDirs, "__HIVE_DEFAULT_PARTITION__") + } else { + PartitionSpec(StructType(Array.empty[StructField]), Array.empty[Partition]) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 5ee0fc892d46e..f2d42a7c3d67c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -269,34 +269,6 @@ class FSBasedRelationSuite extends QueryTest with ParquetTest { } } - ignore("save()/load() - partitioned table - Append - mismatched partition columns") { - withTempPath { file => - partitionedTestDF1.save( - source = classOf[SimpleTextSource].getCanonicalName, - mode = SaveMode.Overwrite, - options = Map("path" -> file.getCanonicalPath), - partitionColumns = Seq("p1", "p2")) - - // Using only a subset of all partition columns - intercept[IllegalArgumentException] { - partitionedTestDF2.save( - source = classOf[SimpleTextSource].getCanonicalName, - mode = SaveMode.Append, - options = Map("path" -> file.getCanonicalPath), - partitionColumns = Seq("p1")) - } - - // Using different order of partition columns - intercept[IllegalArgumentException] { - partitionedTestDF2.save( - source = classOf[SimpleTextSource].getCanonicalName, - mode = SaveMode.Append, - options = Map("path" -> file.getCanonicalPath), - partitionColumns = Seq("p2", "p1")) - } - } - } - test("save()/load() - partitioned table - ErrorIfExists") { withTempDir { file => intercept[RuntimeException] { @@ -452,7 +424,7 @@ class FSBasedRelationSuite extends QueryTest with ParquetTest { } } - ignore("saveAsTable()/load() - partitioned table - Append - mismatched partition columns") { + test("saveAsTable()/load() - partitioned table - Append - mismatched partition columns") { partitionedTestDF1.saveAsTable( tableName = "t", source = classOf[SimpleTextSource].getCanonicalName, @@ -461,21 +433,21 @@ class FSBasedRelationSuite extends QueryTest with ParquetTest { partitionColumns = Seq("p1", "p2")) // Using only a subset of all partition columns - intercept[IllegalArgumentException] { + intercept[Throwable] { partitionedTestDF2.saveAsTable( tableName = "t", source = classOf[SimpleTextSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = SaveMode.Append, options = Map("dataSchema" -> dataSchema.json), partitionColumns = Seq("p1")) } // Using different order of partition columns - intercept[IllegalArgumentException] { + intercept[Throwable] { partitionedTestDF2.saveAsTable( tableName = "t", source = classOf[SimpleTextSource].getCanonicalName, - mode = SaveMode.Overwrite, + mode = SaveMode.Append, options = Map("dataSchema" -> dataSchema.json), partitionColumns = Seq("p2", "p1")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 8d026020feaf6..bc34aed1e5c96 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -92,10 +92,10 @@ class SimpleTextOutputWriter extends OutputWriter { class SimpleTextRelation( paths: Array[String], val maybeDataSchema: Option[StructType], - val partitionColumns: StructType, + partitionsSchema: StructType, parameters: Map[String, String])( @transient val sqlContext: SQLContext) - extends FSBasedRelation(paths, partitionColumns) { + extends FSBasedRelation(paths, partitionsSchema) { import sqlContext.sparkContext @@ -106,7 +106,8 @@ class SimpleTextRelation( case that: SimpleTextRelation => this.paths.sameElements(that.paths) && this.maybeDataSchema == that.maybeDataSchema && - this.dataSchema == that.dataSchema + this.dataSchema == that.dataSchema && + this.partitionColumns == that.partitionColumns case _ => false } From fa543f31cb0b75d8003cfb8d03edea437222c775 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 May 2015 11:51:01 +0800 Subject: [PATCH 30/45] Addresses comments --- .../org/apache/spark/sql/sources/DataSourceStrategy.scala | 2 +- .../main/scala/org/apache/spark/sql/sources/commands.scala | 2 +- .../scala/org/apache/spark/sql/sources/interfaces.scala | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 76ee4b5d7d897..47f265b8db4a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -90,7 +90,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: FSBasedRelation)) => val inputPaths = t.paths.map(new Path(_)).flatMap { path => val fs = path.getFileSystem(t.sqlContext.sparkContext.hadoopConfiguration) - val qualifiedPath = fs.makeQualified(path) + val qualifiedPath = path.makeQualified(fs.getUri, fs.getWorkingDirectory) SparkHadoopUtil.get.listLeafStatuses(fs, qualifiedPath).map(_.getPath).filterNot { path => val name = path.getName name.startsWith("_") || name.startsWith(".") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index e25b818ceaf53..784483b57faea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -71,7 +71,7 @@ private[sql] case class InsertIntoFSBasedRelation( val hadoopConf = sqlContext.sparkContext.hadoopConfiguration val outputPath = new Path(relation.paths.head) val fs = outputPath.getFileSystem(hadoopConf) - val qualifiedOutputPath = fs.makeQualified(outputPath) + val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) val doInsertion = (mode, fs.exists(qualifiedOutputPath)) match { case (SaveMode.ErrorIfExists, true) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index ecf5ad78e1f6b..7d5a87f708f11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -301,7 +301,7 @@ abstract class OutputWriter { * * @constructor This constructor is for internal uses only. The [[PartitionSpec]] argument is for * implementing metastore table conversion. - * @param paths Base paths of this relation. For partitioned relations, it should be either root + * @param paths Base paths of this relation. For partitioned relations, it should be the root * directories of all partition directories. * @param maybePartitionSpec An [[FSBasedRelation]] can be created with an optional * [[PartitionSpec]], so that partition discovery can be skipped. @@ -371,8 +371,8 @@ abstract class FSBasedRelation private[sql]( } /** - * Schema of this relation. It consists of [[dataSchema]] and all partition columns not appeared - * in [[dataSchema]]. + * Schema of this relation. It consists of columns appearing in [[dataSchema]] and all partition + * columns not appearing in [[dataSchema]]. */ override lazy val schema: StructType = { val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet From 0b8cd70da90a837f2d58cd17d02cc3c360810c96 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 May 2015 12:11:41 +0800 Subject: [PATCH 31/45] Adds Scala/Catalyst row conversion when writing non-partitioned tables --- .../org/apache/spark/sql/sources/commands.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 784483b57faea..b8a24800fadb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -30,6 +30,7 @@ import parquet.hadoop.util.ContextUtil import org.apache.spark._ import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand @@ -123,9 +124,17 @@ private[sql] case class InsertIntoFSBasedRelation( writerContainer.executorSideSetup(taskContext) try { - while (iterator.hasNext) { - val row = iterator.next() - writerContainer.outputWriterForRow(row).write(row) + if (relation.needConversion) { + val converter = CatalystTypeConverters.createToScalaConverter(relation.dataSchema) + while (iterator.hasNext) { + val row = converter(iterator.next()).asInstanceOf[Row] + writerContainer.outputWriterForRow(row).write(row) + } + } else { + while (iterator.hasNext) { + val row = iterator.next() + writerContainer.outputWriterForRow(row).write(row) + } } writerContainer.commitTask() } catch { case cause: Throwable => From 795920a66bc4fac8c0184d173b84a06100f0f5ba Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 May 2015 22:27:05 +0800 Subject: [PATCH 32/45] Fixes compilation error after rebasing --- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 7db5ae105c77b..f3107f7b51ad8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -467,8 +467,8 @@ class DataFrame private[sql]( // By the time we get here, since we have already run analysis, all attributes should've been // resolved and become AttributeReference. val cond = plan.condition.map { _.transform { - case EqualTo(a: AttributeReference, b: AttributeReference) if a.sameRef(b) => - EqualTo(plan.left.resolve(a.name), plan.right.resolve(b.name)) + case expressions.EqualTo(a: AttributeReference, b: AttributeReference) if a.sameRef(b) => + expressions.EqualTo(plan.left.resolve(a.name), plan.right.resolve(b.name)) }} plan.copy(condition = cond) } From bc3f9b43b39ba5e3d998c7639877566a7a085597 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 May 2015 22:27:08 +0800 Subject: [PATCH 33/45] Uses projection to separate partition columns and data columns while inserting rows --- .../apache/spark/sql/sources/commands.scala | 96 ++++++++++++------- .../sql/sources/SimpleTextRelation.scala | 6 +- 2 files changed, 61 insertions(+), 41 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index b8a24800fadb5..9b07cb562e776 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -32,6 +32,7 @@ import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateProjection import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} @@ -102,7 +103,7 @@ private[sql] case class InsertIntoFSBasedRelation( } else { val writerContainer = new DynamicPartitionWriterContainer( relation, job, partitionColumns, "__HIVE_DEFAULT_PARTITION__") - insertWithDynamicPartitions(writerContainer, df, partitionColumns) + insertWithDynamicPartitions(sqlContext, writerContainer, df, partitionColumns) } } @@ -110,6 +111,10 @@ private[sql] case class InsertIntoFSBasedRelation( } private def insert(writerContainer: BaseWriterContainer, df: DataFrame): Unit = { + // Uses local vals for serialization + val needsConversion = relation.needConversion + val dataSchema = relation.dataSchema + try { writerContainer.driverSideSetup() df.sqlContext.sparkContext.runJob(df.queryExecution.executedPlan.execute(), writeRows _) @@ -124,8 +129,8 @@ private[sql] case class InsertIntoFSBasedRelation( writerContainer.executorSideSetup(taskContext) try { - if (relation.needConversion) { - val converter = CatalystTypeConverters.createToScalaConverter(relation.dataSchema) + if (needsConversion) { + val converter = CatalystTypeConverters.createToScalaConverter(dataSchema) while (iterator.hasNext) { val row = converter(iterator.next()).asInstanceOf[Row] writerContainer.outputWriterForRow(row).write(row) @@ -145,9 +150,13 @@ private[sql] case class InsertIntoFSBasedRelation( } private def insertWithDynamicPartitions( + sqlContext: SQLContext, writerContainer: BaseWriterContainer, df: DataFrame, partitionColumns: Array[String]): Unit = { + // Uses a local val for serialization + val needsConversion = relation.needConversion + val dataSchema = relation.dataSchema require( df.schema == relation.schema, @@ -156,34 +165,21 @@ private[sql] case class InsertIntoFSBasedRelation( |Relation schema: ${relation.schema} """.stripMargin) - val sqlContext = df.sqlContext - - val (partitionRDD, dataRDD) = { - val fieldNames = relation.schema.fieldNames - val dataCols = fieldNames.filterNot(partitionColumns.contains) - val df = sqlContext.createDataFrame( - DataFrame(sqlContext, query).queryExecution.toRdd, - relation.schema, - needsConversion = false) - - val partitionColumnsInSpec = relation.partitionSpec.partitionColumns.map(_.name) - require( - partitionColumnsInSpec.sameElements(partitionColumns), - s"""Partition columns mismatch. - |Expected: ${partitionColumnsInSpec.mkString(", ")} - |Actual: ${partitionColumns.mkString(", ")} - """.stripMargin) - - val partitionDF = df.select(partitionColumns.head, partitionColumns.tail: _*) - val dataDF = df.select(dataCols.head, dataCols.tail: _*) + val partitionColumnsInSpec = relation.partitionColumns.fieldNames + require( + partitionColumnsInSpec.sameElements(partitionColumns), + s"""Partition columns mismatch. + |Expected: ${partitionColumnsInSpec.mkString(", ")} + |Actual: ${partitionColumns.mkString(", ")} + """.stripMargin) - partitionDF.queryExecution.executedPlan.execute() -> - dataDF.queryExecution.executedPlan.execute() - } + val output = df.queryExecution.executedPlan.output + val (partitionOutput, dataOutput) = output.partition(a => partitionColumns.contains(a.name)) + val codegenEnabled = df.sqlContext.conf.codegenEnabled try { writerContainer.driverSideSetup() - sqlContext.sparkContext.runJob(partitionRDD.zip(dataRDD), writeRows _) + df.sqlContext.sparkContext.runJob(df.queryExecution.executedPlan.execute(), writeRows _) writerContainer.commitJob() relation.refresh() } catch { case cause: Throwable => @@ -192,20 +188,44 @@ private[sql] case class InsertIntoFSBasedRelation( throw new SparkException("Job aborted.", cause) } - def writeRows(taskContext: TaskContext, iterator: Iterator[(Row, Row)]): Unit = { + def writeRows(taskContext: TaskContext, iterator: Iterator[Row]): Unit = { writerContainer.executorSideSetup(taskContext) - try { + val partitionProj = newProjection(codegenEnabled, partitionOutput, output) + val dataProj = newProjection(codegenEnabled, dataOutput, output) + + if (needsConversion) { + val converter = CatalystTypeConverters.createToScalaConverter(dataSchema) while (iterator.hasNext) { - val (partitionValues, data) = iterator.next() - writerContainer.outputWriterForRow(partitionValues).write(data) + val row = converter(iterator.next()).asInstanceOf[Row] + val partitionPart = partitionProj(row) + val dataPart = dataProj(row) + writerContainer.outputWriterForRow(partitionPart).write(dataPart) + } + } else { + while (iterator.hasNext) { + val row = iterator.next() + val partitionPart = partitionProj(row) + val dataPart = dataProj(row) + writerContainer.outputWriterForRow(partitionPart).write(dataPart) } - - writerContainer.commitTask() - } catch { case cause: Throwable => - writerContainer.abortTask() - throw new SparkException("Task failed while writing rows.", cause) } + + writerContainer.commitTask() + } + } + + // This is copied from SparkPlan, probably should move this to a more general place. + private def newProjection( + codegenEnabled: Boolean, + expressions: Seq[Expression], + inputSchema: Seq[Attribute]): Projection = { + log.debug( + s"Creating Projection: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") + if (codegenEnabled) { + GenerateProjection.generate(expressions, inputSchema) + } else { + new InterpretedProjection(expressions, inputSchema) } } } @@ -379,6 +399,10 @@ private[sql] class DynamicPartitionWriterContainer( } private[sql] object DynamicPartitionWriterContainer { + ////////////////////////////////////////////////////////////////////////////////////////////////// + // The following string escaping code is mainly copied from Hive (o.a.h.h.common.FileUtils). + ////////////////////////////////////////////////////////////////////////////////////////////////// + val charToEscape = { val bitSet = new java.util.BitSet(128) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index bc34aed1e5c96..f29dbc4f97a32 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -61,7 +61,6 @@ class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullW } class SimpleTextOutputWriter extends OutputWriter { - private var converter: Any => Any = _ private var recordWriter: RecordWriter[NullWritable, Text] = _ private var taskAttemptContext: TaskAttemptContext = _ @@ -69,15 +68,12 @@ class SimpleTextOutputWriter extends OutputWriter { path: String, dataSchema: StructType, context: TaskAttemptContext): Unit = { - converter = CatalystTypeConverters.createToScalaConverter(dataSchema) recordWriter = new AppendingTextOutputFormat(new Path(path)).getRecordWriter(context) taskAttemptContext = context } override def write(row: Row): Unit = { - // Serializes values in `row` into a comma separated string - val convertedRow = converter(row).asInstanceOf[Row] - val serialized = convertedRow.toSeq.map(_.toString).mkString(",") + val serialized = row.toSeq.map(_.toString).mkString(",") recordWriter.write(null, new Text(serialized)) } From c466de6b5bc47972dcca4e6b4548b19519faa0ca Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 May 2015 23:10:25 +0800 Subject: [PATCH 34/45] Addresses comments --- .../sql/sources/PartitioningOptions.java | 28 ------------------- .../scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../apache/spark/sql/sources/interfaces.scala | 15 ++++++++-- 3 files changed, 13 insertions(+), 32 deletions(-) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java b/sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java deleted file mode 100644 index afe565e7a81e6..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/PartitioningOptions.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.sources; - -public interface PartitioningOptions { - /** - * When set to `false`, partition discovery is not performed. This option - * can be useful when, for example, the user only wants to load data stored - * in a single partition directory without introducing partition columns - * encoded in the directory path. - */ - String PARTITION_DISCOVERY_ENABLED = "partition.discovery.enabled"; -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 2f9784b823173..a9f284a3803b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -66,7 +66,7 @@ private[spark] object SQLConf { // to its length exceeds the threshold. val SCHEMA_STRING_LENGTH_THRESHOLD = "spark.sql.sources.schemaStringLengthThreshold" - // Whether to perform partition discovery when loading external data sources. + // Whether to perform partition discovery when loading external data sources. Default to true. val PARTITION_DISCOVERY_ENABLED = "spark.sql.sources.partitionDiscovery.enabled" // Whether to perform eager analysis when constructing a dataframe. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 7d5a87f708f11..67f756c1c8fa3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode} +import org.apache.spark.sql._ /** * ::DeveloperApi:: @@ -338,9 +338,16 @@ abstract class FSBasedRelation private[sql]( private var _partitionSpec: PartitionSpec = maybePartitionSpec.map { spec => spec.copy(partitionColumns = spec.partitionColumns.asNullable) }.getOrElse { - discoverPartitions() + if (partitionDiscoverEnabled()) { + discoverPartitions() + } else { + PartitionSpec(StructType(Nil), Array.empty[Partition]) + } } + private def partitionDiscoverEnabled() = + sqlContext.conf.getConf(SQLConf.PARTITION_DISCOVERY_ENABLED, "true").toBoolean + private[sql] def partitionSpec: PartitionSpec = _partitionSpec /** @@ -349,7 +356,9 @@ abstract class FSBasedRelation private[sql]( def partitionColumns: StructType = partitionSpec.partitionColumns private[sql] def refresh(): Unit = { - _partitionSpec = discoverPartitions() + if (partitionDiscoverEnabled()) { + _partitionSpec = discoverPartitions() + } } private def discoverPartitions(): PartitionSpec = { From 52b0c9bf7ceca07166b6c0d6220cbe2c2db4e371 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 8 May 2015 23:21:43 +0800 Subject: [PATCH 35/45] Adjusts project/MimaExclude.scala --- project/MimaExcludes.scala | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ad3d8426bdedd..aa844c0dddfe5 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -94,11 +94,23 @@ object MimaExcludes { // This `protected[sql]` method was removed in 1.3.1 ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.sql.SQLContext.checkAnalysis"), - // This `private[sql]` class was removed in 1.4.0: + // These `private[sql]` class was removed in 1.4.0: ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.execution.AddExchange"), ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.execution.AddExchange$"), + "org.spache.spark.sql.execution.AddExchange$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.spache.spark.sql.parquet.PartitionSpec"), + ProblemFilters.exclude[MissingClassProblem]( + "org.spache.spark.sql.parquet.PartitionSpec$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.spache.spark.sql.parquet.Partition"), + ProblemFilters.exclude[MissingClassProblem]( + "org.spache.spark.sql.parquet.Partition$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.spache.spark.sql.parquet.ParquetRelation2$PartitionValues"), + ProblemFilters.exclude[MissingClassProblem]( + "org.spache.spark.sql.parquet.ParquetRelation2$PartitionValues$"), // These test support classes were moved out of src/main and into src/test: ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.parquet.ParquetTestData"), From 0349e098b43fd538687b3ab264b93633bc720ce3 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 9 May 2015 07:56:57 +0800 Subject: [PATCH 36/45] Fixes compilation error introduced while rebasing --- .../org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala | 2 +- .../scala/org/apache/spark/sql/sources/FilteredScanSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/sources/InsertSuite.scala | 2 +- .../scala/org/apache/spark/sql/sources/PrunedScanSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala | 2 +- .../scala/org/apache/spark/sql/sources/TableScanSuite.scala | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index d17391b49031d..6b1aa7e62ea5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { - import caseInsensitiveContext._ + import caseInsensisitiveContext._ var path: File = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 6664e8d64c13a..ca25751b9583d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -64,7 +64,7 @@ case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlCo } class DDLTestSuite extends DataSourceTest { - import caseInsensitiveContext._ + import caseInsensisitiveContext._ before { sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index cce747e7dbf64..cb5e5147ff189 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -97,7 +97,7 @@ object FiltersPushed { class FilteredScanSuite extends DataSourceTest { - import caseInsensitiveContext._ + import caseInsensisitiveContext._ before { sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index d1d427e1790bd..50629ea4dc066 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils class InsertSuite extends DataSourceTest with BeforeAndAfterAll { - import caseInsensitiveContext._ + import caseInsensisitiveContext._ var path: File = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index c2bc52e2120c1..6a1ddf2f8e98b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -52,7 +52,7 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLCo } class PrunedScanSuite extends DataSourceTest { - import caseInsensitiveContext._ + import caseInsensisitiveContext._ before { sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 6567d1acd7644..cb287ba85c1f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { - import caseInsensitiveContext._ + import caseInsensisitiveContext._ var originalDefaultSource: String = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 77af04a491742..3b47b8adf313b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -88,7 +88,7 @@ case class AllDataTypesScan( } class TableScanSuite extends DataSourceTest { - import caseInsensitiveContext._ + import caseInsensisitiveContext._ var tableWithSchemaExpected = (1 to 10).map { i => Row( From 7552168ec458f18d8a13c5e7c56e949f8f23f7bc Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 9 May 2015 12:35:34 +0800 Subject: [PATCH 37/45] Fixes typo in MimaExclude.scala --- project/MimaExcludes.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index aa844c0dddfe5..a47e29e2ef365 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -94,23 +94,23 @@ object MimaExcludes { // This `protected[sql]` method was removed in 1.3.1 ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.sql.SQLContext.checkAnalysis"), - // These `private[sql]` class was removed in 1.4.0: + // These `private[sql]` class were removed in 1.4.0: ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.execution.AddExchange"), ProblemFilters.exclude[MissingClassProblem]( - "org.spache.spark.sql.execution.AddExchange$"), + "org.apache.spark.sql.execution.AddExchange$"), ProblemFilters.exclude[MissingClassProblem]( - "org.spache.spark.sql.parquet.PartitionSpec"), + "org.apache.spark.sql.parquet.PartitionSpec"), ProblemFilters.exclude[MissingClassProblem]( - "org.spache.spark.sql.parquet.PartitionSpec$"), + "org.apache.spark.sql.parquet.PartitionSpec$"), ProblemFilters.exclude[MissingClassProblem]( - "org.spache.spark.sql.parquet.Partition"), + "org.apache.spark.sql.parquet.Partition"), ProblemFilters.exclude[MissingClassProblem]( - "org.spache.spark.sql.parquet.Partition$"), + "org.apache.spark.sql.parquet.Partition$"), ProblemFilters.exclude[MissingClassProblem]( - "org.spache.spark.sql.parquet.ParquetRelation2$PartitionValues"), + "org.apache.spark.sql.parquet.ParquetRelation2$PartitionValues"), ProblemFilters.exclude[MissingClassProblem]( - "org.spache.spark.sql.parquet.ParquetRelation2$PartitionValues$"), + "org.apache.spark.sql.parquet.ParquetRelation2$PartitionValues$"), // These test support classes were moved out of src/main and into src/test: ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.parquet.ParquetTestData"), From c71ac6c277def9a0499a150f54cbfb385ae2937a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 10 May 2015 12:24:39 +0800 Subject: [PATCH 38/45] Addresses comments from @marmbrus --- .../scala/org/apache/spark/sql/SQLConf.scala | 3 ++ .../sql/sources/DataSourceStrategy.scala | 6 ++-- .../apache/spark/sql/sources/interfaces.scala | 29 +++++++++---------- .../org/apache/spark/sql/sources/rules.scala | 8 ++--- .../sql/sources/FSBasedRelationSuite.scala | 1 + 5 files changed, 23 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index a9f284a3803b5..f07bb196c11ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -238,6 +238,9 @@ private[sql] class SQLConf extends Serializable with CatalystConf { private[spark] def defaultDataSourceName: String = getConf(DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.parquet") + private[spark] def partitionDiscoveryEnabled() = + getConf(SQLConf.PARTITION_DISCOVERY_ENABLED, "true").toBoolean + // Do not use a value larger than 4000 as the default value of this property. // See the comments of SCHEMA_STRING_LENGTH_THRESHOLD above for more information. private[spark] def schemaStringLengthThreshold: Int = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 47f265b8db4a0..a5410cda0fe6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -364,13 +364,13 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { translate(child).map(sources.Not) case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringStartsWith(a.name, v.toString())) + Some(sources.StringStartsWith(a.name, v.toString)) case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringEndsWith(a.name, v.toString())) + Some(sources.StringEndsWith(a.name, v.toString)) case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringContains(a.name, v.toString())) + Some(sources.StringContains(a.name, v.toString)) case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 67f756c1c8fa3..c9b474e212318 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.sources +import scala.util.Try + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} @@ -24,9 +26,9 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.sql._ /** * ::DeveloperApi:: @@ -87,7 +89,7 @@ trait SchemaRelationProvider { * ::DeveloperApi:: * Implemented by objects that produce relations for a specific kind of data source * with a given schema and partitioned columns. When Spark SQL is given a DDL operation with a - * USING clause specified (to specify the implemented SchemaRelationProvider), a user defined + * USING clause specified (to specify the implemented [[FSBasedRelationProvider]]), a user defined * schema, and an optional list of partition columns, this interface is used to pass in the * parameters specified by a user. * @@ -114,7 +116,7 @@ trait FSBasedRelationProvider { sqlContext: SQLContext, schema: Option[StructType], partitionColumns: Option[StructType], - parameters: Map[String, String]): BaseRelation + parameters: Map[String, String]): FSBasedRelation } @DeveloperApi @@ -282,12 +284,13 @@ abstract class OutputWriter { * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before * the task output is committed. */ - def close(): Unit = () + def close(): Unit } /** * ::Experimental:: - * A [[BaseRelation]] that abstracts file system based data sources. + * A [[BaseRelation]] that provides much of the common code required for formats that store their + * data to an HDFS compatible filesystem. * * For the read path, similar to [[PrunedFilteredScan]], it can eliminate unneeded columns and * filter using selected predicates before producing an RDD containing all matching tuples as @@ -338,16 +341,13 @@ abstract class FSBasedRelation private[sql]( private var _partitionSpec: PartitionSpec = maybePartitionSpec.map { spec => spec.copy(partitionColumns = spec.partitionColumns.asNullable) }.getOrElse { - if (partitionDiscoverEnabled()) { + if (sqlContext.conf.partitionDiscoveryEnabled()) { discoverPartitions() } else { PartitionSpec(StructType(Nil), Array.empty[Partition]) } } - private def partitionDiscoverEnabled() = - sqlContext.conf.getConf(SQLConf.PARTITION_DISCOVERY_ENABLED, "true").toBoolean - private[sql] def partitionSpec: PartitionSpec = _partitionSpec /** @@ -356,7 +356,7 @@ abstract class FSBasedRelation private[sql]( def partitionColumns: StructType = partitionSpec.partitionColumns private[sql] def refresh(): Unit = { - if (partitionDiscoverEnabled()) { + if (sqlContext.conf.partitionDiscoveryEnabled()) { _partitionSpec = discoverPartitions() } } @@ -365,11 +365,10 @@ abstract class FSBasedRelation private[sql]( val basePaths = paths.map(new Path(_)) val leafDirs = basePaths.flatMap { path => val fs = path.getFileSystem(hadoopConf) - if (fs.exists(path)) { - SparkHadoopUtil.get.listLeafDirStatuses(fs, fs.makeQualified(path)) - } else { - Seq.empty[FileStatus] - } + Try(fs.getFileStatus(path.makeQualified(fs.getUri, fs.getWorkingDirectory))) + .filter(_.isDir) + .map(SparkHadoopUtil.get.listLeafDirStatuses(fs, _)) + .getOrElse(Seq.empty[FileStatus]) }.map(_.getPath) if (leafDirs.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index a5183909d7440..aad1d248d0a28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -101,12 +101,8 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } } - case logical.InsertIntoTable(LogicalRelation(_: InsertableRelation), _, _, _, _) => - // OK - - case logical.InsertIntoTable(LogicalRelation(_: FSBasedRelation), _, _, _, _) => - // OK - + case logical.InsertIntoTable(LogicalRelation(_: InsertableRelation), _, _, _, _) => // OK + case logical.InsertIntoTable(LogicalRelation(_: FSBasedRelation), _, _, _, _) => // OK case logical.InsertIntoTable(l: LogicalRelation, _, _, _, _) => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index f2d42a7c3d67c..3768119020890 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.parquet.ParquetTest import org.apache.spark.sql.types._ +// TODO Don't extend ParquetTest // This test suite extends ParquetTest for some convenient utility methods. These methods should be // moved to some more general places, maybe QueryTest. class FSBasedRelationSuite extends QueryTest with ParquetTest { From 8d12e6930f630a5dd0646b1e309702a71a9815f9 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 10 May 2015 15:58:47 +0800 Subject: [PATCH 39/45] Fixes compilation error --- sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala | 2 +- .../org/apache/spark/sql/sources/SimpleTextRelation.scala | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 182f6122a34f7..fbaac43f974e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -287,7 +287,7 @@ private[sql] object ResolvedDataSource { options) sqlContext.executePlan( InsertIntoFSBasedRelation( - r.asInstanceOf[FSBasedRelation], + r, data.logicalPlan, partitionColumns.toArray, mode)).toRdd diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index f29dbc4f97a32..625a602bb8abb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -27,7 +27,6 @@ import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputForma import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types.{DataType, StructField, StructType} import org.apache.spark.sql.{Row, SQLContext} @@ -40,7 +39,7 @@ class SimpleTextSource extends FSBasedRelationProvider { sqlContext: SQLContext, schema: Option[StructType], partitionColumns: Option[StructType], - parameters: Map[String, String]): BaseRelation = { + parameters: Map[String, String]): FSBasedRelation = { new SimpleTextRelation( Array(parameters("path")), schema, partitionColumns.getOrElse(StructType(Array.empty[StructField])), parameters)(sqlContext) From ad4d4de47d4aba2660cbcf9239942803818e8d3c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 10 May 2015 16:55:47 +0800 Subject: [PATCH 40/45] Enables HDFS style globbing --- .../apache/spark/deploy/SparkHadoopUtil.scala | 7 ++++ .../org/apache/spark/sql/sources/ddl.scala | 34 ++++++++++++++-- .../apache/spark/sql/sources/interfaces.scala | 1 + .../sql/sources/FSBasedRelationSuite.scala | 39 +++++++++++++++++++ .../sql/sources/SimpleTextRelation.scala | 6 +-- 5 files changed, 80 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 31b07998d8630..7fa75ac8c2b54 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -231,6 +231,13 @@ class SparkHadoopUtil extends Logging { recurse(baseStatus) } + def globPath(pattern: Path): Seq[Path] = { + val fs = pattern.getFileSystem(conf) + Option(fs.globStatus(pattern)).map { statuses => + statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toSeq + }.getOrElse(Seq.empty[Path]) + } + /** * Lists all the files in a directory with the specified prefix, and does not end with the * given suffix. The returned {{FileStatus}} instances are sorted by the modification times of diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index fbaac43f974e9..74fac50a770a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -20,7 +20,10 @@ package org.apache.spark.sql.sources import scala.language.{existentials, implicitConversions} import scala.util.matching.Regex +import org.apache.hadoop.fs.Path + import org.apache.spark.Logging +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} @@ -157,7 +160,7 @@ private[sql] class DDLParser( protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")} override implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( - s"identifier matching regex ${regex}", { + s"identifier matching regex $regex", { case lexical.Identifier(str) if regex.unapplySeq(str).isDefined => str case lexical.Keyword(str) if regex.unapplySeq(str).isDefined => str } @@ -230,11 +233,18 @@ private[sql] object ResolvedDataSource { Some(partitionColumnsSchema(schema, partitionColumns)) } + val caseInsensitiveOptions= new CaseInsensitiveMap(options) + val paths = { + val patternPath = new Path(caseInsensitiveOptions("path")) + SparkHadoopUtil.get.globPath(patternPath).map(_.toString).toArray + } + dataSource.createRelation( sqlContext, + paths, Some(schema), maybePartitionsSchema, - new CaseInsensitiveMap(options)) + caseInsensitiveOptions) case dataSource: org.apache.spark.sql.sources.RelationProvider => throw new AnalysisException(s"$className does not allow user-specified schemas.") case _ => @@ -245,7 +255,12 @@ private[sql] object ResolvedDataSource { case dataSource: RelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) case dataSource: FSBasedRelationProvider => - dataSource.createRelation(sqlContext, None, None, new CaseInsensitiveMap(options)) + val caseInsensitiveOptions = new CaseInsensitiveMap(options) + val paths = { + val patternPath = new Path(caseInsensitiveOptions("path")) + SparkHadoopUtil.get.globPath(patternPath).map(_.toString).toArray + } + dataSource.createRelation(sqlContext, paths, None, None, caseInsensitiveOptions) case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => throw new AnalysisException( s"A schema needs to be specified when using $className.") @@ -280,11 +295,22 @@ private[sql] object ResolvedDataSource { case dataSource: CreatableRelationProvider => dataSource.createRelation(sqlContext, mode, options, data) case dataSource: FSBasedRelationProvider => + // Don't glob path for the write path. The contracts here are: + // 1. Only one output path can be specified on the write path; + // 2. Output path must be a legal HDFS style file system path; + // 3. It's OK that the output path doesn't exist yet; + val caseInsensitiveOptions = new CaseInsensitiveMap(options) + val outputPath = { + val path = new Path(caseInsensitiveOptions("path")) + val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + path.makeQualified(fs.getUri, fs.getWorkingDirectory) + } val r = dataSource.createRelation( sqlContext, + Array(outputPath.toString), Some(data.schema), Some(partitionColumnsSchema(data.schema, partitionColumns)), - options) + caseInsensitiveOptions) sqlContext.executePlan( InsertIntoFSBasedRelation( r, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index c9b474e212318..0c59bf33bc1a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -114,6 +114,7 @@ trait FSBasedRelationProvider { */ def createRelation( sqlContext: SQLContext, + paths: Array[String], schema: Option[StructType], partitionColumns: Option[StructType], parameters: Map[String, String]): FSBasedRelation diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala index 3768119020890..415b1cd168848 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala @@ -483,4 +483,43 @@ class FSBasedRelationSuite extends QueryTest with ParquetTest { assert(table("t").collect().isEmpty) } } + + test("Hadoop style globbing") { + withTempPath { file => + partitionedTestDF.save( + source = classOf[SimpleTextSource].getCanonicalName, + mode = SaveMode.Overwrite, + options = Map("path" -> file.getCanonicalPath), + partitionColumns = Seq("p1", "p2")) + + val df = load( + source = classOf[SimpleTextSource].getCanonicalName, + options = Map( + "path" -> s"${file.getCanonicalPath}/p1=*/p2=???", + "dataSchema" -> dataSchema.json)) + + val expectedPaths = Set( + s"${file.getCanonicalFile}/p1=1/p2=foo", + s"${file.getCanonicalFile}/p1=2/p2=foo", + s"${file.getCanonicalFile}/p1=1/p2=bar", + s"${file.getCanonicalFile}/p1=2/p2=bar" + ).map { p => + val path = new Path(p) + val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + path.makeQualified(fs.getUri, fs.getWorkingDirectory).toString + } + + println(df.queryExecution) + + val actualPaths = df.queryExecution.analyzed.collectFirst { + case LogicalRelation(relation: FSBasedRelation) => + relation.paths.toSet + }.getOrElse { + fail("Expect an FSBasedRelation, but none could be found") + } + + assert(actualPaths === expectedPaths) + checkAnswer(df, partitionedTestDF.collect()) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 625a602bb8abb..4d9b4c01c66cd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -37,12 +37,12 @@ import org.apache.spark.sql.{Row, SQLContext} class SimpleTextSource extends FSBasedRelationProvider { override def createRelation( sqlContext: SQLContext, + paths: Array[String], schema: Option[StructType], partitionColumns: Option[StructType], parameters: Map[String, String]): FSBasedRelation = { - new SimpleTextRelation( - Array(parameters("path")), schema, - partitionColumns.getOrElse(StructType(Array.empty[StructField])), parameters)(sqlContext) + val partitionsSchema = partitionColumns.getOrElse(StructType(Array.empty[StructField])) + new SimpleTextRelation(paths, schema, partitionsSchema, parameters)(sqlContext) } } From 348a922273ba4720e954b73418c338b4fb9204f2 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 10 May 2015 17:45:02 +0800 Subject: [PATCH 41/45] Adds projection in FSBasedRelation.buildScan(requiredColumns, inputPaths) --- .../apache/spark/sql/sources/interfaces.scala | 27 +++++++++++++-- .../sql/sources/SimpleTextRelation.scala | 34 ++++++++++++------- 2 files changed, 46 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 0c59bf33bc1a6..c3676cc56e7ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -26,8 +26,9 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.{Row, _} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.types.{StructField, StructType} /** @@ -339,6 +340,8 @@ abstract class FSBasedRelation private[sql]( private val hadoopConf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) + private val codegenEnabled = sqlContext.conf.codegenEnabled + private var _partitionSpec: PartitionSpec = maybePartitionSpec.map { spec => spec.copy(partitionColumns = spec.partitionColumns.asNullable) }.getOrElse { @@ -421,7 +424,25 @@ abstract class FSBasedRelation private[sql]( * selected partition. */ def buildScan(requiredColumns: Array[String], inputPaths: Array[String]): RDD[Row] = { - buildScan(inputPaths) + // Yeah, to workaround serialization... + val dataSchema = this.dataSchema + val codegenEnabled = this.codegenEnabled + + val requiredOutput = requiredColumns.map { col => + val field = dataSchema(col) + BoundReference(dataSchema.fieldIndex(col), field.dataType, field.nullable) + }.toSeq + + val buildProjection = if (codegenEnabled) { + GenerateMutableProjection.generate(requiredOutput, dataSchema.toAttributes) + } else { + () => new InterpretedMutableProjection(requiredOutput, dataSchema.toAttributes) + } + + buildScan(inputPaths).mapPartitions { rows => + val mutableProjection = buildProjection() + rows.map(mutableProjection) + } } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 4d9b4c01c66cd..6a5e7fbf35d69 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -113,19 +113,29 @@ class SimpleTextRelation( override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleTextOutputWriter] - override def buildScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputPaths: Array[String]): RDD[Row] = { - val fields = dataSchema.fields - sparkContext.textFile(inputPaths.mkString(",")).map { record => - val valueMap = record.split(",").zip(fields).map { - case (value, StructField(name, dataType, _, _)) => - name -> Cast(Literal(value), dataType).eval() - }.toMap + override def buildScan(inputPaths: Array[String]): RDD[Row] = { + val fields = dataSchema.map(_.dataType) - // This mocks a simple projection - Row.fromSeq(requiredColumns.map(valueMap).toSeq) + sparkContext.textFile(inputPaths.mkString(",")).map { record => + Row(record.split(",").zip(fields).map { case (value, dataType) => + Cast(Literal(value), dataType).eval() + }: _*) } } + +// override def buildScan( +// requiredColumns: Array[String], +// filters: Array[Filter], +// inputPaths: Array[String]): RDD[Row] = { +// val fields = dataSchema.fields +// sparkContext.textFile(inputPaths.mkString(",")).map { record => +// val valueMap = record.split(",").zip(fields).map { +// case (value, StructField(name, dataType, _, _)) => +// name -> Cast(Literal(value), dataType).eval() +// }.toMap +// +// // This mocks a simple projection +// Row.fromSeq(requiredColumns.map(valueMap).toSeq) +// } +// } } From edf49e7a0e779badd72e28e9533461ca0e53568b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 10 May 2015 17:52:51 +0800 Subject: [PATCH 42/45] Removed commented stale code block --- .../spark/sql/sources/SimpleTextRelation.scala | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 6a5e7fbf35d69..8801aba2f64c3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -122,20 +122,4 @@ class SimpleTextRelation( }: _*) } } - -// override def buildScan( -// requiredColumns: Array[String], -// filters: Array[Filter], -// inputPaths: Array[String]): RDD[Row] = { -// val fields = dataSchema.fields -// sparkContext.textFile(inputPaths.mkString(",")).map { record => -// val valueMap = record.split(",").zip(fields).map { -// case (value, StructField(name, dataType, _, _)) => -// name -> Cast(Literal(value), dataType).eval() -// }.toMap -// -// // This mocks a simple projection -// Row.fromSeq(requiredColumns.map(valueMap).toSeq) -// } -// } } From 43ba50efd0242c015f7bcbb97f98bc198dabe154 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 11 May 2015 11:16:37 +0800 Subject: [PATCH 43/45] Avoids serializing generated projection code --- .../org/apache/spark/sql/sources/interfaces.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index c3676cc56e7ae..5e010d21120f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -433,13 +433,13 @@ abstract class FSBasedRelation private[sql]( BoundReference(dataSchema.fieldIndex(col), field.dataType, field.nullable) }.toSeq - val buildProjection = if (codegenEnabled) { - GenerateMutableProjection.generate(requiredOutput, dataSchema.toAttributes) - } else { - () => new InterpretedMutableProjection(requiredOutput, dataSchema.toAttributes) - } - buildScan(inputPaths).mapPartitions { rows => + val buildProjection = if (codegenEnabled) { + GenerateMutableProjection.generate(requiredOutput, dataSchema.toAttributes) + } else { + () => new InterpretedMutableProjection(requiredOutput, dataSchema.toAttributes) + } + val mutableProjection = buildProjection() rows.map(mutableProjection) } From 1f9b1a54c50888c81236367215385c5f6772c522 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 11 May 2015 20:39:32 +0800 Subject: [PATCH 44/45] Tweaks data schema passed to FSBasedRelations - Only data columns should be passed - Data schema should be converted to nullable version (see SPARK-5950 and PR #4826) --- .../org/apache/spark/sql/sources/commands.scala | 5 +++-- .../org/apache/spark/sql/sources/ddl.scala | 7 +++++-- .../spark/sql/hive/execution/commands.scala | 17 ++++++++--------- 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 9b07cb562e776..127133bfaf662 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -197,10 +197,11 @@ private[sql] case class InsertIntoFSBasedRelation( if (needsConversion) { val converter = CatalystTypeConverters.createToScalaConverter(dataSchema) while (iterator.hasNext) { - val row = converter(iterator.next()).asInstanceOf[Row] + val row = iterator.next() val partitionPart = partitionProj(row) val dataPart = dataProj(row) - writerContainer.outputWriterForRow(partitionPart).write(dataPart) + val convertedDataPart = converter(dataPart).asInstanceOf[Row] + writerContainer.outputWriterForRow(partitionPart).write(convertedDataPart) } } else { while (iterator.hasNext) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 74fac50a770a0..595c5eb40e295 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -239,6 +239,8 @@ private[sql] object ResolvedDataSource { SparkHadoopUtil.get.globPath(patternPath).map(_.toString).toArray } + val dataSchema = StructType(schema.filterNot(f => partitionColumns.contains(f.name))) + dataSource.createRelation( sqlContext, paths, @@ -279,7 +281,7 @@ private[sql] object ResolvedDataSource { schema.find(_.name == col).getOrElse { throw new RuntimeException(s"Partition column $col not found in schema $schema") } - }) + }).asNullable } /** Create a [[ResolvedDataSource]] for saving the content of the given [[DataFrame]]. */ @@ -305,10 +307,11 @@ private[sql] object ResolvedDataSource { val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) path.makeQualified(fs.getUri, fs.getWorkingDirectory) } + val dataSchema = StructType(data.schema.filterNot(f => partitionColumns.contains(f.name))) val r = dataSource.createRelation( sqlContext, Array(outputPath.toString), - Some(data.schema), + Some(dataSchema.asNullable), Some(partitionColumnsSchema(data.schema, partitionColumns)), caseInsensitiveOptions) sqlContext.executePlan( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 70f61c5656562..8e405e080489f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -191,7 +191,7 @@ case class CreateMetastoreDataSourceAsSelect( case SaveMode.Append => // Check if the specified data source match the data source of the existing table. val resolved = ResolvedDataSource( - sqlContext, Some(query.schema), partitionColumns, provider, optionsWithPath) + sqlContext, Some(query.schema.asNullable), partitionColumns, provider, optionsWithPath) val createdRelation = LogicalRelation(resolved.relation) EliminateSubQueries(sqlContext.table(tableName).logicalPlan) match { case l @ LogicalRelation(_: InsertableRelation | _: FSBasedRelation) => @@ -203,14 +203,13 @@ case class CreateMetastoreDataSourceAsSelect( s"table $tableName and using its data source and options." val errorMessage = s""" - |$errorDescription - |== Relations == - |${sideBySide( - s"== Expected Relation ==" :: - l.toString :: Nil, - s"== Actual Relation ==" :: - createdRelation.toString :: Nil).mkString("\n")} - """.stripMargin + |$errorDescription + |== Relations == + |${sideBySide( + s"== Expected Relation ==" :: l.toString :: Nil, + s"== Actual Relation ==" :: createdRelation.toString :: Nil + ).mkString("\n")} + """.stripMargin throw new AnalysisException(errorMessage) } existingSchema = Some(l.schema) From 5351a1b7da2bcefe310e053c280b6caa476fb148 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 12 May 2015 09:27:11 +0800 Subject: [PATCH 45/45] Fixes compilation error introduced while rebasing --- .../apache/spark/sql/sources/CreateTableAsSelectSuite.scala | 6 +++--- .../scala/org/apache/spark/sql/sources/DDLTestSuite.scala | 2 +- .../org/apache/spark/sql/sources/FilteredScanSuite.scala | 2 +- .../scala/org/apache/spark/sql/sources/InsertSuite.scala | 2 +- .../org/apache/spark/sql/sources/PrunedScanSuite.scala | 2 +- .../scala/org/apache/spark/sql/sources/SaveLoadSuite.scala | 2 +- .../scala/org/apache/spark/sql/sources/TableScanSuite.scala | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 6b1aa7e62ea5a..4e54b2eb8df7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -17,16 +17,16 @@ package org.apache.spark.sql.sources -import java.io.{IOException, File} +import java.io.{File, IOException} -import org.apache.spark.sql.{SQLContext, AnalysisException} import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.AnalysisException import org.apache.spark.util.Utils class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { - import caseInsensisitiveContext._ + import caseInsensitiveContext._ var path: File = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index ca25751b9583d..6664e8d64c13a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -64,7 +64,7 @@ case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlCo } class DDLTestSuite extends DataSourceTest { - import caseInsensisitiveContext._ + import caseInsensitiveContext._ before { sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index cb5e5147ff189..cce747e7dbf64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -97,7 +97,7 @@ object FiltersPushed { class FilteredScanSuite extends DataSourceTest { - import caseInsensisitiveContext._ + import caseInsensitiveContext._ before { sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 50629ea4dc066..d1d427e1790bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils class InsertSuite extends DataSourceTest with BeforeAndAfterAll { - import caseInsensisitiveContext._ + import caseInsensitiveContext._ var path: File = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index 6a1ddf2f8e98b..c2bc52e2120c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -52,7 +52,7 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLCo } class PrunedScanSuite extends DataSourceTest { - import caseInsensisitiveContext._ + import caseInsensitiveContext._ before { sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index cb287ba85c1f8..6567d1acd7644 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { - import caseInsensisitiveContext._ + import caseInsensitiveContext._ var originalDefaultSource: String = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 3b47b8adf313b..77af04a491742 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -88,7 +88,7 @@ case class AllDataTypesScan( } class TableScanSuite extends DataSourceTest { - import caseInsensisitiveContext._ + import caseInsensitiveContext._ var tableWithSchemaExpected = (1 to 10).map { i => Row(