## What changes were proposed in this pull request?
PR #14278 is a more general and simpler fix for SPARK-16632 than PR #14272. After merging #14278, we no longer need changes made in #14272. So here I revert them.
This PR targets both master and branch-2.0.
## How was this patch tested?
Existing tests.
Author: Cheng Lian <lian@databricks.com>
Closes#14300 from liancheng/revert-pr-14272.
## What changes were proposed in this pull request?
Elt function doesn't support codegen execution now. We should add the support.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#14277 from viirya/elt-codegen.
## What changes were proposed in this pull request?
In `SpecificParquetRecordReaderBase`, which is used by the vectorized Parquet reader, we convert the Parquet requested schema into a Spark schema to guide column reader initialization. However, the Parquet requested schema is tailored from the schema of the physical file being scanned, and may have inaccurate type information due to bugs of other systems (e.g. HIVE-14294).
On the other hand, we already set the real Spark requested schema into Hadoop configuration in [`ParquetFileFormat`][1]. This PR simply reads out this schema to replace the converted one.
## How was this patch tested?
New test case added in `ParquetQuerySuite`.
[1]: https://github.com/apache/spark/blob/v2.0.0-rc5/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L292-L294
Author: Cheng Lian <lian@databricks.com>
Closes#14278 from liancheng/spark-16632-simpler-fix.
## What changes were proposed in this pull request?
Saving partitions to JDBC in transaction can use a weaker transaction isolation level to reduce locking. Use better method to check if transactions are supported.
## How was this patch tested?
Existing Jenkins tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#14054 from srowen/SPARK-16226.
## What changes were proposed in this pull request?
aggregate expressions can only be executed inside `Aggregate`, if we propagate it up with constraints, the parent operator can not execute it and will fail at runtime.
## How was this patch tested?
new test in SQLQuerySuite
Author: Wenchen Fan <wenchen@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#14281 from cloud-fan/bug.
This allows configuration to be more flexible, for example, when the cluster does
not have a homogeneous configuration (e.g. packages are installed on different
paths in different nodes). By allowing one to reference the environment from
the conf, it becomes possible to work around those in certain cases.
As part of the implementation, ConfigEntry now keeps track of all "known" configs
(i.e. those created through the use of ConfigBuilder), since that list is used
by the resolution code. This duplicates some code in SQLConf, which could potentially
be merged with this now. It will also make it simpler to implement some missing
features such as filtering which configs show up in the UI or in event logs - which
are not part of this change.
Another change is in the way ConfigEntry reads config data; it now takes a string
map and a function that reads env variables, so that it can be called both from
SparkConf and SQLConf. This makes it so both places follow the same read path,
instead of having to replicate certain logic in SQLConf. There are still a
couple of methods in SQLConf that peek into fields of ConfigEntry directly,
though.
Tested via unit tests, and by using the new variable expansion functionality
in a shell session with a custom spark.sql.hive.metastore.jars value.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#14022 from vanzin/SPARK-16272.
## What changes were proposed in this pull request?
Due to backward-compatibility reasons, the following Parquet schema is ambiguous:
```
optional group f (LIST) {
repeated group list {
optional group element {
optional int32 element;
}
}
}
```
According to the parquet-format spec, when interpreted as a standard 3-level layout, this type is equivalent to the following SQL type:
```
ARRAY<STRUCT<element: INT>>
```
However, when interpreted as a legacy 2-level layout, it's equivalent to
```
ARRAY<STRUCT<element: STRUCT<element: INT>>>
```
Historically, to disambiguate these cases, we employed two methods:
- `ParquetSchemaConverter.isElementType()`
Used to disambiguate the above cases while converting Parquet types to Spark types.
- `ParquetRowConverter.isElementType()`
Used to disambiguate the above cases while instantiating row converters that convert Parquet records to Spark rows.
Unfortunately, these two methods make different decision about the above problematic Parquet type, and caused SPARK-16344.
`ParquetRowConverter.isElementType()` is necessary for Spark 1.4 and earlier versions because Parquet requested schemata are directly converted from Spark schemata in these versions. The converted Parquet schemata may be incompatible with actual schemata of the underlying physical files when the files are written by a system/library that uses a schema conversion scheme that is different from Spark when writing Parquet LIST and MAP fields.
In Spark 1.5, Parquet requested schemata are always properly tailored from schemata of physical files to be read. Thus `ParquetRowConverter.isElementType()` is no longer necessary. This PR replaces this method with a simply yet accurate scheme: whenever an ambiguous Parquet type is hit, convert the type in question back to a Spark type using `ParquetSchemaConverter` and check whether it matches the corresponding Spark type.
## How was this patch tested?
New test cases added in `ParquetHiveCompatibilitySuite` and `ParquetQuerySuite`.
Author: Cheng Lian <lian@databricks.com>
Closes#14014 from liancheng/spark-16344-for-master-and-2.0.
Some 1.7 JVMs have a bug that is triggered by certain Scala-generated
bytecode. GenericArrayData suffers from that and fails to load in certain
JVMs.
Moving the offending code out of the constructor and into a helper method
avoids the issue.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#14271 from vanzin/SPARK-16634.
When Hive (or at least certain versions of Hive) creates parquet files
containing tinyint or smallint columns, it stores them as int32, but
doesn't annotate the parquet field as containing the corresponding
int8 / int16 data. When Spark reads those files using the vectorized
reader, it follows the parquet schema for these fields, but when
actually reading the data it tries to use the type fetched from
the metastore, and then fails because data has been loaded into the
wrong fields in OnHeapColumnVector.
So instead of blindly trusting the parquet schema, check whether the
Catalyst-provided schema disagrees with it, and adjust the types so
that the necessary metadata is present when loading the data into
the ColumnVector instance.
Tested with unit tests and with tests that create byte / short columns
in Hive and try to read them from Spark.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#14272 from vanzin/SPARK-16632.
## What changes were proposed in this pull request?
In 2.0, we add a new logic to convert HiveTableScan on ORC tables to Spark's native code path. However, during this conversion, we drop the original metastore schema (https://issues.apache.org/jira/browse/SPARK-15705). Because of this regression, I am changing the default value of `spark.sql.hive.convertMetastoreOrc` to false.
Author: Yin Huai <yhuai@databricks.com>
Closes#14267 from yhuai/SPARK-15705-changeDefaultValue.
## What changes were proposed in this pull request?
`Nvl` function should support numeric-straing cases like Hive/Spark1.6. Currently, `Nvl` finds the tightest common types among numeric types. This PR extends that to consider `String` type, too.
```scala
- TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
+ TypeCoercion.findTightestCommonTypeToString(left.dataType, right.dataType).map { dtype =>
```
**Before**
```scala
scala> sql("select nvl('0', 1)").collect()
org.apache.spark.sql.AnalysisException: cannot resolve `nvl("0", 1)` due to data type mismatch:
input to function coalesce should all be the same type, but it's [string, int]; line 1 pos 7
```
**After**
```scala
scala> sql("select nvl('0', 1)").collect()
res0: Array[org.apache.spark.sql.Row] = Array([0])
```
## How was this patch tested?
Pass the Jenkins tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14251 from dongjoon-hyun/SPARK-16602.
https://issues.apache.org/jira/browse/SPARK-16535
## What changes were proposed in this pull request?
When I scan through the pom.xml of sub projects, I found this warning as below and attached screenshot
```
Definition of groupId is redundant, because it's inherited from the parent
```
![screen shot 2016-07-13 at 3 13 11 pm](https://cloud.githubusercontent.com/assets/3925641/16823121/744f893e-4916-11e6-8a52-042f83b9db4e.png)
I've tried to remove some of the lines with groupId definition, and the build on my local machine is still ok.
```
<groupId>org.apache.spark</groupId>
```
As I just find now `<maven.version>3.3.9</maven.version>` is being used in Spark 2.x, and Maven-3 supports versionless parent elements: Maven 3 will remove the need to specify the parent version in sub modules. THIS is great (in Maven 3.1).
ref: http://stackoverflow.com/questions/3157240/maven-3-worth-it/3166762#3166762
## How was this patch tested?
I've tested by re-building the project, and build succeeded.
Author: Xin Ren <iamshrek@126.com>
Closes#14189 from keypointt/SPARK-16535.
## What changes were proposed in this pull request?
This PR improves `LogicalPlanToSQLSuite` to check the generated SQL directly by **structure**. So far, `LogicalPlanToSQLSuite` relies on `checkHiveQl` to ensure the **successful SQL generation** and **answer equality**. However, it does not guarantee the generated SQL is the same or will not be changed unnoticeably.
## How was this patch tested?
Pass the Jenkins. This is only a testsuite change.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14235 from dongjoon-hyun/SPARK-16590.
## What changes were proposed in this pull request?
In ScriptInputOutputSchema, we read default RecordReader and RecordWriter from conf. Since Spark 2.0 has deleted those config keys from hive conf, we have to set default reader/writer class name by ourselves. Otherwise we will get None for LazySimpleSerde, the data written would not be able to read by script. The test case added worked fine with previous version of Spark, but would fail now.
## How was this patch tested?
added a test case in SQLQuerySuite.
Closes#14169
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#14249 from yhuai/scriptTransformation.
## What changes were proposed in this pull request?
Currently, `JacksonGenerator.apply` is doing type-based dispatch for each row to write appropriate values.
It might not have to be done like this because the schema is already kept.
So, appropriate writers can be created first according to the schema once, and then apply them to each row. This approach is similar with `CatalystWriteSupport`.
This PR corrects `JacksonGenerator` so that it creates all writers for the schema once and then applies them to each row rather than type dispatching for every row.
Benchmark was proceeded with the codes below:
```scala
test("Benchmark for JSON writer") {
val N = 500 << 8
val row =
"""{"struct":{"field1": true, "field2": 92233720368547758070},
"structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]},
"arrayOfString":["str1", "str2"],
"arrayOfInteger":[1, 2147483647, -2147483648],
"arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808],
"arrayOfBigInteger":[922337203685477580700, -922337203685477580800],
"arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308],
"arrayOfBoolean":[true, false, true],
"arrayOfNull":[null, null, null, null],
"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
"arrayOfArray1":[[1, 2, 3], ["str1", "str2"]],
"arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]]
}"""
val df = spark.sqlContext.read.json(spark.sparkContext.parallelize(List.fill(N)(row)))
val benchmark = new Benchmark("JSON writer", N)
benchmark.addCase("writing JSON file", 10) { _ =>
withTempPath { path =>
df.write.format("json").save(path.getCanonicalPath)
}
}
benchmark.run()
}
```
This produced the results below
- **Before**
```
JSON writer: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
writing JSON file 1675 / 1767 0.1 13087.5 1.0X
```
- **After**
```
JSON writer: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
writing JSON file 1597 / 1686 0.1 12477.1 1.0X
```
In addition, I ran this benchmark 10 times for each and calculated the average elapsed time as below:
| **Before** | **After**|
|---------------|------------|
|17478ms |16669ms |
It seems roughly ~5% is improved.
## How was this patch tested?
Existing tests should cover this.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#14028 from HyukjinKwon/SPARK-16351.
## What changes were proposed in this pull request?
This patch moves regexp related unit tests from StringExpressionsSuite to RegexpExpressionsSuite to match the file name for regexp expressions.
## How was this patch tested?
This is a test only change.
Author: Reynold Xin <rxin@databricks.com>
Closes#14230 from rxin/SPARK-16584.
## What changes were proposed in this pull request?
This patch is just a slightly safer way to fix the issue we encountered in https://github.com/apache/spark/pull/14168 should this pattern re-occur at other places in the code.
## How was this patch tested?
Existing tests. Also, I manually tested that it fixes the problem in SPARK-16514 without having the proposed change in https://github.com/apache/spark/pull/14168
Author: Sameer Agarwal <sameerag@cs.berkeley.edu>
Closes#14227 from sameeragarwal/codegen.
## What changes were proposed in this pull request?
Most of the documentation in https://github.com/apache/spark/blob/master/sql/README.md is stale. It would be useful to keep the list of projects to explain what's going on, and everything else should be removed.
## How was this patch tested?
N/A
Author: Reynold Xin <rxin@databricks.com>
Closes#14211 from rxin/SPARK-16557.
## What changes were proposed in this pull request?
There are some calls to methods or fields (getParameters, properties) which are then passed to Java/Scala collection converters. Unfortunately those fields can be null in some cases and then the conversions throws NPE. We fix it by wrapping calls to those fields and methods with option and then do the conversion.
## How was this patch tested?
Manually tested with a custom Hive metastore.
Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>
Closes#14200 from jacek-lewandowski/SPARK-16528.
## What changes were proposed in this pull request?
`SQLTestUtils.withTempDatabase` is a frequently used test harness to setup a temporary table and clean up finally. This issue improves like the following for usability.
```scala
- try f(dbName) finally spark.sql(s"DROP DATABASE $dbName CASCADE")
+ try f(dbName) finally {
+ if (spark.catalog.currentDatabase == dbName) {
+ spark.sql(s"USE ${DEFAULT_DATABASE}")
+ }
+ spark.sql(s"DROP DATABASE $dbName CASCADE")
+ }
```
In case of forgetting to reset the databaes, `withTempDatabase` will not raise Exception.
## How was this patch tested?
This improves test harness.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14184 from dongjoon-hyun/SPARK-16529.
## What changes were proposed in this pull request?
This PR changes the name of columns returned by `SHOW PARTITION` and `SHOW COLUMNS` commands. Currently, both commands uses `result` as a column name.
**Comparison: Column Name**
Command|Spark(Before)|Spark(After)|Hive
----------|--------------|------------|-----
SHOW PARTITIONS|result|partition|partition
SHOW COLUMNS|result|col_name|field
Note that Spark/Hive uses `col_name` in `DESC TABLES`. So, this PR chooses `col_name` for consistency among Spark commands.
**Before**
```scala
scala> sql("show partitions p").show()
+------+
|result|
+------+
| b=2|
+------+
scala> sql("show columns in p").show()
+------+
|result|
+------+
| a|
| b|
+------+
```
**After**
```scala
scala> sql("show partitions p").show
+---------+
|partition|
+---------+
| b=2|
+---------+
scala> sql("show columns in p").show
+--------+
|col_name|
+--------+
| a|
| b|
+--------+
```
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14199 from dongjoon-hyun/SPARK-16543.
#### What changes were proposed in this pull request?
Based on the [Hive SQL syntax](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-ChangeColumnName/Type/Position/Comment), the command to change column name/type/position/comment is `ALTER TABLE CHANGE COLUMN`. However, in our .g4 file, it is `ALTER TABLE CHANGE COLUMNS`. Because it is the last optional keyword, it does not take any effect. Thus, I put the issue as a Trivial level.
cc hvanhovell
#### How was this patch tested?
Existing test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14186 from gatorsmile/changeColumns.
## What changes were proposed in this pull request?
`Alias` with metadata is not a no-op and we should not strip it in `RemoveAliasOnlyProject` rule.
This PR also did some improvement for this rule:
1. extend the semantic of `alias-only`. Now we allow the project list to be partially aliased.
2. add unit test for this rule.
## How was this patch tested?
new `RemoveAliasOnlyProjectSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14106 from cloud-fan/bug.
## What changes were proposed in this pull request?
This patch enables SparkSession to provide spark version.
## How was this patch tested?
Manual test:
```
scala> sc.version
res0: String = 2.1.0-SNAPSHOT
scala> spark.version
res1: String = 2.1.0-SNAPSHOT
```
```
>>> sc.version
u'2.1.0-SNAPSHOT'
>>> spark.version
u'2.1.0-SNAPSHOT'
```
Author: Liwei Lin <lwlin7@gmail.com>
Closes#14165 from lw-lin/add-version.
#### What changes were proposed in this pull request?
If we create a table pointing to a parquet/json datasets without specifying the schema, describe table command does not show the schema at all. It only shows `# Schema of this table is inferred at runtime`. In 1.6, describe table does show the schema of such a table.
~~For data source tables, to infer the schema, we need to load the data source tables at runtime. Thus, this PR calls the function `lookupRelation`.~~
For data source tables, we infer the schema before table creation. Thus, this PR set the inferred schema as the table schema when table creation.
#### How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14148 from gatorsmile/describeSchema.
## What changes were proposed in this pull request?
It's unnecessary. `QueryTest` already sets it.
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#14170 from brkyvz/test-tz.
## What changes were proposed in this pull request?
Currently our Optimizer may reorder the predicates to run them more efficient, but in non-deterministic condition, change the order between deterministic parts and non-deterministic parts may change the number of input rows. For example:
```SELECT a FROM t WHERE rand() < 0.1 AND a = 1```
And
```SELECT a FROM t WHERE a = 1 AND rand() < 0.1```
may call rand() for different times and therefore the output rows differ.
This PR improved this condition by checking whether the predicate is placed before any non-deterministic predicates.
## How was this patch tested?
Expanded related testcases in FilterPushdownSuite.
Author: 蒋星博 <jiangxingbo@meituan.com>
Closes#14012 from jiangxb1987/ppd.
## What changes were proposed in this pull request?
Fix Java style errors and remove unused imports, which are randomly found
## How was this patch tested?
Tested on my local machine.
Author: Xin Ren <iamshrek@126.com>
Closes#14161 from keypointt/SPARK-16437.
## What changes were proposed in this pull request?
A second form of AssertQuery now actually invokes the condition; avoids a build warning too
## How was this patch tested?
Jenkins; running StreamTest
Author: Sean Owen <sowen@cloudera.com>
Closes#14133 from srowen/SPARK-15889.2.
## What changes were proposed in this pull request?
RegexExtract and RegexReplace currently crash on non-nullable input due use of a hard-coded local variable name (e.g. compiles fail with `java.lang.Exception: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 85, Column 26: Redefinition of local variable "m" `).
This changes those variables to use fresh names, and also in a few other places.
## How was this patch tested?
Unit tests. rxin
Author: Eric Liang <ekl@databricks.com>
Closes#14168 from ericl/sc-3906.
## What changes were proposed in this pull request?
This patch implements reflect SQL function, which can be used to invoke a Java method in SQL. Slightly different from Hive, this implementation requires the class name and the method name to be literals. This implementation also supports only a smaller number of data types, and requires the function to be static, as suggested by rxin in #13969.
java_method is an alias for reflect, so this should also resolve SPARK-16277.
## How was this patch tested?
Added expression unit tests and an end-to-end test.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14138 from petermaxlee/reflect-static.
This option is used by Hive to directly delete the files instead of
moving them to the trash. This is needed in certain configurations
where moving the files does not work. For non-Hive tables and partitions,
Spark already behaves as if the PURGE option was set, so there's no
need to do anything.
Hive support for PURGE was added in 0.14 (for tables) and 1.2 (for
partitions), so the code reflects that: trying to use the option with
older versions of Hive will cause an exception to be thrown.
The change is a little noisier than I would like, because of the code
to propagate the new flag through all the interfaces and implementations;
the main changes are in the parser and in HiveShim, aside from the tests
(DDLCommandSuite, VersionsSuite).
Tested by running sql and catalyst unit tests, plus VersionsSuite which
has been updated to test the version-specific behavior. I also ran an
internal test suite that uses PURGE and would not pass previously.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#13831 from vanzin/SPARK-16119.
## What changes were proposed in this pull request?
In code generation, it is incorrect for expressions to reuse variable names across different instances of itself. As an example, SPARK-16488 reports a bug in which pmod expression reuses variable name "r".
This patch updates ExpressionEvalHelper test harness to always project two instances of the same expression, which will help us catch variable reuse problems in expression unit tests. This patch also fixes the bug in crc32 expression.
## How was this patch tested?
This is a test harness change, but I also created a new test suite for testing the test harness.
Author: Reynold Xin <rxin@databricks.com>
Closes#14146 from rxin/SPARK-16489.
## What changes were proposed in this pull request?
when query only use metadata (example: partition key), it can return results based on metadata without scanning files. Hive did it in HIVE-1003.
## How was this patch tested?
add unit tests
Author: Lianhui Wang <lianhuiwang09@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Lianhui Wang <lianhuiwang@users.noreply.github.com>
Closes#13494 from lianhuiwang/metadata-only.
## What changes were proposed in this pull request?
Currently the input `RDD` of `Dataset` is always serialized to `RDD[InternalRow]` prior to being as `Dataset`, but there is a case that we use `map` or `mapPartitions` just after converted to `Dataset`.
In this case, serialize and then deserialize happens but it would not be needed.
This pr adds `ExistingRDD` logical plan for input with `RDD` to have a chance to eliminate serialize/deserialize.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#13890 from ueshin/issues/SPARK-16189.
## What changes were proposed in this pull request?
It would be useful to support listing the columns that are referenced by a filter. This can help simplify data source planning, because with this we would be able to implement unhandledFilters method in HadoopFsRelation.
This is based on rxin's patch (#13901) and adds unit tests.
## How was this patch tested?
Added a new suite FiltersSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Author: Reynold Xin <rxin@databricks.com>
Closes#14120 from petermaxlee/SPARK-16199.
## What changes were proposed in this pull request?
In order to make it clear which filters are fully handled by the
underlying datasource we will mark them with an *. This will give a
clear visual queue to users that the filter is being treated differently
by catalyst than filters which are just presented to the underlying
DataSource.
Examples from the FilteredScanSuite, in this example `c IN (...)` is handled by the source, `b < ...` is not
### Before
```
//SELECT a FROM oneToTenFiltered WHERE a + b > 9 AND b < 16 AND c IN ('bbbbbBBBBB', 'cccccCCCCC', 'dddddDDDDD', 'foo')
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
+- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```
### After
```
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
+- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), *In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```
## How was the this patch tested?
Manually tested with the Spark Cassandra Connector, a source which fully handles underlying filters. Now fully handled filters appear with an * next to their names. I can add an automated test as well if requested
Post 1.6.1
Tested by modifying the FilteredScanSuite to run explains.
Author: Russell Spitzer <Russell.Spitzer@gmail.com>
Closes#11317 from RussellSpitzer/SPARK-12639-Star.
## What changes were proposed in this pull request?
This patch fixes a variable namespace collision bug in pmod and partitionBy
## How was this patch tested?
Regression test for one possible occurrence. A more general fix in `ExpressionEvalHelper.checkEvaluation` will be in a subsequent PR.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#14144 from sameeragarwal/codegen-bug.
## What changes were proposed in this pull request?
Incorrect list of files were being allocated to a batch. This caused a file to read multiple times in the multiple batches.
## How was this patch tested?
Added unit tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#14143 from tdas/SPARK-16430-1.
## What changes were proposed in this pull request?
Display `No physical plan. Waiting for data.` instead of `N/A` for StreamingQuery.explain when no data arrives because `N/A` doesn't provide meaningful information.
## How was this patch tested?
Existing unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#14100 from zsxwing/SPARK-16433.
## What changes were proposed in this pull request?
A structured streaming example with event time windowing.
## How was this patch tested?
Run locally
Author: James Thomas <jamesjoethomas@gmail.com>
Closes#13957 from jjthomas/current.
Some Hadoop classes needed by the Hive metastore client jars are not present
in Spark's packaging (for example, "org/apache/hadoop/mapred/MRVersion"). So
if the parent class loader fails to find a class, try to load it from the
isolated class loader, in case it's available there.
Tested by setting spark.sql.hive.metastore.jars to local paths with Hive/Hadoop
libraries and verifying that Spark can talk to the metastore.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#14020 from vanzin/SPARK-16349.
## What changes were proposed in this pull request?
Temporary tables are used frequently, but `spark.catalog.listColumns` does not support those tables. This PR make `SessionCatalog` supports temporary table column listing.
**Before**
```scala
scala> spark.range(10).createOrReplaceTempView("t1")
scala> spark.catalog.listTables().collect()
res1: Array[org.apache.spark.sql.catalog.Table] = Array(Table[name=`t1`, tableType=`TEMPORARY`, isTemporary=`true`])
scala> spark.catalog.listColumns("t1").collect()
org.apache.spark.sql.AnalysisException: Table `t1` does not exist in database `default`.;
```
**After**
```
scala> spark.catalog.listColumns("t1").collect()
res2: Array[org.apache.spark.sql.catalog.Column] = Array(Column[name='id', description='id', dataType='bigint', nullable='false', isPartition='false', isBucket='false'])
```
## How was this patch tested?
Pass the Jenkins tests including a new testcase.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14114 from dongjoon-hyun/SPARK-16458.
## What changes were proposed in this pull request?
After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number.
## How was this patch tested?
N/A
Author: Reynold Xin <rxin@databricks.com>
Closes#14130 from rxin/SPARK-16477.
## What changes were proposed in this pull request?
This PR prevents dropping the current database to avoid errors like the followings.
```scala
scala> sql("create database delete_db")
scala> sql("use delete_db")
scala> sql("drop database delete_db")
scala> sql("create table t as select 1")
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database `delete_db` not found;
```
## How was this patch tested?
Pass the Jenkins tests including an updated testcase.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14115 from dongjoon-hyun/SPARK-16459.
#### What changes were proposed in this pull request?
**Issue 1:** When a query containing LIMIT/TABLESAMPLE 0, the statistics could be zero. Results are correct but it could cause a huge performance regression. For example,
```Scala
Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v")
.createOrReplaceTempView("test")
val df1 = spark.table("test")
val df2 = spark.table("test").limit(0)
val df = df1.join(df2, Seq("k"), "left")
```
The statistics of both `df` and `df2` are zero. The statistics values should never be zero; otherwise `sizeInBytes` of `BinaryNode` will also be zero (product of children). This PR is to increase it to `1` when the num of rows is equal to 0.
**Issue 2:** When a query containing negative LIMIT/TABLESAMPLE, we should issue exceptions. Negative values could break the implementation assumption of multiple parts. For example, statistics calculation. Below is the example query.
```SQL
SELECT * FROM testData TABLESAMPLE (-1 rows)
SELECT * FROM testData LIMIT -1
```
This PR is to issue an appropriate exception in this case.
**Issue 3:** Spark SQL follows the restriction of LIMIT clause in Hive. The argument to the LIMIT clause must evaluate to a constant value. It can be a numeric literal, or another kind of numeric expression involving operators, casts, and function return values. You cannot refer to a column or use a subquery. Currently, we do not detect whether the expression in LIMIT clause is foldable or not. If non-foldable, we might issue a strange error message. For example,
```SQL
SELECT * FROM testData LIMIT rand() > 0.2
```
Then, a misleading error message is issued, like
```
assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
+- LocalLimit (_nondeterministic#202 > 0.2)
+- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
+- LogicalRDD [key#11, value#12]
java.lang.AssertionError: assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
+- LocalLimit (_nondeterministic#202 > 0.2)
+- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
+- LogicalRDD [key#11, value#12]
```
This PR detects it and then issues a meaningful error message.
#### How was this patch tested?
Added test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14034 from gatorsmile/limit.