## What changes were proposed in this pull request?
In case of any bugs in whole-stage codegen, the generated code can't be compiled, we should fallback to non-codegen to make sure that query could run.
The batch mode of new parquet reader depends on codegen, can't be easily switched to non-batch mode, so we still use codegen for batched scan (for parquet). Because it only support primitive types and the number of columns is less than spark.sql.codegen.maxFields (100), it should not fail.
This could be configurable by `spark.sql.codegen.fallback`
## How was this patch tested?
Manual test it with buggy operator, it worked well.
Author: Davies Liu <davies@databricks.com>
Closes#13501 from davies/codegen_fallback.
## What changes were proposed in this pull request?
Spark currently incorrectly continues to use cached data even if the underlying data is overwritten.
Current behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
sqlContext.read.parquet(dir).count() // outputs 1000 <---- We are still using the cached dataset
```
This patch fixes this bug by adding support for `REFRESH path` that invalidates and refreshes all the cached data (and the associated metadata) for any dataframe that contains the given data source path.
Expected behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
spark.catalog.refreshResource(dir)
sqlContext.read.parquet(dir).count() // outputs 10 <---- We are not using the cached dataset
```
## How was this patch tested?
Unit tests for overwrites and appends in `ParquetQuerySuite` and `CachedTableSuite`.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#13566 from sameeragarwal/refresh-path-2.
## What changes were proposed in this pull request?
The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader.
## How was this patch tested?
Existing tests should be passed.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#13371 from viirya/vectorized-reader-push-down-filter.
## What changes were proposed in this pull request?
Serializer instantiation will consider existing SparkConf
## How was this patch tested?
manual test with `ImmutableList` (Guava) and `kryo-serializers`'s `Immutable*Serializer` implementations.
Added Test Suite.
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: Sela <ansela@paypal.com>
Closes#13424 from amitsela/SPARK-15489.
## What changes were proposed in this pull request?
Currently, we always split the files when it's bigger than maxSplitBytes, but Hadoop LineRecordReader does not respect the splits for compressed files correctly, we should have a API for FileFormat to check whether the file could be splitted or not.
This PR is based on #13442, closes#13442
## How was this patch tested?
add regression tests.
Author: Davies Liu <davies@databricks.com>
Closes#13531 from davies/fix_split.
## What changes were proposed in this pull request?
Code generated `SortMergeJoin` failed with wrong results when using structs as keys. This could (eventually) be traced back to the use of a wrong row reference when comparing structs.
## How was this patch tested?
TBD
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#13589 from hvanhovell/SPARK-15822.
## What changes were proposed in this pull request?
In scala, immutable.List.length is an expensive operation so we should
avoid using Seq.length == 0 or Seq.lenth > 0, and use Seq.isEmpty and Seq.nonEmpty instead.
## How was this patch tested?
existing tests
Author: wangyang <wangyang@haizhi.com>
Closes#13601 from yangw1234/isEmpty.
## What changes were proposed in this pull request?
Replace all occurrences of `None: Option[X]` with `Option.empty[X]`
## How was this patch tested?
Exisiting Tests
Author: Sandeep Singh <sandeep@techaddict.me>
Closes#13591 from techaddict/minor-7.
## What changes were proposed in this pull request?
This PR moves `QueryPlanner.planLater()` method into `GenericStrategy` for extra strategies to be able to use `planLater` in its strategy.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#13147 from ueshin/issues/SPARK-6320.
## What changes were proposed in this pull request?
When saving datasets on storage, `partitionBy` provides an easy way to construct the directory structure. However, if a user choose all columns as partition columns, some exceptions occurs.
- **ORC with all column partitioning**: `AnalysisException` on **future read** due to schema inference failure.
```scala
scala> spark.range(10).write.format("orc").mode("overwrite").partitionBy("id").save("/tmp/data")
scala> spark.read.format("orc").load("/tmp/data").collect()
org.apache.spark.sql.AnalysisException: Unable to infer schema for ORC at /tmp/data. It must be specified manually;
```
- **Parquet with all-column partitioning**: `InvalidSchemaException` on **write execution** due to Parquet limitation.
```scala
scala> spark.range(100).write.format("parquet").mode("overwrite").partitionBy("id").save("/tmp/data")
[Stage 0:> (0 + 8) / 8]16/06/02 16:51:17
ERROR Utils: Aborting task
org.apache.parquet.schema.InvalidSchemaException: A group type can not be empty. Parquet does not support empty group without leaves. Empty group: spark_schema
... (lots of error messages)
```
Although some formats like JSON support all-column partitioning without any problem, it seems not a good idea to make lots of empty directories.
This PR prevents saving with all-column partitioning by consistently raising `AnalysisException` before executing save operation.
## How was this patch tested?
Newly added `PartitioningUtilsSuite`.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13486 from dongjoon-hyun/SPARK-15743.
## What changes were proposed in this pull request?
SparkContext.listAccumulator, by Spark's convention, makes it sound like "list" is a verb and the method should return a list of accumulators. This patch renames the method and the class collection accumulator.
## How was this patch tested?
Updated test case to reflect the names.
Author: Reynold Xin <rxin@databricks.com>
Closes#13594 from rxin/SPARK-15866.
## What changes were proposed in this pull request?
This patch moves some codes in `DataFrameWriter.insertInto` that belongs to `Analyzer`.
## How was this patch tested?
Existing tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#13496 from viirya/move-analyzer-stuff.
## What changes were proposed in this pull request?
When the output mode is complete, then the output of a streaming aggregation essentially will contain the complete aggregates every time. So this is not different from a batch dataset within an incremental execution. Other non-streaming operations should be supported on this dataset. In this PR, I am just adding support for sorting, as it is a common useful functionality. Support for other operations will come later.
## How was this patch tested?
Additional unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#13549 from tdas/SPARK-15812.
## What changes were proposed in this pull request?
* Add DataFrameWriter.foreach to allow the user consuming data in ContinuousQuery
* ForeachWriter is the interface for the user to consume partitions of data
* Add a type parameter T to DataFrameWriter
Usage
```Scala
val ds = spark.read....stream().as[String]
ds.....write
.queryName(...)
.option("checkpointLocation", ...)
.foreach(new ForeachWriter[Int] {
def open(partitionId: Long, version: Long): Boolean = {
// prepare some resources for a partition
// check `version` if possible and return `false` if this is a duplicated data to skip the data processing.
}
override def process(value: Int): Unit = {
// process data
}
def close(errorOrNull: Throwable): Unit = {
// release resources for a partition
// check `errorOrNull` and handle the error if necessary.
}
})
```
## How was this patch tested?
New unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#13342 from zsxwing/foreach.
## What changes were proposed in this pull request?
The fix is pretty simple, just don't make the executedPlan transient in `ScalarSubquery` since it is referenced at execution time.
## How was this patch tested?
I verified the fix manually in non-local mode. It's not clear to me why the problem did not manifest in local mode, any suggestions?
cc davies
Author: Eric Liang <ekl@databricks.com>
Closes#13569 from ericl/fix-scalar-npe.
## What changes were proposed in this pull request?
SparkSession does not have that many functions due to better namespacing, and as a result we probably don't need the function grouping. This patch removes the grouping and also adds missing scaladocs for createDataset functions in SQLContext.
Closes#13577.
## How was this patch tested?
N/A - this is a documentation change.
Author: Reynold Xin <rxin@databricks.com>
Closes#13582 from rxin/SPARK-15850.
## What changes were proposed in this pull request?
This PR closes the input stream created in `HDFSMetadataLog.get`
## How was this patch tested?
Jenkins unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#13583 from zsxwing/leak.
## What changes were proposed in this pull request?
With very wide tables, e.g. thousands of fields, the plan output is unreadable and often causes OOMs due to inefficient string processing. This truncates all struct and operator field lists to a user configurable threshold to limit performance impact.
It would also be nice to optimize string generation to avoid these sort of O(n^2) slowdowns entirely (i.e. use StringBuilder everywhere including expressions), but this is probably too large of a change for 2.0 at this point, and truncation has other benefits for usability.
## How was this patch tested?
Added a microbenchmark that covers this case particularly well. I also ran the microbenchmark while varying the truncation threshold.
```
numFields = 5
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem) 2336 / 2558 0.0 23364.4 0.1X
numFields = 25
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem) 4237 / 4465 0.0 42367.9 0.1X
numFields = 100
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem) 10458 / 11223 0.0 104582.0 0.0X
numFields = Infinity
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
[info] java.lang.OutOfMemoryError: Java heap space
```
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>
Closes#13537 from ericl/truncated-string.
## What changes were proposed in this pull request?
The help function 'toStructType' in the AttributeSeq class doesn't include the metadata when it builds the StructField, so it causes this reported problem https://issues.apache.org/jira/browse/SPARK-15804?jql=project%20%3D%20SPARK when spark writes the the dataframe with the metadata to the parquet datasource.
The code path is when spark writes the dataframe to the parquet datasource through the InsertIntoHadoopFsRelationCommand, spark will build the WriteRelation container, and it will call the help function 'toStructType' to create StructType which contains StructField, it should include the metadata there, otherwise, we will lost the user provide metadata.
## How was this patch tested?
added test case in ParquetQuerySuite.scala
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: Kevin Yu <qyu@us.ibm.com>
Closes#13555 from kevinyu98/spark-15804.
## What changes were proposed in this pull request?
Documentation Fix
## How was this patch tested?
Author: Sandeep Singh <sandeep@techaddict.me>
Closes#13567 from techaddict/minor-4.
## What changes were proposed in this pull request?
On the SparkUI right now we have this SQLTab that displays accumulator values per operator. However, it only displays metrics updated on the executors, not on the driver. It is useful to also include driver metrics, e.g. broadcast time.
This is a different version from https://github.com/apache/spark/pull/12427. This PR sends driver side accumulator updates right after the updating happens, not at the end of execution, by a new event.
## How was this patch tested?
new test in `SQLListenerSuite`
![qq20160606-0](https://cloud.githubusercontent.com/assets/3182036/15841418/0eb137da-2c06-11e6-9068-5694eeb78530.png)
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13189 from cloud-fan/metrics.
## What changes were proposed in this pull request?
revived #13464
Fix Java Lint errors introduced by #13286 and #13280
Before:
```
Using `mvn` from path: /Users/pichu/Project/spark/build/apache-maven-3.3.9/bin/mvn
Java HotSpot(TM) 64-Bit Server VM warning: ignoring option MaxPermSize=512M; support was removed in 8.0
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[340,5] (whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[341,5] (whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[342,5] (whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[343,5] (whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/sql/streaming/OutputMode.java:[41,28] (naming) MethodName: Method name 'Append' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/java/org/apache/spark/sql/streaming/OutputMode.java:[52,28] (naming) MethodName: Method name 'Complete' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[61,8] (imports) UnusedImports: Unused import - org.apache.parquet.schema.PrimitiveType.
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[62,8] (imports) UnusedImports: Unused import - org.apache.parquet.schema.Type.
```
## How was this patch tested?
ran `dev/lint-java` locally
Author: Sandeep Singh <sandeep@techaddict.me>
Closes#13559 from techaddict/minor-3.
## What changes were proposed in this pull request?
This PR adds ContinuousQueryInfo to make ContinuousQueryListener events serializable in order to support writing events into the event log.
## How was this patch tested?
Jenkins unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#13335 from zsxwing/query-info.
## What changes were proposed in this pull request?
The current implementation of "CREATE TEMPORARY TABLE USING datasource..." is NOT creating any intermediate temporary data directory like temporary HDFS folder, instead, it only stores a SQL string in memory. Probably we should use "TEMPORARY VIEW" instead.
This PR assumes a temporary table has to link with some temporary intermediate data. It follows the definition of temporary table like this (from [hortonworks doc](https://docs.hortonworks.com/HDPDocuments/HDP2/HDP-2.3.0/bk_dataintegration/content/temp-tables.html)):
> A temporary table is a convenient way for an application to automatically manage intermediate data generated during a complex query
**Example**:
```
scala> spark.sql("CREATE temporary view my_tab7 (c1: String, c2: String) USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')")
scala> spark.sql("select c1, c2 from my_tab7").show()
+----+-----+
| c1| c2|
+----+-----+
|year| make|
|2012|Tesla|
...
```
It NOW prints a **deprecation warning** if "CREATE TEMPORARY TABLE USING..." is used.
```
scala> spark.sql("CREATE temporary table my_tab7 (c1: String, c2: String) USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')")
16/05/31 10:39:27 WARN SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE tableName USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13414 from clockfly/create_temp_view_using.
## What changes were proposed in this pull request?
This PR allows customization of verbosity in explain output. After change, `dataframe.explain()` and `dataframe.explain(true)` has different verbosity output for physical plan.
Currently, this PR only enables verbosity string for operator `HashAggregateExec` and `SortAggregateExec`. We will gradually enable verbosity string for more operators in future.
**Less verbose mode:** dataframe.explain(extended = false)
`output=[count(a)#85L]` is **NOT** displayed for HashAggregate.
```
scala> Seq((1,2,3)).toDF("a", "b", "c").createTempView("df2")
scala> spark.sql("select count(a) from df2").explain()
== Physical Plan ==
*HashAggregate(key=[], functions=[count(1)])
+- Exchange SinglePartition
+- *HashAggregate(key=[], functions=[partial_count(1)])
+- LocalTableScan
```
**Verbose mode:** dataframe.explain(extended = true)
`output=[count(a)#85L]` is displayed for HashAggregate.
```
scala> spark.sql("select count(a) from df2").explain(true) // "output=[count(a)#85L]" is added
...
== Physical Plan ==
*HashAggregate(key=[], functions=[count(1)], output=[count(a)#85L])
+- Exchange SinglePartition
+- *HashAggregate(key=[], functions=[partial_count(1)], output=[count#87L])
+- LocalTableScan
```
## How was this patch tested?
Manual test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13535 from clockfly/verbose_breakdown_2.
## What changes were proposed in this pull request?
This PR makes sure the typed Filter doesn't change the Dataset schema.
**Before the change:**
```
scala> val df = spark.range(0,9)
scala> df.schema
res12: org.apache.spark.sql.types.StructType = StructType(StructField(id,LongType,false))
scala> val afterFilter = df.filter(_=>true)
scala> afterFilter.schema // !!! schema is CHANGED!!! Column name is changed from id to value, nullable is changed from false to true.
res13: org.apache.spark.sql.types.StructType = StructType(StructField(value,LongType,true))
```
SerializeFromObject and DeserializeToObject are inserted to wrap the Filter, and these two can possibly change the schema of Dataset.
**After the change:**
```
scala> afterFilter.schema // schema is NOT changed.
res47: org.apache.spark.sql.types.StructType = StructType(StructField(id,LongType,false))
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13529 from clockfly/spark-15632.
BindReferences contains a n^2 loop which causes performance issues when operating over large schemas: to determine the ordinal of an attribute reference, we perform a linear scan over the `input` array. Because input can sometimes be a `List`, the call to `input(ordinal).nullable` can also be O(n).
Instead of performing a linear scan, we can convert the input into an array and build a hash map to map from expression ids to ordinals. The greater up-front cost of the map construction is offset by the fact that an expression can contain multiple attribute references, so the cost of the map construction is amortized across a number of lookups.
Perf. benchmarks to follow. /cc ericl
Author: Josh Rosen <joshrosen@databricks.com>
Closes#13505 from JoshRosen/bind-references-improvement.
## What changes were proposed in this pull request?
`an -> a`
Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one.
## How was this patch tested?
manual tests
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#13515 from zhengruifeng/an_a.
## What changes were proposed in this pull request?
This pr fixes the behaviour of `format("csv").option("quote", null)` along with one of spark-csv.
Also, it explicitly sets default values for CSV options in python.
## How was this patch tested?
Added tests in CSVSuite.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#13372 from maropu/SPARK-15585.
## What changes were proposed in this pull request?
This change fixes a crash in TungstenAggregate while executing "Dataset complex Aggregator" test case due to IndexOutOfBoundsException.
jira entry for detail: https://issues.apache.org/jira/browse/SPARK-15704
## How was this patch tested?
Using existing unit tests (including DatasetBenchmark)
Author: Hiroshi Inoue <inouehrs@jp.ibm.com>
Closes#13446 from inouehrs/fix_aggregate.
`PartitionStatistics` uses `foldLeft` and list concatenation (`++`) to flatten an iterator of lists, but this is extremely inefficient compared to simply doing `flatMap`/`flatten` because it performs many unnecessary object allocations. Simply replacing this `foldLeft` by a `flatMap` results in decent performance gains when constructing PartitionStatistics instances for tables with many columns.
This patch fixes this and also makes two similar changes in MLlib and streaming to try to fix all known occurrences of this pattern.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#13491 from JoshRosen/foldleft-to-flatmap.
## What changes were proposed in this pull request?
Now Spark SQL can support 'create table src stored as orc/parquet/avro' for orc/parquet/avro table. But Hive can support both commands: ' stored as orc/parquet/avro' and 'stored as orcfile/parquetfile/avrofile'.
So this PR supports these keywords 'orcfile/parquetfile/avrofile' in Spark SQL.
## How was this patch tested?
add unit tests
Author: Lianhui Wang <lianhuiwang09@gmail.com>
Closes#13500 from lianhuiwang/SPARK-15756.
## What changes were proposed in this pull request?
Currently, the memory for temporary buffer used by TimSort is always allocated as on-heap without bookkeeping, it could cause OOM both in on-heap and off-heap mode.
This PR will try to manage that by preallocate it together with the pointer array, same with RadixSort. It both works for on-heap and off-heap mode.
This PR also change the loadFactor of BytesToBytesMap to 0.5 (it was 0.70), it enables use to radix sort also makes sure that we have enough memory for timsort.
## How was this patch tested?
Existing tests.
Author: Davies Liu <davies@databricks.com>
Closes#13318 from davies/fix_timsort.
## What changes were proposed in this pull request?
As of this patch, the following throws an exception because the schemas may not match:
```
CREATE TABLE students (age INT, name STRING) AS SELECT * FROM boxes
```
but this is OK:
```
CREATE TABLE students AS SELECT * FROM boxes
```
## How was this patch tested?
SQLQuerySuite, HiveDDLCommandSuite
Author: Andrew Or <andrew@databricks.com>
Closes#13490 from andrewor14/ctas-no-column.
## What changes were proposed in this pull request?
For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow row to be null, only its columns can be null.
This PR explicitly add this constraint and throw exception if users break it.
## How was this patch tested?
several new tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13469 from cloud-fan/null-object.
## What changes were proposed in this pull request?
Queries with scalar sub-query in the SELECT list run against a local, in-memory relation throw
UnsupportedOperationException exception.
Problem repro:
```SQL
scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1")
scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2")
scala> sql("select (select min(c1) from t2) from t1").show()
java.lang.UnsupportedOperationException: Cannot evaluate expression: scalar-subquery#62 []
at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.eval(Expression.scala:215)
at org.apache.spark.sql.catalyst.expressions.ScalarSubquery.eval(subquery.scala:62)
at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142)
at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:45)
at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:29)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.immutable.List.foreach(List.scala:381)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.immutable.List.map(List.scala:285)
at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$37.applyOrElse(Optimizer.scala:1473)
```
The problem is specific to local, in memory relations. It is caused by rule ConvertToLocalRelation, which attempts to push down
a scalar-subquery expression to the local tables.
The solution prevents the rule to apply if Project references scalar subqueries.
## How was this patch tested?
Added regression tests to SubquerySuite.scala
Author: Ioana Delaney <ioanamdelaney@gmail.com>
Closes#13418 from ioana-delaney/scalarSubV2.
## What changes were proposed in this pull request?
Our encoder framework has been evolved a lot, this PR tries to clean up the code to make it more readable and emphasise the concept that encoder should be used as a container of serde expressions.
1. move validation logic to analyzer instead of encoder
2. only have a `resolveAndBind` method in encoder instead of `resolve` and `bind`, as we don't have the encoder life cycle concept anymore.
3. `Dataset` don't need to keep a resolved encoder, as there is no such concept anymore. bound encoder is still needed to do serialization outside of query framework.
4. Using `BoundReference` to represent an unresolved field in deserializer expression is kind of weird, this PR adds a `GetColumnByOrdinal` for this purpose. (serializer expression still use `BoundReference`, we can replace it with `GetColumnByOrdinal` in follow-ups)
## How was this patch tested?
existing test
Author: Wenchen Fan <wenchen@databricks.com>
Author: Cheng Lian <lian@databricks.com>
Closes#13269 from cloud-fan/clean-encoder.
## What changes were proposed in this pull request?
For consistency, this PR updates some remaining `TungstenAggregation/SortBasedAggregate` after SPARK-15728.
- Update a comment in codegen in `VectorizedHashMapGenerator.scala`.
- `TungstenAggregationQuerySuite` --> `HashAggregationQuerySuite`
- `TungstenAggregationQueryWithControlledFallbackSuite` --> `HashAggregationQueryWithControlledFallbackSuite`
- Update two error messages in `SQLQuerySuite.scala` and `AggregationQuerySuite.scala`.
- Update several comments.
## How was this patch tested?
Manual (Only comment changes and test suite renamings).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13487 from dongjoon-hyun/SPARK-15744.
## What changes were proposed in this pull request?
##### The root cause:
When `DataSource.resolveRelation` is trying to build `ListingFileCatalog` object, `ListLeafFiles` is invoked where a list of `FileStatus` objects are retrieved from the provided path. These FileStatus objects include directories for the partitions (id=0 and id=2 in the jira). However, these directory `FileStatus` objects also try to invoke `getFileBlockLocations` where directory is not allowed for `DistributedFileSystem`, hence the exception happens.
This PR is to remove the block of code that invokes `getFileBlockLocations` for every FileStatus object of the provided path. Instead, we call `HadoopFsRelation.listLeafFiles` directly because this utility method filters out the directories before calling `getFileBlockLocations` for generating `LocatedFileStatus` objects.
## How was this patch tested?
Regtest is run. Manual test:
```
scala> spark.read.format("parquet").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_part").show
+-----+---+
| text| id|
+-----+---+
|hello| 0|
|world| 0|
|hello| 1|
|there| 1|
+-----+---+
spark.read.format("orc").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_orc").show
+-----+---+
| text| id|
+-----+---+
|hello| 0|
|world| 0|
|hello| 1|
|there| 1|
+-----+---+
```
I also tried it with 2 level of partitioning.
I have not found a way to add test case in the unit test bucket that can test a real hdfs file location. Any suggestions will be appreciated.
Author: Xin Wu <xinwu@us.ibm.com>
Closes#13463 from xwu0226/SPARK-14959.
## What changes were proposed in this pull request?
This adds microbenchmarks for tracking performance of queries over very wide or deeply nested DataFrames. It seems performance degrades when DataFrames get thousands of columns wide or hundreds of fields deep.
## How was this patch tested?
Current results included.
cc rxin JoshRosen
Author: Eric Liang <ekl@databricks.com>
Closes#13456 from ericl/sc-3468.
## What changes were proposed in this pull request?
When users create a case class and use java reserved keyword as field name, spark sql will generate illegal java code and throw exception at runtime.
This PR checks the field names when building the encoder, and if illegal field names are used, throw exception immediately with a good error message.
## How was this patch tested?
new test in DatasetSuite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13485 from cloud-fan/java.
## What changes were proposed in this pull request?
Currently we don't support bucketing for `save` and `insertInto`.
For `save`, we just write the data out into a directory users specified, and it's not a table, we don't keep its metadata. When we read it back, we have no idea if the data is bucketed or not, so it doesn't make sense to use `save` to write bucketed data, as we can't use the bucket information anyway.
We can support it in the future, once we have features like bucket discovery, or we save bucket information in the data directory too, so that we don't need to rely on a metastore.
For `insertInto`, it inserts data into an existing table, so it doesn't make sense to specify bucket information, as we should get the bucket information from the existing table.
This PR improves the error message for the above 2 cases.
## How was this patch tested?
new test in `BukctedWriteSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13452 from cloud-fan/error-msg.
## What changes were proposed in this pull request?
This PR disables writing Parquet summary files by default (i.e., when Hadoop configuration "parquet.enable.summary-metadata" is not set).
Please refer to [SPARK-15719][1] for more details.
## How was this patch tested?
New test case added in `ParquetQuerySuite` to check no summary files are written by default.
[1]: https://issues.apache.org/jira/browse/SPARK-15719
Author: Cheng Lian <lian@databricks.com>
Closes#13455 from liancheng/spark-15719-disable-parquet-summary-files.
## What changes were proposed in this pull request?
This PR bans syntax like `CREATE TEMPORARY TABLE USING AS SELECT`
`CREATE TEMPORARY TABLE ... USING ... AS ...` is not properly implemented, the temporary data is not cleaned up when the session exits. Before a full fix, we probably should ban this syntax.
This PR only impact syntax like `CREATE TEMPORARY TABLE ... USING ... AS ...`.
Other syntax like `CREATE TEMPORARY TABLE .. USING ...` and `CREATE TABLE ... USING ...` are not impacted.
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13451 from clockfly/ban_create_temp_table_using_as.
#### What changes were proposed in this pull request?
This PR is to address the following issues:
- **ISSUE 1:** For ORC source format, we are reporting the strange error message when we did not enable Hive support:
```SQL
SQL Example:
select id from `org.apache.spark.sql.hive.orc`.`file_path`
Error Message:
Table or view not found: `org.apache.spark.sql.hive.orc`.`file_path`
```
Instead, we should issue the error message like:
```
Expected Error Message:
The ORC data source must be used with Hive support enabled
```
- **ISSUE 2:** For the Avro format, we report the strange error message like:
The example query is like
```SQL
SQL Example:
select id from `avro`.`file_path`
select id from `com.databricks.spark.avro`.`file_path`
Error Message:
Table or view not found: `com.databricks.spark.avro`.`file_path`
```
The desired message should be like:
```
Expected Error Message:
Failed to find data source: avro. Please use Spark package http://spark-packages.org/package/databricks/spark-avro"
```
- ~~**ISSUE 3:** Unable to detect incompatibility libraries for Spark 2.0 in Data Source Resolution. We report a strange error message:~~
**Update**: The latest code changes contains
- For JDBC format, we added an extra checking in the rule `ResolveRelations` of `Analyzer`. Without the PR, Spark will return the error message like: `Option 'url' not specified`. Now, we are reporting `Unsupported data source type for direct query on files: jdbc`
- Make data source format name case incensitive so that error handling behaves consistent with the normal cases.
- Added the test cases for all the supported formats.
#### How was this patch tested?
Added test cases to cover all the above issues
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#13283 from gatorsmile/runSQLAgainstFile.