aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19107][SQL] support creating hive table with DataFrameWriter and CatalogWenchen Fan2017-01-106-40/+93
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? After unifying the CREATE TABLE syntax in https://github.com/apache/spark/pull/16296, it's pretty easy to support creating hive table with `DataFrameWriter` and `Catalog` now. This PR basically just removes the hive provider check in `DataFrameWriter.saveAsTable` and `Catalog.createExternalTable`, and add tests. ## How was this patch tested? new tests in `HiveDDLSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #16487 from cloud-fan/hive-table.
* [SPARK-18952] Regex strings not properly escaped in codegen for aggregationsBurak Yavuz2017-01-093-10/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? If I use the function regexp_extract, and then in my regex string, use `\`, i.e. escape character, this fails codegen, because the `\` character is not properly escaped when codegen'd. Example stack trace: ``` /* 059 */ private int maxSteps = 2; /* 060 */ private int numRows = 0; /* 061 */ private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add("date_format(window#325.start, yyyy-MM-dd HH:mm)", org.apache.spark.sql.types.DataTypes.StringType) /* 062 */ .add("regexp_extract(source#310.description, ([a-zA-Z]+)\[.*, 1)", org.apache.spark.sql.types.DataTypes.StringType); /* 063 */ private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add("sum", org.apache.spark.sql.types.DataTypes.LongType); /* 064 */ private Object emptyVBase; ... org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 62, Column 58: Invalid escape sequence at org.codehaus.janino.Scanner.scanLiteralCharacter(Scanner.java:918) at org.codehaus.janino.Scanner.produce(Scanner.java:604) at org.codehaus.janino.Parser.peekRead(Parser.java:3239) at org.codehaus.janino.Parser.parseArguments(Parser.java:3055) at org.codehaus.janino.Parser.parseSelector(Parser.java:2914) at org.codehaus.janino.Parser.parseUnaryExpression(Parser.java:2617) at org.codehaus.janino.Parser.parseMultiplicativeExpression(Parser.java:2573) at org.codehaus.janino.Parser.parseAdditiveExpression(Parser.java:2552) ``` In the codegend expression, the literal should use `\\` instead of `\` A similar problem was solved here: https://github.com/apache/spark/pull/15156. ## How was this patch tested? Regression test in `DataFrameAggregationSuite` Author: Burak Yavuz <brkyvz@gmail.com> Closes #16361 from brkyvz/reg-break.
* [SPARK-19020][SQL] Cardinality estimation of aggregate operatorZhenhua Wang2017-01-094-3/+198
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Support cardinality estimation of aggregate operator ## How was this patch tested? Add test cases Author: Zhenhua Wang <wzh_zju@163.com> Author: wangzhenhua <wangzhenhua@huawei.com> Closes #16431 from wzhfy/aggEstimation.
* [SPARK-17077][SQL] Cardinality estimation for project operatorZhenhua Wang2017-01-086-0/+196
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Support cardinality estimation for project operator. ## How was this patch tested? Add a test suite and a base class in the catalyst package. Author: Zhenhua Wang <wzh_zju@163.com> Closes #16430 from wzhfy/projectEstimation.
* [SPARK-19126][DOCS] Update Join Documentation Across Languagesanabranch2017-01-081-4/+12
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - [X] Make sure all join types are clearly mentioned - [X] Make join labeling/style consistent - [X] Make join label ordering docs the same - [X] Improve join documentation according to above for Scala - [X] Improve join documentation according to above for Python - [X] Improve join documentation according to above for R ## How was this patch tested? No tests b/c docs. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Closes #16504 from anabranch/SPARK-19126.
* [SPARK-19127][DOCS] Update Rank Function Documentationanabranch2017-01-081-6/+10
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - [X] Fix inconsistencies in function reference for dense rank and dense - [X] Make all languages equivalent in their reference to `dense_rank` and `rank`. ## How was this patch tested? N/A for docs. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Closes #16505 from anabranch/SPARK-19127.
* [SPARK-19093][SQL] Cached tables are not used in SubqueryExpressionDilip Biswal2017-01-082-1/+78
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Consider the plans inside subquery expressions while looking up cache manager to make use of cached data. Currently CacheManager.useCachedData does not consider the subquery expressions in the plan. SQL ``` select * from rows where not exists (select * from rows) ``` Before the fix ``` == Optimized Logical Plan == Join LeftAnti :- InMemoryRelation [_1#3775, _2#3776], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) : +- *FileScan parquet [_1#3775,_2#3776] Batched: true, Format: Parquet, Location: InMemoryFileIndex[dbfs:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string> +- Project [_1#3775 AS _1#3775#4001, _2#3776 AS _2#3776#4002] +- Relation[_1#3775,_2#3776] parquet ``` After ``` == Optimized Logical Plan == Join LeftAnti :- InMemoryRelation [_1#256, _2#257], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) : +- *FileScan parquet [_1#256,_2#257] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string> +- Project [_1#256 AS _1#256#298, _2#257 AS _2#257#299] +- InMemoryRelation [_1#256, _2#257], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) +- *FileScan parquet [_1#256,_2#257] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string> ``` Query2 ``` SELECT * FROM t1 WHERE c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1)) ``` Before ``` == Analyzed Logical Plan == c1: int Project [c1#3] +- Filter predicate-subquery#47 [(c1#3 = c1#10)] : +- Project [c1#10] : +- Filter predicate-subquery#46 [(c1#10 = c1#17)] : : +- Project [c1#17] : : +- Filter (c1#17 = 1) : : +- SubqueryAlias t3, `t3` : : +- Project [value#15 AS c1#17] : : +- LocalRelation [value#15] : +- SubqueryAlias t2, `t2` : +- Project [value#8 AS c1#10] : +- LocalRelation [value#8] +- SubqueryAlias t1, `t1` +- Project [value#1 AS c1#3] +- LocalRelation [value#1] == Optimized Logical Plan == Join LeftSemi, (c1#3 = c1#10) :- InMemoryRelation [c1#3], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1 : +- LocalTableScan [c1#3] +- Project [value#8 AS c1#10] +- Join LeftSemi, (value#8 = c1#17) :- LocalRelation [value#8] +- Project [value#15 AS c1#17] +- Filter (value#15 = 1) +- LocalRelation [value#15] ``` After ``` == Analyzed Logical Plan == c1: int Project [c1#3] +- Filter predicate-subquery#47 [(c1#3 = c1#10)] : +- Project [c1#10] : +- Filter predicate-subquery#46 [(c1#10 = c1#17)] : : +- Project [c1#17] : : +- Filter (c1#17 = 1) : : +- SubqueryAlias t3, `t3` : : +- Project [value#15 AS c1#17] : : +- LocalRelation [value#15] : +- SubqueryAlias t2, `t2` : +- Project [value#8 AS c1#10] : +- LocalRelation [value#8] +- SubqueryAlias t1, `t1` +- Project [value#1 AS c1#3] +- LocalRelation [value#1] == Optimized Logical Plan == Join LeftSemi, (c1#3 = c1#10) :- InMemoryRelation [c1#3], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1 : +- LocalTableScan [c1#3] +- Join LeftSemi, (c1#10 = c1#17) :- InMemoryRelation [c1#10], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t2 : +- LocalTableScan [c1#10] +- Filter (c1#17 = 1) +- InMemoryRelation [c1#17], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1 +- LocalTableScan [c1#3] ``` ## How was this patch tested? Added new tests in CachedTableSuite. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #16493 from dilipbiswal/SPARK-19093.
* [SPARK-19085][SQL] cleanup OutputWriterFactory and OutputWriterWenchen Fan2017-01-0811-63/+27
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `OutputWriterFactory`/`OutputWriter` are internal interfaces and we can remove some unnecessary APIs: 1. `OutputWriterFactory.newWriter(path: String)`: no one calls it and no one implements it. 2. `OutputWriter.write(row: Row)`: during execution we only call `writeInternal`, which is weird as `OutputWriter` is already an internal interface. We should rename `writeInternal` to `write` and remove `def write(row: Row)` and it's related converter code. All implementations should just implement `def write(row: InternalRow)` ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #16479 from cloud-fan/hive-writer.
* [SPARK-19074][SS][DOCS] Updated Structured Streaming Programming Guide for ↵Tathagata Das2017-01-061-4/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | update mode and source/sink options ## What changes were proposed in this pull request? Updates - Updated Late Data Handling section by adding a figure for Update Mode. Its more intuitive to explain late data handling with Update Mode, so I added the new figure before the Append Mode figure. - Updated Output Modes section with Update mode - Added options for all the sources and sinks --------------------------- --------------------------- ![image](https://cloud.githubusercontent.com/assets/663212/21665176/f150b224-d29f-11e6-8372-14d32da21db9.png) --------------------------- --------------------------- <img width="931" alt="screen shot 2017-01-03 at 6 09 11 pm" src="https://cloud.githubusercontent.com/assets/663212/21629740/d21c9bb8-d1df-11e6-915b-488a59589fa6.png"> <img width="933" alt="screen shot 2017-01-03 at 6 10 00 pm" src="https://cloud.githubusercontent.com/assets/663212/21629749/e22bdabe-d1df-11e6-86d3-7e51d2f28dbc.png"> --------------------------- --------------------------- ![image](https://cloud.githubusercontent.com/assets/663212/21665200/108e18fc-d2a0-11e6-8640-af598cab090b.png) ![image](https://cloud.githubusercontent.com/assets/663212/21665148/cfe414fa-d29f-11e6-9baa-4124ccbab093.png) ![image](https://cloud.githubusercontent.com/assets/663212/21665226/2e8f39e4-d2a0-11e6-85b1-7657e2df5491.png) Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16468 from tdas/SPARK-19074.
* [SPARK-16792][SQL] Dataset containing a Case Class with a List type causes a ↵Michal Senkyr2017-01-064-22/+231
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | CompileException (converting sequence to list) ## What changes were proposed in this pull request? Added a `to` call at the end of the code generated by `ScalaReflection.deserializerFor` if the requested type is not a supertype of `WrappedArray[_]` that uses `CanBuildFrom[_, _, _]` to convert result into an arbitrary subtype of `Seq[_]`. Care was taken to preserve the original deserialization where it is possible to avoid the overhead of conversion in cases where it is not needed `ScalaReflection.serializerFor` could already be used to serialize any `Seq[_]` so it was not altered `SQLImplicits` had to be altered and new implicit encoders added to permit serialization of other sequence types Also fixes [SPARK-16815] Dataset[List[T]] leads to ArrayStoreException ## How was this patch tested? ```bash ./build/mvn -DskipTests clean package && ./dev/run-tests ``` Also manual execution of the following sets of commands in the Spark shell: ```scala case class TestCC(key: Int, letters: List[String]) val ds1 = sc.makeRDD(Seq( (List("D")), (List("S","H")), (List("F","H")), (List("D","L","L")) )).map(x=>(x.length,x)).toDF("key","letters").as[TestCC] val test1=ds1.map{_.key} test1.show ``` ```scala case class X(l: List[String]) spark.createDataset(Seq(List("A"))).map(X).show ``` ```scala spark.sqlContext.createDataset(sc.parallelize(List(1) :: Nil)).collect ``` After adding arbitrary sequence support also tested with the following commands: ```scala case class QueueClass(q: scala.collection.immutable.Queue[Int]) spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect ``` Author: Michal Senkyr <mike.senkyr@gmail.com> Closes #16240 from michalsenkyr/sql-caseclass-list-fix.
* [SPARK-18871][SQL] New test cases for IN/NOT IN subqueryKevin Yu2017-01-054-0/+884
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR extends the existing IN/NOT IN subquery test cases coverage, adds more test cases to the IN subquery test suite. Based on the discussion, we will create `subquery/in-subquery` sub structure under `sql/core/src/test/resources/sql-tests/inputs` directory. This is the high level grouping for IN subquery: `subquery/in-subquery/` `subquery/in-subquery/simple-in.sql` `subquery/in-subquery/in-group-by.sql (in parent side, subquery, and both)` `subquery/in-subquery/not-in-group-by.sql` `subquery/in-subquery/in-order-by.sql` `subquery/in-subquery/in-limit.sql` `subquery/in-subquery/in-having.sql` `subquery/in-subquery/in-joins.sql` `subquery/in-subquery/not-in-joins.sql` `subquery/in-subquery/in-set-operations.sql` `subquery/in-subquery/in-with-cte.sql` `subquery/in-subquery/not-in-with-cte.sql` subquery/in-subquery/in-multiple-columns.sql` We will deliver it through multiple prs, this is the first pr for the IN subquery, it has `subquery/in-subquery/simple-in.sql` `subquery/in-subquery/in-group-by.sql (in parent side, subquery, and both)` These are the results from running on DB2. [Modified test file of in-group-by.sql used to run on DB2](https://github.com/apache/spark/files/683367/in-group-by.sql.db2.txt) [Output of the run result on DB2](https://github.com/apache/spark/files/683362/in-group-by.sql.db2.out.txt) [Modified test file of simple-in.sql used to run on DB2](https://github.com/apache/spark/files/683378/simple-in.sql.db2.txt) [Output of the run result on DB2](https://github.com/apache/spark/files/683379/simple-in.sql.db2.out.txt) ## How was this patch tested? This patch is adding tests. Author: Kevin Yu <qyu@us.ibm.com> Closes #16337 from kevinyu98/spark-18871.
* [SPARK-18885][SQL] unify CREATE TABLE syntax for data source and hive serde ↵Wenchen Fan2017-01-0519-131/+470
| | | | | | | | | | | | | | | | | | | | | | | tables ## What changes were proposed in this pull request? Today we have different syntax to create data source or hive serde tables, we should unify them to not confuse users and step forward to make hive a data source. Please read https://issues.apache.org/jira/secure/attachment/12843835/CREATE-TABLE.pdf for details. TODO(for follow-up PRs): 1. TBLPROPERTIES is not added to the new syntax, we should decide if we wanna add it later. 2. `SHOW CREATE TABLE` should be updated to use the new syntax. 3. we should decide if we wanna change the behavior of `SET LOCATION`. ## How was this patch tested? new tests Author: Wenchen Fan <wenchen@databricks.com> Closes #16296 from cloud-fan/create-table.
* [SPARK-19058][SQL] fix partition related behaviors with ↵Wenchen Fan2017-01-057-112/+142
| | | | | | | | | | | | | | | | | | | | DataFrameWriter.saveAsTable ## What changes were proposed in this pull request? When we append data to a partitioned table with `DataFrameWriter.saveAsTable`, there are 2 issues: 1. doesn't work when the partition has custom location. 2. will recover all partitions This PR fixes them by moving the special partition handling code from `DataSourceAnalysis` to `InsertIntoHadoopFsRelationCommand`, so that the `DataFrameWriter.saveAsTable` code path can also benefit from it. ## How was this patch tested? newly added regression tests Author: Wenchen Fan <wenchen@databricks.com> Closes #16460 from cloud-fan/append.
* [SPARK-19070] Clean-up dataset actionsHerman van Hovell2017-01-041-50/+26
| | | | | | | | | | | | ## What changes were proposed in this pull request? Dataset actions currently spin off a new `Dataframe` only to track query execution. This PR simplifies this code path by using the `Dataset.queryExecution` directly. This PR also merges the typed and untyped action evaluation paths. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16466 from hvanhovell/SPARK-19070.
* [MINOR][DOCS] Remove consecutive duplicated words/typo in Spark RepoNiranjan Padmanabhan2017-01-0417-19/+19
| | | | | | | | | | | | ## What changes were proposed in this pull request? There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words. ## How was this patch tested? N/A since only docs or comments were updated. Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com> Closes #16455 from neurons/np.structure_streaming_doc.
* [SPARK-19060][SQL] remove the supportsPartial flag in AggregateFunctionWenchen Fan2017-01-047-48/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Now all aggregation functions support partial aggregate, we can remove the `supportsPartual` flag in `AggregateFunction` ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #16461 from cloud-fan/partial.
* [SPARK-19072][SQL] codegen of Literal should not output boxed valueWenchen Fan2017-01-032-11/+24
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In https://github.com/apache/spark/pull/16402 we made a mistake that, when double/float is infinity, the `Literal` codegen will output boxed value and cause wrong result. This PR fixes this by special handling infinity to not output boxed value. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16469 from cloud-fan/literal.
* [SPARK-19048][SQL] Delete Partition Location when Dropping Managed ↵gatorsmile2017-01-034-9/+113
| | | | | | | | | | | | | | | | Partitioned Tables in InMemoryCatalog ### What changes were proposed in this pull request? The data in the managed table should be deleted after table is dropped. However, if the partition location is not under the location of the partitioned table, it is not deleted as expected. Users can specify any location for the partition when they adding a partition. This PR is to delete partition location when dropping managed partitioned tables stored in `InMemoryCatalog`. ### How was this patch tested? Added test cases for both HiveExternalCatalog and InMemoryCatalog Author: gatorsmile <gatorsmile@gmail.com> Closes #16448 from gatorsmile/unsetSerdeProp.
* [SPARK-18877][SQL] `CSVInferSchema.inferField` on DecimalType should find a ↵Dongjoon Hyun2017-01-032-1/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | common type with `typeSoFar` ## What changes were proposed in this pull request? CSV type inferencing causes `IllegalArgumentException` on decimal numbers with heterogeneous precisions and scales because the current logic uses the last decimal type in a **partition**. Specifically, `inferRowType`, the **seqOp** of **aggregate**, returns the last decimal type. This PR fixes it to use `findTightestCommonType`. **decimal.csv** ``` 9.03E+12 1.19E+11 ``` **BEFORE** ```scala scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").printSchema root |-- _c0: decimal(3,-9) (nullable = true) scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").show 16/12/16 14:32:49 ERROR Executor: Exception in task 0.0 in stage 4.0 (TID 4) java.lang.IllegalArgumentException: requirement failed: Decimal precision 4 exceeds max precision 3 ``` **AFTER** ```scala scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").printSchema root |-- _c0: decimal(4,-9) (nullable = true) scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").show +---------+ | _c0| +---------+ |9.030E+12| | 1.19E+11| +---------+ ``` ## How was this patch tested? Pass the newly add test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16320 from dongjoon-hyun/SPARK-18877.
* [SPARK-18932][SQL] Support partial aggregation for collect_set/collect_listLiang-Chi Hsieh2017-01-034-43/+39
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently collect_set/collect_list aggregation expression don't support partial aggregation. This patch is to enable partial aggregation for them. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16371 from viirya/collect-partial-support.
* [SPARK-18998][SQL] Add a cbo conf to switch between default statistics and ↵Zhenhua Wang2017-01-037-10/+117
| | | | | | | | | | | | | | | | | estimated statistics ## What changes were proposed in this pull request? We add a cbo configuration to switch between default stats and estimated stats. We also define a new statistics method `planStats` in LogicalPlan with conf as its parameter, in order to pass the cbo switch and other estimation related configurations in the future. `planStats` is used on the caller sides (i.e. in Optimizer and Strategies) to make transformation decisions based on stats. ## How was this patch tested? Add a test case using a dummy LogicalPlan. Author: Zhenhua Wang <wzh_zju@163.com> Closes #16401 from wzhfy/cboSwitch.
* [SPARK-19029][SQL] Remove databaseName from SimpleCatalogRelationgatorsmile2017-01-033-10/+5
| | | | | | | | | | | | ### What changes were proposed in this pull request? Remove useless `databaseName ` from `SimpleCatalogRelation`. ### How was this patch tested? Existing test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #16438 from gatorsmile/removeDBFromSimpleCatalogRelation.
* [SPARK-19002][BUILD][PYTHON] Check pep8 against all Python scriptshyukjinkwon2017-01-029-31/+27
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to check pep8 against all other Python scripts and fix the errors as below: ```bash ./dev/create-release/generate-contributors.py ./dev/create-release/releaseutils.py ./dev/create-release/translate-contributors.py ./dev/lint-python ./python/docs/epytext.py ./examples/src/main/python/mllib/decision_tree_classification_example.py ./examples/src/main/python/mllib/decision_tree_regression_example.py ./examples/src/main/python/mllib/gradient_boosting_classification_example.py ./examples/src/main/python/mllib/gradient_boosting_regression_example.py ./examples/src/main/python/mllib/linear_regression_with_sgd_example.py ./examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py ./examples/src/main/python/mllib/naive_bayes_example.py ./examples/src/main/python/mllib/random_forest_classification_example.py ./examples/src/main/python/mllib/random_forest_regression_example.py ./examples/src/main/python/mllib/svm_with_sgd_example.py ./examples/src/main/python/streaming/network_wordjoinsentiments.py ./sql/hive/src/test/resources/data/scripts/cat.py ./sql/hive/src/test/resources/data/scripts/cat_error.py ./sql/hive/src/test/resources/data/scripts/doubleescapedtab.py ./sql/hive/src/test/resources/data/scripts/dumpdata_script.py ./sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py ./sql/hive/src/test/resources/data/scripts/escapednewline.py ./sql/hive/src/test/resources/data/scripts/escapedtab.py ./sql/hive/src/test/resources/data/scripts/input20_script.py ./sql/hive/src/test/resources/data/scripts/newline.py ``` ## How was this patch tested? - `./python/docs/epytext.py` ```bash cd ./python/docs $$ make html ``` - pep8 check (Python 2.7 / Python 3.3.6) ``` ./dev/lint-python ``` - `./dev/merge_spark_pr.py` (Python 2.7 only / Python 3.3.6 not working) ```bash python -m doctest -v ./dev/merge_spark_pr.py ``` - `./dev/create-release/releaseutils.py` `./dev/create-release/generate-contributors.py` `./dev/create-release/translate-contributors.py` (Python 2.7 only / Python 3.3.6 not working) ```bash python generate-contributors.py python translate-contributors.py ``` - Examples (Python 2.7 / Python 3.3.6) ```bash ./bin/spark-submit examples/src/main/python/mllib/decision_tree_classification_example.py ./bin/spark-submit examples/src/main/python/mllib/decision_tree_regression_example.py ./bin/spark-submit examples/src/main/python/mllib/gradient_boosting_classification_example.py ./bin/spark-submit examples/src/main/python/mllib/gradient_boosting_regression_example.p ./bin/spark-submit examples/src/main/python/mllib/random_forest_classification_example.py ./bin/spark-submit examples/src/main/python/mllib/random_forest_regression_example.py ``` - Examples (Python 2.7 only / Python 3.3.6 not working) ``` ./bin/spark-submit examples/src/main/python/mllib/linear_regression_with_sgd_example.py ./bin/spark-submit examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py ./bin/spark-submit examples/src/main/python/mllib/naive_bayes_example.py ./bin/spark-submit examples/src/main/python/mllib/svm_with_sgd_example.py ``` - `sql/hive/src/test/resources/data/scripts/*.py` (Python 2.7 / Python 3.3.6 within suggested changes) Manually tested only changed ones. - `./dev/github_jira_sync.py` (Python 2.7 only / Python 3.3.6 not working) Manually tested this after disabling actually adding comments and links. And also via Jenkins tests. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16405 from HyukjinKwon/minor-pep8.
* [SPARK-19022][TESTS] Fix tests dependent on OS due to different newline ↵hyukjinkwon2017-01-021-6/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | characters ## What changes were proposed in this pull request? There are two tests failing on Windows due to the different newlines. ``` - StreamingQueryProgress - prettyJson *** FAILED *** (0 milliseconds) "{ "id" : "39788670-6722-48b7-a248-df6ba08722ac", "runId" : "422282f1-3b81-4b47-a15d-82dda7e69390", "name" : "myName", ... }" did not equal "{ "id" : "39788670-6722-48b7-a248-df6ba08722ac", "runId" : "422282f1-3b81-4b47-a15d-82dda7e69390", "name" : "myName", ... }" ... ``` ``` - StreamingQueryStatus - prettyJson *** FAILED *** (0 milliseconds) "{ "message" : "active", "isDataAvailable" : true, "isTriggerActive" : false }" did not equal "{ "message" : "active", "isDataAvailable" : true, "isTriggerActive" : false }" ... ``` The reason is, `pretty` in `org.json4s.pretty` writes OS-dependent newlines but the string defined in the tests are `\n`. This ends up with test failures. This PR proposes to compare these regardless of newline concerns. ## How was this patch tested? Manually tested via AppVeyor. **Before** https://ci.appveyor.com/project/spark-test/spark/build/417-newlines-fix-before **After** https://ci.appveyor.com/project/spark-test/spark/build/418-newlines-fix Author: hyukjinkwon <gurwls223@gmail.com> Closes #16433 from HyukjinKwon/tests-StreamingQueryStatusAndProgressSuite.
* [SPARK-19050][SS][TESTS] Fix EventTimeWatermarkSuite 'delay in months and ↵Shixiong Zhu2017-01-011-1/+4
| | | | | | | | | | | | | | | | years handled correctly' ## What changes were proposed in this pull request? `monthsSinceEpoch` in this test is like `math.floor(num)`, so `monthDiff` has two possible values. ## How was this patch tested? Jenkins. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16449 from zsxwing/watermark-test-hotfix.
* [SPARK-19028][SQL] Fixed non-thread-safe functions used in SessionCataloggatorsmile2016-12-312-18/+20
| | | | | | | | | | | | | | ### What changes were proposed in this pull request? Fixed non-thread-safe functions used in SessionCatalog: - refreshTable - lookupRelation ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #16437 from gatorsmile/addSyncToLookUpTable.
* [SPARK-18123][SQL] Use db column names instead of RDD column ones during ↵Dongjoon Hyun2016-12-303-25/+95
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | JDBC Writing ## What changes were proposed in this pull request? Apache Spark supports the following cases **by quoting RDD column names** while saving through JDBC. - Allow reserved keyword as a column name, e.g., 'order'. - Allow mixed-case colume names like the following, e.g., `[a: int, A: int]`. ``` scala scala> val df = sql("select 1 a, 1 A") df: org.apache.spark.sql.DataFrame = [a: int, A: int] ... scala> df.write.mode("overwrite").format("jdbc").options(option).save() scala> df.write.mode("append").format("jdbc").options(option).save() ``` This PR aims to use **database column names** instead of RDD column ones in order to support the following additionally. Note that this case succeeds with `MySQL`, but fails on `Postgres`/`Oracle` before. ``` scala val df1 = sql("select 1 a") val df2 = sql("select 1 A") ... df1.write.mode("overwrite").format("jdbc").options(option).save() df2.write.mode("append").format("jdbc").options(option).save() ``` ## How was this patch tested? Pass the Jenkins test with a new testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Author: gatorsmile <gatorsmile@gmail.com> Closes #15664 from dongjoon-hyun/SPARK-18123.
* [SPARK-18922][TESTS] Fix more path-related test failures on Windowshyukjinkwon2016-12-3020-105/+106
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to fix the test failures due to different format of paths on Windows. Failed tests are as below: ``` ColumnExpressionSuite: - input_file_name, input_file_block_start, input_file_block_length - FileScanRDD *** FAILED *** (187 milliseconds) "file:///C:/projects/spark/target/tmp/spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce/part-00001-c083a03a-e55e-4b05-9073-451de352d006.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce" (ColumnExpressionSuite.scala:545) - input_file_name, input_file_block_start, input_file_block_length - HadoopRDD *** FAILED *** (172 milliseconds) "file:/C:/projects/spark/target/tmp/spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f/part-00000-f6530138-9ad3-466d-ab46-0eeb6f85ed0b.txt" did not contain "C:\projects\spark\target\tmp\spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f" (ColumnExpressionSuite.scala:569) - input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD *** FAILED *** (156 milliseconds) "file:/C:/projects/spark/target/tmp/spark-a894c7df-c74d-4d19-82a2-a04744cb3766/part-00000-29674e3f-3fcf-4327-9b04-4dab1d46338d.txt" did not contain "C:\projects\spark\target\tmp\spark-a894c7df-c74d-4d19-82a2-a04744cb3766" (ColumnExpressionSuite.scala:598) ``` ``` DataStreamReaderWriterSuite: - source metadataPath *** FAILED *** (62 milliseconds) org.mockito.exceptions.verification.junit.ArgumentsAreDifferent: Argument(s) are different! Wanted: streamSourceProvider.createSource( org.apache.spark.sql.SQLContext3b04133b, "C:\projects\spark\target\tmp\streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0", None, "org.apache.spark.sql.streaming.test", Map() ); -> at org.apache.spark.sql.streaming.test.DataStreamReaderWriterSuite$$anonfun$12.apply$mcV$sp(DataStreamReaderWriterSuite.scala:374) Actual invocation has different arguments: streamSourceProvider.createSource( org.apache.spark.sql.SQLContext3b04133b, "/C:/projects/spark/target/tmp/streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0", None, "org.apache.spark.sql.streaming.test", Map() ); ``` ``` GlobalTempViewSuite: - CREATE GLOBAL TEMP VIEW USING *** FAILED *** (110 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-960398ba-a0a1-45f6-a59a-d98533f9f519; ``` ``` CreateTableAsSelectSuite: - CREATE TABLE USING AS SELECT *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - create a table, drop it and create another one with the same name *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - create table using as select - with partitioned by *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - create table using as select - with non-zero buckets *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ``` ``` HiveMetadataCacheSuite: - partitioned table is cached when partition pruning is true *** FAILED *** (532 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - partitioned table is cached when partition pruning is false *** FAILED *** (297 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` MultiDatabaseSuite: - createExternalTable() to non-default database - with USE *** FAILED *** (954 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-0839d9a7-5e29-467a-9e3e-3e4cd618ee09; - createExternalTable() to non-default database - without USE *** FAILED *** (500 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c7e24d73-1d8f-45e8-ab7d-53a83087aec3; - invalid database name and table names *** FAILED *** (31 milliseconds) "Path does not exist: file:/C:projectsspark arget mpspark-15a2a494-3483-4876-80e5-ec396e704b77;" did not contain "`t:a` is not a valid name for tables/databases. Valid names only contain alphabet characters, numbers and _." (MultiDatabaseSuite.scala:296) ``` ``` OrcQuerySuite: - SPARK-8501: Avoids discovery schema from empty ORC files *** FAILED *** (15 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC *** FAILED *** (78 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - converted ORC table supports resolving mixed case field *** FAILED *** (297 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite: - Locality support for FileScanRDD *** FAILED *** (15 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-383d1f13-8783-47fd-964d-9c75e5eec50f, expected: file:/// ``` ``` HiveQuerySuite: - CREATE TEMPORARY FUNCTION *** FAILED *** (0 milliseconds) java.net.MalformedURLException: For input string: "%5Cprojects%5Cspark%5Csql%5Chive%5Ctarget%5Cscala-2.11%5Ctest-classes%5CTestUDTF.jar" - ADD FILE command *** FAILED *** (500 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\sql\hive\target\scala-2.11\test-classes\data\files\v1.txt - ADD JAR command 2 *** FAILED *** (110 milliseconds) org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilessample.json; ``` ``` PruneFileSourcePartitionsSuite: - PruneFileSourcePartitions should not change the output of LogicalRelation *** FAILED *** (15 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` HiveCommandSuite: - LOAD DATA LOCAL *** FAILED *** (109 milliseconds) org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilesemployee.dat; - LOAD DATA *** FAILED *** (93 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark arget mpemployee.dat7496657117354281006.tmp - Truncate Table *** FAILED *** (78 milliseconds) org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilesemployee.dat; ``` ``` HiveExternalCatalogBackwardCompatibilitySuite: - make sure we can read table created by old version of Spark *** FAILED *** (0 milliseconds) "[/C:/projects/spark/target/tmp/]spark-0554d859-74e1-..." did not equal "[C:\projects\spark\target\tmp\]spark-0554d859-74e1-..." (HiveExternalCatalogBackwardCompatibilitySuite.scala:213) org.scalatest.exceptions.TestFailedException - make sure we can alter table location created by old version of Spark *** FAILED *** (110 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark arget mpspark-0e9b2c5f-49a1-4e38-a32a-c0ab1813a79f ``` ``` ExternalCatalogSuite: - create/drop/rename partitions should create/delete/rename the directory *** FAILED *** (610 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-4c24f010-18df-437b-9fed-990c6f9adece ``` ``` SQLQuerySuite: - describe functions - temporary user defined functions *** FAILED *** (16 milliseconds) java.net.URISyntaxException: Illegal character in opaque part at index 22: C:projectssparksqlhive argetscala-2.11 est-classesTestUDTF.jar - specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-a34c9814-a483-43f2-be29-37f616b6df91; ``` ``` PartitionProviderCompatibilitySuite: - convert partition provider to hive with repair table *** FAILED *** (281 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-ee5fc96d-8c7d-4ebf-8571-a1d62736473e; - when partition management is enabled, new tables have partition provider hive *** FAILED *** (187 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-803ad4d6-3e8c-498d-9ca5-5cda5d9b2a48; - when partition management is disabled, new tables have no partition provider *** FAILED *** (172 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c9fda9e2-4020-465f-8678-52cd72d0a58f; - when partition management is disabled, we preserve the old behavior even for new tables *** FAILED *** (203 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e13; - insert overwrite partition of legacy datasource table *** FAILED *** (188 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e79; - insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (219 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-6ba3a88d-6f6c-42c5-a9f4-6d924a0616ff; - SPARK-18544 append with saveAsTable - partition management true *** FAILED *** (173 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-cd234a6d-9cb4-4d1d-9e51-854ae9543bbd; - SPARK-18635 special chars in partition values - partition management true *** FAILED *** (2 seconds, 967 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-18635 special chars in partition values - partition management false *** FAILED *** (62 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-18659 insert overwrite table with lowercase - partition management true *** FAILED *** (63 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-18544 append with saveAsTable - partition management false *** FAILED *** (266 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-18659 insert overwrite table files - partition management false *** FAILED *** (63 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-18659 insert overwrite table with lowercase - partition management false *** FAILED *** (78 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - sanity check table setup *** FAILED *** (31 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - insert into partial dynamic partitions *** FAILED *** (47 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - insert into fully dynamic partitions *** FAILED *** (62 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - insert into static partition *** FAILED *** (78 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - overwrite partial dynamic partitions *** FAILED *** (63 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - overwrite fully dynamic partitions *** FAILED *** (47 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - overwrite static partition *** FAILED *** (63 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` MetastoreDataSourcesSuite: - check change without refresh *** FAILED *** (203 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-00713fe4-ca04-448c-bfc7-6c5e9a2ad2a1; - drop, change, recreate *** FAILED *** (78 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-2030a21b-7d67-4385-a65b-bb5e2bed4861; - SPARK-15269 external data source table creation *** FAILED *** (78 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4d50fd4a-14bc-41d6-9232-9554dd233f86; - CTAS *** FAILED *** (109 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - CTAS with IF NOT EXISTS *** FAILED *** (109 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - CTAS: persisted partitioned bucketed data source table *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - SPARK-15025: create datasource table with path with select *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - CTAS: persisted partitioned data source table *** FAILED *** (47 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ``` ``` HiveMetastoreCatalogSuite: - Persist non-partitioned parquet relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string - Persist non-partitioned orc relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ``` ``` HiveUDFSuite: - SPARK-11522 select input_file_name from non-parquet table *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` QueryPartitionSuite: - SPARK-13709: reading partitioned Avro table with nested schema *** FAILED *** (250 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` ParquetHiveCompatibilitySuite: - simple primitives *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-10177 timestamp *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - array *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - map *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - struct *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-16344: array of struct with a single field named 'array_element' *** FAILED *** (15 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ## How was this patch tested? Manually tested via AppVeyor. ``` ColumnExpressionSuite: - input_file_name, input_file_block_start, input_file_block_length - FileScanRDD (234 milliseconds) - input_file_name, input_file_block_start, input_file_block_length - HadoopRDD (235 milliseconds) - input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD (203 milliseconds) ``` ``` DataStreamReaderWriterSuite: - source metadataPath (63 milliseconds) ``` ``` GlobalTempViewSuite: - CREATE GLOBAL TEMP VIEW USING (436 milliseconds) ``` ``` CreateTableAsSelectSuite: - CREATE TABLE USING AS SELECT (171 milliseconds) - create a table, drop it and create another one with the same name (422 milliseconds) - create table using as select - with partitioned by (141 milliseconds) - create table using as select - with non-zero buckets (125 milliseconds) ``` ``` HiveMetadataCacheSuite: - partitioned table is cached when partition pruning is true (3 seconds, 211 milliseconds) - partitioned table is cached when partition pruning is false (1 second, 781 milliseconds) ``` ``` MultiDatabaseSuite: - createExternalTable() to non-default database - with USE (797 milliseconds) - createExternalTable() to non-default database - without USE (640 milliseconds) - invalid database name and table names (62 milliseconds) ``` ``` OrcQuerySuite: - SPARK-8501: Avoids discovery schema from empty ORC files (703 milliseconds) - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC (750 milliseconds) - converted ORC table supports resolving mixed case field (625 milliseconds) ``` ``` HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite: - Locality support for FileScanRDD (296 milliseconds) ``` ``` HiveQuerySuite: - CREATE TEMPORARY FUNCTION (125 milliseconds) - ADD FILE command (250 milliseconds) - ADD JAR command 2 (609 milliseconds) ``` ``` PruneFileSourcePartitionsSuite: - PruneFileSourcePartitions should not change the output of LogicalRelation (359 milliseconds) ``` ``` HiveCommandSuite: - LOAD DATA LOCAL (1 second, 829 milliseconds) - LOAD DATA (1 second, 735 milliseconds) - Truncate Table (1 second, 641 milliseconds) ``` ``` HiveExternalCatalogBackwardCompatibilitySuite: - make sure we can read table created by old version of Spark (32 milliseconds) - make sure we can alter table location created by old version of Spark (125 milliseconds) - make sure we can rename table created by old version of Spark (281 milliseconds) ``` ``` ExternalCatalogSuite: - create/drop/rename partitions should create/delete/rename the directory (625 milliseconds) ``` ``` SQLQuerySuite: - describe functions - temporary user defined functions (31 milliseconds) - specifying database name for a temporary table is not allowed (390 milliseconds) ``` ``` PartitionProviderCompatibilitySuite: - convert partition provider to hive with repair table (813 milliseconds) - when partition management is enabled, new tables have partition provider hive (562 milliseconds) - when partition management is disabled, new tables have no partition provider (344 milliseconds) - when partition management is disabled, we preserve the old behavior even for new tables (422 milliseconds) - insert overwrite partition of legacy datasource table (750 milliseconds) - SPARK-18544 append with saveAsTable - partition management true (985 milliseconds) - SPARK-18635 special chars in partition values - partition management true (3 seconds, 328 milliseconds) - SPARK-18635 special chars in partition values - partition management false (2 seconds, 891 milliseconds) - SPARK-18659 insert overwrite table with lowercase - partition management true (750 milliseconds) - SPARK-18544 append with saveAsTable - partition management false (656 milliseconds) - SPARK-18659 insert overwrite table files - partition management false (922 milliseconds) - SPARK-18659 insert overwrite table with lowercase - partition management false (469 milliseconds) - sanity check table setup (937 milliseconds) - insert into partial dynamic partitions (2 seconds, 985 milliseconds) - insert into fully dynamic partitions (1 second, 937 milliseconds) - insert into static partition (1 second, 578 milliseconds) - overwrite partial dynamic partitions (7 seconds, 561 milliseconds) - overwrite fully dynamic partitions (1 second, 766 milliseconds) - overwrite static partition (1 second, 797 milliseconds) ``` ``` MetastoreDataSourcesSuite: - check change without refresh (610 milliseconds) - drop, change, recreate (437 milliseconds) - SPARK-15269 external data source table creation (297 milliseconds) - CTAS with IF NOT EXISTS (437 milliseconds) - CTAS: persisted partitioned bucketed data source table (422 milliseconds) - SPARK-15025: create datasource table with path with select (265 milliseconds) - CTAS (438 milliseconds) - CTAS with IF NOT EXISTS (469 milliseconds) - CTAS: persisted partitioned bucketed data source table (406 milliseconds) ``` ``` HiveMetastoreCatalogSuite: - Persist non-partitioned parquet relation into metastore as managed table using CTAS (406 milliseconds) - Persist non-partitioned orc relation into metastore as managed table using CTAS (313 milliseconds) ``` ``` HiveUDFSuite: - SPARK-11522 select input_file_name from non-parquet table (3 seconds, 144 milliseconds) ``` ``` QueryPartitionSuite: - SPARK-13709: reading partitioned Avro table with nested schema (1 second, 67 milliseconds) ``` ``` ParquetHiveCompatibilitySuite: - simple primitives (745 milliseconds) - SPARK-10177 timestamp (375 milliseconds) - array (407 milliseconds) - map (409 milliseconds) - struct (437 milliseconds) - SPARK-16344: array of struct with a single field named 'array_element' (391 milliseconds) ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16397 from HyukjinKwon/SPARK-18922-paths.
* [SPARK-19012][SQL] Fix `createTempViewCommand` to throw AnalysisException ↵Dongjoon Hyun2016-12-292-11/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | instead of ParseException ## What changes were proposed in this pull request? Currently, `createTempView`, `createOrReplaceTempView`, and `createGlobalTempView` show `ParseExceptions` on invalid table names. We had better show better error message. Also, this PR also adds and updates the missing description on the API docs correctly. **BEFORE** ``` scala> spark.range(10).createOrReplaceTempView("11111") org.apache.spark.sql.catalyst.parser.ParseException: mismatched input '11111' expecting {'SELECT', 'FROM', 'ADD', ...}(line 1, pos 0) == SQL == 11111 ... ``` **AFTER** ``` scala> spark.range(10).createOrReplaceTempView("11111") org.apache.spark.sql.AnalysisException: Invalid view name: 11111; ... ``` ## How was this patch tested? Pass the Jenkins with updated a test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16427 from dongjoon-hyun/SPARK-19012.
* [SPARK-18567][SQL] Simplify CreateDataSourceTableAsSelectCommandWenchen Fan2016-12-284-198/+213
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The `CreateDataSourceTableAsSelectCommand` is quite complex now, as it has a lot of work to do if the table already exists: 1. throw exception if we don't want to ignore it. 2. do some check and adjust the schema if we want to append data. 3. drop the table and create it again if we want to overwrite. The work 2 and 3 should be done by analyzer, so that we can also apply it to hive tables. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #15996 from cloud-fan/append.
* [SPARK-16213][SQL] Reduce runtime overhead of a program that creates an ↵Kazuaki Ishizaki2016-12-299-91/+230
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | primitive array in DataFrame ## What changes were proposed in this pull request? This PR reduces runtime overhead of a program the creates an primitive array in DataFrame by using the similar approach to #15044. Generated code performs boxing operation in an assignment from InternalRow to an `Object[]` temporary array (at Lines 051 and 061 in the generated code before without this PR). If we know that type of array elements is primitive, we apply the following optimizations: 1. Eliminate a pair of `isNullAt()` and a null assignment 2. Allocate an primitive array instead of `Object[]` (eliminate boxing operations) 3. Create `UnsafeArrayData` by using `UnsafeArrayWriter` to keep a primitive array in a row format instead of doing non-lightweight operations in constructor of `GenericArrayData` The PR also performs the same things for `CreateMap`. Here are performance results of [DataFrame programs](https://github.com/kiszk/spark/blob/6bf54ec5e227689d69f6db991e9ecbc54e153d0a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala#L83-L112) by up to 17.9x over without this PR. ``` Without SPARK-16043 OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Read a primitive array in DataFrame: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 3805 / 4150 0.0 507308.9 1.0X Double 3593 / 3852 0.0 479056.9 1.1X With SPARK-16043 Read a primitive array in DataFrame: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 213 / 271 0.0 28387.5 1.0X Double 204 / 223 0.0 27250.9 1.0X ``` Note : #15780 is enabled for these measurements An motivating example ``` java val df = sparkContext.parallelize(Seq(0.0d, 1.0d), 1).toDF df.selectExpr("Array(value + 1.1d, value + 2.2d)").show ``` Generated code without this PR ``` java /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 006 */ private Object[] references; /* 007 */ private scala.collection.Iterator[] inputs; /* 008 */ private scala.collection.Iterator inputadapter_input; /* 009 */ private UnsafeRow serializefromobject_result; /* 010 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter; /* 012 */ private Object[] project_values; /* 013 */ private UnsafeRow project_result; /* 014 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder; /* 015 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter; /* 016 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter; /* 017 */ /* 018 */ public GeneratedIterator(Object[] references) { /* 019 */ this.references = references; /* 020 */ } /* 021 */ /* 022 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 023 */ partitionIndex = index; /* 024 */ this.inputs = inputs; /* 025 */ inputadapter_input = inputs[0]; /* 026 */ serializefromobject_result = new UnsafeRow(1); /* 027 */ this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0); /* 028 */ this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 029 */ this.project_values = null; /* 030 */ project_result = new UnsafeRow(1); /* 031 */ this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32); /* 032 */ this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1); /* 033 */ this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(); /* 034 */ /* 035 */ } /* 036 */ /* 037 */ protected void processNext() throws java.io.IOException { /* 038 */ while (inputadapter_input.hasNext()) { /* 039 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 040 */ double inputadapter_value = inputadapter_row.getDouble(0); /* 041 */ /* 042 */ final boolean project_isNull = false; /* 043 */ this.project_values = new Object[2]; /* 044 */ boolean project_isNull1 = false; /* 045 */ /* 046 */ double project_value1 = -1.0; /* 047 */ project_value1 = inputadapter_value + 1.1D; /* 048 */ if (false) { /* 049 */ project_values[0] = null; /* 050 */ } else { /* 051 */ project_values[0] = project_value1; /* 052 */ } /* 053 */ /* 054 */ boolean project_isNull4 = false; /* 055 */ /* 056 */ double project_value4 = -1.0; /* 057 */ project_value4 = inputadapter_value + 2.2D; /* 058 */ if (false) { /* 059 */ project_values[1] = null; /* 060 */ } else { /* 061 */ project_values[1] = project_value4; /* 062 */ } /* 063 */ /* 064 */ final ArrayData project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_values); /* 065 */ this.project_values = null; /* 066 */ project_holder.reset(); /* 067 */ /* 068 */ project_rowWriter.zeroOutNullBytes(); /* 069 */ /* 070 */ if (project_isNull) { /* 071 */ project_rowWriter.setNullAt(0); /* 072 */ } else { /* 073 */ // Remember the current cursor so that we can calculate how many bytes are /* 074 */ // written later. /* 075 */ final int project_tmpCursor = project_holder.cursor; /* 076 */ /* 077 */ if (project_value instanceof UnsafeArrayData) { /* 078 */ final int project_sizeInBytes = ((UnsafeArrayData) project_value).getSizeInBytes(); /* 079 */ // grow the global buffer before writing data. /* 080 */ project_holder.grow(project_sizeInBytes); /* 081 */ ((UnsafeArrayData) project_value).writeToMemory(project_holder.buffer, project_holder.cursor); /* 082 */ project_holder.cursor += project_sizeInBytes; /* 083 */ /* 084 */ } else { /* 085 */ final int project_numElements = project_value.numElements(); /* 086 */ project_arrayWriter.initialize(project_holder, project_numElements, 8); /* 087 */ /* 088 */ for (int project_index = 0; project_index < project_numElements; project_index++) { /* 089 */ if (project_value.isNullAt(project_index)) { /* 090 */ project_arrayWriter.setNullDouble(project_index); /* 091 */ } else { /* 092 */ final double project_element = project_value.getDouble(project_index); /* 093 */ project_arrayWriter.write(project_index, project_element); /* 094 */ } /* 095 */ } /* 096 */ } /* 097 */ /* 098 */ project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor); /* 099 */ } /* 100 */ project_result.setTotalSize(project_holder.totalSize()); /* 101 */ append(project_result); /* 102 */ if (shouldStop()) return; /* 103 */ } /* 104 */ } /* 105 */ } ``` Generated code with this PR ``` java /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 006 */ private Object[] references; /* 007 */ private scala.collection.Iterator[] inputs; /* 008 */ private scala.collection.Iterator inputadapter_input; /* 009 */ private UnsafeRow serializefromobject_result; /* 010 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter; /* 012 */ private UnsafeArrayData project_arrayData; /* 013 */ private UnsafeRow project_result; /* 014 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder; /* 015 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter; /* 016 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter; /* 017 */ /* 018 */ public GeneratedIterator(Object[] references) { /* 019 */ this.references = references; /* 020 */ } /* 021 */ /* 022 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 023 */ partitionIndex = index; /* 024 */ this.inputs = inputs; /* 025 */ inputadapter_input = inputs[0]; /* 026 */ serializefromobject_result = new UnsafeRow(1); /* 027 */ this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0); /* 028 */ this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 029 */ /* 030 */ project_result = new UnsafeRow(1); /* 031 */ this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32); /* 032 */ this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1); /* 033 */ this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(); /* 034 */ /* 035 */ } /* 036 */ /* 037 */ protected void processNext() throws java.io.IOException { /* 038 */ while (inputadapter_input.hasNext()) { /* 039 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 040 */ double inputadapter_value = inputadapter_row.getDouble(0); /* 041 */ /* 042 */ byte[] project_array = new byte[32]; /* 043 */ project_arrayData = new UnsafeArrayData(); /* 044 */ Platform.putLong(project_array, 16, 2); /* 045 */ project_arrayData.pointTo(project_array, 16, 32); /* 046 */ /* 047 */ boolean project_isNull1 = false; /* 048 */ /* 049 */ double project_value1 = -1.0; /* 050 */ project_value1 = inputadapter_value + 1.1D; /* 051 */ if (false) { /* 052 */ project_arrayData.setNullAt(0); /* 053 */ } else { /* 054 */ project_arrayData.setDouble(0, project_value1); /* 055 */ } /* 056 */ /* 057 */ boolean project_isNull4 = false; /* 058 */ /* 059 */ double project_value4 = -1.0; /* 060 */ project_value4 = inputadapter_value + 2.2D; /* 061 */ if (false) { /* 062 */ project_arrayData.setNullAt(1); /* 063 */ } else { /* 064 */ project_arrayData.setDouble(1, project_value4); /* 065 */ } /* 066 */ project_holder.reset(); /* 067 */ /* 068 */ // Remember the current cursor so that we can calculate how many bytes are /* 069 */ // written later. /* 070 */ final int project_tmpCursor = project_holder.cursor; /* 071 */ /* 072 */ if (project_arrayData instanceof UnsafeArrayData) { /* 073 */ final int project_sizeInBytes = ((UnsafeArrayData) project_arrayData).getSizeInBytes(); /* 074 */ // grow the global buffer before writing data. /* 075 */ project_holder.grow(project_sizeInBytes); /* 076 */ ((UnsafeArrayData) project_arrayData).writeToMemory(project_holder.buffer, project_holder.cursor); /* 077 */ project_holder.cursor += project_sizeInBytes; /* 078 */ /* 079 */ } else { /* 080 */ final int project_numElements = project_arrayData.numElements(); /* 081 */ project_arrayWriter.initialize(project_holder, project_numElements, 8); /* 082 */ /* 083 */ for (int project_index = 0; project_index < project_numElements; project_index++) { /* 084 */ if (project_arrayData.isNullAt(project_index)) { /* 085 */ project_arrayWriter.setNullDouble(project_index); /* 086 */ } else { /* 087 */ final double project_element = project_arrayData.getDouble(project_index); /* 088 */ project_arrayWriter.write(project_index, project_element); /* 089 */ } /* 090 */ } /* 091 */ } /* 092 */ /* 093 */ project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor); /* 094 */ project_result.setTotalSize(project_holder.totalSize()); /* 095 */ append(project_result); /* 096 */ if (shouldStop()) return; /* 097 */ } /* 098 */ } /* 099 */ } ``` ## How was this patch tested? Added unit tests into `DataFrameComplexTypeSuite` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #13909 from kiszk/SPARK-16213.
* [MINOR][DOC] Fix doc of ForeachWriter to use writeStreamCarson Wang2016-12-281-2/+2
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix the document of `ForeachWriter` to use `writeStream` instead of `write` for a streaming dataset. ## How was this patch tested? Docs only. Author: Carson Wang <carson.wang@intel.com> Closes #16419 from carsonwang/FixDoc.
* [SPARK-18960][SQL][SS] Avoid double reading file which is being copied.uncleGen2016-12-282-3/+9
| | | | | | | | | | | | | ## What changes were proposed in this pull request? In HDFS, when we copy a file into target directory, there will a temporary `._COPY_` file for a period of time. The duration depends on file size. If we do not skip this file, we will may read the same data for two times. ## How was this patch tested? update unit test Author: uncleGen <hustyugm@gmail.com> Closes #16370 from uncleGen/SPARK-18960.
* [SPARK-18992][SQL] Move spark.sql.hive.thriftServer.singleSession to SQLConfgatorsmile2016-12-285-51/+87
| | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? Since `spark.sql.hive.thriftServer.singleSession` is a configuration of SQL component, this conf can be moved from `SparkConf` to `StaticSQLConf`. When we introduced `spark.sql.hive.thriftServer.singleSession`, all the SQL configuration are session specific. They can be modified in different sessions. In Spark 2.1, static SQL configuration is added. It is a perfect fit for `spark.sql.hive.thriftServer.singleSession`. Previously, we did the same move for `spark.sql.warehouse.dir` from `SparkConf` to `StaticSQLConf` ### How was this patch tested? Added test cases in HiveThriftServer2Suites.scala Author: gatorsmile <gatorsmile@gmail.com> Closes #16392 from gatorsmile/hiveThriftServerSingleSession.
* Revert "[SPARK-18990][SQL] make DatasetBenchmark fairer for Dataset"Yin Huai2016-12-271-42/+33
| | | | This reverts commit a05cc425a0a7d18570b99883993a04ad175aa071.
* [SPARK-18990][SQL] make DatasetBenchmark fairer for DatasetWenchen Fan2016-12-271-33/+42
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently `DatasetBenchmark` use `case class Data(l: Long, s: String)` as the record type of `RDD` and `Dataset`, which introduce serialization overhead only to `Dataset` and is unfair. This PR use `Long` as the record type, to be fairer for `Dataset` ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #16391 from cloud-fan/benchmark.
* [SPARK-19004][SQL] Fix `JDBCWriteSuite.testH2Dialect` by removing ↵Dongjoon Hyun2016-12-271-3/+0
| | | | | | | | | | | | | | | | | | `getCatalystType` ## What changes were proposed in this pull request? `JDBCSuite` and `JDBCWriterSuite` have their own `testH2Dialect`s for their testing purposes. This PR fixes `testH2Dialect` in `JDBCWriterSuite` by removing `getCatalystType` implementation in order to return correct types. Currently, it always returns `Some(StringType)` incorrectly. Note that, for the `testH2Dialect` in `JDBCSuite`, it's intentional because of the test case `Remap types via JdbcDialects`. ## How was this patch tested? This is a test only update. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16409 from dongjoon-hyun/SPARK-H2-DIALECT.
* [SPARK-18999][SQL][MINOR] simplify Literal codegenWenchen Fan2016-12-274-35/+18
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `Literal` can use `CodegenContex.addReferenceObj` to implement codegen, instead of `CodegenFallback`. This can also simplify the generated code a little bit, before we will generate: `((Expression) references[1]).eval(null)`, now it's just `references[1]`. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16402 from cloud-fan/minor.
* [SPARK-18989][SQL] DESC TABLE should not fail with format class not foundWenchen Fan2016-12-262-2/+55
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? When we describe a table, we only wanna see the information of this table, not read it, so it's ok even if the format class is not present at the classpath. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16388 from cloud-fan/hive.
* [SPARK-18980][SQL] implement Aggregator with TypedImperativeAggregateWenchen Fan2016-12-2610-56/+212
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we implement `Aggregator` with `DeclarativeAggregate`, which will serialize/deserialize the buffer object every time we process an input. This PR implements `Aggregator` with `TypedImperativeAggregate` and avoids to serialize/deserialize buffer object many times. The benchmark shows we get about 2 times speed up. For simple buffer object that doesn't need serialization, we still go with `DeclarativeAggregate`, to avoid performance regression. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16383 from cloud-fan/aggregator.
* [SPARK-18943][SQL] Avoid per-record type dispatch in CSV when readinghyukjinkwon2016-12-243-159/+184
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `CSVRelation.csvParser` does type dispatch for each value in each row. We can prevent this because the schema is already kept in `CSVRelation`. So, this PR proposes that converters are created first according to the schema, and then apply them to each. I just ran some small benchmarks as below after resembling the logics in https://github.com/apache/spark/blob/7c33b0fd050f3d2b08c1cfd7efbff8166832c1af/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala#L170-L178 to test the updated logics. ```scala test("Benchmark for CSV converter") { var numMalformedRecords = 0 val N = 500 << 12 val schema = StructType( StructField("a", StringType) :: StructField("b", StringType) :: StructField("c", StringType) :: StructField("d", StringType) :: Nil) val row = Array("1.0", "test", "2015-08-20 14:57:00", "FALSE") val data = spark.sparkContext.parallelize(List.fill(N)(row)) val parser = CSVRelation.csvParser(schema, schema.fieldNames, CSVOptions()) val benchmark = new Benchmark("CSV converter", N) benchmark.addCase("cast CSV string tokens", 10) { _ => data.flatMap { recordTokens => parser(recordTokens, numMalformedRecords) }.collect() } benchmark.run() } ``` **Before** ``` CSV converter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ cast CSV string tokens 1061 / 1130 1.9 517.9 1.0X ``` **After** ``` CSV converter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ cast CSV string tokens 940 / 1011 2.2 459.2 1.0X ``` ## How was this patch tested? Tests in `CSVTypeCastSuite` and `CSVRelation` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16351 from HyukjinKwon/type-dispatch.
* [SPARK-18800][SQL] Correct the assert in UnsafeKVExternalSorter which ↵Liang-Chi Hsieh2016-12-241-1/+3
| | | | | | | | | | | | | | | | | | | | | | ensures array size ## What changes were proposed in this pull request? `UnsafeKVExternalSorter` uses `UnsafeInMemorySorter` to sort the records of `BytesToBytesMap` if it is given a map. Currently we use the number of keys in `BytesToBytesMap` to determine if the array used for sort is enough or not. We has an assert that ensures the size of the array is enough: `map.numKeys() <= map.getArray().size() / 2`. However, each record in the map takes two entries in the array, one is record pointer, another is key prefix. So the correct assert should be `map.numKeys() * 2 <= map.getArray().size() / 2`. ## How was this patch tested? N/A Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16232 from viirya/SPARK-18800-fix-UnsafeKVExternalSorter.
* [SPARK-18911][SQL] Define CatalogStatistics to interact with metastore and ↵wangzhenhua2016-12-249-23/+95
| | | | | | | | | | | | | | | | | | | convert it to Statistics in relations ## What changes were proposed in this pull request? Statistics in LogicalPlan should use attributes to refer to columns rather than column names, because two columns from two relations can have the same column name. But CatalogTable doesn't have the concepts of attribute or broadcast hint in Statistics. Therefore, putting Statistics in CatalogTable is confusing. We define a different statistic structure in CatalogTable, which is only responsible for interacting with metastore, and is converted to statistics in LogicalPlan when it is used. ## How was this patch tested? add test cases Author: wangzhenhua <wangzhenhua@huawei.com> Author: Zhenhua Wang <wzh_zju@163.com> Closes #16323 from wzhfy/nameToAttr.
* [SPARK-18985][SS] Add missing @InterfaceStability.Evolving for Structured ↵Shixiong Zhu2016-12-229-9/+28
| | | | | | | | | | | | | | | | Streaming APIs ## What changes were proposed in this pull request? Add missing InterfaceStability.Evolving for Structured Streaming APIs ## How was this patch tested? Compiling the codes. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16385 from zsxwing/SPARK-18985.
* [SPARK-18973][SQL] Remove SortPartitions and RedistributeDataReynold Xin2016-12-228-58/+26
| | | | | | | | | | | | ## What changes were proposed in this pull request? SortPartitions and RedistributeData logical operators are not actually used and can be removed. Note that we do have a Sort operator (with global flag false) that subsumed SortPartitions. ## How was this patch tested? Also updated test cases to reflect the removal. Author: Reynold Xin <rxin@databricks.com> Closes #16381 from rxin/SPARK-18973.
* [SPARK-16975][SQL][FOLLOWUP] Do not duplicately check file paths in data ↵hyukjinkwon2016-12-224-15/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | sources implementing FileFormat ## What changes were proposed in this pull request? This PR cleans up duplicated checking for file paths in implemented data sources and prevent to attempt to list twice in ORC data source. https://github.com/apache/spark/pull/14585 handles a problem for the partition column name having `_` and the issue itself is resolved correctly. However, it seems the data sources implementing `FileFormat` are validating the paths duplicately. Assuming from the comment in `CSVFileFormat`, `// TODO: Move filtering.`, I guess we don't have to check this duplicately. Currently, this seems being filtered in `PartitioningAwareFileIndex.shouldFilterOut` and`PartitioningAwareFileIndex.isDataPath`. So, `FileFormat.inferSchema` will always receive leaf files. For example, running to codes below: ``` scala spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet") spark.read.parquet("/tmp/parquet") ``` gives the paths below without directories but just valid data files: ``` bash /tmp/parquet/_col=0/part-r-00000-094a8efa-bece-4b50-b54c-7918d1f7b3f8.snappy.parquet /tmp/parquet/_col=1/part-r-00000-094a8efa-bece-4b50-b54c-7918d1f7b3f8.snappy.parquet /tmp/parquet/_col=2/part-r-00000-25de2b50-225a-4bcf-a2bc-9eb9ed407ef6.snappy.parquet ... ``` to `FileFormat.inferSchema`. ## How was this patch tested? Unit test added in `HadoopFsRelationTest` and related existing tests. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14627 from HyukjinKwon/SPARK-16975.
* [SPARK-18922][TESTS] Fix more resource-closing-related and path-related test ↵hyukjinkwon2016-12-224-8/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | failures in identified ones on Windows ## What changes were proposed in this pull request? There are several tests failing due to resource-closing-related and path-related problems on Windows as below. - `SQLQuerySuite`: ``` - specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-1f4471ab-aac0-4239-ae35-833d54b37e52; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370) ``` - `JsonSuite`: ``` - Loading a JSON dataset from a text file with SQL *** FAILED *** (94 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c918a8b7-fc09-433c-b9d0-36c0f78ae918; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370) ``` - `StateStoreSuite`: ``` - SPARK-18342: commit fails when rename fails *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: StateStoreSuite29777261fs://C:%5Cprojects%5Cspark%5Ctarget%5Ctmp%5Cspark-ef349862-7281-4963-aaf3-add0d670a4ad%5C?????-2218c2f8-2cf6-4f80-9cdf-96354e8246a77685899733421033312/0 at org.apache.hadoop.fs.Path.initialize(Path.java:206) at org.apache.hadoop.fs.Path.<init>(Path.java:116) at org.apache.hadoop.fs.Path.<init>(Path.java:89) ... Cause: java.net.URISyntaxException: Relative path in absolute URI: StateStoreSuite29777261fs://C:%5Cprojects%5Cspark%5Ctarget%5Ctmp%5Cspark-ef349862-7281-4963-aaf3-add0d670a4ad%5C?????-2218c2f8-2cf6-4f80-9cdf-96354e8246a77685899733421033312/0 at java.net.URI.checkPath(URI.java:1823) at java.net.URI.<init>(URI.java:745) at org.apache.hadoop.fs.Path.initialize(Path.java:203) ``` - `HDFSMetadataLogSuite`: ``` - FileManager: FileContextManager *** FAILED *** (94 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-415bb0bd-396b-444d-be82-04599e025f21 at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010) at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:127) at org.apache.spark.sql.execution.streaming.HDFSMetadataLogSuite.withTempDir(HDFSMetadataLogSuite.scala:38) - FileManager: FileSystemManager *** FAILED *** (78 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ef8222cd-85aa-47c0-a396-bc7979e15088 at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010) at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:127) at org.apache.spark.sql.execution.streaming.HDFSMetadataLogSuite.withTempDir(HDFSMetadataLogSuite.scala:38) ``` And, there are some tests being failed due to the length limitation on cmd in Windows as below: - `LauncherBackendSuite`: ``` - local: launcher handle *** FAILED *** (30 seconds, 120 milliseconds) The code passed to eventually never returned normally. Attempted 283 times over 30.0960053 seconds. Last failure message: The reference was null. (LauncherBackendSuite.scala:56) org.scalatest.exceptions.TestFailedDueToTimeoutException: at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) - standalone/client: launcher handle *** FAILED *** (30 seconds, 47 milliseconds) The code passed to eventually never returned normally. Attempted 282 times over 30.037987100000002 seconds. Last failure message: The reference was null. (LauncherBackendSuite.scala:56) org.scalatest.exceptions.TestFailedDueToTimeoutException: at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) ``` The executed command is, https://gist.github.com/HyukjinKwon/d3fdd2e694e5c022992838a618a516bd, which is 16K length; however, the length limitation is 8K on Windows. So, it is being failed to launch. This PR proposes to fix the test failures on Windows and skip the tests failed due to the length limitation ## How was this patch tested? Manually tested via AppVeyor **Before** `SQLQuerySuite `: https://ci.appveyor.com/project/spark-test/spark/build/306-pr-references `JsonSuite`: https://ci.appveyor.com/project/spark-test/spark/build/307-pr-references `StateStoreSuite` : https://ci.appveyor.com/project/spark-test/spark/build/305-pr-references `HDFSMetadataLogSuite`: https://ci.appveyor.com/project/spark-test/spark/build/304-pr-references `LauncherBackendSuite`: https://ci.appveyor.com/project/spark-test/spark/build/303-pr-references **After** `SQLQuerySuite`: https://ci.appveyor.com/project/spark-test/spark/build/293-SQLQuerySuite `JsonSuite`: https://ci.appveyor.com/project/spark-test/spark/build/294-JsonSuite `StateStoreSuite`: https://ci.appveyor.com/project/spark-test/spark/build/297-StateStoreSuite `HDFSMetadataLogSuite`: https://ci.appveyor.com/project/spark-test/spark/build/319-pr-references `LauncherBackendSuite`: failed test skipped. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16335 from HyukjinKwon/more-fixes-on-windows.
* [DOC] bucketing is applicable to all file-based data sourcesReynold Xin2016-12-211-3/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? Starting Spark 2.1.0, bucketing feature is available for all file-based data sources. This patch fixes some function docs that haven't yet been updated to reflect that. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #16349 from rxin/ds-doc.
* [SQL] Minor readability improvement for partition handling codeReynold Xin2016-12-224-44/+49
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch includes minor changes to improve readability for partition handling code. I'm in the middle of implementing some new feature and found some naming / implicit type inference not as intuitive. ## How was this patch tested? This patch should have no semantic change and the changes should be covered by existing test cases. Author: Reynold Xin <rxin@databricks.com> Closes #16378 from rxin/minor-fix.
* [SPARK-18908][SS] Creating StreamingQueryException should check if ↵Shixiong Zhu2016-12-217-104/+165
| | | | | | | | | | | | | | | | | | | | logicalPlan is created ## What changes were proposed in this pull request? This PR audits places using `logicalPlan` in StreamExecution and ensures they all handles the case that `logicalPlan` cannot be created. In addition, this PR also fixes the following issues in `StreamingQueryException`: - `StreamingQueryException` and `StreamExecution` are cycle-dependent because in the `StreamingQueryException`'s constructor, it calls `StreamExecution`'s `toDebugString` which uses `StreamingQueryException`. Hence it will output `null` value in the error message. - Duplicated stack trace when calling Throwable.printStackTrace because StreamingQueryException's toString contains the stack trace. ## How was this patch tested? The updated `test("max files per trigger - incorrect values")`. I found this issue when I switched from `testStream` to the real codes to verify the failure in this test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16322 from zsxwing/SPARK-18907.