#### What changes were proposed in this pull request?
This PR is to support star expansion in hash. For example,
```SQL
val structDf = testData2.select("a", "b").as("record")
structDf.select(hash($"*")
```
In addition, it refactors the codes for the rule `ResolveStar` and fixes a regression for star expansion in group by when using SQL API. For example,
```SQL
SELECT * FROM testData2 group by a, b
```
cc cloud-fan Now, the code for star resolution is much cleaner. The coverage is better. Could you check if this refactoring is good? Thanks!
#### How was this patch tested?
Added a few test cases to cover it.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11904 from gatorsmile/starResolution.
## What changes were proposed in this pull request?
`SessionCatalog`, introduced in #11750, is a catalog that keeps track of temporary functions and tables, and delegates metastore operations to `ExternalCatalog`. This functionality overlaps a lot with the existing `analysis.Catalog`.
As of this commit, `SessionCatalog` and `ExternalCatalog` will no longer be dead code. There are still things that need to be done after this patch, namely:
- SPARK-14013: Properly implement temporary functions in `SessionCatalog`
- SPARK-13879: Decide which DDL/DML commands to support natively in Spark
- SPARK-?????: Implement the ones we do want to support through `SessionCatalog`.
- SPARK-?????: Merge SQL/HiveContext
## How was this patch tested?
This is largely a refactoring task so there are no new tests introduced. The particularly relevant tests are `SessionCatalogSuite` and `ExternalCatalogSuite`.
Author: Andrew Or <andrew@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#11836 from andrewor14/use-session-catalog.
This PR adds a new `Sink` implementation that writes out Parquet files. In order to correctly handle partial failures while maintaining exactly once semantics, the files for each batch are written out to a unique directory and then atomically appended to a metadata log. When a parquet based `DataSource` is initialized for reading, we first check for this log directory and use it instead of file listing when present.
Unit tests are added, as well as a stress test that checks the answer after non-deterministic injected failures.
Author: Michael Armbrust <michael@databricks.com>
Closes#11897 from marmbrus/fileSink.
This PR introduces a 64-bit hashcode expression. Such an expression is especially usefull for HyperLogLog++ and other probabilistic datastructures.
I have implemented xxHash64 which is a 64-bit hashing algorithm created by Yann Colet and Mathias Westerdahl. This is a high speed (C implementation runs at memory bandwidth) and high quality hashcode. It exploits both Instruction Level Parralellism (for speed) and the multiplication and rotation techniques (for quality) like MurMurHash does.
The initial results are promising. I have added a CG'ed test to the `HashBenchmark`, and this results in the following results (running from SBT):
Running benchmark: Hash For simple
Running case: interpreted version
Running case: codegen version
Running case: codegen version 64-bit
Intel(R) Core(TM) i7-4750HQ CPU 2.00GHz
Hash For simple: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
interpreted version 1011 / 1016 132.8 7.5 1.0X
codegen version 1864 / 1869 72.0 13.9 0.5X
codegen version 64-bit 1614 / 1644 83.2 12.0 0.6X
Running benchmark: Hash For normal
Running case: interpreted version
Running case: codegen version
Running case: codegen version 64-bit
Intel(R) Core(TM) i7-4750HQ CPU 2.00GHz
Hash For normal: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
interpreted version 2467 / 2475 0.9 1176.1 1.0X
codegen version 2008 / 2115 1.0 957.5 1.2X
codegen version 64-bit 728 / 758 2.9 347.0 3.4X
Running benchmark: Hash For array
Running case: interpreted version
Running case: codegen version
Running case: codegen version 64-bit
Intel(R) Core(TM) i7-4750HQ CPU 2.00GHz
Hash For array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
interpreted version 1544 / 1707 0.1 11779.6 1.0X
codegen version 2728 / 2745 0.0 20815.5 0.6X
codegen version 64-bit 2508 / 2549 0.1 19132.8 0.6X
Running benchmark: Hash For map
Running case: interpreted version
Running case: codegen version
Running case: codegen version 64-bit
Intel(R) Core(TM) i7-4750HQ CPU 2.00GHz
Hash For map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
interpreted version 1819 / 1826 0.0 444014.3 1.0X
codegen version 183 / 194 0.0 44642.9 9.9X
codegen version 64-bit 173 / 174 0.0 42120.9 10.5X
This shows that algorithm is consistently faster than MurMurHash32 in all cases and up to 3x (!) in the normal case.
I have also added this to HyperLogLog++ and it cuts the processing time of the following code in half:
val df = sqlContext.range(1<<25).agg(approxCountDistinct("id"))
df.explain()
val t = System.nanoTime()
df.show()
val ns = System.nanoTime() - t
// Before
ns: Long = 5821524302
// After
ns: Long = 2836418963
cc cloud-fan (you have been working on hashcodes) / rxin
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#11209 from hvanhovell/xxHash.
## What changes were proposed in this pull request?
In this PR, I am implementing a new abstraction for management of streaming state data - State Store. It is a key-value store for persisting running aggregates for aggregate operations in streaming dataframes. The motivation and design is discussed here.
https://docs.google.com/document/d/1-ncawFx8JS5Zyfq1HAEGBx56RDet9wfVp_hDM8ZL254/edit#
## How was this patch tested?
- [x] Unit tests
- [x] Cluster tests
**Coverage from unit tests**
<img width="952" alt="screen shot 2016-03-21 at 3 09 40 pm" src="https://cloud.githubusercontent.com/assets/663212/13935872/fdc8ba86-ef76-11e5-93e8-9fa310472c7b.png">
## TODO
- [x] Fix updates() iterator to avoid duplicate updates for same key
- [x] Use Coordinator in ContinuousQueryManager
- [x] Plugging in hadoop conf and other confs
- [x] Unit tests
- [x] StateStore object lifecycle and methods
- [x] StateStoreCoordinator communication and logic
- [x] StateStoreRDD fault-tolerance
- [x] StateStoreRDD preferred location using StateStoreCoordinator
- [ ] Cluster tests
- [ ] Whether preferred locations are set correctly
- [ ] Whether recovery works correctly with distributed storage
- [x] Basic performance tests
- [x] Docs
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#11645 from tdas/state-store.
## What changes were proposed in this pull request?
This PR adds support for TimestampType in the vectorized parquet reader
## How was this patch tested?
1. `VectorizedColumnReader` initially had a gating condition on `primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT96)` that made us fall back on parquet-mr for handling timestamps. This condition is now removed.
2. The `ParquetHadoopFsRelationSuite` (that tests for all supported hive types -- including `TimestampType`) fails when the gating condition is removed (https://github.com/apache/spark/pull/11808) and should now pass with this change. Similarly, the `ParquetHiveCompatibilitySuite.SPARK-10177 timestamp` test that fails when the gating condition is removed, should now pass as well.
3. Added tests in `HadoopFsRelationTest` that test both the dictionary encoded and non-encoded versions across all supported datatypes.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11882 from sameeragarwal/timestamp-parquet.
## What changes were proposed in this pull request?
This PR rollback some changes in #11274 , which introduced some performance regression when do a simple aggregation on parquet scan with one integer column.
Does not really understand how this change introduce this huge impact, maybe related show JIT compiler inline functions. (saw very different stats from profiling).
## How was this patch tested?
Manually run the parquet reader benchmark, before this change:
```
Intel(R) Core(TM) i7-4558U CPU 2.80GHz
Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
SQL Parquet Vectorized 2391 / 3107 43.9 22.8 1.0X
```
After this change
```
Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5
Intel(R) Core(TM) i7-4558U CPU 2.80GHz
Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
SQL Parquet Vectorized 2032 / 2626 51.6 19.4 1.0X```
Author: Davies Liu <davies@databricks.com>
Closes#11912 from davies/fix_regression.
This patch refactors the `MemoryStore` so that it can be tested without needing to construct / mock an entire `BlockManager`.
- The block manager's serialization- and compression-related methods have been moved from `BlockManager` to `SerializerManager`.
- `BlockInfoManager `is now passed directly to classes that need it, rather than being passed via the `BlockManager`.
- The `MemoryStore` now calls `dropFromMemory` via a new `BlockEvictionHandler` interface rather than directly calling the `BlockManager`. This change helps to enforce a narrow interface between the `MemoryStore` and `BlockManager` functionality and makes this interface easier to mock in tests.
- Several of the block unrolling tests have been moved from `BlockManagerSuite` into a new `MemoryStoreSuite`.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#11899 from JoshRosen/reduce-memorystore-blockmanager-coupling.
#### What changes were proposed in this pull request?
The PR https://github.com/apache/spark/pull/10541 changed the rule `CollapseProject` by enabling collapsing `Project` into `Aggregate`. It leaves a to-do item to remove the duplicate code. This PR is to finish this to-do item. Also added a test case for covering this change.
#### How was this patch tested?
Added a new test case.
liancheng Could you check if the code refactoring is fine? Thanks!
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11427 from gatorsmile/collapseProjectRefactor.
## What changes were proposed in this pull request?
This PR does the renaming as suggested by marmbrus in [this comment][1].
## How was this patch tested?
Existing tests.
[1]: 6d37e1eb90 (commitcomment-16654694)
Author: Cheng Lian <lian@databricks.com>
Closes#11889 from liancheng/spark-13817-follow-up.
## What changes were proposed in this pull request?
Try to fix a flaky hang
## How was this patch tested?
Existing Jenkins test
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#11909 from zsxwing/hotfix2.
## What changes were proposed in this pull request?
1. Deprecated unionAll. It is pretty confusing to have both "union" and "unionAll" when the two do the same thing in Spark but are different in SQL.
2. Rename reduce in KeyValueGroupedDataset to reduceGroups so it is more consistent with rest of the functions in KeyValueGroupedDataset. Also makes it more obvious what "reduce" and "reduceGroups" mean. Previously it was confusing because it could be reducing a Dataset, or just reducing groups.
3. Added a "name" function, which is more natural to name columns than "as" for non-SQL users.
4. Remove "subtract" function since it is just an alias for "except".
## How was this patch tested?
All changes should be covered by existing tests. Also added couple test cases to cover "name".
Author: Reynold Xin <rxin@databricks.com>
Closes#11908 from rxin/SPARK-14088.
## What changes were proposed in this pull request?
This PR updates `sql/README.md` according to the latest console output and removes some unused imports in `sql` module. This is done by manually, so there is no guarantee to remove all unused imports.
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11907 from dongjoon-hyun/update_sql_module.
## What changes were proposed in this pull request?
This fix tries to fix several SQL test warnings under the sql/core/src/test directory. The fixed warnings includes "[unchecked]", "[rawtypes]", and "[varargs]".
## How was this patch tested?
All existing tests passed.
Author: Yong Tang <yong.tang.github@outlook.com>
Closes#11857 from yongtang/SPARK-13401.
## What changes were proposed in this pull request?
Round() in database usually round the number up (away from zero), it's different than Math.round() in Java.
For example:
```
scala> java.lang.Math.round(-3.5)
res3: Long = -3
```
In Database, we should return -4.0 in this cases.
This PR remove the buggy special case for scale=0.
## How was this patch tested?
Add tests for negative values with tie.
Author: Davies Liu <davies@databricks.com>
Closes#11894 from davies/fix_round.
## What changes were proposed in this pull request?
Fix an issue that DataFrameReaderWriterSuite may hang forever.
## How was this patch tested?
Existing tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#11902 from zsxwing/hotfix.
## What changes were proposed in this pull request?
This patch moves StringToColumn implicit class into SQLImplicits. This was kept in SQLContext.implicits object for binary backward compatibility, in the Spark 1.x series. It makes more sense for this API to be in SQLImplicits since that's the single class that defines all the SQL implicits.
## How was this patch tested?
Should be covered by existing unit tests.
Author: Reynold Xin <rxin@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11878 from rxin/SPARK-14060.
## What changes were proposed in this pull request?
This patch changed the return type for SQLContext.range from `Dataset[Long]` (Scala primitive) to `Dataset[java.lang.Long]` (Java boxed long).
Previously, SPARK-13894 changed the return type of range from `Dataset[Row]` to `Dataset[Long]`. The problem is that due to https://issues.scala-lang.org/browse/SI-4388, Scala compiles primitive types in generics into just Object, i.e. range at bytecode level now just returns `Dataset[Object]`. This is really bad for Java users because they are losing type safety and also need to add a type cast every time they use range.
Talked to Jason Zaugg from Lightbend (Typesafe) who suggested the best approach is to return `Dataset[java.lang.Long]`. The downside is that when Scala users want to explicitly type a closure used on the dataset returned by range, they would need to use `java.lang.Long` instead of the Scala `Long`.
## How was this patch tested?
The signature change should be covered by existing unit tests and API tests. I also added a new test case in DatasetSuite for range.
Author: Reynold Xin <rxin@databricks.com>
Closes#11880 from rxin/SPARK-14063.
This PR relaxes the requirements of a `Sink` for structured streaming to only require idempotent appending of data. Previously the `Sink` needed to be able to transactionally append data while recording an opaque offset indicated how far in a stream we have processed.
In order to do this, a new write-ahead-log has been added to stream execution, which records the offsets that will are present in each batch. The log is created in the newly added `checkpointLocation`, which defaults to `${spark.sql.streaming.checkpointLocation}/${queryName}` but can be overriden by setting `checkpointLocation` in `DataFrameWriter`.
In addition to making sinks easier to write the addition of batchIds and a checkpoint location is done in anticipation of integration with the the `StateStore` (#11645).
Author: Michael Armbrust <michael@databricks.com>
Closes#11804 from marmbrus/batchIds.
## What changes were proposed in this pull request?
Currently, **BooleanSimplification** optimization can handle the following cases.
* a && (!a || b ) ==> a && b
* a && (b || !a ) ==> a && b
However, it can not handle the followings cases since those equations fail at the comparisons between their canonicalized forms.
* a < 1 && (!(a < 1) || b) ==> (a < 1) && b
* a <= 1 && (!(a <= 1) || b) ==> (a <= 1) && b
* a > 1 && (!(a > 1) || b) ==> (a > 1) && b
* a >= 1 && (!(a >= 1) || b) ==> (a >= 1) && b
This PR implements the above cases and also the followings, too.
* a < 1 && ((a >= 1) || b ) ==> (a < 1) && b
* a <= 1 && ((a > 1) || b ) ==> (a <= 1) && b
* a > 1 && ((a <= 1) || b) ==> (a > 1) && b
* a >= 1 && ((a < 1) || b) ==> (a >= 1) && b
## How was this patch tested?
Pass the Jenkins tests including new test cases in BooleanSimplicationSuite.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11851 from dongjoon-hyun/SPARK-14029.
SPARK-13774: IllegalArgumentException: Can not create a Path from an empty string for incorrect file path
**Overview:**
- If a non-existent path is given in this call
``
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
``
it throws the following error:
`java.lang.IllegalArgumentException: Can not create a Path from an empty string` …..
`It gets called from inferSchema call in org.apache.spark.sql.execution.datasources.DataSource.resolveRelation`
- The purpose of this JIRA is to throw a better error message.
- With the fix, you will now get a _Path does not exist_ error message.
```
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
org.apache.spark.sql.AnalysisException: Path does not exist: file:/Users/ksunitha/trunk/spark/file-path-is-incorrect.csv;
at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:215)
at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:204)
...
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:204)
at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:131)
at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:141)
... 49 elided
```
**Details**
_Changes include:_
- Check if path exists or not in resolveRelation in DataSource, and throw an AnalysisException with message like “Path does not exist: $path”
- AnalysisException is thrown similar to the exceptions thrown in resolveRelation.
- The glob path and the non glob path is checked with minimal calls to path exists. If the globPath is empty, then it is a nonexistent glob pattern and an error will be thrown. In the scenario that it is not globPath, it is necessary to only check if the first element in the Seq is valid or not.
_Test modifications:_
- Changes went in for 3 tests to account for this error checking.
- SQLQuerySuite:test("run sql directly on files") – Error message needed to be updated.
- 2 tests failed in MetastoreDataSourcesSuite because they had a dummy path and so test is modified to give a tempdir and allow it to move past so it can continue to test the codepath it meant to test
_New Tests:_
2 new tests are added to DataFrameSuite to validate that glob and non-glob path will throw the new error message.
_Testing:_
Unit tests were run with the fix.
**Notes/Questions to reviewers:**
- There is some code duplication in DataSource.scala in resolveRelation method and also createSource with respect to getting the paths. I have not made any changes to the createSource codepath. Should we make the change there as well ?
- From other JIRAs, I know there is restructuring and changes going on in this area, not sure how that will affect these changes, but since this seemed like a starter issue, I looked into it. If we prefer not to add the overhead of the checks, or if there is a better place to do so, let me know.
I would appreciate your review. Thanks for your time and comments.
Author: Sunitha Kambhampati <skambha@us.ibm.com>
Closes#11775 from skambha/improve_errmsg.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13953
Currently, JSON data source creates a new field in `PERMISSIVE` mode for storing malformed string.
This field can be renamed via `spark.sql.columnNameOfCorruptRecord` option but it is a global configuration.
This PR make that option can be applied per read and can be specified via `option()`. This will overwrites `spark.sql.columnNameOfCorruptRecord` if it is set.
## How was this patch tested?
Unit tests were used and `./dev/run_tests` for coding style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11881 from HyukjinKwon/SPARK-13953.
## What changes were proposed in this pull request?
This is a follow-up of PR #11348.
After PR #11348, a predicate is never pushed through a project as long as the project contains any non-deterministic fields. Thus, it's impossible that the candidate filter condition can reference any non-deterministic projected fields, and related logic can be safely cleaned up.
To be more specific, the following optimization is allowed:
```scala
// From:
df.select('a, 'b).filter('c > rand(42))
// To:
df.filter('c > rand(42)).select('a, 'b)
```
while this isn't:
```scala
// From:
df.select('a, rand('b) as 'rb, 'c).filter('c > 'rb)
// To:
df.filter('c > rand('b)).select('a, rand('b) as 'rb, 'c)
```
## How was this patch tested?
Existing test cases should do the work.
Author: Cheng Lian <lian@databricks.com>
Closes#11864 from liancheng/spark-13473-cleanup.
## What changes were proposed in this pull request?
As we have completed the `SQLBuilder`, we can safely turn on native view by default.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11872 from cloud-fan/native-view.
This PR add implements the new `buildReader` interface for the Parquet `FileFormat`. An simple implementation of `FileScanRDD` is also included.
This code should be tested by the many existing tests for parquet.
Author: Michael Armbrust <michael@databricks.com>
Author: Sameer Agarwal <sameer@databricks.com>
Author: Nong Li <nong@databricks.com>
Closes#11709 from marmbrus/parquetReader.
## What changes were proposed in this pull request?
This patch adds support for reading `DecimalTypes` with high (> 18) precision in `VectorizedColumnReader`
## How was this patch tested?
1. `VectorizedColumnReader` initially had a gating condition on `primitiveType.getDecimalMetadata().getPrecision() > Decimal.MAX_LONG_DIGITS()` that made us fall back on parquet-mr for handling high-precision decimals. This condition is now removed.
2. In particular, the `ParquetHadoopFsRelationSuite` (that tests for all supported hive types -- including `DecimalType(25, 5)`) fails when the gating condition is removed (https://github.com/apache/spark/pull/11808) and should now pass with this change.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11869 from sameeragarwal/bigdecimal-parquet.
This PR resolves two issues:
First, expanding * inside aggregate functions of structs when using Dataframe/Dataset APIs. For example,
```scala
structDf.groupBy($"a").agg(min(struct($"record.*")))
```
Second, it improves the error messages when having invalid star usage when using Dataframe/Dataset APIs. For example,
```scala
pagecounts4PartitionsDS
.map(line => (line._1, line._3))
.toDF()
.groupBy($"_1")
.agg(sum("*") as "sumOccurances")
```
Before the fix, the invalid usage will issue a confusing error message, like:
```
org.apache.spark.sql.AnalysisException: cannot resolve '_1' given input columns _1, _2;
```
After the fix, the message is like:
```
org.apache.spark.sql.AnalysisException: Invalid usage of '*' in function 'sum'
```
cc: rxin nongli cloud-fan
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11208 from gatorsmile/sumDataSetResolution.
## What changes were proposed in this pull request?
This patch merges DatasetHolder and DataFrameHolder. This makes more sense because DataFrame/Dataset are now one class.
In addition, fixed some minor issues with pull request #11732.
## How was this patch tested?
Updated existing unit tests that test these implicits.
Author: Reynold Xin <rxin@databricks.com>
Closes#11737 from rxin/SPARK-13898.
## What changes were proposed in this pull request?
WholeStageCodegen naturally breaks the execution into pipelines that are easier to
measure duration. This is more granular than the task timings (a task can be multiple
pipelines) and is integrated with the web ui.
We currently report total time (across all tasks), min/mask/median to get a sense of how long each is taking.
## How was this patch tested?
Manually tested looking at the web ui.
Author: Nong Li <nong@databricks.com>
Closes#11741 from nongli/spark-13916.
## What changes were proposed in this pull request?
There is only one exception: `PythonUDF`. However, I don't think the `PythonUDF#` prefix is useful, as we can only create python udf under python context. This PR removes the `PythonUDF#` prefix from `PythonUDF.toString`, so that it doesn't need to overrde `sql`.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11859 from cloud-fan/tmp.
## What changes were proposed in this pull request?
This PR generates code that get a value in each column from ```ColumnVector``` instead of creating ```InternalRow``` when ```ColumnarBatch``` is accessed. This PR improves benchmark program by up to 15%.
This PR consists of two parts:
1. Get an ```ColumnVector ``` by using ```ColumnarBatch.column()``` method
2. Get a value of each column by using ```rdd_col${COLIDX}.getInt(ROWIDX)``` instead of ```rdd_row.getInt(COLIDX)```
This is a motivated example.
````
sqlContext.conf.setConfString(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true")
sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true")
val values = 10
withTempPath { dir =>
withTempTable("t1", "tempTable") {
sqlContext.range(values).registerTempTable("t1")
sqlContext.sql("select id % 2 as p, cast(id as INT) as id from t1")
.write.partitionBy("p").parquet(dir.getCanonicalPath)
sqlContext.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable")
sqlContext.sql("select sum(p) from tempTable").collect
}
}
````
The original code
````java
...
/* 072 */ while (!shouldStop() && rdd_batchIdx < numRows) {
/* 073 */ InternalRow rdd_row = rdd_batch.getRow(rdd_batchIdx++);
/* 074 */ /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
/* 075 */ /* input[0, int] */
/* 076 */ boolean rdd_isNull = rdd_row.isNullAt(0);
/* 077 */ int rdd_value = rdd_isNull ? -1 : (rdd_row.getInt(0));
...
````
The code generated by this PR
````java
/* 072 */ while (!shouldStop() && rdd_batchIdx < numRows) {
/* 073 */ org.apache.spark.sql.execution.vectorized.ColumnVector rdd_col0 = rdd_batch.column(0);
/* 074 */ /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
/* 075 */ /* input[0, int] */
/* 076 */ boolean rdd_isNull = rdd_col0.getIsNull(rdd_batchIdx);
/* 077 */ int rdd_value = rdd_isNull ? -1 : (rdd_col0.getInt(rdd_batchIdx));
...
/* 128 */ rdd_batchIdx++;
/* 129 */ }
/* 130 */ if (shouldStop()) return;
````
Performance
Without this PR
````
model name : Intel(R) Xeon(R) CPU E5-2667 v2 3.30GHz
Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
Read data column 434 / 488 36.3 27.6 1.0X
Read partition column 302 / 346 52.1 19.2 1.4X
Read both columns 588 / 643 26.8 37.4 0.7X
````
With this PR
````
model name : Intel(R) Xeon(R) CPU E5-2667 v2 3.30GHz
Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
Read data column 392 / 516 40.1 24.9 1.0X
Read partition column 256 / 318 61.4 16.3 1.5X
Read both columns 523 / 539 30.1 33.3 0.7X
````
## How was this patch tested?
Tested by existing test suites and benchmark
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#11636 from kiszk/SPARK-13805.
## What changes were proposed in this pull request?
This PR try acquire the memory for hash map in shuffled hash join, fail the task if there is no enough memory (otherwise it could OOM the executor).
It also removed unused HashedRelation.
## How was this patch tested?
Existing unit tests. Manual tests with TPCDS Q78.
Author: Davies Liu <davies@databricks.com>
Closes#11826 from davies/cleanup_hash2.
## What changes were proposed in this pull request?
This is a more aggressive version of PR #11820, which not only fixes the original problem, but also does the following updates to enforce the at-most-one-qualifier constraint:
- Renames `NamedExpression.qualifiers` to `NamedExpression.qualifier`
- Uses `Option[String]` rather than `Seq[String]` for `NamedExpression.qualifier`
Quoted PR description of #11820 here:
> Current implementations of `AttributeReference.sql` and `Alias.sql` joins all available qualifiers, which is logically wrong. But this implementation mistake doesn't cause any real SQL generation bugs though, since there is always at most one qualifier for any given `AttributeReference` or `Alias`.
## How was this patch tested?
Existing tests should be enough.
Author: Cheng Lian <lian@databricks.com>
Closes#11822 from liancheng/spark-14004-aggressive.
## What changes were proposed in this pull request?
case classes defined in REPL are wrapped by line classes, and we have a trick for scala 2.10 REPL to automatically register the wrapper classes to `OuterScope` so that we can use when create encoders.
However, this trick doesn't work right after we upgrade to scala 2.11, and unfortunately the tests are only in scala 2.10, which makes this bug hidden until now.
This PR moves the encoder tests to scala 2.11 `ReplSuite`, and fixes this bug by another approach(the previous trick can't port to scala 2.11 REPL): make `OuterScope` smarter that can detect classes defined in REPL and load the singleton of line wrapper classes automatically.
## How was this patch tested?
the migrated encoder tests in `ReplSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11410 from cloud-fan/repl.
## What changes were proposed in this pull request?
Ad-hoc Dataset API ScalaDoc fixes
## How was this patch tested?
By building and checking ScalaDoc locally.
Author: Cheng Lian <lian@databricks.com>
Closes#11862 from liancheng/ds-doc-fixes.
## What changes were proposed in this pull request?
`SubqueryHolder` is only used when generate SQL string in `SQLBuilder`, it's more clear to make it an inner class in `SQLBuilder`.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11861 from cloud-fan/gensql.
## What changes were proposed in this pull request?
When we validate an encoder, we may call `dataType` on unresolved expressions. This PR fix the validation so that we will resolve attributes first.
## How was this patch tested?
a new test in `DatasetSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11816 from cloud-fan/encoder.
#### What changes were proposed in this pull request?
This PR is to support order by position in SQL, e.g.
```SQL
select c1, c2, c3 from tbl order by 1 desc, 3
```
should be equivalent to
```SQL
select c1, c2, c3 from tbl order by c1 desc, c3 asc
```
This is controlled by config option `spark.sql.orderByOrdinal`.
- When true, the ordinal numbers are treated as the position in the select list.
- When false, the ordinal number in order/sort By clause are ignored.
- Only convert integer literals (not foldable expressions). If found foldable expressions, ignore them
- This also works with select *.
**Question**: Do we still need sort by columns that contain zero reference? In this case, it will have no impact on the sorting results. IMO, we should not allow users do it. rxin cloud-fan marmbrus yhuai hvanhovell
-- Update: In these cases, they are ignored in this case.
**Note**: This PR is taken from https://github.com/apache/spark/pull/10731. When merging this PR, please give the credit to zhichao-li
Also cc all the people who are involved in the previous discussion: adrian-wang chenghao-intel tejasapatil
#### How was this patch tested?
Added a few test cases for both positive and negative test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11815 from gatorsmile/orderByPosition.
## What changes were proposed in this pull request?
This PR adds some proper periods and spaces to Spark CLI help messages and SQL/YARN conf docs for consistency.
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11848 from dongjoon-hyun/add_proper_period_and_space.
## What changes were proposed in this pull request?
[Spark Coding Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide) has 100-character limit on lines, but it's disabled for Java since 11/09/15. This PR enables **LineLength** checkstyle again. To help that, this also introduces **RedundantImport** and **RedundantModifier**, too. The following is the diff on `checkstyle.xml`.
```xml
- <!-- TODO: 11/09/15 disabled - the lengths are currently > 100 in many places -->
- <!--
<module name="LineLength">
<property name="max" value="100"/>
<property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
</module>
- -->
<module name="NoLineWrap"/>
<module name="EmptyBlock">
<property name="option" value="TEXT"/>
-167,5 +164,7
</module>
<module name="CommentsIndentation"/>
<module name="UnusedImports"/>
+ <module name="RedundantImport"/>
+ <module name="RedundantModifier"/>
```
## How was this patch tested?
Currently, `lint-java` is disabled in Jenkins. It needs a manual test.
After passing the Jenkins tests, `dev/lint-java` should passes locally.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11831 from dongjoon-hyun/SPARK-14011.
## What changes were proposed in this pull request?
Currently, there is no way to control the behaviour when fails to parse corrupt records in JSON data source .
This PR adds the support for parse modes just like CSV data source. There are three modes below:
- `PERMISSIVE` : When it fails to parse, this sets `null` to to field. This is a default mode when it has been this mode.
- `DROPMALFORMED`: When it fails to parse, this drops the whole record.
- `FAILFAST`: When it fails to parse, it just throws an exception.
This PR also make JSON data source share the `ParseModes` in CSV data source.
## How was this patch tested?
Unit tests were used and `./dev/run_tests` for code style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11756 from HyukjinKwon/SPARK-13764.
#### What changes were proposed in this pull request?
This PR is to add a new Optimizer rule for pruning Sort if its SortOrder is no-op. In the phase of **Optimizer**, if a specific `SortOrder` does not have any reference, it has no effect on the sorting results. If `Sort` is empty, remove the whole `Sort`.
For example, in the following SQL query
```SQL
SELECT * FROM t ORDER BY NULL + 5
```
Before the fix, the plan is like
```
== Analyzed Logical Plan ==
a: int, b: int
Sort [(cast(null as int) + 5) ASC], true
+- Project [a#92,b#93]
+- SubqueryAlias t
+- Project [_1#89 AS a#92,_2#90 AS b#93]
+- LocalRelation [_1#89,_2#90], [[1,2],[1,2]]
== Optimized Logical Plan ==
Sort [null ASC], true
+- LocalRelation [a#92,b#93], [[1,2],[1,2]]
== Physical Plan ==
WholeStageCodegen
: +- Sort [null ASC], true, 0
: +- INPUT
+- Exchange rangepartitioning(null ASC, 5), None
+- LocalTableScan [a#92,b#93], [[1,2],[1,2]]
```
After the fix, the plan is like
```
== Analyzed Logical Plan ==
a: int, b: int
Sort [(cast(null as int) + 5) ASC], true
+- Project [a#92,b#93]
+- SubqueryAlias t
+- Project [_1#89 AS a#92,_2#90 AS b#93]
+- LocalRelation [_1#89,_2#90], [[1,2],[1,2]]
== Optimized Logical Plan ==
LocalRelation [a#92,b#93], [[1,2],[1,2]]
== Physical Plan ==
LocalTableScan [a#92,b#93], [[1,2],[1,2]]
```
cc rxin cloud-fan marmbrus Thanks!
#### How was this patch tested?
Added a test suite for covering this rule
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11840 from gatorsmile/sortElimination.
## What changes were proposed in this pull request?
Previously, Dataset.groupBy returns a GroupedData, and Dataset.groupByKey returns a GroupedDataset. The naming is very similar, and unfortunately does not convey the real differences between the two.
Assume we are grouping by some keys (K). groupByKey is a key-value style group by, in which the schema of the returned dataset is a tuple of just two fields: key and value. groupBy, on the other hand, is a relational style group by, in which the schema of the returned dataset is flattened and contain |K| + |V| fields.
This pull request also removes the experimental tag from RelationalGroupedDataset. It has been with DataFrame since 1.3, and we have enough confidence now to stabilize it.
## How was this patch tested?
This is a rename to improve API understandability. Should be covered by all existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#11841 from rxin/SPARK-13897.
## What changes were proposed in this pull request?
500L << 20 is actually pretty close to 32-bit int limit. I was trying to increase this to 500L << 23 and got negative numbers instead.
## How was this patch tested?
I'm only modifying test code.
Author: Reynold Xin <rxin@databricks.com>
Closes#11839 from rxin/SPARK-14018.
## What changes were proposed in this pull request?
This is a minor followup on https://github.com/apache/spark/pull/11799 that extracts out the `VectorizedColumnReader` from `VectorizedParquetRecordReader` into its own file.
## How was this patch tested?
N/A (refactoring only)
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11834 from sameeragarwal/rename.
## What changes were proposed in this pull request?
This PR cleans up the new parquet record reader with the following changes:
1. Removes the non-vectorized parquet reader code from `UnsafeRowParquetRecordReader`.
2. Removes the non-vectorized column reader code from `ColumnReader`.
3. Renames `UnsafeRowParquetRecordReader` to `VectorizedParquetRecordReader` and `ColumnReader` to `VectorizedColumnReader`
4. Deprecate `PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED`
## How was this patch tested?
Refactoring only; Existing tests should reveal any problems.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11799 from sameeragarwal/vectorized-parquet.
## What changes were proposed in this pull request?
As part of testing generating SQL query from a analyzed SQL plan, we run the generated SQL for tests in HiveComparisonTest. This PR makes the generated SQL get eagerly analyzed. So, when a generated SQL has any analysis error, we can see the error message created by
```
case NonFatal(e) => fail(
s"""Failed to analyze the converted SQL string:
|
|# Original HiveQL query string:
|$queryString
|
|# Resolved query plan:
|${originalQuery.analyzed.treeString}
|
|# Converted SQL query string:
|$convertedSQL
""".stripMargin, e)
```
Right now, if we can parse a generated SQL but fail to analyze it, we will see error message generated by the following code (it only mentions that we cannot execute the original query, i.e. `queryString`).
```
case e: Throwable =>
val errorMessage =
s"""
|Failed to execute query using catalyst:
|Error: ${e.getMessage}
|${stackTraceToString(e)}
|$queryString
|$query
|== HIVE - ${hive.size} row(s) ==
|${hive.mkString("\n")}
""".stripMargin
```
## How was this patch tested?
Existing tests.
Author: Yin Huai <yhuai@databricks.com>
Closes#11825 from yhuai/SPARK-13972-follow-up.
## What changes were proposed in this pull request?
ShuffledHashJoin (also outer join) is removed in 1.6, in favor of SortMergeJoin, which is more robust and also fast.
ShuffledHashJoin is still useful in this case: 1) one table is much smaller than the other one, then cost to build a hash table on smaller table is smaller than sorting the larger table 2) any partition of the small table could fit in memory.
This PR brings back ShuffledHashJoin, basically revert #9645, and fix the conflict. Also merging outer join and left-semi join into the same class. This PR does not implement full outer join, because it's not implemented efficiently (requiring build hash table on both side).
A simple benchmark (one table is 5x smaller than other one) show that ShuffledHashJoin could be 2X faster than SortMergeJoin.
## How was this patch tested?
Added new unit tests for ShuffledHashJoin.
Author: Davies Liu <davies@databricks.com>
Closes#11788 from davies/shuffle_join.
## What changes were proposed in this pull request?
Current implementations of `AttributeReference.sql` and `Alias.sql` joins all available qualifiers, which is logically wrong. But this implementation mistake doesn't cause any real SQL generation bugs though, since there is always at most one qualifier for any given `AttributeReference` or `Alias`.
This PR fixes this issue by only picking the first qualifiers.
## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Existing tests should be enough.
Author: Cheng Lian <lian@databricks.com>
Closes#11820 from liancheng/spark-14004-single-qualifier.
## What changes were proposed in this pull request?
Now we should be able to convert all logical plans to SQL string, if they are parsed from hive query. This PR changes the error handling to throw exceptions instead of just log.
We will send new PRs for spotted bugs, and merge this one after all bugs are fixed.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11782 from cloud-fan/test.
## What changes were proposed in this pull request?
The fix is simple, use the existing `CombineUnions` rule to combine adjacent Unions before build SQL string.
## How was this patch tested?
The re-enabled test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11818 from cloud-fan/bug-fix.
## What changes were proposed in this pull request?
This patch updates documentations for Datasets. I also updated some internal documentation for exchange/broadcast.
## How was this patch tested?
Just documentation/api stability update.
Author: Reynold Xin <rxin@databricks.com>
Closes#11814 from rxin/dataset-docs.
## What changes were proposed in this pull request?
JIRA: https://issues.apache.org/jira/browse/SPARK-13930
Recently the fast serialization has been introduced to collecting DataFrame/Dataset (#11664). The same technology can be used on collect limit operator too.
## How was this patch tested?
Add a benchmark for collect limit to `BenchmarkWholeStageCodegen`.
Without this patch:
model name : Westmere E56xx/L56xx/X56xx (Nehalem-C)
collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
collect limit 1 million 3413 / 3768 0.3 3255.0 1.0X
collect limit 2 millions 9728 / 10440 0.1 9277.3 0.4X
With this patch:
model name : Westmere E56xx/L56xx/X56xx (Nehalem-C)
collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
collect limit 1 million 833 / 1284 1.3 794.4 1.0X
collect limit 2 millions 3348 / 4005 0.3 3193.3 0.2X
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#11759 from viirya/execute-take.
## What changes were proposed in this pull request?
This PR revises Dataset API ScalaDoc. All public methods are divided into the following groups
* `groupname basic`: Basic Dataset functions
* `groupname action`: Actions
* `groupname untypedrel`: Untyped Language Integrated Relational Queries
* `groupname typedrel`: Typed Language Integrated Relational Queries
* `groupname func`: Functional Transformations
* `groupname rdd`: RDD Operations
* `groupname output`: Output Operations
`since` tag and sample code are also updated. We may want to add more sample code for typed APIs.
## How was this patch tested?
Documentation change. Checked by building unidoc locally.
Author: Cheng Lian <lian@databricks.com>
Closes#11769 from liancheng/spark-13826-ds-api-doc.
PR #11696 introduced a complex pattern match that broke Scala 2.10 match unreachability check and caused build failure. This PR fixes this issue by expanding this pattern match into several simpler ones.
Note that tuning or turning off `-Dscalac.patmat.analysisBudget` doesn't work for this case.
Compilation against Scala 2.10
Author: tedyu <yuzhihong@gmail.com>
Closes#11798 from yy2016/master.
## What changes were proposed in this pull request?
We haven't figured out the corrected logical to add sub-queries yet, so we should not clear all sub-queries before generate SQL. This PR changed the logic to only remove sub-queries above table relation.
an example for this bug, original SQL: `SELECT a FROM (SELECT a FROM tbl) t WHERE a = 1`
before this PR, we will generate:
```
SELECT attr_1 AS a FROM
SELECT attr_1 FROM (
SELECT a AS attr_1 FROM tbl
) AS sub_q0
WHERE attr_1 = 1
```
We missed a sub-query and this SQL string is illegal.
After this PR, we will generate:
```
SELECT attr_1 AS a FROM (
SELECT attr_1 FROM (
SELECT a AS attr_1 FROM tbl
) AS sub_q0
WHERE attr_1 = 1
) AS t
```
TODO: for long term, we should find a way to add sub-queries correctly, so that arbitrary logical plans can be converted to SQL string.
## How was this patch tested?
`LogicalPlanToSQLSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11786 from cloud-fan/bug-fix.
## What changes were proposed in this pull request?
We only need to make sub-query names unique every time we generate a SQL string, but not all the time. This PR moves the `newSubqueryName` method to `class SQLBuilder` and remove `object SQLBuilder`.
also addressed 2 minor comments in https://github.com/apache/spark/pull/11696
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11783 from cloud-fan/tmp.
## What changes were proposed in this pull request?
Compilation against Scala 2.10 fails with:
```
[error] [warn] /home/jenkins/workspace/spark-master-compile-sbt-scala-2.10/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala:483: Cannot check match for unreachability.
[error] (The analysis required more space than allowed. Please try with scalac -Dscalac.patmat.analysisBudget=512 or -Dscalac.patmat.analysisBudget=off.)
[error] [warn] private def addSubqueryIfNeeded(plan: LogicalPlan): LogicalPlan = plan match {
```
## How was this patch tested?
Compilation against Scala 2.10
Author: tedyu <yuzhihong@gmail.com>
Closes#11787 from yy2016/master.
JIRA: https://issues.apache.org/jira/browse/SPARK-13838
## What changes were proposed in this pull request?
We should also clear the variable code in `BoundReference.genCode` to prevent it to be evaluated twice, as we did in `evaluateVariables`.
## How was this patch tested?
Existing tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#11674 from viirya/avoid-reevaluate.
## What changes were proposed in this pull request?
Support queries that JOIN tables with USING clause.
SELECT * from table1 JOIN table2 USING <column_list>
USING clause can be used as a means to simplify the join condition
when :
1) Equijoin semantics is desired and
2) The column names in the equijoin have the same name.
We already have the support for Natural Join in Spark. This PR makes
use of the already existing infrastructure for natural join to
form the join condition and also the projection list.
## How was the this patch tested?
Have added unit tests in SQLQuerySuite, CatalystQlSuite, ResolveNaturalJoinSuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#11297 from dilipbiswal/spark-13427.
## What changes were proposed in this pull request?
This PR adds SQL generation support for `Generate` operator. It always converts `Generate` operator into `LATERAL VIEW` format as there are many limitations to put UDTF in project list.
This PR is based on https://github.com/apache/spark/pull/11658, please see the last commit to review the real changes.
Thanks dilipbiswal for his initial work! Takes over https://github.com/apache/spark/pull/11596
## How was this patch tested?
new tests in `LogicalPlanToSQLSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11696 from cloud-fan/generate.
## What changes were proposed in this pull request?
Logging was made private in Spark 2.0. If we move it, then users would be able to create a Logging trait themselves to avoid changing their own code.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11764 from cloud-fan/logger.
This commit updates the HiveContext so that sc.hadoopConfiguration is used to instantiate its internal instances of HiveConf.
I tested this by overriding the S3 FileSystem implementation from spark-defaults.conf as "spark.hadoop.fs.s3.impl" (to avoid [HADOOP-12810](https://issues.apache.org/jira/browse/HADOOP-12810)).
Author: Ryan Blue <blue@apache.org>
Closes#11273 from rdblue/SPARK-13403-new-hive-conf-from-hadoop-conf.
Because ClassTags are available when constructing ShuffledRDD we can use them to automatically use Kryo for shuffle serialization when the RDD's types are known to be compatible with Kryo.
This patch introduces `SerializerManager`, a component which picks the "best" serializer for a shuffle given the elements' ClassTags. It will automatically pick a Kryo serializer for ShuffledRDDs whose key, value, and/or combiner types are primitives, arrays of primitives, or strings. In the future we can use this class as a narrow extension point to integrate specialized serializers for other types, such as ByteBuffers.
In a planned followup patch, I will extend the BlockManager APIs so that we're able to use similar automatic serializer selection when caching RDDs (this is a little trickier because the ClassTags need to be threaded through many more places).
Author: Josh Rosen <joshrosen@databricks.com>
Closes#11755 from JoshRosen/automatically-pick-best-serializer.
## What changes were proposed in this pull request?
Since developer API of plug-able parser has been removed in #10801 , docs should be updated accordingly.
## How was this patch tested?
This patch will not affect the real code path.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#11758 from adrian-wang/spark12855.
## What changes were proposed in this pull request?
This PR removes three minor duplicated lines. First one is making the following unreachable code warning.
```
JoinSuite.scala:52: unreachable code
[warn] case j: BroadcastHashJoin => j
```
The other two are just consecutive repetitions in `Seq` of MiMa filters.
## How was this patch tested?
Pass the existing Jenkins test.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11773 from dongjoon-hyun/remove_duplicated_line.
## What changes were proposed in this pull request?
Fix expression generation for optional types.
Standard Java reflection causes issues when dealing with synthetic Scala objects (things that do not map to Java and thus contain a dollar sign in their name). This patch introduces Scala reflection in such cases.
This patch also adds a regression test for Dataset's handling of classes defined in package objects (which was the initial purpose of this PR).
## How was this patch tested?
A new test in ExpressionEncoderSuite that tests optional inner classes and a regression test for Dataset's handling of package objects.
Author: Jakob Odersky <jakob@odersky.com>
Closes#11708 from jodersky/SPARK-13118-package-objects.
## What changes were proposed in this pull request?
We need to copy the UnsafeRow since a Join could produce multiple rows from single input rows. We could avoid that if there is no join (or the join will not produce multiple rows) inside WholeStageCodegen.
Updated the benchmark for `collect`, we could see 20-30% speedup.
## How was this patch tested?
existing unit tests.
Author: Davies Liu <davies@databricks.com>
Closes#11740 from davies/avoid_copy2.
## What changes were proposed in this pull request?
This https://github.com/apache/spark/pull/2400 added the support to parse JSON rows wrapped with an array. However, this throws an exception when the given data contains array data and struct data in the same field as below:
```json
{"a": {"b": 1}}
{"a": []}
```
and the schema is given as below:
```scala
val schema =
StructType(
StructField("a", StructType(
StructField("b", StringType) :: Nil
)) :: Nil)
```
- **Before**
```scala
sqlContext.read.schema(schema).json(path).show()
```
```scala
Exception in thread "main" org.apache.spark.SparkException: Job aborted due to stage failure: Task 7 in stage 0.0 failed 4 times, most recent failure: Lost task 7.3 in stage 0.0 (TID 10, 192.168.1.170): java.lang.ClassCastException: org.apache.spark.sql.types.GenericArrayData cannot be cast to org.apache.spark.sql.catalyst.InternalRow
at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getStruct(rows.scala:50)
at org.apache.spark.sql.catalyst.expressions.GenericMutableRow.getStruct(rows.scala:247)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source)
...
```
- **After**
```scala
sqlContext.read.schema(schema).json(path).show()
```
```bash
+----+
| a|
+----+
| [1]|
|null|
+----+
```
For other data types, in this case it converts the given values are `null` but only this case emits an exception.
This PR makes the support for wrapped rows applied only at the top level.
## How was this patch tested?
Unit tests were used and `./dev/run_tests` for code style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11752 from HyukjinKwon/SPARK-3308-follow-up.
## What changes were proposed in this pull request?
As part of the effort to merge `SQLContext` and `HiveContext`, this patch implements an internal catalog called `SessionCatalog` that handles temporary functions and tables and delegates metastore operations to `ExternalCatalog`. Currently, this is still dead code, but in the future it will be part of `SessionState` and will replace `o.a.s.sql.catalyst.analysis.Catalog`.
A recent patch #11573 parses Hive commands ourselves in Spark, but still passes the entire query text to Hive. In a future patch, we will use `SessionCatalog` to implement the parsed commands.
## How was this patch tested?
800+ lines of tests in `SessionCatalogSuite`.
Author: Andrew Or <andrew@databricks.com>
Closes#11750 from andrewor14/temp-catalog.
## What changes were proposed in this pull request?
Narrow down the parameter type of `UserDefinedType#serialize()`. Currently, the parameter type is `Any`, however it would logically make more sense to narrow it down to the type of the actual user defined type.
## How was this patch tested?
Existing tests were successfully run on local machine.
Author: Jakob Odersky <jakob@odersky.com>
Closes#11379 from jodersky/SPARK-11011-udt-types.
## What changes were proposed in this pull request?
**[I'll link it to the JIRA once ASF JIRA is back online]**
This PR modifies the existing `CombineFilters` rule to remove redundant conditions while combining individual filter predicates. For instance, queries of the form `table.where('a === 1 && 'b === 1).where('a === 1 && 'c === 1)` will now be optimized to ` table.where('a === 1 && 'b === 1 && 'c === 1)` (instead of ` table.where('a === 1 && 'a === 1 && 'b === 1 && 'c === 1)`)
## How was this patch tested?
Unit test in `FilterPushdownSuite`
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11670 from sameeragarwal/combine-filters.
## What changes were proposed in this pull request?
This PR generalizes the `NullFiltering` optimizer rule in catalyst to `InferFiltersFromConstraints` that can automatically infer all relevant filters based on an operator's constraints while making sure of 2 things:
(a) no redundant filters are generated, and
(b) filters that do not contribute to any further optimizations are not generated.
## How was this patch tested?
Extended all tests in `InferFiltersFromConstraintsSuite` (that were initially based on `NullFilteringSuite` to test filter inference in `Filter` and `Join` operators.
In particular the 2 tests ( `single inner join with pre-existing filters: filter out values on either side` and `multiple inner joins: filter out values on all sides on equi-join keys` attempts to highlight/test the real potential of this rule for join optimization.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11665 from sameeragarwal/infer-filters.
#### What changes were proposed in this pull request?
This PR is to convert to SQL from analyzed logical plans containing operator `ScriptTransformation`.
For example, below is the SQL containing `Transform`
```
SELECT TRANSFORM (a, b, c, d) USING 'cat' FROM parquet_t2
```
Its logical plan is like
```
ScriptTransformation [a#210L,b#211L,c#212L,d#213L], cat, [key#208,value#209], HiveScriptIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim, )),List((field.delim, )),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),true)
+- SubqueryAlias parquet_t2
+- Relation[a#210L,b#211L,c#212L,d#213L] ParquetRelation
```
The generated SQL will be like
```
SELECT TRANSFORM (`parquet_t2`.`a`, `parquet_t2`.`b`, `parquet_t2`.`c`, `parquet_t2`.`d`) USING 'cat' AS (`key` string, `value` string) FROM `default`.`parquet_t2`
```
#### How was this patch tested?
Seven test cases are added to `LogicalPlanToSQLSuite`.
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#11503 from gatorsmile/transformToSQL.
## What changes were proposed in this pull request?
This PR tries to solve a fundamental issue in the `SQLBuilder`. When we want to turn a logical plan into SQL string and put it after FROM clause, we need to wrap it with a sub-query. However, a logical plan is allowed to have same-name outputs with different qualifiers(e.g. the `Join` operator), and this kind of plan can't be put under a subquery as we will erase and assign a new qualifier to all outputs and make it impossible to distinguish same-name outputs.
To solve this problem, this PR renames all attributes with globally unique names(using exprId), so that we don't need qualifiers to resolve ambiguity anymore.
For example, `SELECT x.key, MAX(y.key) OVER () FROM t x JOIN t y`, we will parse this SQL to a Window operator and a Project operator, and add a sub-query between them. The generated SQL looks like:
```
SELECT sq_1.key, sq_1.max
FROM (
SELECT sq_0.key, sq_0.key, MAX(sq_0.key) OVER () AS max
FROM (
SELECT x.key, y.key FROM t1 AS x JOIN t2 AS y
) AS sq_0
) AS sq_1
```
You can see, the `key` columns become ambiguous after `sq_0`.
After this PR, it will generate something like:
```
SELECT attr_30 AS key, attr_37 AS max
FROM (
SELECT attr_30, attr_37
FROM (
SELECT attr_30, attr_35, MAX(attr_35) AS attr_37
FROM (
SELECT attr_30, attr_35 FROM
(SELECT key AS attr_30 FROM t1) AS sq_0
INNER JOIN
(SELECT key AS attr_35 FROM t1) AS sq_1
) AS sq_2
) AS sq_3
) AS sq_4
```
The outermost SELECT is used to turn the generated named to real names back, and the innermost SELECT is used to alias real columns to our generated names. Between them, there is no name ambiguity anymore.
## How was this patch tested?
existing tests and new tests in LogicalPlanToSQLSuite.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11658 from cloud-fan/gensql.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13894
Change the return type of the `SQLContext.range` API from `DataFrame` to `Dataset`.
## How was this patch tested?
No additional unit test required.
Author: Cheng Hao <hao.cheng@intel.com>
Closes#11730 from chenghao-intel/range.
## What changes were proposed in this pull request?
There is a feature of hive SQL called multi-insert. For example:
```
FROM src
INSERT OVERWRITE TABLE dest1
SELECT key + 1
INSERT OVERWRITE TABLE dest2
SELECT key WHERE key > 2
INSERT OVERWRITE TABLE dest3
SELECT col EXPLODE(arr) exp AS col
...
```
We partially support it currently, with some limitations: 1) WHERE can't reference columns produced by LATERAL VIEW. 2) It's not executed eagerly, i.e. `sql("...multi-insert clause...")` won't take place right away like other commands, e.g. CREATE TABLE.
This PR removes these limitations and make us fully support multi-insert.
## How was this patch tested?
new tests in `SQLQuerySuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11754 from cloud-fan/lateral-view.
## What changes were proposed in this pull request?
Follow up to https://github.com/apache/spark/pull/11657
- Also update `String.getBytes("UTF-8")` to use `StandardCharsets.UTF_8`
- And fix one last new Coverity warning that turned up (use of unguarded `wait()` replaced by simpler/more robust `java.util.concurrent` classes in tests)
- And while we're here cleaning up Coverity warnings, just fix about 15 more build warnings
## How was this patch tested?
Jenkins tests
Author: Sean Owen <sowen@cloudera.com>
Closes#11725 from srowen/SPARK-13823.2.
## What changes were proposed in this pull request?
The purpose of [SPARK-12653](https://issues.apache.org/jira/browse/SPARK-12653) is re-enabling a regression test.
Historically, the target regression test is added by [SPARK-8498](093c34838d), but is temporarily disabled by [SPARK-12615](8ce645d4ee) due to binary compatibility error.
The following is the current error message at the submitting spark job with the pre-built `test.jar` file in the target regression test.
```
Exception in thread "main" java.lang.NoSuchMethodError: org.apache.spark.SparkContext$.$lessinit$greater$default$6()Lscala/collection/Map;
```
Simple rebuilding `test.jar` can not recover the purpose of testcase since we need to support both Scala 2.10 and 2.11 for a while. For example, we will face the following Scala 2.11 error if we use `test.jar` built by Scala 2.10.
```
Exception in thread "main" java.lang.NoSuchMethodError: scala.reflect.api.JavaUniverse.runtimeMirror(Ljava/lang/ClassLoader;)Lscala/reflect/api/JavaMirrors$JavaMirror;
```
This PR replace the existing `test.jar` with `test-2.10.jar` and `test-2.11.jar` and improve the regression test to use the suitable jar file.
## How was this patch tested?
Pass the existing Jenkins test.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11744 from dongjoon-hyun/SPARK-12653.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13899
This PR makes CSV data source produce `InternalRow` instead of `Row`.
Basically, this resembles JSON data source. It uses the same codes for casting.
## How was this patch tested?
Unit tests were used within IDE and code style was checked by `./dev/run_tests`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11717 from HyukjinKwon/SPARK-13899.
## What changes were proposed in this pull request?
This PR brings codegen support for broadcast left-semi join.
## How was this patch tested?
Existing tests. Added benchmark, the result show 7X speedup.
Author: Davies Liu <davies@databricks.com>
Closes#11742 from davies/gen_semi.
## What changes were proposed in this pull request?
Remove the wrong "expected" parameter in MathFunctionsSuite.scala's checkNaNWithoutCodegen.
This function is to check NaN value, so the "expected" parameter is useless. The Callers do not pass "expected" value and the similar function like checkNaNWithGeneratedProjection and checkNaNWithOptimization do not use it also.
Author: Yucai Yu <yucai.yu@intel.com>
Closes#11718 from yucai/unused_expected.
## What changes were proposed in this pull request?
This PR just move some code from SortMergeOuterJoin into SortMergeJoin.
This is for support codegen for outer join.
## How was this patch tested?
existing tests.
Author: Davies Liu <davies@databricks.com>
Closes#11743 from davies/gen_smjouter.
## What changes were proposed in this pull request?
This patch changes DataFrameReader.text()'s return type from DataFrame to Dataset[String].
Closes#11731.
## How was this patch tested?
Updated existing integration tests to reflect the change.
Author: Reynold Xin <rxin@databricks.com>
Closes#11739 from rxin/SPARK-13895.
## What changes were proposed in this pull request?
Change the return type of toJson in Dataset class
## How was this patch tested?
No additional unit test required.
Author: Stavros Kontopoulos <stavros.kontopoulos@typesafe.com>
Closes#11732 from skonto/fix_toJson.
## What changes were proposed in this pull request?
Our internal code can go through SessionState.catalog and SessionState.analyzer. This brings two small benefits:
1. Reduces internal dependency on SQLContext.
2. Removes 2 public methods in Java (Java does not obey package private visibility).
More importantly, according to the design in SPARK-13485, we'd need to claim this catalog function for the user-facing public functions, rather than having an internal field.
## How was this patch tested?
Existing unit/integration test code.
Author: Reynold Xin <rxin@databricks.com>
Closes#11716 from rxin/SPARK-13893.
## What changes were proposed in this pull request?
Use method 'testQuietly' to avoid ContinuousQuerySuite flooding the console logs with garbage
Make ContinuousQuerySuite not output logs to the console. The logs will still output to unit-tests.log.
## How was this patch tested?
Just check Jenkins output.
Author: Xin Ren <iamshrek@126.com>
Closes#11703 from keypointt/SPARK-13660.
#### What changes were proposed in this pull request?
Before this PR, two Optimizer rules `ColumnPruning` and `PushPredicateThroughProject` reverse each other's effects. Optimizer always reaches the max iteration when optimizing some queries. Extra `Project` are found in the plan. For example, below is the optimized plan after reaching 100 iterations:
```
Join Inner, Some((cast(id1#16 as bigint) = id1#18L))
:- Project [id1#16]
: +- Filter isnotnull(cast(id1#16 as bigint))
: +- Project [id1#16]
: +- Relation[id1#16,newCol#17] JSON part: struct<>, data: struct<id1:int,newCol:int>
+- Filter isnotnull(id1#18L)
+- Relation[id1#18L] JSON part: struct<>, data: struct<id1:bigint>
```
This PR splits the optimizer rule `ColumnPruning` to `ColumnPruning` and `EliminateOperators`
The issue becomes worse when having another rule `NullFiltering`, which could add extra Filters for `IsNotNull`. We have to be careful when introducing extra `Filter` if the benefit is not large enough. Another PR will be submitted by sameeragarwal to handle this issue.
cc sameeragarwal marmbrus
In addition, `ColumnPruning` should not push `Project` through non-deterministic `Filter`. This could cause wrong results. This will be put in a separate PR.
cc davies cloud-fan yhuai
#### How was this patch tested?
Modified the existing test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11682 from gatorsmile/viewDuplicateNames.
## What changes were proposed in this pull request?
In general it is better for internal classes to not depend on the external class (in this case SQLContext) to reduce coupling between user-facing APIs and the internal implementations. This patch removes SQLContext dependency from some internal classes such as SparkPlanner, SparkOptimizer.
As part of this patch, I also removed the following internal methods from SQLContext:
```
protected[sql] def functionRegistry: FunctionRegistry
protected[sql] def optimizer: Optimizer
protected[sql] def sqlParser: ParserInterface
protected[sql] def planner: SparkPlanner
protected[sql] def continuousQueryManager
protected[sql] def prepareForExecution: RuleExecutor[SparkPlan]
```
## How was this patch tested?
Existing unit/integration tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#11712 from rxin/sqlContext-planner.
## What changes were proposed in this pull request?
When initial creating `CVSSuite.scala` in SPARK-12833, there was a typo on `scalastyle:on`: `scalstyle:on`. So, it turns off ScalaStyle checking for the rest of the file mistakenly. So, it can not find a violation on the code of `SPARK-12668` added recently. This issue fixes the existing escaping correctly and adds a new escaping for `SPARK-12668` code like the following.
```scala
test("test aliases sep and encoding for delimiter and charset") {
+ // scalastyle:off
val cars = sqlContext
...
.load(testFile(carsFile8859))
+ // scalastyle:on
```
This will prevent future potential problems, too.
## How was this patch tested?
Pass the Jenkins test.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11700 from dongjoon-hyun/SPARK-13870.
## What changes were proposed in this pull request?
This patch removes DescribeCommand's dependency on LogicalPlan. After this patch, DescribeCommand simply accepts a TableIdentifier. It minimizes the dependency, and blocks my next patch (removes SQLContext dependency from SparkPlanner).
## How was this patch tested?
Should be covered by existing unit tests and Hive compatibility tests that run describe table.
Author: Reynold Xin <rxin@databricks.com>
Closes#11710 from rxin/SPARK-13884.
## What changes were proposed in this pull request?
When we call DataFrame/Dataset.collect(), Java serializer (or Kryo Serializer) will be used to serialize the UnsafeRows in executor, then deserialize them into UnsafeRows in driver. Java serializer (and Kyro serializer) are slow on millions rows, because they try to find out the same rows, but usually there is no same rows.
This PR will serialize the UnsafeRows as byte array by packing them together, then Java serializer (or Kyro serializer) serialize the bytes very fast (there are fewer blocks and byte array are not compared by content).
The UnsafeRow format is highly compressible, the serialized bytes are also compressed (configurable by spark.io.compression.codec).
## How was this patch tested?
Existing unit tests.
Add a benchmark for collect, before this patch:
```
Intel(R) Core(TM) i7-4558U CPU 2.80GHz
collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
collect 1 million 3991 / 4311 0.3 3805.7 1.0X
collect 2 millions 10083 / 10637 0.1 9616.0 0.4X
collect 4 millions 29551 / 30072 0.0 28182.3 0.1X
```
```
Intel(R) Core(TM) i7-4558U CPU 2.80GHz
collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------
collect 1 million 775 / 1170 1.4 738.9 1.0X
collect 2 millions 1153 / 1758 0.9 1099.3 0.7X
collect 4 millions 4451 / 5124 0.2 4244.9 0.2X
```
We can see about 5-7X speedup.
Author: Davies Liu <davies@databricks.com>
Closes#11664 from davies/serialize_row.
## What changes were proposed in this pull request?
Avoid the copy in HashedRelation, since most of the HashedRelation are built with Array[Row], added the copy() for LeftSemiJoinHash. This could help to reduce the memory consumption for Broadcast join.
## How was this patch tested?
Existing tests.
Author: Davies Liu <davies@databricks.com>
Closes#11666 from davies/remove_copy.
## What changes were proposed in this pull request?
1. Rename DataFrame.scala Dataset.scala, since the class is now named Dataset.
2. Remove LegacyFunctions. It was introduced in Spark 1.6 for backward compatibility, and can be removed in Spark 2.0.
## How was this patch tested?
Should be covered by existing unit/integration tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#11704 from rxin/SPARK-13880.
## What changes were proposed in this pull request?
- Add a MetadataLog interface for metadata reliably storage.
- Add HDFSMetadataLog as a MetadataLog implementation based on HDFS.
- Update FileStreamSource to use HDFSMetadataLog instead of managing metadata by itself.
## How was this patch tested?
unit tests
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#11625 from zsxwing/metadata-log.
## What changes were proposed in this pull request?
We introduced some local operators in org.apache.spark.sql.execution.local package but never fully wired the engine to actually use these. We still plan to implement a full local mode, but it's probably going to be fairly different from what the current iterator-based local mode would look like. Based on what we know right now, we might want a push-based columnar version of these operators.
Let's just remove them for now, and we can always re-introduced them in the future by looking at branch-1.6.
## How was this patch tested?
This is simply dead code removal.
Author: Reynold Xin <rxin@databricks.com>
Closes#11705 from rxin/SPARK-13882.
This PR adds a new strategy, `FileSourceStrategy`, that can be used for planning scans of collections of files that might be partitioned or bucketed.
Compared with the existing planning logic in `DataSourceStrategy` this version has the following desirable properties:
- It removes the need to have `RDD`, `broadcastedHadoopConf` and other distributed concerns in the public API of `org.apache.spark.sql.sources.FileFormat`
- Partition column appending is delegated to the format to avoid an extra copy / devectorization when appending partition columns
- It minimizes the amount of data that is shipped to each executor (i.e. it does not send the whole list of files to every worker in the form of a hadoop conf)
- it natively supports bucketing files into partitions, and thus does not require coalescing / creating a `UnionRDD` with the correct partitioning.
- Small files are automatically coalesced into fewer tasks using an approximate bin-packing algorithm.
Currently only a testing source is planned / tested using this strategy. In follow-up PRs we will port the existing formats to this API.
A stub for `FileScanRDD` is also added, but most methods remain unimplemented.
Other minor cleanups:
- partition pruning is pushed into `FileCatalog` so both the new and old code paths can use this logic. This will also allow future implementations to use indexes or other tricks (i.e. a MySQL metastore)
- The partitions from the `FileCatalog` now propagate information about file sizes all the way up to the planner so we can intelligently spread files out.
- `Array` -> `Seq` in some internal APIs to avoid unnecessary `toArray` calls
- Rename `Partition` to `PartitionDirectory` to differentiate partitions used earlier in pruning from those where we have already enumerated the files and their sizes.
Author: Michael Armbrust <michael@databricks.com>
Closes#11646 from marmbrus/fileStrategy.
Three different things were needed to get rid of spurious warnings:
- silence deprecation warnings when cloning configuration
- change the way SparkHadoopUtil instantiates SparkConf to silence
warnings
- avoid creating new SparkConf instances where it's not needed.
On top of that, I changed the way that Logging.scala detects the repl;
now it uses a method that is overridden in the repl's Main class, and
the hack in Utils.scala is not needed anymore. This makes the 2.11 repl
behave like the 2.10 one and set the default log level to WARN, which
is a lot better. Previously, this wasn't working because the 2.11 repl
triggers log initialization earlier than the 2.10 one.
I also removed and simplified some other code in the 2.11 repl's Main
to avoid replicating logic that already exists elsewhere in Spark.
Tested the 2.11 repl in local and yarn modes.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#11510 from vanzin/SPARK-13626.
Addressing outstanding comments in #11573.
Jenkins, new test case in `DDLCommandSuite`
Author: Andrew Or <andrew@databricks.com>
Closes#11667 from andrewor14/ddl-parser-followups.
If a _SUCCESS appears in the inner partitioning dir, partition discovery will treat that _SUCCESS file as a data file. Then, partition discovery will fail because it finds that the dir structure is not valid. We should ignore those `_SUCCESS` files.
In future, it is better to ignore all files/dirs starting with `_` or `.`. This PR does not make this change. I am thinking about making this change simple, so we can consider of getting it in branch 1.6.
To ignore all files/dirs starting with `_` or `, the main change is to let ParquetRelation have another way to get metadata files. Right now, it relies on FileStatusCache's cachedLeafStatuses, which returns file statuses of both metadata files (e.g. metadata files used by parquet) and data files, which requires more changes.
https://issues.apache.org/jira/browse/SPARK-13207
Author: Yin Huai <yhuai@databricks.com>
Closes#11088 from yhuai/SPARK-13207.
## What changes were proposed in this pull request?
This PR fixes 135 typos over 107 files:
* 121 typos in comments
* 11 typos in testcase name
* 3 typos in log messages
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11689 from dongjoon-hyun/fix_more_typos.
## What changes were proposed in this pull request?
- Fixes calls to `new String(byte[])` or `String.getBytes()` that rely on platform default encoding, to use UTF-8
- Same for `InputStreamReader` and `OutputStreamWriter` constructors
- Standardizes on UTF-8 everywhere
- Standardizes specifying the encoding with `StandardCharsets.UTF-8`, not the Guava constant or "UTF-8" (which means handling `UnuspportedEncodingException`)
- (also addresses the other remaining Coverity scan issues, which are pretty trivial; these are separated into commit 1deecd8d9c )
## How was this patch tested?
Jenkins tests
Author: Sean Owen <sowen@cloudera.com>
Closes#11657 from srowen/SPARK-13823.
## What changes were proposed in this pull request?
fix typo in DataSourceRegister
## How was this patch tested?
found when going through latest code
Author: Jacky Li <jacky.likun@huawei.com>
Closes#11686 from jackylk/patch-12.
## What changes were proposed in this pull request?
This PR removes two methods, `collectRows()` and `takeRows()`, from `Dataset[T]`. These methods were added in PR #11443, and were later considered not useful.
## How was this patch tested?
Existing tests should do the work.
Author: Cheng Lian <lian@databricks.com>
Closes#11678 from liancheng/remove-collect-rows-and-take-rows.
PR #11443 added an extra `plan: Option[LogicalPlan]` argument to `AnalysisException` and attached partially analyzed plan to thrown `AnalysisException` in `QueryExecution.assertAnalyzed()`. However, the original stack trace wasn't properly inherited. This PR fixes this issue by inheriting the stack trace.
A test case is added to verify that the first entry of `AnalysisException` stack trace isn't from `QueryExecution`.
Author: Cheng Lian <lian@databricks.com>
Closes#11677 from liancheng/analysis-exception-stacktrace.
## What changes were proposed in this pull request?
This PR split the PhysicalRDD into two classes, PhysicalRDD and PhysicalScan. PhysicalRDD is used for DataFrames that is created from existing RDD. PhysicalScan is used for DataFrame that is created from data sources. This enable use to apply different optimization on both of them.
Also fix the problem for sameResult() on two DataSourceScan.
Also fix the equality check to toString for `In`. It's better to use Seq there, but we can't break this public API (sad).
## How was this patch tested?
Existing tests. Manually tested with TPCDS query Q59 and Q64, all those duplicated exchanges can be re-used now, also saw there are 40+% performance improvement (saving half of the scan).
Author: Davies Liu <davies@databricks.com>
Closes#11514 from davies/existing_rdd.
## What changes were proposed in this pull request?
This patch is ported over from viirya's changes in #11048. Currently for most DDLs we just pass the query text directly to Hive. Instead, we should parse these commands ourselves and in the future (not part of this patch) use the `HiveCatalog` to process these DDLs. This is a pretext to merging `SQLContext` and `HiveContext`.
Note: As of this patch we still pass the query text to Hive. The difference is that we now parse the commands ourselves so in the future we can just use our own catalog.
## How was this patch tested?
Jenkins, new `DDLCommandSuite`, which comprises of about 40% of the changes here.
Author: Andrew Or <andrew@databricks.com>
Closes#11573 from andrewor14/parser-plus-plus.
This is needed to avoid odd compiler errors when building just the
sql package with maven, because of odd interactions between scalac
and shaded classes.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#11640 from vanzin/SPARK-13780.
## What changes were proposed in this pull request?
PR #11443 temporarily disabled MiMA check, this PR re-enables it.
One extra change is that `object DataFrame` is also removed. The only purpose of introducing `object DataFrame` was to use it as an internal factory for creating `Dataset[Row]`. By replacing this internal factory with `Dataset.newDataFrame`, both `DataFrame` and `DataFrame$` are entirely removed from the API, so that we can simply put a `MissingClassProblem` filter in `MimaExcludes.scala` for most DataFrame API changes.
## How was this patch tested?
Tested by MiMA check triggered by Jenkins.
Author: Cheng Lian <lian@databricks.com>
Closes#11656 from liancheng/re-enable-mima.
Fix the compilation failure introduced by https://github.com/apache/spark/pull/11555 because of a merge conflict.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11648 from cloud-fan/hotbug.
## What changes were proposed in this pull request?
Add SQL generation support for window functions. The idea is simple, just treat `Window` operator like `Project`, i.e. add subquery to its child when necessary, generate a `SELECT ... FROM ...` SQL string, implement `sql` method for window related expressions, e.g. `WindowSpecDefinition`, `WindowFrame`, etc.
This PR also fixed SPARK-13720 by improving the process of adding extra `SubqueryAlias`(the `RecoverScopingInfo` rule). Before this PR, we update the qualifiers in project list while adding the subquery. However, this is incomplete as we need to update qualifiers in all ancestors that refer attributes here. In this PR, we split `RecoverScopingInfo` into 2 rules: `AddSubQuery` and `UpdateQualifier`. `AddSubQuery` only add subquery if necessary, and `UpdateQualifier` will re-propagate and update qualifiers bottom up.
Ideally we should put the bug fix part in an individual PR, but this bug also blocks the window stuff, so I put them together here.
Many thanks to gatorsmile for the initial discussion and test cases!
## How was this patch tested?
new tests in `LogicalPlanToSQLSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11555 from cloud-fan/window.
#### What changes were proposed in this pull request?
`projectList` is useless. Its value is always the same as the child.output. Remove it from the class `Window`. Removal can simplify the codes in Analyzer and Optimizer.
This PR is based on the discussion started by cloud-fan in a separate PR:
https://github.com/apache/spark/pull/5604#discussion_r55140466
This PR also eliminates useless `Window`.
cloud-fan yhuai
#### How was this patch tested?
Existing test cases cover it.
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#11565 from gatorsmile/removeProjListWindow.
## What changes were proposed in this pull request?
This PR adds support for inferring an additional set of data constraints based on attribute equality. For e.g., if an operator has constraints of the form (`a = 5`, `a = b`), we can now automatically infer an additional constraint of the form `b = 5`
## How was this patch tested?
Tested that new constraints are properly inferred for filters (by adding a new test) and equi-joins (by modifying an existing test)
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11618 from sameeragarwal/infer-isequal-constraints.
## What changes were proposed in this pull request?
This PR unifies DataFrame and Dataset by migrating existing DataFrame operations to Dataset and make `DataFrame` a type alias of `Dataset[Row]`.
Most Scala code changes are source compatible, but Java API is broken as Java knows nothing about Scala type alias (mostly replacing `DataFrame` with `Dataset<Row>`).
There are several noticeable API changes related to those returning arrays:
1. `collect`/`take`
- Old APIs in class `DataFrame`:
```scala
def collect(): Array[Row]
def take(n: Int): Array[Row]
```
- New APIs in class `Dataset[T]`:
```scala
def collect(): Array[T]
def take(n: Int): Array[T]
def collectRows(): Array[Row]
def takeRows(n: Int): Array[Row]
```
Two specialized methods `collectRows` and `takeRows` are added because Java doesn't support returning generic arrays. Thus, for example, `DataFrame.collect(): Array[T]` actually returns `Object` instead of `Array<T>` from Java side.
Normally, Java users may fall back to `collectAsList` and `takeAsList`. The two new specialized versions are added to avoid performance regression in ML related code (but maybe I'm wrong and they are not necessary here).
1. `randomSplit`
- Old APIs in class `DataFrame`:
```scala
def randomSplit(weights: Array[Double], seed: Long): Array[DataFrame]
def randomSplit(weights: Array[Double]): Array[DataFrame]
```
- New APIs in class `Dataset[T]`:
```scala
def randomSplit(weights: Array[Double], seed: Long): Array[Dataset[T]]
def randomSplit(weights: Array[Double]): Array[Dataset[T]]
```
Similar problem as above, but hasn't been addressed for Java API yet. We can probably add `randomSplitAsList` to fix this one.
1. `groupBy`
Some original `DataFrame.groupBy` methods have conflicting signature with original `Dataset.groupBy` methods. To distinguish these two, typed `Dataset.groupBy` methods are renamed to `groupByKey`.
Other noticeable changes:
1. Dataset always do eager analysis now
We used to support disabling DataFrame eager analysis to help reporting partially analyzed malformed logical plan on analysis failure. However, Dataset encoders requires eager analysi during Dataset construction. To preserve the error reporting feature, `AnalysisException` now takes an extra `Option[LogicalPlan]` argument to hold the partially analyzed plan, so that we can check the plan tree when reporting test failures. This plan is passed by `QueryExecution.assertAnalyzed`.
## How was this patch tested?
Existing tests do the work.
## TODO
- [ ] Fix all tests
- [ ] Re-enable MiMA check
- [ ] Update ScalaDoc (`since`, `group`, and example code)
Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Cheng Lian <liancheng@users.noreply.github.com>
Closes#11443 from liancheng/ds-to-df.
## What changes were proposed in this pull request?
Since the opening curly brace, '{', has many usages as discussed in [SPARK-3854](https://issues.apache.org/jira/browse/SPARK-3854), this PR adds a ScalaStyle rule to prevent '){' pattern for the following majority pattern and fixes the code accordingly. If we enforce this in ScalaStyle from now, it will improve the Scala code quality and reduce review time.
```
// Correct:
if (true) {
println("Wow!")
}
// Incorrect:
if (true){
println("Wow!")
}
```
IntelliJ also shows new warnings based on this.
## How was this patch tested?
Pass the Jenkins ScalaStyle test.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11637 from dongjoon-hyun/SPARK-3854.
## What changes were proposed in this pull request?
ContinuousQueryManager is sometimes flaky on Jenkins. I could not reproduce it on my machine, so I guess it about the waiting times which causes problems if Jenkins is loaded. I have increased the wait time in the hope that it will be less flaky.
## How was this patch tested?
I reran the unit test many times on a loop in my machine. I am going to run it a few time in Jenkins, that's the real test.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#11638 from tdas/cqm-flaky-test.
## What changes were proposed in this pull request?
We should reuse an object similar to the other non-primitive type getters. For
a query that computes averages over decimal columns, this shows a 10% speedup
on overall query times.
## How was this patch tested?
Existing tests and this benchmark
```
TPCDS Snappy: Best/Avg Time(ms) Rate(M/s) Per Row(ns)
--------------------------------------------------------------------------------
q27-agg (master) 10627 / 11057 10.8 92.3
q27-agg (this patch) 9722 / 9832 11.8 84.4
```
Author: Nong Li <nong@databricks.com>
Closes#11624 from nongli/spark-13790.
## What changes were proposed in this pull request?
This PR adds support for inferring `IsNotNull` constraints from expressions with an `!==`. More specifically, if an operator has a condition on `a !== b`, we know that both `a` and `b` in the operator output can no longer be null.
## How was this patch tested?
1. Modified a test in `ConstraintPropagationSuite` to test for expressions with an inequality.
2. Added a test in `NullFilteringSuite` for making sure an Inner join with a "non-equal" condition appropriately filters out null from their input.
cc nongli
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11594 from sameeragarwal/isnotequal-constraints.
JIRA: https://issues.apache.org/jira/browse/SPARK-13636
## What changes were proposed in this pull request?
As shown in the wholestage codegen verion of Sort operator, when Sort is top of Exchange (or other operator that produce UnsafeRow), we will create variables from UnsafeRow, than create another UnsafeRow using these variables. We should avoid the unnecessary unpack and pack variables from UnsafeRows.
## How was this patch tested?
All existing wholestage codegen tests should be passed.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#11484 from viirya/direct-consume-unsaferow.
## What changes were proposed in this pull request?
According to #11627 , this PR replace `DataFrameWriter.stream()` with `startStream()` in comments of `ContinuousQueryListener.java`.
## How was this patch tested?
Manual. (It changes on comments.)
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11629 from dongjoon-hyun/minor_rename.
## What changes were proposed in this pull request?
The new name makes it more obvious with the verb "start" that we are actually starting some execution.
## How was this patch tested?
This is just a rename. Existing unit tests should cover it.
Author: Reynold Xin <rxin@databricks.com>
Closes#11627 from rxin/SPARK-13794.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13766
This PR makes the file extensions (written by internal datasource) consistent.
**Before**
- TEXT, CSV and JSON
```
[.COMPRESSION_CODEC_NAME]
```
- Parquet
```
[.COMPRESSION_CODEC_NAME].parquet
```
- ORC
```
.orc
```
**After**
- TEXT, CSV and JSON
```
.txt[.COMPRESSION_CODEC_NAME]
.csv[.COMPRESSION_CODEC_NAME]
.json[.COMPRESSION_CODEC_NAME]
```
- Parquet
```
[.COMPRESSION_CODEC_NAME].parquet
```
- ORC
```
[.COMPRESSION_CODEC_NAME].orc
```
When the compression codec is set,
- For Parquet and ORC, each still stays in Parquet and ORC format but just have compressed data internally. So, I think it is okay to name `.parquet` and `.orc` at the end.
- For Text, CSV and JSON, each does not stays in each format but it has different data format according to compression codec. So, each has the names `.json`, `.csv` and `.txt` before the compression extension.
## How was this patch tested?
Unit tests are used and `./dev/run_tests` for coding style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11604 from HyukjinKwon/SPARK-13766.
## What changes were proposed in this pull request?
A very minor change for using `BigDecimal.decimal(f: Float)` instead of `BigDecimal(f: float)`. The latter is deprecated and can result in inconsistencies due to an implicit conversion to `Double`.
## How was this patch tested?
N/A
cc yhuai
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11597 from sameeragarwal/bigdecimal.
## What changes were proposed in this pull request?
Fix this use case, which was already fixed in SPARK-10548 in 1.6 but was broken in master due to #9264:
```
(1 to 100).par.foreach { _ => sc.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count() }
```
This threw `IllegalArgumentException` consistently before this patch. For more detail, see the JIRA.
## How was this patch tested?
New test in `SQLExecutionSuite`.
Author: Andrew Or <andrew@databricks.com>
Closes#11586 from andrewor14/fix-concurrent-sql.
## What changes were proposed in this pull request?
This PR is a small follow up on https://github.com/apache/spark/pull/11338 (https://issues.apache.org/jira/browse/SPARK-13092) to use `ExpressionSet` as part of the verification logic in `ConstraintPropagationSuite`.
## How was this patch tested?
No new tests added. Just changes the verification logic in `ConstraintPropagationSuite`.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11611 from sameeragarwal/expression-set.
#### What changes were proposed in this pull request?
Remove all the deterministic conditions in a [[Filter]] that are contained in the Child's Constraints.
For example, the first query can be simplified to the second one.
```scala
val queryWithUselessFilter = tr1
.where("tr1.a".attr > 10 || "tr1.c".attr < 10)
.join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr))
.where(
("tr1.a".attr > 10 || "tr1.c".attr < 10) &&
'd.attr < 100 &&
"tr2.a".attr === "tr1.a".attr)
```
```scala
val query = tr1
.where("tr1.a".attr > 10 || "tr1.c".attr < 10)
.join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr))
```
#### How was this patch tested?
Six test cases are added.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11406 from gatorsmile/FilterRemoval.
## What changes were proposed in this pull request?
It’s possible to have common parts in a query, for example, self join, it will be good to avoid the duplicated part to same CPUs and memory (Broadcast or cache).
Exchange will materialize the underlying RDD by shuffle or collect, it’s a great point to check duplicates and reuse them. Duplicated exchanges means they generate exactly the same result inside a query.
In order to find out the duplicated exchanges, we should be able to compare SparkPlan to check that they have same results or not. We already have that for LogicalPlan, so we should move that into QueryPlan to make it available for SparkPlan.
Once we can find the duplicated exchanges, we should replace all of them with same SparkPlan object (could be wrapped by ReusedExchage for explain), then the plan tree become a DAG. Since all the planner only work with tree, so this rule should be the last one for the entire planning.
After the rule, the plan will looks like:
```
WholeStageCodegen
: +- Project [id#0L]
: +- BroadcastHashJoin [id#0L], [id#2L], Inner, BuildRight, None
: :- Project [id#0L]
: : +- BroadcastHashJoin [id#0L], [id#1L], Inner, BuildRight, None
: : :- Range 0, 1, 4, 1024, [id#0L]
: : +- INPUT
: +- INPUT
:- BroadcastExchange HashedRelationBroadcastMode(true,List(id#1L),List(id#1L))
: +- WholeStageCodegen
: : +- Range 0, 1, 4, 1024, [id#1L]
+- ReusedExchange [id#2L], BroadcastExchange HashedRelationBroadcastMode(true,List(id#1L),List(id#1L))
```
![bjoin](https://cloud.githubusercontent.com/assets/40902/13414787/209e8c5c-df0a-11e5-8a0f-edff69d89e83.png)
For three ways SortMergeJoin,
```
== Physical Plan ==
WholeStageCodegen
: +- Project [id#0L]
: +- SortMergeJoin [id#0L], [id#4L], None
: :- INPUT
: +- INPUT
:- WholeStageCodegen
: : +- Project [id#0L]
: : +- SortMergeJoin [id#0L], [id#3L], None
: : :- INPUT
: : +- INPUT
: :- WholeStageCodegen
: : : +- Sort [id#0L ASC], false, 0
: : : +- INPUT
: : +- Exchange hashpartitioning(id#0L, 200), None
: : +- WholeStageCodegen
: : : +- Range 0, 1, 4, 33554432, [id#0L]
: +- WholeStageCodegen
: : +- Sort [id#3L ASC], false, 0
: : +- INPUT
: +- ReusedExchange [id#3L], Exchange hashpartitioning(id#0L, 200), None
+- WholeStageCodegen
: +- Sort [id#4L ASC], false, 0
: +- INPUT
+- ReusedExchange [id#4L], Exchange hashpartitioning(id#0L, 200), None
```
![sjoin](https://cloud.githubusercontent.com/assets/40902/13414790/27aea61c-df0a-11e5-8cbf-fbc985c31d95.png)
If the same ShuffleExchange or BroadcastExchange, execute()/executeBroadcast() will be called by different parents, they should cached the RDD/Broadcast, return the same one for all the parents.
## How was this patch tested?
Added some unit tests for this. Had done some manual tests on TPCDS query Q59 and Q64, we can see some exchanges are re-used (this requires a change in PhysicalRDD to for sameResult, is be done in #11514 ).
Author: Davies Liu <davies@databricks.com>
Closes#11403 from davies/dedup.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13728https://github.com/apache/spark/pull/11509 makes the output only single ORC file.
It was 10 files but this PR writes only single file. So, this could not skip stripes in ORC by the pushed down filters.
So, this PR simply repartitions data into 10 so that the test could pass.
## How was this patch tested?
unittest and `./dev/run_tests` for code style test.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#11593 from HyukjinKwon/SPARK-13728.
#### What changes were proposed in this pull request?
As shown in another PR: https://github.com/apache/spark/pull/11596, we are using `SELECT 1` as a dummy table, when the table is used for SQL statements in which a table reference is required, but the contents of the table are not important. For example,
```SQL
SELECT value FROM (select 1) dummyTable Lateral View explode(array(1,2,3)) adTable as value
```
Before the PR, the optimized plan contains a useless `Project` after Optimizer executing the `ColumnPruning` rule, as shown below:
```
== Analyzed Logical Plan ==
value: int
Project [value#22]
+- Generate explode(array(1, 2, 3)), true, false, Some(adtable), [value#22]
+- SubqueryAlias dummyTable
+- Project [1 AS 1#21]
+- OneRowRelation$
== Optimized Logical Plan ==
Generate explode([1,2,3]), false, false, Some(adtable), [value#22]
+- Project
+- OneRowRelation$
```
After the fix, the optimized plan removed the useless `Project`, as shown below:
```
== Optimized Logical Plan ==
Generate explode([1,2,3]), false, false, Some(adtable), [value#22]
+- OneRowRelation$
```
This PR is to remove `Project` when its Child's output is Nil
#### How was this patch tested?
Added a new unit test case into the suite `ColumnPruningSuite.scala`
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11599 from gatorsmile/projectOneRowRelation.
## What changes were proposed in this pull request?
If there are many branches in a CaseWhen expression, the generated code could go above the 64K limit for single java method, will fail to compile. This PR change it to fallback to interpret mode if there are more than 20 branches.
This PR is based on #11243 and #11221, thanks to joehalliwell
Closes#11243Closes#11221
## How was this patch tested?
Add a test with 50 branches.
Author: Davies Liu <davies@databricks.com>
Closes#11592 from davies/fix_when.
## What changes were proposed in this pull request?
Analysis exception occurs while running the following query.
```
SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) `a` AS `ints`
```
```
Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`ints`' given input columns: [a, `ints`]; line 1 pos 7
'Project ['ints]
+- Generate explode(a#0.b), true, false, Some(a), [`ints`#8]
+- SubqueryAlias nestedarray
+- LocalRelation [a#0], [[[[1,2,3]]]]
```
## How was this patch tested?
Added new unit tests in SQLQuerySuite and HiveQlSuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#11538 from dilipbiswal/SPARK-13698.
## What changes were proposed in this pull request?
In order to make `docs/examples` (and other related code) more simple/readable/user-friendly, this PR replaces existing codes like the followings by using `diamond` operator.
```
- final ArrayList<Product2<Object, Object>> dataToWrite =
- new ArrayList<Product2<Object, Object>>();
+ final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>();
```
Java 7 or higher supports **diamond** operator which replaces the type arguments required to invoke the constructor of a generic class with an empty set of type parameters (<>). Currently, Spark Java code use mixed usage of this.
## How was this patch tested?
Manual.
Pass the existing tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11541 from dongjoon-hyun/SPARK-13702.
## What changes were proposed in this pull request?
`ScalaReflection.mirror` method should be synchronized when scala version is `2.10` because `universe.runtimeMirror` is not thread safe.
## How was this patch tested?
I added a test to check thread safety of `ScalaRefection.mirror` method in `ScalaReflectionSuite`, which will throw the following Exception in Scala `2.10` without this patch:
```
[info] - thread safety of mirror *** FAILED *** (49 milliseconds)
[info] java.lang.UnsupportedOperationException: tail of empty list
[info] at scala.collection.immutable.Nil$.tail(List.scala:339)
[info] at scala.collection.immutable.Nil$.tail(List.scala:334)
[info] at scala.reflect.internal.SymbolTable.popPhase(SymbolTable.scala:172)
[info] at scala.reflect.internal.Symbols$Symbol.unsafeTypeParams(Symbols.scala:1477)
[info] at scala.reflect.internal.Symbols$TypeSymbol.tpe(Symbols.scala:2777)
[info] at scala.reflect.internal.Mirrors$RootsBase.init(Mirrors.scala:235)
[info] at scala.reflect.runtime.JavaMirrors$class.createMirror(JavaMirrors.scala:34)
[info] at scala.reflect.runtime.JavaMirrors$class.runtimeMirror(JavaMirrors.scala:61)
[info] at scala.reflect.runtime.JavaUniverse.runtimeMirror(JavaUniverse.scala:12)
[info] at scala.reflect.runtime.JavaUniverse.runtimeMirror(JavaUniverse.scala:12)
[info] at org.apache.spark.sql.catalyst.ScalaReflection$.mirror(ScalaReflection.scala:36)
[info] at org.apache.spark.sql.catalyst.ScalaReflectionSuite$$anonfun$12$$anonfun$apply$mcV$sp$1$$anonfun$apply$1$$anonfun$apply$2.apply(ScalaReflectionSuite.scala:256)
[info] at org.apache.spark.sql.catalyst.ScalaReflectionSuite$$anonfun$12$$anonfun$apply$mcV$sp$1$$anonfun$apply$1$$anonfun$apply$2.apply(ScalaReflectionSuite.scala:252)
[info] at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
[info] at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
[info] at scala.concurrent.impl.ExecutionContextImpl$$anon$3.exec(ExecutionContextImpl.scala:107)
[info] at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
[info] at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
[info] at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
[info] at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
```
Notice that the test will pass when Scala version is `2.11`.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#11487 from ueshin/issues/SPARK-13640.
## What changes were proposed in this pull request?
This issue fixes the following potential bugs and Java coding style detected by Coverity and Checkstyle.
- Implement both null and type checking in equals functions.
- Fix wrong type casting logic in SimpleJavaBean2.equals.
- Add `implement Cloneable` to `UTF8String` and `SortedIterator`.
- Remove dereferencing before null check in `AbstractBytesToBytesMapSuite`.
- Fix coding style: Add '{}' to single `for` statement in mllib examples.
- Remove unused imports in `ColumnarBatch` and `JavaKinesisStreamSuite`.
- Remove unused fields in `ChunkFetchIntegrationSuite`.
- Add `stop()` to prevent resource leak.
Please note that the last two checkstyle errors exist on newly added commits after [SPARK-13583](https://issues.apache.org/jira/browse/SPARK-13583).
## How was this patch tested?
manual via `./dev/lint-java` and Coverity site.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11530 from dongjoon-hyun/SPARK-13692.
This PR replaces #9925 which had issues with CI. **Please see the original PR for any previous discussions.**
## What changes were proposed in this pull request?
Deprecate the SparkSQL column operator !== and use =!= as an alternative.
Fixes subtle issues related to operator precedence (basically, !== does not have the same priority as its logical negation, ===).
## How was this patch tested?
All currently existing tests.
Author: Jakob Odersky <jodersky@gmail.com>
Closes#11588 from jodersky/SPARK-7286.
## Motivation
CSV data source was contributed by Databricks. It is the inlined version of https://github.com/databricks/spark-csv. The data source name was `com.databricks.spark.csv`. As a result there are many tables created on older versions of spark with that name as the source. For backwards compatibility we should keep the old name.
## Proposed changes
`com.databricks.spark.csv` was added to list of `backwardCompatibilityMap` in `ResolvedDataSource.scala`
## Tests
A unit test was added to `CSVSuite` to parse a csv file using the old name.
Author: Hossein <hossein@databricks.com>
Closes#11589 from falaki/SPARK-13754.
## What changes were proposed in this pull request?
This PR fix the sizeInBytes of HadoopFsRelation.
## How was this patch tested?
Added regression test for that.
Author: Davies Liu <davies@databricks.com>
Closes#11590 from davies/fix_sizeInBytes.
When generating Graphviz DOT files in the SQL query visualization we need to escape double-quotes inside node labels. This is a followup to #11309, which fixed a similar graph in Spark Core's DAG visualization.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#11587 from JoshRosen/graphviz-escaping.
## What changes were proposed in this pull request?
If a filter predicate or a join condition consists of `IsNotNull` checks, we should reorder these checks such that these non-nullability checks are evaluated before the rest of the predicates.
For e.g., if a filter predicate is of the form `a > 5 && isNotNull(b)`, we should rewrite this as `isNotNull(b) && a > 5` during physical plan generation.
## How was this patch tested?
new unit tests that verify the physical plan for both filters and joins in `ReorderedPredicateSuite`
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11511 from sameeragarwal/reorder-isnotnull.
Follow-up to #11509, that simply refactors the interface that we use when resolving a pluggable `DataSource`.
- Multiple functions share the same set of arguments so we make this a case class, called `DataSource`. Actual resolution is now done by calling a function on this class.
- Instead of having multiple methods named `apply` (some of which do writing some of which do reading) we now explicitly have `resolveRelation()` and `write(mode, df)`.
- Get rid of `Array[String]` since this is an internal API and was forcing us to awkwardly call `toArray` in a bunch of places.
Author: Michael Armbrust <michael@databricks.com>
Closes#11572 from marmbrus/dataSourceResolution.
## What changes were proposed in this pull request?
This removes the remaining deprecated Octal escape literals. The followings are the warnings on those two lines.
```
LiteralExpressionSuite.scala:99: Octal escape literals are deprecated, use \u0000 instead.
HiveQlSuite.scala:74: Octal escape literals are deprecated, use \u002c instead.
```
## How was this patch tested?
Manual.
During building, there should be no warning on `Octal escape literals`.
```
mvn -DskipTests clean install
```
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11584 from dongjoon-hyun/SPARK-13400.
## What changes were proposed in this pull request?
This PR add SQL generation support for aggregate with multi-distinct, by simply moving the `DistinctAggregationRewriter` rule to optimizer.
More discussions are needed as this breaks an import contract: analyzed plan should be able to run without optimization. However, the `ComputeCurrentTime` rule has kind of broken it already, and I think maybe we should add a new phase for this kind of rules, because strictly speaking they don't belong to analysis and is coupled with the physical plan implementation.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11579 from cloud-fan/distinct.
## What changes were proposed in this pull request?
In order to avoid StackOverflow when parse a expression with hundreds of ORs, we should use loop instead of recursive functions to flatten the tree as list. This PR also build a balanced tree to reduce the depth of generated And/Or expression, to avoid StackOverflow in analyzer/optimizer.
## How was this patch tested?
Add new unit tests. Manually tested with TPCDS Q3 with hundreds predicates in it [1]. These predicates help to reduce the number of partitions, then the query time went from 60 seconds to 8 seconds.
[1] https://github.com/cloudera/impala-tpcds-kit/blob/master/queries/q3.sql
Author: Davies Liu <davies@databricks.com>
Closes#11501 from davies/long_or.