## What changes were proposed in this pull request?
As adaptive query execution may change the number of partitions in different batches, it may break streaming queries. Hence, we should disallow this feature in Structured Streaming.
## How was this patch tested?
`test("SPARK-19268: Adaptive query execution should be disallowed")`.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16683 from zsxwing/SPARK-19268.
## What changes were proposed in this pull request?
Currently, running the codes in Java
```java
spark.udf().register("inc", new UDF1<Long, Long>() {
Override
public Long call(Long i) {
return i + 1;
}
}, DataTypes.LongType);
spark.range(10).toDF("x").createOrReplaceTempView("tmp");
Row result = spark.sql("SELECT inc(x) FROM tmp GROUP BY inc(x)").head();
Assert.assertEquals(7, result.getLong(0));
```
fails as below:
```
org.apache.spark.sql.AnalysisException: expression 'tmp.`x`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
Aggregate [UDF(x#19L)], [UDF(x#19L) AS UDF(x)#23L]
+- SubqueryAlias tmp, `tmp`
+- Project [id#16L AS x#19L]
+- Range (0, 10, step=1, splits=Some(8))
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40)
at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:57)
```
The root cause is because we were creating the function every time when it needs to build as below:
```scala
scala> def inc(i: Int) = i + 1
inc: (i: Int)Int
scala> (inc(_: Int)).hashCode
res15: Int = 1231799381
scala> (inc(_: Int)).hashCode
res16: Int = 2109839984
scala> (inc(_: Int)) == (inc(_: Int))
res17: Boolean = false
```
This seems leading to the comparison failure between `ScalaUDF`s created from Java UDF API, for example, in `Expression.semanticEquals`.
In case of Scala one, it seems already fine.
Both can be tested easily as below if any reviewer is more comfortable with Scala:
```scala
val df = Seq((1, 10), (2, 11), (3, 12)).toDF("x", "y")
val javaUDF = new UDF1[Int, Int] {
override def call(i: Int): Int = i + 1
}
// spark.udf.register("inc", javaUDF, IntegerType) // Uncomment this for Java API
// spark.udf.register("inc", (i: Int) => i + 1) // Uncomment this for Scala API
df.createOrReplaceTempView("tmp")
spark.sql("SELECT inc(y) FROM tmp GROUP BY inc(y)").show()
```
## How was this patch tested?
Unit test in `JavaUDFSuite.java` and `./dev/lint-java`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16553 from HyukjinKwon/SPARK-9435.
## What changes were proposed in this pull request?
Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable.
This PR brought in the following changes:
1. Remove the param `viewOriginalText` from `CatalogTable`;
2. Update the output of command `DescribeTableCommand`.
## How was this patch tested?
Tested by exsiting test cases, also updated the failed test cases.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16679 from jiangxb1987/catalogTable.
## What changes were proposed in this pull request?
To implement DDL commands, we added several analyzer rules in sql/hive module to analyze DDL related plans. However, our `Analyzer` currently only have one extending interface: `extendedResolutionRules`, which defines extra rules that will be run together with other rules in the resolution batch, and doesn't fit DDL rules well, because:
1. DDL rules may do some checking and normalization, but we may do it many times as the resolution batch will run rules again and again, until fixed point, and it's hard to tell if a DDL rule has already done its checking and normalization. It's fine because DDL rules are idempotent, but it's bad for analysis performance
2. some DDL rules may depend on others, and it's pretty hard to write `if` conditions to guarantee the dependencies. It will be good if we have a batch which run rules in one pass, so that we can guarantee the dependencies by rules order.
This PR adds a new extending interface in `Analyzer`: `postHocResolutionRules`, which defines rules that will be run only once in a batch runs right after the resolution batch.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16645 from cloud-fan/analyzer.
## What changes were proposed in this pull request?
when we append data to a existed partitioned datasource table, the InsertIntoHadoopFsRelationCommand.getCustomPartitionLocations currently
return the same location with Hive default, it should return None.
## How was this patch tested?
Author: windpiger <songjun@outlook.com>
Closes#16642 from windpiger/appendSchema.
## What changes were proposed in this pull request?
As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed.
Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression.
This PR simply stops common subexpression elimination for conditional expressions, with some cleanup.
## How was this patch tested?
regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16659 from cloud-fan/codegen.
### What changes were proposed in this pull request?
It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.
### How was this patch tested?
Fixed the test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16587 from gatorsmile/blockHiveTable.
## What changes were proposed in this pull request?
This PR refactors CSV read path to be consistent with JSON data source. It makes the methods in classes have consistent arguments with JSON ones.
`UnivocityParser` and `JacksonParser`
``` scala
private[csv] class UnivocityParser(
schema: StructType,
requiredSchema: StructType,
options: CSVOptions) extends Logging {
...
def parse(input: String): Seq[InternalRow] = {
...
```
``` scala
class JacksonParser(
schema: StructType,
columnNameOfCorruptRecord: String,
options: JSONOptions) extends Logging {
...
def parse(input: String): Option[InternalRow] = {
...
```
These allow parsing an iterator (`String` to `InternalRow`) as below for both JSON and CSV:
```scala
iter.flatMap(parser.parse)
```
## How was this patch tested?
Existing tests should cover this.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16669 from HyukjinKwon/SPARK-16101-read.
## What changes were proposed in this pull request?
For data source tables, we will always reorder the specified table schema, or the query in CTAS, to put partition columns at the end. e.g. `CREATE TABLE t(a int, b int, c int, d int) USING parquet PARTITIONED BY (d, b)` will create a table with schema `<a, c, d, b>`
Hive serde tables don't have this problem before, because its CREATE TABLE syntax specifies data schema and partition schema individually.
However, after we unifed the CREATE TABLE syntax, Hive serde table also need to do the reorder. This PR puts the reorder logic in a analyzer rule, which works with both data source tables and Hive serde tables.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16655 from cloud-fan/schema.
## What changes were proposed in this pull request?
JDBC read is failing with NPE due to missing null value check for array data type if the source table has null values in the array type column. For null values Resultset.getArray() returns null.
This PR adds null safe check to the Resultset.getArray() value before invoking method on the Array object.
## How was this patch tested?
Updated the PostgresIntegration test suite to test null values. Ran docker integration tests on my laptop.
Author: sureshthalamati <suresh.thalamati@gmail.com>
Closes#15192 from sureshthalamati/jdbc_array_null_fix-SPARK-14536.
## What changes were proposed in this pull request?
This PR refactors CSV write path to be consistent with JSON data source.
This PR makes the methods in classes have consistent arguments with JSON ones.
- `UnivocityGenerator` and `JacksonGenerator`
``` scala
private[csv] class UnivocityGenerator(
schema: StructType,
writer: Writer,
options: CSVOptions = new CSVOptions(Map.empty[String, String])) {
...
def write ...
def close ...
def flush ...
```
``` scala
private[sql] class JacksonGenerator(
schema: StructType,
writer: Writer,
options: JSONOptions = new JSONOptions(Map.empty[String, String])) {
...
def write ...
def close ...
def flush ...
```
- This PR also makes the classes put in together in a consistent manner with JSON.
- `CsvFileFormat`
``` scala
CsvFileFormat
CsvOutputWriter
```
- `JsonFileFormat`
``` scala
JsonFileFormat
JsonOutputWriter
```
## How was this patch tested?
Existing tests should cover this.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16496 from HyukjinKwon/SPARK-16101-write.
## What changes were proposed in this pull request?
There is a race condition when stopping StateStore which makes `StateStoreSuite.maintenance` flaky. `StateStore.stop` doesn't wait for the running task to finish, and an out-of-date task may fail `doMaintenance` and cancel the new task. Here is a reproducer: dde1b5b106
This PR adds MaintenanceTask to eliminate the race condition.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#16627 from zsxwing/SPARK-19267.
## What changes were proposed in this pull request?
When we query a table with a filter on partitioned columns, we will push the partition filter to the metastore to get matched partitions directly.
In `HiveExternalCatalog.listPartitionsByFilter`, we assume the column names in partition filter are already normalized and we don't need to consider case sensitivity. However, `HiveTableScanExec` doesn't follow this assumption. This PR fixes it.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16647 from cloud-fan/bug.
## What changes were proposed in this pull request?
This PR refactors the code generation part to get data from `ColumnarVector` and `ColumnarBatch` by using a trait `ColumnarBatchScan` for ease of reuse. This is because this part will be reused by several components (e.g. parquet reader, Dataset.cache, and others) since `ColumnarBatch` will be first citizen.
This PR is a part of https://github.com/apache/spark/pull/15219. In advance, this PR makes the code generation for `ColumnarVector` and `ColumnarBatch` reuseable as a trait. In general, this is very useful for other components from the reuseability view, too.
## How was this patch tested?
tested existing test suites
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#15467 from kiszk/columnarrefactor.
## What changes were proposed in this pull request?
The initial shouldFilterOut() method invocation filter the root path name(table name in the intial call) and remove if it contains _. I moved the check one level below, so it first list files/directories in the given root path and then apply filter.
(Please fill in changes proposed in this fix)
## How was this patch tested?
Added new test case for this scenario
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: jayadevanmurali <jayadevan.m@tcs.com>
Author: jayadevan <jayadevan.m@tcs.com>
Closes#16635 from jayadevanmurali/branch-0.1-SPARK-19059.
## What changes were proposed in this pull request?
We have a table relation plan cache in `HiveMetastoreCatalog`, which caches a lot of things: file status, resolved data source, inferred schema, etc.
However, it doesn't make sense to limit this cache with hive support, we should move it to SQL core module so that users can use this cache without hive support.
It can also reduce the size of `HiveMetastoreCatalog`, so that it's easier to remove it eventually.
main changes:
1. move the table relation cache to `SessionCatalog`
2. `SessionCatalog.lookupRelation` will return `SimpleCatalogRelation` and the analyzer will convert it to `LogicalRelation` or `MetastoreRelation` later, then `HiveSessionCatalog` doesn't need to override `lookupRelation` anymore
3. `FindDataSourceTable` will read/write the table relation cache.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16621 from cloud-fan/plan-cache.
## What changes were proposed in this pull request?
We should call `StateStore.abort()` when there should be any error before the store is committed.
## How was this patch tested?
Manually.
Author: Liwei Lin <lwlin7@gmail.com>
Closes#16547 from lw-lin/append-filter.
## What changes were proposed in this pull request?
On CREATE/ALTER a view, it's no longer needed to generate a SQL text string from the LogicalPlan, instead we store the SQL query text、the output column names of the query plan, and current database to CatalogTable. Permanent views created by this approach can be resolved by current view resolution approach.
The main advantage includes:
1. If you update an underlying view, the current view also gets updated;
2. That gives us a change to get ride of SQL generation for operators.
Major changes of this PR:
1. Generate the view-specific properties(e.g. view default database, view query output column names) during permanent view creation and store them as properties in the CatalogTable;
2. Update the commands `CreateViewCommand` and `AlterViewAsCommand`, get rid of SQL generation from them.
## How was this patch tested?
Existing tests.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16613 from jiangxb1987/view-write-path.
## What changes were proposed in this pull request?
Inserting data into Hive tables has its own implementation that is distinct from data sources: `InsertIntoHiveTable`, `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`.
Note that one other major difference is that data source tables write directly to the final destination without using some staging directory, and then Spark itself adds the partitions/tables to the catalog. Hive tables actually write to some staging directory, and then call Hive metastore's loadPartition/loadTable function to load those data in. So we still need to keep `InsertIntoHiveTable` to put this special logic. In the future, we should think of writing to the hive table location directly, so that we don't need to call `loadTable`/`loadPartition` at the end and remove `InsertIntoHiveTable`.
This PR removes `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`, and create a `HiveFileFormat` to implement the write logic. In the future, we should also implement the read logic in `HiveFileFormat`.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16517 from cloud-fan/insert-hive.
## What changes were proposed in this pull request?
Added outer_explode, outer_posexplode, outer_inline functions and expressions.
Some bug fixing in GenerateExec.scala for CollectionGenerator. Previously it was not correctly handling the case of outer with empty collections, only with nulls.
## How was this patch tested?
New tests added to GeneratorFunctionSuite
Author: Bogdan Raducanu <bogdan.rdc@gmail.com>
Closes#16608 from bogdanrdc/SPARK-13721.
## What changes were proposed in this pull request?
In append mode, we check whether the schema of the write is compatible with the schema of the existing data. It can be a significant performance issue in cloud environment to find the existing schema for files. This patch removes the check.
Note that for catalog tables, we always do the check, as discussed in https://github.com/apache/spark/pull/16339#discussion_r96208357
## How was this patch tested?
N/A
Closes#16339.
Author: Reynold Xin <rxin@databricks.com>
Closes#16622 from rxin/SPARK-18917.
## What changes were proposed in this pull request?
`dropDuplicates` will create an Alias using the same exprId, so `StreamExecution` should also replace Alias if necessary.
## How was this patch tested?
test("SPARK-19065: dropDuplicates should not create expressions using the same id")
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16564 from zsxwing/SPARK-19065.
## What changes were proposed in this pull request?
This PR proposes to fix ambiguous link warnings by simply making them as code blocks for both javadoc and scaladoc.
```
[warn] .../spark/core/src/main/scala/org/apache/spark/Accumulator.scala:20: The link target "SparkContext#accumulator" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala:281: The link target "runMiniBatchSGD" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala:83: The link target "run" is ambiguous. Several members fit the target:
...
```
This PR also fixes javadoc8 break as below:
```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error] ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error] ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error] ^
[info] 3 errors
```
## How was this patch tested?
Manually via `sbt unidoc > output.txt` and the checked it via `cat output.txt | grep ambiguous`
and `sbt unidoc | grep error`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16604 from HyukjinKwon/SPARK-3249.
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/16296 , we reached a consensus that we should hide the external/managed table concept to users and only expose custom table path.
This PR renames `Catalog.createExternalTable` to `createTable`(still keep the old versions for backward compatibility), and only set the table type to EXTERNAL if `path` is specified in options.
## How was this patch tested?
new tests in `CatalogSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16528 from cloud-fan/create-table.
## What changes were proposed in this pull request?
We have a config `spark.sql.files.ignoreCorruptFiles` which can be used to ignore corrupt files when reading files in SQL. Currently the `ignoreCorruptFiles` config has two issues and can't work for Parquet:
1. We only ignore corrupt files in `FileScanRDD` . Actually, we begin to read those files as early as inferring data schema from the files. For corrupt files, we can't read the schema and fail the program. A related issue reported at http://apache-spark-developers-list.1001551.n3.nabble.com/Skip-Corrupted-Parquet-blocks-footer-tc20418.html
2. In `FileScanRDD`, we assume that we only begin to read the files when starting to consume the iterator. However, it is possibly the files are read before that. In this case, `ignoreCorruptFiles` config doesn't work too.
This patch targets Parquet datasource. If this direction is ok, we can address the same issue for other datasources like Orc.
Two main changes in this patch:
1. Replace `ParquetFileReader.readAllFootersInParallel` by implementing the logic to read footers in multi-threaded manner
We can't ignore corrupt files if we use `ParquetFileReader.readAllFootersInParallel`. So this patch implements the logic to do the similar thing in `readParquetFootersInParallel`.
2. In `FileScanRDD`, we need to ignore corrupt file too when we call `readFunction` to return iterator.
One thing to notice is:
We read schema from Parquet file's footer. The method to read footer `ParquetFileReader.readFooter` throws `RuntimeException`, instead of `IOException`, if it can't successfully read the footer. Please check out df9d8e4154/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java (L470). So this patch catches `RuntimeException`. One concern is that it might also shadow other runtime exceptions other than reading corrupt files.
## 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#16474 from viirya/fix-ignorecorrupted-parquet-files.
### What changes were proposed in this pull request?
```Scala
sql("CREATE TABLE tab (a STRING) STORED AS PARQUET")
// This table fetch is to fill the cache with zero leaf files
spark.table("tab").show()
sql(
s"""
|LOAD DATA LOCAL INPATH '$newPartitionDir' OVERWRITE
|INTO TABLE tab
""".stripMargin)
spark.table("tab").show()
```
In the above example, the returned result is empty after table loading. The metadata cache could be out of dated after loading new data into the table, because loading/inserting does not update the cache. So far, the metadata cache is only used for data source tables. Thus, for Hive serde tables, only `parquet` and `orc` formats are facing such issues, because the Hive serde tables in the format of parquet/orc could be converted to data source tables when `spark.sql.hive.convertMetastoreParquet`/`spark.sql.hive.convertMetastoreOrc` is on.
This PR is to refresh the metadata cache after processing the `LOAD DATA` command.
In addition, Spark SQL does not convert **partitioned** Hive tables (orc/parquet) to data source tables in the write path, but the read path is using the metadata cache for both **partitioned** and non-partitioned Hive tables (orc/parquet). That means, writing the partitioned parquet/orc tables still use `InsertIntoHiveTable`, instead of `InsertIntoHadoopFsRelationCommand`. To avoid reading the out-of-dated cache, `InsertIntoHiveTable` needs to refresh the metadata cache for partitioned tables. Note, it does not need to refresh the cache for non-partitioned parquet/orc tables, because it does not call `InsertIntoHiveTable` at all. Based on the comments, this PR will keep the existing logics unchanged. That means, we always refresh the table no matter whether the table is partitioned or not.
### How was this patch tested?
Added test cases in parquetSuites.scala
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16500 from gatorsmile/refreshInsertIntoHiveTable.
## What changes were proposed in this pull request?
After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.
This PR implement:
DataFrameWriter.saveAsTable work with hive format with overwrite mode
## How was this patch tested?
unit test added
Author: windpiger <songjun@outlook.com>
Closes#16549 from windpiger/saveAsTableWithHiveOverwrite.
## What changes were proposed in this pull request?
the offset of short is 4 in OffHeapColumnVector's putShorts, but actually it should be 2.
## How was this patch tested?
unit test
Author: Yucai Yu <yucai.yu@intel.com>
Closes#16555 from yucai/offheap_short.
Otherwise the open parentheses isn't closed in query plan descriptions of batch scans.
PushedFilters: [In(COL_A, [1,2,4,6,10,16,219,815], IsNotNull(COL_B), ...
Author: Andrew Ash <andrew@andrewash.com>
Closes#16558 from ash211/patch-9.
### What changes were proposed in this pull request?
`DataFrameWriter`'s [save() API](5d38f09f47/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala (L207)) is performing a unnecessary full filesystem scan for the saved files. The save() API is the most basic/core API in `DataFrameWriter`. We should avoid it.
The related PR: https://github.com/apache/spark/pull/16090
### How was this patch tested?
Updated the existing test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16481 from gatorsmile/saveFileScan.
## What changes were proposed in this pull request?
Currently in SQL we implement overwrites by calling fs.delete() directly on the original data. This is not ideal since we the original files end up deleted even if the job aborts. We should extend the commit protocol to allow file overwrites to be managed as well.
## How was this patch tested?
Existing tests. I also fixed a bunch of tests that were depending on the commit protocol implementation being set to the legacy mapreduce one.
cc rxin cloud-fan
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>
Closes#16554 from ericl/add-delete-protocol.
## What changes were proposed in this pull request?
This PR proposes to throw an exception for both jdbc APIs when user specified schemas are not allowed or useless.
**DataFrameReader.jdbc(...)**
``` scala
spark.read.schema(StructType(Nil)).jdbc(...)
```
**DataFrameReader.table(...)**
```scala
spark.read.schema(StructType(Nil)).table("usrdb.test")
```
## How was this patch tested?
Unit test in `JDBCSuite` and `DataFrameReaderWriterSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14451 from HyukjinKwon/SPARK-16848.
## What changes were proposed in this pull request?
We should be able to resolve a nested view. The main advantage is that if you update an underlying view, the current view also gets updated.
The new approach should be compatible with older versions of SPARK/HIVE, that means:
1. The new approach should be able to resolve the views that created by older versions of SPARK/HIVE;
2. The new approach should be able to resolve the views that are currently supported by SPARK SQL.
The new approach mainly brings in the following changes:
1. Add a new operator called `View` to keep track of the CatalogTable that describes the view, and the output attributes as well as the child of the view;
2. Update the `ResolveRelations` rule to resolve the relations and views, note that a nested view should be resolved correctly;
3. Add `viewDefaultDatabase` variable to `CatalogTable` to keep track of the default database name used to resolve a view, if the `CatalogTable` is not a view, then the variable should be `None`;
4. Add `AnalysisContext` to enable us to still support a view created with CTE/Windows query;
5. Enables the view support without enabling Hive support (i.e., enableHiveSupport);
6. Fix a weird behavior: the result of a view query may have different schema if the referenced table has been changed. After this PR, we try to cast the child output attributes to that from the view schema, throw an AnalysisException if cast is not allowed.
Note this is compatible with the views defined by older versions of Spark(before 2.2), which have empty `defaultDatabase` and all the relations in `viewText` have database part defined.
## How was this patch tested?
1. Add new tests in `SessionCatalogSuite` to test the function `lookupRelation`;
2. Add new test case in `SQLViewSuite` to test resolve a nested view.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16233 from jiangxb1987/resolve-view.
## What changes were proposed in this pull request?
Currently we have two sets of statistics in LogicalPlan: a simple stats and a stats estimated by cbo, but the computing logic and naming are quite confusing, we need to unify these two sets of stats.
## How was this patch tested?
Just modify existing tests.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#16529 from wzhfy/unifyStats.
## What changes were proposed in this pull request?
The analyzer rule that supports to query files directly will be added to `Analyzer.extendedResolutionRules` when SparkSession is created, according to the `spark.sql.runSQLOnFiles` flag. If the flag is off when we create `SparkSession`, this rule is not added and we can not query files directly even we turn on the flag later.
This PR fixes this bug by always adding that rule to `Analyzer.extendedResolutionRules`.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16531 from cloud-fan/sql-on-files.
## What changes were proposed in this pull request?
This PR allow update mode for non-aggregation streaming queries. It will be same as the append mode if a query has no aggregations.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16520 from zsxwing/update-without-agg.
## What changes were proposed in this pull request?
To support `FETCH_FIRST`, SPARK-16563 used Scala `Iterator.duplicate`. However,
Scala `Iterator.duplicate` uses a **queue to buffer all items between both iterators**,
this causes GC and hangs for queries with large number of rows. We should not use this,
especially for `spark.sql.thriftServer.incrementalCollect`.
https://github.com/scala/scala/blob/2.12.x/src/library/scala/collection/Iterator.scala#L1262-L1300
## How was this patch tested?
Pass the existing tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#16440 from dongjoon-hyun/SPARK-18857.
## What changes were proposed in this pull request?
This PR proposes to fix all the test failures identified by testing with AppVeyor.
**Scala - aborted tests**
```
WindowQuerySuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.WindowQuerySuite *** ABORTED *** (156 milliseconds)
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilespart_tiny.txt;
OrcSourceSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.orc.OrcSourceSuite *** ABORTED *** (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);
ParquetMetastoreSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetMetastoreSuite *** ABORTED *** (4 seconds, 703 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);
ParquetSourceSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetSourceSuite *** ABORTED *** (3 seconds, 907 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-581a6575-454f-4f21-a516-a07f95266143;
KafkaRDDSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaRDDSuite *** ABORTED *** (5 seconds, 212 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-4722304d-213e-4296-b556-951df1a46807
DirectKafkaStreamSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.DirectKafkaStreamSuite *** ABORTED *** (7 seconds, 127 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d0d3eba7-4215-4e10-b40e-bb797e89338e
at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010)
ReliableKafkaStreamSuite
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.ReliableKafkaStreamSuite *** ABORTED *** (5 seconds, 498 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d33e45a0-287e-4bed-acae-ca809a89d888
KafkaStreamSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaStreamSuite *** ABORTED *** (2 seconds, 892 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-59c9d169-5a56-4519-9ef0-cefdbd3f2e6c
KafkaClusterSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaClusterSuite *** ABORTED *** (1 second, 690 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-3ef402b0-8689-4a60-85ae-e41e274f179d
DirectKafkaStreamSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.DirectKafkaStreamSuite *** ABORTED *** (59 seconds, 626 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-426107da-68cf-4d94-b0d6-1f428f1c53f6
KafkaRDDSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.KafkaRDDSuite *** ABORTED *** (2 minutes, 6 seconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-b9ce7929-5dae-46ab-a0c4-9ef6f58fbc2
```
**Java - failed tests**
```
Test org.apache.spark.streaming.kafka.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-1cee32f4-4390-4321-82c9-e8616b3f0fb0, took 9.61 sec
Test org.apache.spark.streaming.kafka.JavaKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-f42695dd-242e-4b07-847c-f299b8e4676e, took 11.797 sec
Test org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-85c0d062-78cf-459c-a2dd-7973572101ce, took 1.581 sec
Test org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-49eb6b5c-8366-47a6-83f2-80c443c48280, took 17.895 sec
org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-898cf826-d636-4b1c-a61a-c12a364c02e7, took 8.858 sec
```
**Scala - failed tests**
```
PartitionProviderCompatibilitySuite:
- insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (828 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-bb6337b9-4f99-45ab-ad2c-a787ab965c09
- SPARK-18635 special chars in partition values - partition management true *** FAILED *** (5 seconds, 360 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 *** (141 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);
```
```
UtilsSuite:
- reading offset bytes of a file (compressed) *** FAILED *** (0 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ecb2b7d5-db8b-43a7-b268-1bf242b5a491
- reading offset bytes across multiple files (compressed) *** FAILED *** (0 milliseconds)
java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-25cc47a8-1faa-4da5-8862-cf174df63ce0
```
```
StatisticsSuite:
- MetastoreRelations fallback to HDFS for size estimation *** FAILED *** (110 milliseconds)
org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'csv_table' not found in database 'default';
```
```
SQLQuerySuite:
- permanent UDTF *** FAILED *** (125 milliseconds)
org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count_temp'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 24
- describe functions - user defined functions *** FAILED *** (125 milliseconds)
org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7
- CTAS without serde with location *** FAILED *** (16 milliseconds)
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-ed673d73-edfc-404e-829e-2e2b9725d94e/c1
- derived from Hive query file: drop_database_removes_partition_dirs.q *** FAILED *** (47 milliseconds)
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_database_removes_partition_dirs_table
- derived from Hive query file: drop_table_removes_partition_dirs.q *** FAILED *** (0 milliseconds)
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_table_removes_partition_dirs_table2
- SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH *** FAILED *** (109 milliseconds)
java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/sql/hive/projectsspark arget mpspark-1a122f8c-dfb3-46c4-bab1-f30764baee0e/*part-r*
```
```
HiveDDLSuite:
- drop external tables in default database *** 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);
- add/drop partitions - external 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);
- create/drop database - location without pre-created directory *** 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);
- create/drop database - location with pre-created directory *** FAILED *** (32 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);
- drop database containing tables - CASCADE *** FAILED *** (94 milliseconds)
CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)
- drop an empty database - CASCADE *** FAILED *** (63 milliseconds)
CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)
- drop database containing tables - RESTRICT *** FAILED *** (47 milliseconds)
CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)
- drop an empty database - RESTRICT *** FAILED *** (47 milliseconds)
CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)
- CREATE TABLE LIKE an external data source table *** FAILED *** (140 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c5eba16d-07ae-4186-95bb-21c5811cf888;
- CREATE TABLE LIKE an external Hive serde 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);
- desc table for data source table - no user-defined schema *** FAILED *** (125 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-e8bf5bf5-721a-4cbe-9d6 at scala.collection.immutable.List.foreach(List.scala:381)d-5543a8301c1d;
```
```
MetastoreDataSourcesSuite
- CTAS: persisted bucketed data source table *** FAILED *** (16 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
```
```
ShowCreateTableSuite:
- simple external hive table *** 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);
```
```
PartitionedTablePerfStatsSuite:
- hive table: partitioned pruned table reports only selected files *** FAILED *** (313 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);
- datasource table: partitioned pruned table reports only selected files *** FAILED *** (219 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-311f45f8-d064-4023-a4bb-e28235bff64d;
- hive table: lazy partition pruning reads only necessary partition data *** FAILED *** (203 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);
- datasource table: lazy partition pruning reads only necessary partition data *** FAILED *** (187 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-fde874ca-66bd-4d0b-a40f-a043b65bf957;
- hive table: lazy partition pruning with file status caching enabled *** FAILED *** (188 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);
- datasource table: lazy partition pruning with file status caching enabled *** FAILED *** (187 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-e6d20183-dd68-4145-acbe-4a509849accd;
- hive table: file status caching respects refresh table and refreshByPath *** FAILED *** (172 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);
- datasource table: file status caching respects refresh table and refreshByPath *** FAILED *** (203 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-8b2c9651-2adf-4d58-874f-659007e21463;
- hive table: file status cache respects size limit *** FAILED *** (219 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);
- datasource table: file status cache respects size limit *** FAILED *** (171 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-7835ab57-cb48-4d2c-bb1d-b46d5a4c47e4;
- datasource table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-20598d76-c004-42a7-8061-6c56f0eda5e2;
- hive table: table setup does not scan filesystem *** 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);
- hive table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 281 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);
- datasource table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 422 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4cfed321-4d1d-4b48-8d34-5c169afff383;
- hive table: files read and cached when filesource partition management is off *** FAILED *** (234 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);
- datasource table: all partition data cached in memory when partition management is off *** FAILED *** (203 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4bcc0398-15c9-4f6a-811e-12d40f3eec12;
- SPARK-18700: table loaded only once even when resolved concurrently *** FAILED *** (1 second, 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);
```
```
HiveSparkSubmitSuite:
- temporary Hive UDF: define a UDF and use it *** FAILED *** (2 seconds, 94 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- permanent Hive UDF: define a UDF and use it *** FAILED *** (281 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- permanent Hive UDF: use a already defined permanent function *** FAILED *** (718 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-8368: includes jars passed in through --jars *** FAILED *** (3 seconds, 521 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-8020: set sql conf in spark conf *** FAILED *** (0 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-8489: MissingRequirementError during reflection *** FAILED *** (94 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-9757 Persist Parquet relation with decimal column *** FAILED *** (16 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-11009 fix wrong result of Window function in cluster mode *** FAILED *** (16 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-14244 fix window partition size attribute binding failure *** FAILED *** (78 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- set spark.sql.warehouse.dir *** FAILED *** (16 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- set hive.metastore.warehouse.dir *** FAILED *** (15 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-16901: set javax.jdo.option.ConnectionURL *** FAILED *** (16 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
- SPARK-18360: default table path of tables in default database should depend on the location of default database *** FAILED *** (15 milliseconds)
java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```
```
UtilsSuite:
- resolveURIs with multiple paths *** FAILED *** (0 milliseconds)
".../jar3,file:/C:/pi.py[%23]py.pi,file:/C:/path%..." did not equal ".../jar3,file:/C:/pi.py[#]py.pi,file:/C:/path%..." (UtilsSuite.scala:468)
```
```
CheckpointSuite:
- recovery with file input stream *** FAILED *** (10 seconds, 205 milliseconds)
The code passed to eventually never returned normally. Attempted 660 times over 10.014272499999999 seconds. Last failure message: Unexpected internal error near index 1
\
^. (CheckpointSuite.scala:680)
```
## How was this patch tested?
Manually via AppVeyor as below:
**Scala - aborted tests**
```
WindowQuerySuite - all passed
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
ParquetMetastoreSuite - all passed
ParquetSourceSuite - all passed
KafkaRDDSuite - all passed
DirectKafkaStreamSuite - all passed
ReliableKafkaStreamSuite - all passed
KafkaStreamSuite - all passed
KafkaClusterSuite - all passed
DirectKafkaStreamSuite - all passed
KafkaRDDSuite - all passed
```
**Java - failed tests**
```
org.apache.spark.streaming.kafka.JavaKafkaRDDSuite - all passed
org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka.JavaKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite - all passed
```
**Scala - failed tests**
```
PartitionProviderCompatibilitySuite:
- insert overwrite partition of new datasource table overwrites just partition (1 second, 953 milliseconds)
- SPARK-18635 special chars in partition values - partition management true (6 seconds, 31 milliseconds)
- SPARK-18635 special chars in partition values - partition management false (4 seconds, 578 milliseconds)
```
```
UtilsSuite:
- reading offset bytes of a file (compressed) (203 milliseconds)
- reading offset bytes across multiple files (compressed) (0 milliseconds)
```
```
StatisticsSuite:
- MetastoreRelations fallback to HDFS for size estimation (94 milliseconds)
```
```
SQLQuerySuite:
- permanent UDTF (407 milliseconds)
- describe functions - user defined functions (441 milliseconds)
- CTAS without serde with location (2 seconds, 831 milliseconds)
- derived from Hive query file: drop_database_removes_partition_dirs.q (734 milliseconds)
- derived from Hive query file: drop_table_removes_partition_dirs.q (563 milliseconds)
- SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH (453 milliseconds)
```
```
HiveDDLSuite:
- drop external tables in default database (3 seconds, 5 milliseconds)
- add/drop partitions - external table (2 seconds, 750 milliseconds)
- create/drop database - location without pre-created directory (500 milliseconds)
- create/drop database - location with pre-created directory (407 milliseconds)
- drop database containing tables - CASCADE (453 milliseconds)
- drop an empty database - CASCADE (375 milliseconds)
- drop database containing tables - RESTRICT (328 milliseconds)
- drop an empty database - RESTRICT (391 milliseconds)
- CREATE TABLE LIKE an external data source table (953 milliseconds)
- CREATE TABLE LIKE an external Hive serde table (3 seconds, 782 milliseconds)
- desc table for data source table - no user-defined schema (1 second, 150 milliseconds)
```
```
MetastoreDataSourcesSuite
- CTAS: persisted bucketed data source table (875 milliseconds)
```
```
ShowCreateTableSuite:
- simple external hive table (78 milliseconds)
```
```
PartitionedTablePerfStatsSuite:
- hive table: partitioned pruned table reports only selected files (1 second, 109 milliseconds)
- datasource table: partitioned pruned table reports only selected files (860 milliseconds)
- hive table: lazy partition pruning reads only necessary partition data (859 milliseconds)
- datasource table: lazy partition pruning reads only necessary partition data (1 second, 219 milliseconds)
- hive table: lazy partition pruning with file status caching enabled (875 milliseconds)
- datasource table: lazy partition pruning with file status caching enabled (890 milliseconds)
- hive table: file status caching respects refresh table and refreshByPath (922 milliseconds)
- datasource table: file status caching respects refresh table and refreshByPath (640 milliseconds)
- hive table: file status cache respects size limit (469 milliseconds)
- datasource table: file status cache respects size limit (453 milliseconds)
- datasource table: table setup does not scan filesystem (328 milliseconds)
- hive table: table setup does not scan filesystem (313 milliseconds)
- hive table: num hive client calls does not scale with partition count (5 seconds, 431 milliseconds)
- datasource table: num hive client calls does not scale with partition count (4 seconds, 79 milliseconds)
- hive table: files read and cached when filesource partition management is off (656 milliseconds)
- datasource table: all partition data cached in memory when partition management is off (484 milliseconds)
- SPARK-18700: table loaded only once even when resolved concurrently (2 seconds, 578 milliseconds)
```
```
HiveSparkSubmitSuite:
- temporary Hive UDF: define a UDF and use it (1 second, 745 milliseconds)
- permanent Hive UDF: define a UDF and use it (406 milliseconds)
- permanent Hive UDF: use a already defined permanent function (375 milliseconds)
- SPARK-8368: includes jars passed in through --jars (391 milliseconds)
- SPARK-8020: set sql conf in spark conf (156 milliseconds)
- SPARK-8489: MissingRequirementError during reflection (187 milliseconds)
- SPARK-9757 Persist Parquet relation with decimal column (157 milliseconds)
- SPARK-11009 fix wrong result of Window function in cluster mode (156 milliseconds)
- SPARK-14244 fix window partition size attribute binding failure (156 milliseconds)
- set spark.sql.warehouse.dir (172 milliseconds)
- set hive.metastore.warehouse.dir (156 milliseconds)
- SPARK-16901: set javax.jdo.option.ConnectionURL (157 milliseconds)
- SPARK-18360: default table path of tables in default database should depend on the location of default database (172 milliseconds)
```
```
UtilsSuite:
- resolveURIs with multiple paths (0 milliseconds)
```
```
CheckpointSuite:
- recovery with file input stream (4 seconds, 452 milliseconds)
```
Note: after resolving the aborted tests, there is a test failure identified as below:
```
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
```
This does not look due to this problem so this PR does not fix it here.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16451 from HyukjinKwon/all-path-resource-fixes.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.