## What changes were proposed in this pull request?
In forType function of object RandomDataGenerator, the code following:
if (maybeSqlTypeGenerator.isDefined){
....
Some(generator)
} else{
None
}
will be changed. Instead, maybeSqlTypeGenerator.map will be used.
## How was this patch tested?
All of the current unit tests passed.
Author: Weiqing Yang <yangweiqing001@gmail.com>
Closes#13448 from Sherry302/master.
## What changes were proposed in this pull request?
For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow row to be null, only its columns can be null.
This PR explicitly add this constraint and throw exception if users break it.
## How was this patch tested?
several new tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13469 from cloud-fan/null-object.
## What changes were proposed in this pull request?
There are 2 kinds of `GetStructField`:
1. resolved from `UnresolvedExtractValue`, and it will have a `name` property.
2. created when we build deserializer expression for nested tuple, no `name` property.
When we want to validate the ordinals of nested tuple, we should only catch `GetStructField` without the name property.
## How was this patch tested?
new test in `EncoderResolutionSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13474 from cloud-fan/ordinal-check.
## What changes were proposed in this pull request?
Our encoder framework has been evolved a lot, this PR tries to clean up the code to make it more readable and emphasise the concept that encoder should be used as a container of serde expressions.
1. move validation logic to analyzer instead of encoder
2. only have a `resolveAndBind` method in encoder instead of `resolve` and `bind`, as we don't have the encoder life cycle concept anymore.
3. `Dataset` don't need to keep a resolved encoder, as there is no such concept anymore. bound encoder is still needed to do serialization outside of query framework.
4. Using `BoundReference` to represent an unresolved field in deserializer expression is kind of weird, this PR adds a `GetColumnByOrdinal` for this purpose. (serializer expression still use `BoundReference`, we can replace it with `GetColumnByOrdinal` in follow-ups)
## How was this patch tested?
existing test
Author: Wenchen Fan <wenchen@databricks.com>
Author: Cheng Lian <lian@databricks.com>
Closes#13269 from cloud-fan/clean-encoder.
## What changes were proposed in this pull request?
This command didn't work for Hive tables. Now it does:
```
ALTER TABLE boxes PARTITION (width=3)
SET SERDE 'com.sparkbricks.serde.ColumnarSerDe'
WITH SERDEPROPERTIES ('compress'='true')
```
## How was this patch tested?
`HiveExternalCatalogSuite`
Author: Andrew Or <andrew@databricks.com>
Closes#13453 from andrewor14/alter-partition-storage.
## What changes were proposed in this pull request?
This patch fixes a number of `com.esotericsoftware.kryo.KryoException: java.lang.NullPointerException` exceptions reported in [SPARK-15604], [SPARK-14752] etc. (while executing sparkSQL queries with the kryo serializer) by explicitly implementing `KryoSerialization` for `LazilyGenerateOrdering`.
## How was this patch tested?
1. Modified `OrderingSuite` so that all tests in the suite also test kryo serialization (for both interpreted and generated ordering).
2. Manually verified TPC-DS q1.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#13466 from sameeragarwal/kryo.
## What changes were proposed in this pull request?
This PR add a rule at the end of analyzer to correct nullable fields of attributes in a logical plan by using nullable fields of the corresponding attributes in its children logical plans (these plans generate the input rows).
This is another approach for addressing SPARK-13484 (the first approach is https://github.com/apache/spark/pull/11371).
Close#113711
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#13290 from yhuai/SPARK-13484.
## What changes were proposed in this pull request?
This patch moves all user-facing structured streaming classes into sql.streaming. As part of this, I also added some since version annotation to methods and classes that don't have them.
## How was this patch tested?
Updated tests to reflect the moves.
Author: Reynold Xin <rxin@databricks.com>
Closes#13429 from rxin/SPARK-15686.
## What changes were proposed in this pull request?
Currently structured streaming only supports append output mode. This PR adds the following.
- Added support for Complete output mode in the internal state store, analyzer and planner.
- Added public API in Scala and Python for users to specify output mode
- Added checks for unsupported combinations of output mode and DF operations
- Plans with no aggregation should support only Append mode
- Plans with aggregation should support only Update and Complete modes
- Default output mode is Append mode (**Question: should we change this to automatically set to Complete mode when there is aggregation?**)
- Added support for Complete output mode in Memory Sink. So Memory Sink internally supports append and complete, update. But from public API only Complete and Append output modes are supported.
## How was this patch tested?
Unit tests in various test suites
- StreamingAggregationSuite: tests for complete mode
- MemorySinkSuite: tests for checking behavior in Append and Complete modes.
- UnsupportedOperationSuite: tests for checking unsupported combinations of DF ops and output modes
- DataFrameReaderWriterSuite: tests for checking that output mode cannot be called on static DFs
- Python doc test and existing unit tests modified to call write.outputMode.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#13286 from tdas/complete-mode.
In this case, the result type of the expression becomes DECIMAL(38, 36) as we promote the individual string literals to DECIMAL(38, 18) when we handle string promotions for `BinaryArthmaticExpression`.
I think we need to cast the string literals to Double type instead. I looked at the history and found that this was changed to use decimal instead of double to avoid potential loss of precision when we cast decimal to double.
To double check i ran the query against hive, mysql. This query returns non NULL result for both the databases and both promote the expression to use double.
Here is the output.
- Hive
```SQL
hive> create table l2 as select (cast(99 as decimal(19,6)) + '2') from l1;
OK
hive> describe l2;
OK
_c0 double
```
- MySQL
```SQL
mysql> create table foo2 as select (cast(99 as decimal(19,6)) + '2') from test;
Query OK, 1 row affected (0.01 sec)
Records: 1 Duplicates: 0 Warnings: 0
mysql> describe foo2;
+-----------------------------------+--------+------+-----+---------+-------+
| Field | Type | Null | Key | Default | Extra |
+-----------------------------------+--------+------+-----+---------+-------+
| (cast(99 as decimal(19,6)) + '2') | double | NO | | 0 | |
+-----------------------------------+--------+------+-----+---------+-------+
```
## How was this patch tested?
Added a new test in SQLQuerySuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#13368 from dilipbiswal/spark-15557.
## What changes were proposed in this pull request?
This change resolves a number of build warnings that have accumulated, before 2.x. It does not address a large number of deprecation warnings, especially related to the Accumulator API. That will happen separately.
## How was this patch tested?
Jenkins
Author: Sean Owen <sowen@cloudera.com>
Closes#13377 from srowen/BuildWarnings.
## What changes were proposed in this pull request?
This patch removes the last two commands defined in the catalyst module: DescribeFunction and ShowFunctions. They were unnecessary since the parser could just generate DescribeFunctionCommand and ShowFunctionsCommand directly.
## How was this patch tested?
Created a new SparkSqlParserSuite.
Author: Reynold Xin <rxin@databricks.com>
Closes#13292 from rxin/SPARK-15436.
## What changes were proposed in this pull request?
This PR fixes 3 slow tests:
1. `ParquetQuerySuite.read/write wide table`: This is not a good unit test as it runs more than 5 minutes. This PR removes it and add a new regression test in `CodeGenerationSuite`, which is more "unit".
2. `ParquetQuerySuite.returning batch for wide table`: reduce the threshold and use smaller data size.
3. `DatasetSuite.SPARK-14554: Dataset.map may generate wrong java code for wide table`: Improve `CodeFormatter.format`(introduced at https://github.com/apache/spark/pull/12979) can dramatically speed this it up.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13273 from cloud-fan/test.
## What changes were proposed in this pull request?
in hive, `locate("aa", "aaa", 0)` would yield 0, `locate("aa", "aaa", 1)` would yield 1 and `locate("aa", "aaa", 2)` would yield 2, while in Spark, `locate("aa", "aaa", 0)` would yield 1, `locate("aa", "aaa", 1)` would yield 2 and `locate("aa", "aaa", 2)` would yield 0. This results from the different understanding of the third parameter in udf `locate`. It means the starting index and starts from 1, so when we use 0, the return would always be 0.
## How was this patch tested?
tested with modified `StringExpressionsSuite` and `StringFunctionsSuite`
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#13186 from adrian-wang/locate.
## What changes were proposed in this pull request?
When invalid date string like "2015-02-29 00:00:00" are cast as date or timestamp using spark sql, it used to not return null but another valid date (2015-03-01 in this case).
In this pr, invalid date string like "2016-02-29" and "2016-04-31" are returned as null when cast as date or timestamp.
## How was this patch tested?
Unit tests are added.
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: wangyang <wangyang@haizhi.com>
Closes#13169 from wangyang1992/invalid_date.
## What changes were proposed in this pull request?
Incrementalizing plans of with multiple streaming aggregation is tricky and we dont have the necessary support for "delta" to implement correctly. So disabling the support for multiple streaming aggregations.
## How was this patch tested?
Additional unit tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#13210 from tdas/SPARK-15428.
## What changes were proposed in this pull request?
This patch simplifies the implementation of Range operator and make the explain string consistent between logical plan and physical plan. To do this, I changed RangeExec to embed a Range logical plan in it.
Before this patch (note that the logical Range and physical Range actually output different information):
```
== Optimized Logical Plan ==
Range 0, 100, 2, 2, [id#8L]
== Physical Plan ==
*Range 0, 2, 2, 50, [id#8L]
```
After this patch:
If step size is 1:
```
== Optimized Logical Plan ==
Range(0, 100, splits=2)
== Physical Plan ==
*Range(0, 100, splits=2)
```
If step size is not 1:
```
== Optimized Logical Plan ==
Range (0, 100, step=2, splits=2)
== Physical Plan ==
*Range (0, 100, step=2, splits=2)
```
## How was this patch tested?
N/A
Author: Reynold Xin <rxin@databricks.com>
Closes#13239 from rxin/SPARK-15459.
#### What changes were proposed in this pull request?
When there are duplicate keys in the partition specs or table properties, we always use the last value and ignore all the previous values. This is caused by the function call `toMap`.
partition specs or table properties are widely used in multiple DDL statements.
This PR is to detect the duplicates and issue an exception if found.
#### How was this patch tested?
Added test cases in DDLSuite
Author: gatorsmile <gatorsmile@gmail.com>
Closes#13095 from gatorsmile/detectDuplicate.
## What changes were proposed in this pull request?
In only `catalyst` module, there exists 8 evaluation test cases on unresolved expressions. But, in real-world situation, those cases doesn't happen since they occurs exceptions before evaluations.
```scala
scala> sql("select format_number(null, 3)")
res0: org.apache.spark.sql.DataFrame = [format_number(CAST(NULL AS DOUBLE), 3): string]
scala> sql("select format_number(cast(null as NULL), 3)")
org.apache.spark.sql.catalyst.parser.ParseException:
DataType null() is not supported.(line 1, pos 34)
```
This PR makes those testcases more realistic.
```scala
- checkEvaluation(FormatNumber(Literal.create(null, NullType), Literal(3)), null)
+ assert(FormatNumber(Literal.create(null, NullType), Literal(3)).resolved === false)
```
Also, this PR also removes redundant `resolved` checking in `FoldablePropagation` optimizer.
## How was this patch tested?
Pass the modified Jenkins tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13241 from dongjoon-hyun/SPARK-15462.
## What changes were proposed in this pull request?
Right now inferring the schema for case classes happens before searching the SQLUserDefinedType annotation, so the SQLUserDefinedType annotation for case classes doesn't work.
This PR simply changes the inferring order to resolve it. I also reenabled the java.math.BigDecimal test and added two tests for `List`.
## How was this patch tested?
`encodeDecodeTest(UDTCaseClass(new java.net.URI("http://spark.apache.org/")), "udt with case class")`
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#12965 from zsxwing/SPARK-15190.
## What changes were proposed in this pull request?
This PR introduce place holder for comment in generated code and the purpose is same for #12939 but much safer.
Generated code to be compiled doesn't include actual comments but includes place holder instead.
Place holders in generated code will be replaced with actual comments only at the time of logging.
Also, this PR can resolve SPARK-15205.
## How was this patch tested?
Existing tests.
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Closes#12979 from sarutak/SPARK-15205.
## What changes were proposed in this pull request?
`CreateNamedStruct` and `CreateNamedStructUnsafe` should preserve metadata of value expressions if it is `NamedExpression` like `CreateStruct` or `CreateStructUnsafe` are doing.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#13193 from ueshin/issues/SPARK-15400.
## What changes were proposed in this pull request?
The following code:
```
val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
ds.filter(_._1 == "b").select(expr("_1").as[String]).foreach(println(_))
```
throws an Exception:
```
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: _1#420
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50)
at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88)
at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87)
...
Cause: java.lang.RuntimeException: Couldn't find _1#420 in [_1#416,_2#417]
at scala.sys.package$.error(package.scala:27)
at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:94)
at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:88)
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49)
at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88)
at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87)
...
```
This is because `EmbedSerializerInFilter` rule drops the `exprId`s of output of surrounded `SerializeFromObject`.
The analyzed and optimized plans of the above example are as follows:
```
== Analyzed Logical Plan ==
_1: string
Project [_1#420]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421]
+- Filter <function1>.apply
+- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2
+- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]
== Optimized Logical Plan ==
!Project [_1#420]
+- Filter <function1>.apply
+- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]
```
This PR fixes `EmbedSerializerInFilter` rule to keep `exprId`s of output of surrounded `SerializeFromObject`.
The plans after this patch are as follows:
```
== Analyzed Logical Plan ==
_1: string
Project [_1#420]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421]
+- Filter <function1>.apply
+- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2
+- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]
== Optimized Logical Plan ==
Project [_1#416]
+- Filter <function1>.apply
+- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]
```
## How was this patch tested?
Existing tests and I added a test to check if `filter and then select` works.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#13096 from ueshin/issues/SPARK-15313.
## What changes were proposed in this pull request?
This patch fixes a bug in TypeUtils.checkForSameTypeInputExpr. Previously the code was testing on strict equality, which does not taking nullability into account.
This is based on https://github.com/apache/spark/pull/12768. This patch fixed a bug there (with empty expression) and added a test case.
## How was this patch tested?
Added a new test suite and test case.
Closes#12768.
Author: Reynold Xin <rxin@databricks.com>
Author: Oleg Danilov <oleg.danilov@wandisco.com>
Closes#13208 from rxin/SPARK-14990.
Hello : Can you help check this PR? I am adding support for the java.math.BigInteger for java bean code path. I saw internally spark is converting the BigInteger to BigDecimal in ColumnType.scala and CatalystRowConverter.scala. I use the similar way and convert the BigInteger to the BigDecimal. .
Author: Kevin Yu <qyu@us.ibm.com>
Closes#10125 from kevinyu98/working_on_spark-11827.
## What changes were proposed in this pull request?
Fix `MapObjects.itemAccessorMethod` to handle `TimestampType`. Without this fix, `Array[Timestamp]` cannot be properly encoded or decoded. To reproduce this, in `ExpressionEncoderSuite`, if you add the following test case:
`encodeDecodeTest(Array(Timestamp.valueOf("2016-01-29 10:00:00")), "array of timestamp")
`
... you will see that (without this fix) it fails with the following output:
```
- encode/decode for array of timestamp: [Ljava.sql.Timestamp;fd9ebde *** FAILED ***
Exception thrown while decoding
Converted: [0,1000000010,800000001,52a7ccdc36800]
Schema: value#61615
root
-- value: array (nullable = true)
|-- element: timestamp (containsNull = true)
Encoder:
class[value[0]: array<timestamp>] (ExpressionEncoderSuite.scala:312)
```
## How was this patch tested?
Existing tests
Author: Sumedh Mungee <smungee@gmail.com>
Closes#13108 from smungee/fix-itemAccessorMethod.
## What changes were proposed in this pull request?
This PR aims to add new **FoldablePropagation** optimizer that propagates foldable expressions by replacing all attributes with the aliases of original foldable expression. Other optimizations will take advantage of the propagated foldable expressions: e.g. `EliminateSorts` optimizer now can handle the following Case 2 and 3. (Case 1 is the previous implementation.)
1. Literals and foldable expression, e.g. "ORDER BY 1.0, 'abc', Now()"
2. Foldable ordinals, e.g. "SELECT 1.0, 'abc', Now() ORDER BY 1, 2, 3"
3. Foldable aliases, e.g. "SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, z"
This PR has been generalized based on cloud-fan 's key ideas many times; he should be credited for the work he did.
**Before**
```
scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain
== Physical Plan ==
WholeStageCodegen
: +- Sort [1.0#5 ASC,x#0 ASC], true, 0
: +- INPUT
+- Exchange rangepartitioning(1.0#5 ASC, x#0 ASC, 200), None
+- WholeStageCodegen
: +- Project [1.0 AS 1.0#5,1461873043577000 AS x#0]
: +- INPUT
+- Scan OneRowRelation[]
```
**After**
```
scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain
== Physical Plan ==
WholeStageCodegen
: +- Project [1.0 AS 1.0#5,1461873079484000 AS x#0]
: +- INPUT
+- Scan OneRowRelation[]
```
## How was this patch tested?
Pass the Jenkins tests including a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12719 from dongjoon-hyun/SPARK-14939.
## What changes were proposed in this pull request?
toCommentSafeString method replaces "\u" with "\\\\u" to avoid codegen breaking.
But if the even number of "\" is put before "u", like "\\\\u", in the string literal in the query, codegen can break.
Following code causes compilation error.
```
val df = Seq(...).toDF
df.select("'\\\\\\\\u002A/'").show
```
The reason of the compilation error is because "\\\\\\\\\\\\\\\\u002A/" is translated into "*/" (the end of comment).
Due to this unsafety, arbitrary code can be injected like as follows.
```
val df = Seq(...).toDF
// Inject "System.exit(1)"
df.select("'\\\\\\\\u002A/{System.exit(1);}/*'").show
```
## How was this patch tested?
Added new test cases.
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Author: sarutak <sarutak@oss.nttdata.co.jp>
Closes#12939 from sarutak/SPARK-15165.
## What changes were proposed in this pull request?
This PR improves `RowEncoder` and `MapObjects`, to support array as the external type for `ArrayType`. The idea is straightforward, we use `Object` as the external input type for `ArrayType`, and determine its type at runtime in `MapObjects`.
## How was this patch tested?
new test in `RowEncoderSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13138 from cloud-fan/map-object.
## What changes were proposed in this pull request?
We originally designed the type coercion rules to match Hive, but over time we have diverged. It does not make sense to call it HiveTypeCoercion anymore. This patch renames it TypeCoercion.
## How was this patch tested?
Updated unit tests to reflect the rename.
Author: Reynold Xin <rxin@databricks.com>
Closes#13091 from rxin/SPARK-15310.
## What changes were proposed in this pull request?
This patch moves all the object related expressions into expressions.objects package, for better code organization.
## How was this patch tested?
N/A
Author: Reynold Xin <rxin@databricks.com>
Closes#13085 from rxin/SPARK-15306.
#### What changes were proposed in this pull request?
~~Currently, multiple partitions are allowed to drop by using a single DDL command: Alter Table Drop Partition. However, the internal implementation could break atomicity. That means, we could just drop a subset of qualified partitions, if hitting an exception when dropping one of qualified partitions~~
~~This PR contains the following behavior changes:~~
~~- disallow dropping multiple partitions by a single command ~~
~~- allow users to input predicates in partition specification and issue a nicer error message if the predicate's comparison operator is not `=`.~~
~~- verify the partition spec in SessionCatalog. This can ensure each partition spec in `Drop Partition` does not correspond to multiple partitions.~~
This PR has two major parts:
- Verify the partition spec in SessionCatalog for fixing the following issue:
```scala
sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')")
```
Above example uses an invalid partition spec. Without this PR, we will drop all the partitions. The reason is Hive megastores getPartitions API returns all the partitions if we provide an invalid spec.
- Re-implemented the `dropPartitions` in `HiveClientImpl`. Now, we always check if all the user-specified partition specs exist before attempting to drop the partitions. Previously, we start drop the partition before completing checking the existence of all the partition specs. If any failure happened after we start to drop the partitions, we will log an error message to indicate which partitions have been dropped and which partitions have not been dropped.
#### How was this patch tested?
Modified the existing test cases and added new test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#12801 from gatorsmile/banDropMultiPart.
## What changes were proposed in this pull request?
Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView.
## How was this patch tested?
Unit tests.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#12945 from clockfly/spark-15171.
## What changes were proposed in this pull request?
SPARK-15241: We now support java decimal and catalyst decimal in external row, it makes sense to also support scala decimal.
SPARK-15242: This is a long-standing bug, and is exposed after https://github.com/apache/spark/pull/12364, which eliminate the `If` expression if the field is not nullable:
```
val fieldValue = serializerFor(
GetExternalRowField(inputObject, i, externalDataTypeForInput(f.dataType)),
f.dataType)
if (f.nullable) {
If(
Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil),
Literal.create(null, f.dataType),
fieldValue)
} else {
fieldValue
}
```
Previously, we always use `DecimalType.SYSTEM_DEFAULT` as the output type of converted decimal field, which is wrong as it doesn't match the real decimal type. However, it works well because we always put converted field into `If` expression to do the null check, and `If` use its `trueValue`'s data type as its output type.
Now if we have a not nullable decimal field, then the converted field's output type will be `DecimalType.SYSTEM_DEFAULT`, and we will write wrong data into unsafe row.
The fix is simple, just use the given decimal type as the output type of converted decimal field.
These 2 issues was found at https://github.com/apache/spark/pull/13008
## How was this patch tested?
new tests in RowEncoderSuite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13019 from cloud-fan/encoder-decimal.
Since we cannot really trust if the underlying external catalog can throw exceptions when there is an invalid metadata operation, let's do it in SessionCatalog.
- [X] The first step is to unify the error messages issued in Hive-specific Session Catalog and general Session Catalog.
- [X] The second step is to verify the inputs of metadata operations for partitioning-related operations. This is moved to a separate PR: https://github.com/apache/spark/pull/12801
- [X] The third step is to add database existence verification in `SessionCatalog`
- [X] The fourth step is to add table existence verification in `SessionCatalog`
- [X] The fifth step is to add function existence verification in `SessionCatalog`
Add test cases and verify the error messages we issued
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#12385 from gatorsmile/verifySessionAPIs.
#### What changes were proposed in this pull request?
This PR is to address a few existing issues in `EXPLAIN`:
- The `EXPLAIN` options `LOGICAL | FORMATTED | EXTENDED | CODEGEN` should not be 0 or more match. It should 0 or one match. Parser does not allow users to use more than one option in a single command.
- The option `LOGICAL` is not supported. Issue an exception when users specify this option in the command.
- The output of `EXPLAIN ` contains a weird empty line when the output of analyzed plan is empty. We should remove it. For example:
```
== Parsed Logical Plan ==
CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io. HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false
== Analyzed Logical Plan ==
CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io. HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false
== Optimized Logical Plan ==
CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io. HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false
...
```
#### How was this patch tested?
Added and modified a few test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#12991 from gatorsmile/explainCreateTable.
## What changes were proposed in this pull request?
following operations have file system operation now:
1. CREATE DATABASE: create a dir
2. DROP DATABASE: delete the dir
3. CREATE TABLE: create a dir
4. DROP TABLE: delete the dir
5. RENAME TABLE: rename the dir
6. CREATE PARTITIONS: create a dir
7. RENAME PARTITIONS: rename the dir
8. DROP PARTITIONS: drop the dir
## How was this patch tested?
new tests in `ExternalCatalogSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#12871 from cloud-fan/catalog.
#### What changes were proposed in this pull request?
So far, in the implementation of InMemoryCatalog, we do not check if the new/destination table/function/partition exists or not. Thus, we just silently remove the existent table/function/partition.
This PR is to detect them and issue an appropriate exception.
#### How was this patch tested?
Added the related test cases. They also verify if HiveExternalCatalog also detects these errors.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#12960 from gatorsmile/renameInMemoryCatalog.
#### What changes were proposed in this pull request?
When Describe a UDTF, the command returns a wrong result. The command is unable to find the function, which has been created and cataloged in the catalog but not in the functionRegistry.
This PR is to correct it. If the function is not in the functionRegistry, we will check the catalog for collecting the information of the UDTF function.
#### How was this patch tested?
Added test cases to verify the results
Author: gatorsmile <gatorsmile@gmail.com>
Closes#12885 from gatorsmile/showFunction.
## What changes were proposed in this pull request?
The problem is: In `RowEncoder`, we use `Invoke` to get the field of an external row, which lose the nullability information. This PR creates a `GetExternalRowField` expression, so that we can preserve the nullability info.
TODO: simplify the null handling logic in `RowEncoder`, to remove so many if branches, in follow-up PR.
## How was this patch tested?
new tests in `RowEncoderSuite`
Note that, This PR takes over https://github.com/apache/spark/pull/11980, with a little simplification, so all credits should go to koertkuipers
Author: Wenchen Fan <wenchen@databricks.com>
Author: Koert Kuipers <koert@tresata.com>
Closes#12364 from cloud-fan/nullable.
## What changes were proposed in this pull request?
This PR improve the error message for `Generate` in 3 cases:
1. generator is nested in expressions, e.g. `SELECT explode(list) + 1 FROM tbl`
2. generator appears more than one time in SELECT, e.g. `SELECT explode(list), explode(list) FROM tbl`
3. generator appears in other operator which is not project, e.g. `SELECT * FROM tbl SORT BY explode(list)`
## How was this patch tested?
new tests in `AnalysisErrorSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#12810 from cloud-fan/bug.
## What changes were proposed in this pull request?
Just a bunch of small tweaks on DDL exception messages.
## How was this patch tested?
`DDLCommandSuite` et al.
Author: Andrew Or <andrew@databricks.com>
Closes#12853 from andrewor14/make-exceptions-consistent.
#### What changes were proposed in this pull request?
Compared with the current Spark parser, there are two extra syntax are supported in Hive for sampling
- In `On` clauses, `rand()` is used for indicating sampling on the entire row instead of an individual column. For example,
```SQL
SELECT * FROM source TABLESAMPLE(BUCKET 3 OUT OF 32 ON rand()) s;
```
- Users can specify the total length to be read. For example,
```SQL
SELECT * FROM source TABLESAMPLE(100M) s;
```
Below is the link for references:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Sampling
This PR is to parse and capture these two extra syntax, and issue a better error message.
#### How was this patch tested?
Added test cases to verify the thrown exceptions
Author: gatorsmile <gatorsmile@gmail.com>
Closes#12838 from gatorsmile/bucketOnRand.
## What changes were proposed in this pull request?
Make serializer correctly inferred if the input type is `List[_]`, since `List[_]` is type of `Seq[_]`, before it was matched to different case (`case t if definedByConstructorParams(t)`).
## How was this patch tested?
New test case was added.
Author: bomeng <bmeng@us.ibm.com>
Closes#12849 from bomeng/SPARK-15062.
## What changes were proposed in this pull request?
This PR addresses a few minor issues in SQL parser:
- Removes some unused rules and keywords in the grammar.
- Removes code path for fallback SQL parsing (was needed for Hive native parsing).
- Use `UnresolvedGenerator` instead of hard-coding `Explode` & `JsonTuple`.
- Adds a more generic way of creating error messages for unsupported Hive features.
- Use `visitFunctionName` as much as possible.
- Interpret a `CatalogColumn`'s `DataType` directly instead of parsing it again.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#12826 from hvanhovell/SPARK-15047.
## What changes were proposed in this pull request?
In this PR we add support for correlated scalar subqueries. An example of such a query is:
```SQL
select * from tbl1 a where a.value > (select max(value) from tbl2 b where b.key = a.key)
```
The implementation adds the `RewriteCorrelatedScalarSubquery` rule to the Optimizer. This rule plans these subqueries using `LEFT OUTER` joins. It currently supports rewrites for `Project`, `Aggregate` & `Filter` logical plans.
I could not find a well defined semantics for the use of scalar subqueries in an `Aggregate`. The current implementation currently evaluates the scalar subquery *before* aggregation. This means that you either have to make scalar subquery part of the grouping expression, or that you have to aggregate it further on. I am open to suggestions on this.
The implementation currently forces the uniqueness of a scalar subquery by enforcing that it is aggregated and that the resulting column is wrapped in an `AggregateExpression`.
## How was this patch tested?
Added tests to `SubquerySuite`.
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#12822 from hvanhovell/SPARK-14785.
## What changes were proposed in this pull request?
In order to support nested predicate subquery, this PR introduce an internal join type ExistenceJoin, which will emit all the rows from left, plus an additional column, which presents there are any rows matched from right or not (it's not null-aware right now). This additional column could be used to replace the subquery in Filter.
In theory, all the predicate subquery could use this join type, but it's slower than LeftSemi and LeftAnti, so it's only used for nested subquery (subquery inside OR).
For example, the following SQL:
```sql
SELECT a FROM t WHERE EXISTS (select 0) OR EXISTS (select 1)
```
This PR also fix a bug in predicate subquery push down through join (they should not).
Nested null-aware subquery is still not supported. For example, `a > 3 OR b NOT IN (select bb from t)`
After this, we could run TPCDS query Q10, Q35, Q45
## How was this patch tested?
Added unit tests.
Author: Davies Liu <davies@databricks.com>
Closes#12820 from davies/or_exists.
## What changes were proposed in this pull request?
This PR adds `fromPrimitiveArray` and `toPrimitiveArray` in `UnsafeArrayData`, so that we can do the conversion much faster in VectorUDT/MatrixUDT.
## How was this patch tested?
existing tests and new test suite `UnsafeArraySuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#12640 from cloud-fan/ml.
## What changes were proposed in this pull request?
CatalystSqlParser can parse data types. So, we do not need to have an individual DataTypeParser.
## How was this patch tested?
Existing tests
Author: Yin Huai <yhuai@databricks.com>
Closes#12796 from yhuai/removeDataTypeParser.
## What changes were proposed in this pull request?
This patch fixes a null handling bug in EqualNullSafe's code generation.
## How was this patch tested?
Updated unit test so they would fail without the fix.
Closes#12628.
Author: Reynold Xin <rxin@databricks.com>
Author: Arash Nabili <arash@levyx.com>
Closes#12799 from rxin/equalnullsafe.
The previous subquery PRs did not include support for pushing subqueries used in filters (`WHERE`/`HAVING`) down. This PR adds this support. For example :
```scala
range(0, 10).registerTempTable("a")
range(5, 15).registerTempTable("b")
range(7, 25).registerTempTable("c")
range(3, 12).registerTempTable("d")
val plan = sql("select * from a join b on a.id = b.id left join c on c.id = b.id where a.id in (select id from d)")
plan.explain(true)
```
Leads to the following Analyzed & Optimized plans:
```
== Parsed Logical Plan ==
...
== Analyzed Logical Plan ==
id: bigint, id: bigint, id: bigint
Project [id#0L,id#4L,id#8L]
+- Filter predicate-subquery#16 [(id#0L = id#12L)]
: +- SubqueryAlias predicate-subquery#16 [(id#0L = id#12L)]
: +- Project [id#12L]
: +- SubqueryAlias d
: +- Range 3, 12, 1, 8, [id#12L]
+- Join LeftOuter, Some((id#8L = id#4L))
:- Join Inner, Some((id#0L = id#4L))
: :- SubqueryAlias a
: : +- Range 0, 10, 1, 8, [id#0L]
: +- SubqueryAlias b
: +- Range 5, 15, 1, 8, [id#4L]
+- SubqueryAlias c
+- Range 7, 25, 1, 8, [id#8L]
== Optimized Logical Plan ==
Join LeftOuter, Some((id#8L = id#4L))
:- Join Inner, Some((id#0L = id#4L))
: :- Join LeftSemi, Some((id#0L = id#12L))
: : :- Range 0, 10, 1, 8, [id#0L]
: : +- Range 3, 12, 1, 8, [id#12L]
: +- Range 5, 15, 1, 8, [id#4L]
+- Range 7, 25, 1, 8, [id#8L]
== Physical Plan ==
...
```
I have also taken the opportunity to move quite a bit of code around:
- Rewriting subqueris and pulling out correlated predicated from subqueries has been moved into the analyzer. The analyzer transforms `Exists` and `InSubQuery` into `PredicateSubquery` expressions. A PredicateSubquery exposes the 'join' expressions and the proper references. This makes things like type coercion, optimization and planning easier to do.
- I have added support for `Aggregate` plans in subqueries. Any correlated expressions will be added to the grouping expressions. I have removed support for `Union` plans, since pulling in an outer reference from beneath a Union has no value (a filtered value could easily be part of another Union child).
- Resolution of subqueries is now done using `OuterReference`s. These are used to wrap any outer reference; this makes the identification of these references easier, and also makes dealing with duplicate attributes in the outer and inner plans easier. The resolution of subqueries initially used a resolution loop which would alternate between calling the analyzer and trying to resolve the outer references. We now use a dedicated analyzer which uses a special rule for outer reference resolution.
These changes are a stepping stone for enabling correlated scalar subqueries, enabling all Hive tests & allowing us to use predicate subqueries anywhere.
Current tests and added test cases in FilterPushdownSuite.
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#12720 from hvanhovell/SPARK-14858.
#### What changes were proposed in this pull request?
Replaces a logical `Except` operator with a `Left-anti Join` operator. This way, we can take advantage of all the benefits of join implementations (e.g. managed memory, code generation, broadcast joins).
```SQL
SELECT a1, a2 FROM Tab1 EXCEPT SELECT b1, b2 FROM Tab2
==> SELECT DISTINCT a1, a2 FROM Tab1 LEFT ANTI JOIN Tab2 ON a1<=>b1 AND a2<=>b2
```
Note:
1. This rule is only applicable to EXCEPT DISTINCT. Do not use it for EXCEPT ALL.
2. This rule has to be done after de-duplicating the attributes; otherwise, the enerated
join conditions will be incorrect.
This PR also corrects the existing behavior in Spark. Before this PR, the behavior is like
```SQL
test("except") {
val df_left = Seq(1, 2, 2, 3, 3, 4).toDF("id")
val df_right = Seq(1, 3).toDF("id")
checkAnswer(
df_left.except(df_right),
Row(2) :: Row(2) :: Row(4) :: Nil
)
}
```
After this PR, the result is corrected. We strictly follow the SQL compliance of `Except Distinct`.
#### How was this patch tested?
Modified and added a few test cases to verify the optimization rule and the results of operators.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#12736 from gatorsmile/exceptByAntiJoin.
## What changes were proposed in this pull request?
This PR aims to implement decimal aggregation optimization for window queries by improving existing `DecimalAggregates`. Historically, `DecimalAggregates` optimizer is designed to transform general `sum/avg(decimal)`, but it breaks recently added windows queries like the followings. The following queries work well without the current `DecimalAggregates` optimizer.
**Sum**
```scala
scala> sql("select sum(a) over () from (select explode(array(1.0,2.0)) a) t").head
java.lang.RuntimeException: Unsupported window function: MakeDecimal((sum(UnscaledValue(a#31)),mode=Complete,isDistinct=false),12,1)
scala> sql("select sum(a) over () from (select explode(array(1.0,2.0)) a) t").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [sum(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#23]
: +- INPUT
+- Window [MakeDecimal((sum(UnscaledValue(a#21)),mode=Complete,isDistinct=false),12,1) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS sum(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#23]
+- Exchange SinglePartition, None
+- Generate explode([1.0,2.0]), false, false, [a#21]
+- Scan OneRowRelation[]
```
**Average**
```scala
scala> sql("select avg(a) over () from (select explode(array(1.0,2.0)) a) t").head
java.lang.RuntimeException: Unsupported window function: cast(((avg(UnscaledValue(a#40)),mode=Complete,isDistinct=false) / 10.0) as decimal(6,5))
scala> sql("select avg(a) over () from (select explode(array(1.0,2.0)) a) t").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [avg(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#44]
: +- INPUT
+- Window [cast(((avg(UnscaledValue(a#42)),mode=Complete,isDistinct=false) / 10.0) as decimal(6,5)) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS avg(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#44]
+- Exchange SinglePartition, None
+- Generate explode([1.0,2.0]), false, false, [a#42]
+- Scan OneRowRelation[]
```
After this PR, those queries work fine and new optimized physical plans look like the followings.
**Sum**
```scala
scala> sql("select sum(a) over () from (select explode(array(1.0,2.0)) a) t").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [sum(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#35]
: +- INPUT
+- Window [MakeDecimal((sum(UnscaledValue(a#33)),mode=Complete,isDistinct=false) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING),12,1) AS sum(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#35]
+- Exchange SinglePartition, None
+- Generate explode([1.0,2.0]), false, false, [a#33]
+- Scan OneRowRelation[]
```
**Average**
```scala
scala> sql("select avg(a) over () from (select explode(array(1.0,2.0)) a) t").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [avg(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#47]
: +- INPUT
+- Window [cast(((avg(UnscaledValue(a#45)),mode=Complete,isDistinct=false) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) / 10.0) as decimal(6,5)) AS avg(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#47]
+- Exchange SinglePartition, None
+- Generate explode([1.0,2.0]), false, false, [a#45]
+- Scan OneRowRelation[]
```
In this PR, *SUM over window* pattern matching is based on the code of hvanhovell ; he should be credited for the work he did.
## How was this patch tested?
Pass the Jenkins tests (with newly added testcases)
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12421 from dongjoon-hyun/SPARK-14664.
## What changes were proposed in this pull request?
#12625 exposed a new user-facing conf interface in `SparkSession`. This patch adds a catalog interface.
## How was this patch tested?
See `CatalogSuite`.
Author: Andrew Or <andrew@databricks.com>
Closes#12713 from andrewor14/user-facing-catalog.
## What changes were proposed in this pull request?
This patch changes UnresolvedFunction and UnresolvedGenerator to use a FunctionIdentifier rather than just a String for function name. Also changed SessionCatalog to accept FunctionIdentifier in lookupFunction.
## How was this patch tested?
Updated related unit tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#12659 from rxin/SPARK-14888.
#### What changes were proposed in this pull request?
For performance, predicates can be pushed through Window if and only if the following conditions are satisfied:
1. All the expressions are part of window partitioning key. The expressions can be compound.
2. Deterministic
#### How was this patch tested?
TODO:
- [X] DSL needs to be modified for window
- [X] more tests will be added.
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#11635 from gatorsmile/pushPredicateThroughWindow.
!< means not less than which is equivalent to >=
!> means not greater than which is equivalent to <=
I'd to create a PR to support these two operators.
I've added new test cases in: DataFrameSuite, ExpressionParserSuite, JDBCSuite, PlanParserSuite, SQLQuerySuite
dilipbiswal viirya gatorsmile
Author: jliwork <jiali@us.ibm.com>
Closes#12316 from jliwork/SPARK-14548.
## What changes were proposed in this pull request?
Currently, `OptimizeIn` optimizer replaces `In` expression into `InSet` expression if the size of set is greater than a constant, 10.
This issue aims to make a configuration `spark.sql.optimizer.inSetConversionThreshold` for that.
After this PR, `OptimizerIn` is configurable.
```scala
scala> sql("select a in (1,2,3) from (select explode(array(1,2)) a) T").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [a#7 IN (1,2,3) AS (a IN (1, 2, 3))#8]
: +- INPUT
+- Generate explode([1,2]), false, false, [a#7]
+- Scan OneRowRelation[]
scala> sqlContext.setConf("spark.sql.optimizer.inSetConversionThreshold", "2")
scala> sql("select a in (1,2,3) from (select explode(array(1,2)) a) T").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [a#16 INSET (1,2,3) AS (a IN (1, 2, 3))#17]
: +- INPUT
+- Generate explode([1,2]), false, false, [a#16]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Pass the Jenkins tests (with a new testcase)
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12562 from dongjoon-hyun/SPARK-14796.
## What changes were proposed in this pull request?
Currently, a column could be resolved wrongly if there are columns from both outer table and subquery have the same name, we should only resolve the attributes that can't be resolved within subquery. They may have same exprId than other attributes in subquery, so we should create alias for them.
Also, the column in IN subquery could have same exprId, we should create alias for them.
## How was this patch tested?
Added regression tests. Manually tests TPCDS Q70 and Q95, work well after this patch.
Author: Davies Liu <davies@databricks.com>
Closes#12539 from davies/fix_subquery.
### What changes were proposed in this pull request?
TPCDS Q90 fails to parse because it uses a reserved keyword as an Identifier; `AT` was used as an alias for one of the subqueries. `AT` is not a reserved keyword and should have been registerd as a in the `nonReserved` rule.
In order to prevent this from happening again I have added tests for all keywords that are non-reserved in Hive. See the `nonReserved`, `sql11ReservedKeywordsUsedAsCastFunctionName` & `sql11ReservedKeywordsUsedAsIdentifier` rules in https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g.
### How was this patch tested?
Added tests to for all Hive non reserved keywords to `TableIdentifierParserSuite`.
cc davies
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#12537 from hvanhovell/SPARK-14762.
## What changes were proposed in this pull request?
Implement some `hashCode` and `equals` together in order to enable the scalastyle.
This is a first batch, I will continue to implement them but I wanted to know your thoughts.
Author: Joan <joan@goyeau.com>
Closes#12157 from joan38/SPARK-6429-HashCode-Equals.
## What changes were proposed in this pull request?
Code generation for complex type, `CreateArray`, `CreateMap`, `CreateStruct`, `CreateNamedStruct`, exceeds JVM size limit for large elements.
We should split generated code into multiple `apply` functions if the complex types have large elements, like `UnsafeProjection` or others for large expressions.
## How was this patch tested?
I added some tests to check if the generated codes for the expressions exceed or not.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#12559 from ueshin/issues/SPARK-14793.
## What changes were proposed in this pull request?
This patch moves analyze table parsing into SparkSqlAstBuilder and removes HiveSqlAstBuilder.
In order to avoid extensive refactoring, I created a common trait for CatalogRelation and MetastoreRelation, and match on that. In the future we should probably just consolidate the two into a single thing so we don't need this common trait.
## How was this patch tested?
Updated unit tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#12584 from rxin/SPARK-14821.
`MutableProjection` is not thread-safe and we won't use it in multiple threads. I think the reason that we return `() => MutableProjection` is not about thread safety, but to save the costs of generating code when we need same but individual mutable projections.
However, I only found one place that use this [feature](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala#L122-L123), and comparing to the troubles it brings, I think we should generate `MutableProjection` directly instead of return a function.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#7373 from cloud-fan/project.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-14600
This PR makes `Expand.output` have different attributes from the grouping attributes produced by the underlying `Project`, as they have different meaning, so that we can safely push down filter through `Expand`
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#12496 from cloud-fan/expand.
## What changes were proposed in this pull request?
Enable ScalaReflection and User Defined Types for plain Scala classes.
This involves the move of `schemaFor` from `ScalaReflection` trait (which is Runtime and Compile time (macros) reflection) to the `ScalaReflection` object (runtime reflection only) as I believe this code wouldn't work at compile time anyway as it manipulates `Class`'s that are not compiled yet.
## How was this patch tested?
Unit test
Author: Joan <joan@goyeau.com>
Closes#12149 from joan38/SPARK-13929-Scala-reflection.
### What changes were proposed in this pull request?
This PR adds support for in/exists predicate subqueries to Spark. Predicate sub-queries are used as a filtering condition in a query (this is the only supported use case). A predicate sub-query comes in two forms:
- `[NOT] EXISTS(subquery)`
- `[NOT] IN (subquery)`
This PR is (loosely) based on the work of davies (https://github.com/apache/spark/pull/10706) and chenghao-intel (https://github.com/apache/spark/pull/9055). They should be credited for the work they did.
### How was this patch tested?
Modified parsing unit tests.
Added tests to `org.apache.spark.sql.SQLQuerySuite`
cc rxin, davies & chenghao-intel
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#12306 from hvanhovell/SPARK-4226.
When `Await.result` throws an exception which originated from a different thread, the resulting stacktrace doesn't include the path leading to the `Await.result` call itself, making it difficult to identify the impact of these exceptions. For example, I've seen cases where broadcast cleaning errors propagate to the main thread and crash it but the resulting stacktrace doesn't include any of the main thread's code, making it difficult to pinpoint which exception crashed that thread.
This patch addresses this issue by explicitly catching, wrapping, and re-throwing exceptions that are thrown by `Await.result`.
I tested this manually using 16b31c8251, a patch which reproduces an issue where an RPC exception which occurs while unpersisting RDDs manages to crash the main thread without any useful stacktrace, and verified that informative, full stacktraces were generated after applying the fix in this PR.
/cc rxin nongli yhuai anabranch
Author: Josh Rosen <joshrosen@databricks.com>
Closes#12433 from JoshRosen/wrap-and-rethrow-await-exceptions.
## What changes were proposed in this pull request?
This PR tries to separate the serialization and deserialization logic from object operators, so that it's easier to eliminate unnecessary serializations in optimizer.
Typed aggregate related operators are special, they will deserialize the input row to multiple objects and it's difficult to simply use a deserializer operator to abstract it, so we still mix the deserialization logic there.
## How was this patch tested?
existing tests and new test in `EliminateSerializationSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#12260 from cloud-fan/encoder.
## What changes were proposed in this pull request?
We currently disable codegen for `CaseWhen` if the number of branches is greater than 20 (in CaseWhen.MAX_NUM_CASES_FOR_CODEGEN). It would be better if this value is a non-public config defined in SQLConf.
## How was this patch tested?
Pass the Jenkins tests (including a new testcase `Support spark.sql.codegen.maxCaseBranches option`)
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12353 from dongjoon-hyun/SPARK-14577.
## What changes were proposed in this pull request?
The `doGenCode` method currently takes in an `ExprCode`, mutates it and returns the java code to evaluate the given expression. It should instead just return a new `ExprCode` to avoid passing around mutable objects during code generation.
## How was this patch tested?
Existing Tests
Author: Sameer Agarwal <sameer@databricks.com>
Closes#12483 from sameeragarwal/new-exprcode-2.
## What changes were proposed in this pull request?
Per rxin's suggestions, this patch renames `s/gen/genCode` and `s/genCode/doGenCode` to better reflect the semantics of these 2 function calls.
## How was this patch tested?
N/A (refactoring only)
Author: Sameer Agarwal <sameer@databricks.com>
Closes#12475 from sameeragarwal/gencode.
## What changes were proposed in this pull request?
Currently, `HiveTypeCoercion.IfCoercion` removes all predicates whose return-type are null. However, some UDFs need evaluations because they are designed to throw exceptions. This PR fixes that to preserve the predicates. Also, `assert_true` is implemented as Spark SQL function.
**Before**
```
scala> sql("select if(assert_true(false),2,3)").head
res2: org.apache.spark.sql.Row = [3]
```
**After**
```
scala> sql("select if(assert_true(false),2,3)").head
... ASSERT_TRUE ...
```
**Hive**
```
hive> select if(assert_true(false),2,3);
OK
Failed with exception java.io.IOException:org.apache.hadoop.hive.ql.metadata.HiveException: ASSERT_TRUE(): assertion failed.
```
## How was this patch tested?
Pass the Jenkins tests (including a new testcase in `HivePlanTest`)
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12340 from dongjoon-hyun/SPARK-14580.
## What changes were proposed in this pull request?
There are many operations that are currently not supported in the streaming execution. For example:
- joining two streams
- unioning a stream and a batch source
- sorting
- window functions (not time windows)
- distinct aggregates
Furthermore, executing a query with a stream source as a batch query should also fail.
This patch add an additional step after analysis in the QueryExecution which will check that all the operations in the analyzed logical plan is supported or not.
## How was this patch tested?
unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#12246 from tdas/SPARK-14473.
## What changes were proposed in this pull request?
This PR aims to add `bound` function (aka Banker's round) by extending current `round` implementation. [Hive supports `bround` since 1.3.0.](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF)
**Hive (1.3 ~ 2.0)**
```
hive> select round(2.5), bround(2.5);
OK
3.0 2.0
```
**After this PR**
```scala
scala> sql("select round(2.5), bround(2.5)").head
res0: org.apache.spark.sql.Row = [3,2]
```
## How was this patch tested?
Pass the Jenkins tests (with extended tests).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12376 from dongjoon-hyun/SPARK-14614.
## What changes were proposed in this pull request?
We currently hard code the max number of optimizer/analyzer iterations to 100. This patch makes it configurable. While I'm at it, I also added the SessionCatalog to the optimizer, so we can use information there in optimization.
## How was this patch tested?
Updated unit tests to reflect the change.
Author: Reynold Xin <rxin@databricks.com>
Closes#12434 from rxin/SPARK-14677.
## What changes were proposed in this pull request?
This PR moves `CurrentDatabase` from sql/hive package to sql/catalyst. It also adds the function description, which looks like the following.
```
scala> sqlContext.sql("describe function extended current_database").collect.foreach(println)
[Function: current_database]
[Class: org.apache.spark.sql.execution.command.CurrentDatabase]
[Usage: current_database() - Returns the current database.]
[Extended Usage:
> SELECT current_database()]
```
## How was this patch tested?
Existing tests
Author: Yin Huai <yhuai@databricks.com>
Closes#12424 from yhuai/SPARK-14668.
## What changes were proposed in this pull request?
Current `LikeSimplification` handles the following four rules.
- 'a%' => expr.StartsWith("a")
- '%b' => expr.EndsWith("b")
- '%a%' => expr.Contains("a")
- 'a' => EqualTo("a")
This PR adds the following rule.
- 'a%b' => expr.Length() >= 2 && expr.StartsWith("a") && expr.EndsWith("b")
Here, 2 is statically calculated from "a".size + "b".size.
**Before**
```
scala> sql("select a from (select explode(array('abc','adc')) a) T where a like 'a%c'").explain()
== Physical Plan ==
WholeStageCodegen
: +- Filter a#5 LIKE a%c
: +- INPUT
+- Generate explode([abc,adc]), false, false, [a#5]
+- Scan OneRowRelation[]
```
**After**
```
scala> sql("select a from (select explode(array('abc','adc')) a) T where a like 'a%c'").explain()
== Physical Plan ==
WholeStageCodegen
: +- Filter ((length(a#5) >= 2) && (StartsWith(a#5, a) && EndsWith(a#5, c)))
: +- INPUT
+- Generate explode([abc,adc]), false, false, [a#5]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Pass the Jenkins tests (including new testcase).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12312 from dongjoon-hyun/SPARK-14545.
## What changes were proposed in this pull request?
Right now, filter push down only works with Project, Aggregate, Generate and Join, they can't be pushed through many other plans.
This PR added support for Union, Intersect, Except and all unary plans.
## How was this patch tested?
Added tests.
Author: Davies Liu <davies@databricks.com>
Closes#12342 from davies/filter_hint.
## What changes were proposed in this pull request?
This patch implements the `CREATE TABLE` command using the `SessionCatalog`. Previously we handled only `CTAS` and `CREATE TABLE ... USING`. This requires us to refactor `CatalogTable` to accept various fields (e.g. bucket and skew columns) and pass them to Hive.
WIP: Note that I haven't verified whether this actually works yet! But I believe it does.
## How was this patch tested?
Tests will come in a future commit.
Author: Andrew Or <andrew@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#12271 from andrewor14/create-table-ddl.
## What changes were proposed in this pull request?
Currently, Union only takes intersect of the constraints from it's children, all others are dropped, we should try to merge them together.
This PR try to merge the constraints that have the same reference but came from different children, for example: `a > 10` and `a < 100` could be merged as `a > 10 || a < 100`.
## How was this patch tested?
Added more cases in existing test.
Author: Davies Liu <davies@databricks.com>
Closes#12328 from davies/union_const.
## What changes were proposed in this pull request?
According to the [Spark Code Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide) and [Scala Style Guide](http://docs.scala-lang.org/style/control-structures.html#curlybraces), we had better enforce the following rule.
```
case: Always omit braces in case clauses.
```
This PR makes a new ScalaStyle rule, 'OmitBracesInCase', and enforces it to the code.
## How was this patch tested?
Pass the Jenkins tests (including Scala style checking)
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12280 from dongjoon-hyun/SPARK-14508.
## What changes were proposed in this pull request?
This implements a few alter table partition commands using the `SessionCatalog`. In particular:
```
ALTER TABLE ... ADD PARTITION ...
ALTER TABLE ... DROP PARTITION ...
ALTER TABLE ... RENAME PARTITION ... TO ...
```
The following operations are not supported, and an `AnalysisException` with a helpful error message will be thrown if the user tries to use them:
```
ALTER TABLE ... EXCHANGE PARTITION ...
ALTER TABLE ... ARCHIVE PARTITION ...
ALTER TABLE ... UNARCHIVE PARTITION ...
ALTER TABLE ... TOUCH ...
ALTER TABLE ... COMPACT ...
ALTER TABLE ... CONCATENATE
MSCK REPAIR TABLE ...
```
## How was this patch tested?
`DDLSuite`, `DDLCommandSuite` and `HiveDDLCommandSuite`
Author: Andrew Or <andrew@databricks.com>
Closes#12220 from andrewor14/alter-partition-ddl.
## What changes were proposed in this pull request?
We can simplifies binary comparisons with semantically-equal operands:
1. Replace '<=>' with 'true' literal.
2. Replace '=', '<=', and '>=' with 'true' literal if both operands are non-nullable.
3. Replace '<' and '>' with 'false' literal if both operands are non-nullable.
For example, the following example plan
```
scala> sql("SELECT * FROM (SELECT explode(array(1,2,3)) a) T WHERE a BETWEEN a AND a+7").explain()
...
: +- Filter ((a#59 >= a#59) && (a#59 <= (a#59 + 7)))
...
```
will be optimized into the following.
```
: +- Filter (a#47 <= (a#47 + 7))
```
## How was this patch tested?
Pass the Jenkins tests including new `BinaryComparisonSimplificationSuite`.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12267 from dongjoon-hyun/SPARK-14502.
## What changes were proposed in this pull request?
This PR fix resultResult() for Union.
## How was this patch tested?
Added regression test.
Author: Davies Liu <davies@databricks.com>
Closes#12295 from davies/fix_sameResult.
## What changes were proposed in this pull request?
For an external table's metadata (in Hive's representation), its table type needs to be EXTERNAL_TABLE. Also, there needs to be a field called EXTERNAL set in the table property with a value of TRUE (for a MANAGED_TABLE it will be FALSE) based on https://github.com/apache/hive/blob/release-1.2.1/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1095-L1105. HiveClientImpl's toHiveTable misses to set this table property.
## How was this patch tested?
Added a new test.
Author: Yin Huai <yhuai@databricks.com>
Closes#12275 from yhuai/SPARK-14506.
#### What changes were proposed in this pull request?
This PR is to provide a native support for DDL `DROP VIEW` and `DROP TABLE`. The PR includes native parsing and native analysis.
Based on the HIVE DDL document for [DROP_VIEW_WEB_LINK](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-
DropView
), `DROP VIEW` is defined as,
**Syntax:**
```SQL
DROP VIEW [IF EXISTS] [db_name.]view_name;
```
- to remove metadata for the specified view.
- illegal to use DROP TABLE on a view.
- illegal to use DROP VIEW on a table.
- this command only works in `HiveContext`. In `SQLContext`, we will get an exception.
This PR also handles `DROP TABLE`.
**Syntax:**
```SQL
DROP TABLE [IF EXISTS] table_name [PURGE];
```
- Previously, the `DROP TABLE` command only can drop Hive tables in `HiveContext`. Now, after this PR, this command also can drop temporary table, external table, external data source table in `SQLContext`.
- In `HiveContext`, we will not issue an exception if the to-be-dropped table does not exist and users did not specify `IF EXISTS`. Instead, we just log an error message. If `IF EXISTS` is specified, we will not issue any error message/exception.
- In `SQLContext`, we will issue an exception if the to-be-dropped table does not exist, unless `IF EXISTS` is specified.
- Data will not be deleted if the tables are `external`, unless table type is `managed_table`.
#### How was this patch tested?
For verifying command parsing, added test cases in `spark/sql/hive/HiveDDLCommandSuite.scala`
For verifying command analysis, added test cases in `spark/sql/hive/execution/HiveDDLSuite.scala`
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#12146 from gatorsmile/dropView.
## What changes were proposed in this pull request?
We implement typed filter by `MapPartitions`, which doesn't work well with whole stage codegen. This PR use `Filter` to implement typed filter and we can get the whole stage codegen support for free.
This PR also introduced `DeserializeToObject` and `SerializeFromObject`, to seperate serialization logic from object operator, so that it's eaiser to write optimization rules for adjacent object operators.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#12061 from cloud-fan/whole-stage-codegen.
## What changes were proposed in this pull request?
This is a followup to #12117 and addresses some of the TODOs introduced there. In particular, the resolution of database is now pushed into session catalog, which knows about the current database. Further, the logic for checking whether a function exists is pushed into the external catalog.
No change in functionality is expected.
## How was this patch tested?
`SessionCatalogSuite`, `DDLSuite`
Author: Andrew Or <andrew@databricks.com>
Closes#12198 from andrewor14/function-exists.
The current package name uses a dash, which is a little weird but seemed
to work. That is, until a new test tried to mock a class that references
one of those shaded types, and then things started failing.
Most changes are just noise to fix the logging configs.
For reference, SPARK-8815 also raised this issue, although at the time it
did not cause any issues in Spark, so it was not addressed.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#11941 from vanzin/SPARK-14134.
### What changes were proposed in this pull request?
This PR adds support for `LEFT ANTI JOIN` to Spark SQL. A `LEFT ANTI JOIN` is the exact opposite of a `LEFT SEMI JOIN` and can be used to identify rows in one dataset that are not in another dataset. Note that `nulls` on the left side of the join cannot match a row on the right hand side of the join; the result is that left anti join will always select a row with a `null` in one or more of its keys.
We currently add support for the following SQL join syntax:
SELECT *
FROM tbl1 A
LEFT ANTI JOIN tbl2 B
ON A.Id = B.Id
Or using a dataframe:
tbl1.as("a").join(tbl2.as("b"), $"a.id" === $"b.id", "left_anti)
This PR provides serves as the basis for implementing `NOT EXISTS` and `NOT IN (...)` correlated sub-queries. It would also serve as good basis for implementing an more efficient `EXCEPT` operator.
The PR has been (losely) based on PR's by both davies (https://github.com/apache/spark/pull/10706) and chenghao-intel (https://github.com/apache/spark/pull/10563); credit should be given where credit is due.
This PR adds supports for `LEFT ANTI JOIN` to `BroadcastHashJoin` (including codegeneration), `ShuffledHashJoin` and `BroadcastNestedLoopJoin`.
### How was this patch tested?
Added tests to `JoinSuite` and ported `ExistenceJoinSuite` from https://github.com/apache/spark/pull/10563.
cc davies chenghao-intel rxin
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#12214 from hvanhovell/SPARK-12610.
LIKE <pattern> is commonly used in SHOW TABLES / FUNCTIONS etc DDL. In the pattern, user can use `|` or `*` as wildcards.
1. Currently, we used `replaceAll()` to replace `*` with `.*`, but the replacement was scattered in several places; I have created an utility method and use it in all the places;
2. Consistency with Hive: the pattern is case insensitive in Hive and white spaces will be trimmed, but current pattern matching does not do that. For example, suppose we have tables (t1, t2, t3), `SHOW TABLES LIKE ' T* ' ` will list all the t-tables. Please use Hive to verify it.
3. Combined with `|`, the result will be sorted. For pattern like `' B*|a* '`, it will list the result in a-b order.
I've made some changes to the utility method to make sure we will get the same result as Hive does.
A new method was created in StringUtil and test cases were added.
andrewor14
Author: bomeng <bmeng@us.ibm.com>
Closes#12206 from bomeng/SPARK-14429.
## What changes were proposed in this pull request?
We have ParserUtils and ParseUtils which are both utility collections for use during the parsing process.
Those names and what they are used for is very similar so I think we can merge them.
Also, the original unescapeSQLString method may have a fault. When "\u0061" style character literals are passed to the method, it's not unescaped successfully.
This patch fix the bug.
## How was this patch tested?
Added a new test case.
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Closes#12199 from sarutak/merge-ParseUtils-and-ParserUtils.
## What changes were proposed in this pull request?
Current, SparkSQL `initCap` is using `toTitleCase` function. However, `UTF8String.toTitleCase` implementation changes only the first letter and just copy the other letters: e.g. sParK --> SParK. This is the correct implementation `toTitleCase`.
```
hive> select initcap('sParK');
Spark
```
```
scala> sql("select initcap('sParK')").head
res0: org.apache.spark.sql.Row = [SParK]
```
This PR updates the implementation of `initcap` using `toLowerCase` and `toTitleCase`.
## How was this patch tested?
Pass the Jenkins tests (including new testcase).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12175 from dongjoon-hyun/SPARK-14402.
## What changes were proposed in this pull request?
The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.
With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
- `window(timeColumn, windowDuration)`
- `window(timeColumn, windowDuration, slideDuration)`
- `window(timeColumn, windowDuration, slideDuration, startTime)`
In Python, users can access all APIs above, but in addition they can do
- In Python:
`window(timeColumn, windowDuration, startTime=...)`
that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.
## How was this patch tested?
Unit tests + manual tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#12136 from brkyvz/python-windows.
## What changes were proposed in this pull request?
This PR implements CreateFunction and DropFunction commands. Besides implementing these two commands, we also change how to manage functions. Here are the main changes.
* `FunctionRegistry` will be a container to store all functions builders and it will not actively load any functions. Because of this change, we do not need to maintain a separate registry for HiveContext. So, `HiveFunctionRegistry` is deleted.
* SessionCatalog takes care the job of loading a function if this function is not in the `FunctionRegistry` but its metadata is stored in the external catalog. For this case, SessionCatalog will (1) load the metadata from the external catalog, (2) load all needed resources (i.e. jars and files), (3) create a function builder based on the function definition, (4) register the function builder in the `FunctionRegistry`.
* A `UnresolvedGenerator` is created. So, the parser will not need to call `FunctionRegistry` directly during parsing, which is not a good time to create a Hive UDTF. In the analysis phase, we will resolve `UnresolvedGenerator`.
This PR is based on viirya's https://github.com/apache/spark/pull/12036/
## How was this patch tested?
Existing tests and new tests.
## TODOs
[x] Self-review
[x] Cleanup
[x] More tests for create/drop functions (we need to more tests for permanent functions).
[ ] File JIRAs for all TODOs
[x] Standardize the error message when a function does not exist.
Author: Yin Huai <yhuai@databricks.com>
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#12117 from yhuai/function.
#### What changes were proposed in this pull request?
Currently, the weird error messages are issued if we use Hive Context-only operations in SQL Context.
For example,
- When calling `Drop Table` in SQL Context, we got the following message:
```
Expected exception org.apache.spark.sql.catalyst.parser.ParseException to be thrown, but java.lang.ClassCastException was thrown.
```
- When calling `Script Transform` in SQL Context, we got the message:
```
assertion failed: No plan for ScriptTransformation [key#9,value#10], cat, [tKey#155,tValue#156], null
+- LogicalRDD [key#9,value#10], MapPartitionsRDD[3] at beforeAll at BeforeAndAfterAll.scala:187
```
Updates:
Based on the investigation from hvanhovell , the root cause is `visitChildren`, which is the default implementation. It always returns the result of the last defined context child. After merging the code changes from hvanhovell , it works! Thank you hvanhovell !
#### How was this patch tested?
A few test cases are added.
Not sure if the same issue exist for the other operators/DDL/DML. hvanhovell
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Herman van Hovell <hvanhovell@questtec.nl>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#12134 from gatorsmile/hiveParserCommand.
## What changes were proposed in this pull request?
This PR contains the following 5 types of maintenance fix over 59 files (+94 lines, -93 lines).
- Fix typos(exception/log strings, testcase name, comments) in 44 lines.
- Fix lint-java errors (MaxLineLength) in 6 lines. (New codes after SPARK-14011)
- Use diamond operators in 40 lines. (New codes after SPARK-13702)
- Fix redundant semicolon in 5 lines.
- Rename class `InferSchemaSuite` to `CSVInferSchemaSuite` in CSVInferSchemaSuite.scala.
## How was this patch tested?
Manual and pass the Jenkins tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12139 from dongjoon-hyun/SPARK-14355.
## What changes were proposed in this pull request?
This PR aims to fix all Scala-Style multiline comments into Java-Style multiline comments in Scala codes.
(All comment-only changes over 77 files: +786 lines, −747 lines)
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12130 from dongjoon-hyun/use_multiine_javadoc_comments.
## What changes were proposed in this pull request?
Currently, `SimplifyConditionals` handles `true` and `false` to optimize branches. This PR improves `SimplifyConditionals` to take advantage of `null` conditions for `if` and `CaseWhen` expressions, too.
**Before**
```
scala> sql("SELECT IF(null, 1, 0)").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [if (null) 1 else 0 AS (IF(CAST(NULL AS BOOLEAN), 1, 0))#4]
: +- INPUT
+- Scan OneRowRelation[]
scala> sql("select case when cast(null as boolean) then 1 else 2 end").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [CASE WHEN null THEN 1 ELSE 2 END AS CASE WHEN CAST(NULL AS BOOLEAN) THEN 1 ELSE 2 END#14]
: +- INPUT
+- Scan OneRowRelation[]
```
**After**
```
scala> sql("SELECT IF(null, 1, 0)").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [0 AS (IF(CAST(NULL AS BOOLEAN), 1, 0))#4]
: +- INPUT
+- Scan OneRowRelation[]
scala> sql("select case when cast(null as boolean) then 1 else 2 end").explain()
== Physical Plan ==
WholeStageCodegen
: +- Project [2 AS CASE WHEN CAST(NULL AS BOOLEAN) THEN 1 ELSE 2 END#4]
: +- INPUT
+- Scan OneRowRelation[]
```
**Hive**
```
hive> select if(null,1,2);
OK
2
hive> select case when cast(null as boolean) then 1 else 2 end;
OK
2
```
## How was this patch tested?
Pass the Jenkins tests (including new extended test cases).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12122 from dongjoon-hyun/SPARK-14338.
This PR adds the ability to perform aggregations inside of a `ContinuousQuery`. In order to implement this feature, the planning of aggregation has augmented with a new `StatefulAggregationStrategy`. Unlike batch aggregation, stateful-aggregation uses the `StateStore` (introduced in #11645) to persist the results of partial aggregation across different invocations. The resulting physical plan performs the aggregation using the following progression:
- Partial Aggregation
- Shuffle
- Partial Merge (now there is at most 1 tuple per group)
- StateStoreRestore (now there is 1 tuple from this batch + optionally one from the previous)
- Partial Merge (now there is at most 1 tuple per group)
- StateStoreSave (saves the tuple for the next batch)
- Complete (output the current result of the aggregation)
The following refactoring was also performed to allow us to plug into existing code:
- The get/put implementation is taken from #12013
- The logic for breaking down and de-duping the physical execution of aggregation has been move into a new pattern `PhysicalAggregation`
- The `AttributeReference` used to identify the result of an `AggregateFunction` as been moved into the `AggregateExpression` container. This change moves the reference into the same object as the other intermediate references used in aggregation and eliminates the need to pass around a `Map[(AggregateFunction, Boolean), Attribute]`. Further clean up (using a different aggregation container for logical/physical plans) is deferred to a followup.
- Some planning logic is moved from the `SessionState` into the `QueryExecution` to make it easier to override in the streaming case.
- The ability to write a `StreamTest` that checks only the output of the last batch has been added to simulate the future addition of output modes.
Author: Michael Armbrust <michael@databricks.com>
Closes#12048 from marmbrus/statefulAgg.
## What changes were proposed in this pull request?
This PR adds the function `window` as a column expression.
`window` can be used to bucket rows into time windows given a time column. With this expression, performing time series analysis on batch data, as well as streaming data should become much more simpler.
### Usage
Assume the following schema:
`sensor_id, measurement, timestamp`
To average 5 minute data every 1 minute (window length of 5 minutes, slide duration of 1 minute), we will use:
```scala
df.groupBy(window("timestamp", “5 minutes”, “1 minute”), "sensor_id")
.agg(mean("measurement").as("avg_meas"))
```
This will generate windows such as:
```
09:00:00-09:05:00
09:01:00-09:06:00
09:02:00-09:07:00 ...
```
Intervals will start at every `slideDuration` starting at the unix epoch (1970-01-01 00:00:00 UTC).
To start intervals at a different point of time, e.g. 30 seconds after a minute, the `startTime` parameter can be used.
```scala
df.groupBy(window("timestamp", “5 minutes”, “1 minute”, "30 second"), "sensor_id")
.agg(mean("measurement").as("avg_meas"))
```
This will generate windows such as:
```
09:00:30-09:05:30
09:01:30-09:06:30
09:02:30-09:07:30 ...
```
Support for Python will be made in a follow up PR after this.
## How was this patch tested?
This patch has some basic unit tests for the `TimeWindow` expression testing that the parameters pass validation, and it also has some unit/integration tests testing the correctness of the windowing and usability in complex operations (multi-column grouping, multi-column projections, joins).
Author: Burak Yavuz <brkyvz@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes#12008 from brkyvz/df-time-window.
`Expand` operator now uses its child plan's constraints as its valid constraints (i.e., the base of constraints). This is not correct because `Expand` will set its group by attributes to null values. So the nullability of these attributes should be true.
E.g., for an `Expand` operator like:
val input = LocalRelation('a.int, 'b.int, 'c.int).where('c.attr > 10 && 'a.attr < 5 && 'b.attr > 2)
Expand(
Seq(
Seq('c, Literal.create(null, StringType), 1),
Seq('c, 'a, 2)),
Seq('c, 'a, 'gid.int),
Project(Seq('a, 'c), input))
The `Project` operator has the constraints `IsNotNull('a)`, `IsNotNull('b)` and `IsNotNull('c)`. But the `Expand` should not have `IsNotNull('a)` in its constraints.
This PR is the first step for this issue and remove invalid constraints of `Expand` operator.
A test is added to `ConstraintPropagationSuite`.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Author: Michael Armbrust <michael@databricks.com>
Closes#11995 from viirya/fix-expand-constraints.
## What changes were proposed in this pull request?
JIRA: https://issues.apache.org/jira/browse/SPARK-13995
We infer relative `IsNotNull` constraints from logical plan's expressions in `constructIsNotNullConstraints` now. However, we don't consider the case of (nested) `Cast`.
For example:
val tr = LocalRelation('a.int, 'b.long)
val plan = tr.where('a.attr === 'b.attr).analyze
Then, the plan's constraints will have `IsNotNull(Cast(resolveColumn(tr, "a"), LongType))`, instead of `IsNotNull(resolveColumn(tr, "a"))`. This PR fixes it.
Besides, as `IsNotNull` constraints are most useful for `Attribute`, we should do recursing through any `Expression` that is null intolerant and construct `IsNotNull` constraints for all `Attribute`s under these Expressions.
For example, consider the following constraints:
val df = Seq((1,2,3)).toDF("a", "b", "c")
df.where("a + b = c").queryExecution.analyzed.constraints
The inferred isnotnull constraints should be isnotnull(a), isnotnull(b), isnotnull(c), instead of isnotnull(a + c) and isnotnull(c).
## How was this patch tested?
Test is added into `ConstraintPropagationSuite`.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#11809 from viirya/constraint-cast.
### What changes were proposed in this pull request?
This PR removes the ANTLR3 based parser, and moves the new ANTLR4 based parser into the `org.apache.spark.sql.catalyst.parser package`.
### How was this patch tested?
Existing unit tests.
cc rxin andrewor14 yhuai
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#12071 from hvanhovell/SPARK-14211.
## What changes were proposed in this pull request?
In `ExpressionEncoder`, we use `constructorFor` to build `fromRowExpression` as the `deserializer` in `ObjectOperator`. It's kind of confusing, we should make the name consistent.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#12058 from cloud-fan/rename.
## What changes were proposed in this pull request?
Builds on https://github.com/apache/spark/pull/12022 and (a) appends "..." to truncated comment strings and (b) fixes indentation in lines after the commented strings if they happen to have a `(`, `{`, `)` or `}`
## How was this patch tested?
Manually examined the generated code.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#12044 from sameeragarwal/comment.
## What changes were proposed in this pull request?
Session catalog was added in #11750. However, it doesn't really support temporary functions properly; right now we only store the metadata in the form of `CatalogFunction`, but this doesn't make sense for temporary functions because there is no class name.
This patch moves the `FunctionRegistry` into the `SessionCatalog`. With this, the user can call `catalog.createTempFunction` and `catalog.lookupFunction` to use the function they registered previously. This is currently still dead code, however.
## How was this patch tested?
`SessionCatalogSuite`.
Author: Andrew Or <andrew@databricks.com>
Closes#11972 from andrewor14/temp-functions.
## What changes were proposed in this pull request?
This patch addresses the remaining comments left in #11750 and #11918 after they are merged. For a full list of changes in this patch, just trace the commits.
## How was this patch tested?
`SessionCatalogSuite` and `CatalogTestCases`
Author: Andrew Or <andrew@databricks.com>
Closes#12006 from andrewor14/session-catalog-followup.
### What changes were proposed in this pull request?
The current ANTLR3 parser is quite complex to maintain and suffers from code blow-ups. This PR introduces a new parser that is based on ANTLR4.
This parser is based on the [Presto's SQL parser](https://github.com/facebook/presto/blob/master/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4). The current implementation can parse and create Catalyst and SQL plans. Large parts of the HiveQl DDL and some of the DML functionality is currently missing, the plan is to add this in follow-up PRs.
This PR is a work in progress, and work needs to be done in the following area's:
- [x] Error handling should be improved.
- [x] Documentation should be improved.
- [x] Multi-Insert needs to be tested.
- [ ] Naming and package locations.
### How was this patch tested?
Catalyst and SQL unit tests.
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#11557 from hvanhovell/ngParser.
## What changes were proposed in this pull request?
This PR adds support for automatically inferring `IsNotNull` constraints from any non-nullable attributes that are part of an operator's output. This also fixes the issue that causes the optimizer to hit the maximum number of iterations for certain queries in https://github.com/apache/spark/pull/11828.
## How was this patch tested?
Unit test in `ConstraintPropagationSuite`
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11953 from sameeragarwal/infer-isnotnull.
## What changes were proposed in this pull request?
JIRA: https://issues.apache.org/jira/browse/SPARK-12443
`constructorFor` will call `dataTypeFor` to determine if a type is `ObjectType` or not. If there is not case for `Decimal`, it will be recognized as `ObjectType` and causes the bug.
## How was this patch tested?
Test is added into `ExpressionEncoderSuite`.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#10399 from viirya/fix-encoder-decimal.
## What changes were proposed in this pull request?
As we have `CreateArray` and `CreateStruct`, we should also have `CreateMap`. This PR adds the `CreateMap` expression, and the DataFrame API, and python API.
## How was this patch tested?
various new tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11879 from cloud-fan/create_map.
## What changes were proposed in this pull request?
This PR fix the conflict between ColumnPruning and PushPredicatesThroughProject, because ColumnPruning will try to insert a Project before Filter, but PushPredicatesThroughProject will move the Filter before Project.This is fixed by remove the Project before Filter, if the Project only do column pruning.
The RuleExecutor will fail the test if reached max iterations.
Closes#11745
## How was this patch tested?
Existing tests.
This is a test case still failing, disabled for now, will be fixed by https://issues.apache.org/jira/browse/SPARK-14137
Author: Davies Liu <davies@databricks.com>
Closes#11828 from davies/fail_rule.
## What changes were proposed in this pull request?
This reopens#11836, which was merged but promptly reverted because it introduced flaky Hive tests.
## How was this patch tested?
See `CatalogTestCases`, `SessionCatalogSuite` and `HiveContextSuite`.
Author: Andrew Or <andrew@databricks.com>
Closes#11938 from andrewor14/session-catalog-again.
## What changes were proposed in this pull request?
unionAll has been deprecated in SPARK-14088.
## How was this patch tested?
Should be covered by all existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#11946 from rxin/SPARK-14142.
## 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 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?
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?
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?
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.
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 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?
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 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?
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?
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?
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?
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?
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?
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?
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?
`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?
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?
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.
## 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?
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?
`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 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?
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.
#### What changes were proposed in this pull request?
Non-deterministic predicates should not be pushed through Generate.
#### How was this patch tested?
Added a test case in `FilterPushdownSuite.scala`
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11562 from gatorsmile/pushPredicateDownWindow.
## What changes were proposed in this pull request?
This PR adds an optimizer rule to eliminate reading (unnecessary) NULL values if they are not required for correctness by inserting `isNotNull` filters is the query plan. These filters are currently inserted beneath existing `Filter` and `Join` operators and are inferred based on their data constraints.
Note: While this optimization is applicable to all types of join, it primarily benefits `Inner` and `LeftSemi` joins.
## How was this patch tested?
1. Added a new `NullFilteringSuite` that tests for `IsNotNull` filters in the query plan for joins and filters. Also, tests interaction with the `CombineFilters` optimizer rules.
2. Test generated ExpressionTrees via `OrcFilterSuite`
3. Test filter source pushdown logic via `SimpleTextHadoopFsRelationSuite`
cc yhuai nongli
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11372 from sameeragarwal/gen-isnotnull.
## What changes were proposed in this pull request?
Today we have `analysis.Catalog` and `catalog.Catalog`. In the future the former will call the latter. When that happens, if both of them are still called `Catalog` it will be very confusing. This patch renames the latter `ExternalCatalog` because it is expected to talk to external systems.
## How was this patch tested?
Jenkins.
Author: Andrew Or <andrew@databricks.com>
Closes#11526 from andrewor14/rename-catalog.
## What changes were proposed in this pull request?
This patch simply moves things to existing package `o.a.s.sql.catalyst.parser` in an effort to reduce the size of the diff in #11048. This is conceptually the same as a recently merged patch #11482.
## How was this patch tested?
Jenkins.
Author: Andrew Or <andrew@databricks.com>
Closes#11506 from andrewor14/parser-package.
## What changes were proposed in this pull request?
After SPARK-6990, `dev/lint-java` keeps Java code healthy and helps PR review by saving much time.
This issue aims remove unused imports from Java/Scala code and add `UnusedImports` checkstyle rule to help developers.
## How was this patch tested?
```
./dev/lint-java
./build/sbt compile
```
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11438 from dongjoon-hyun/SPARK-13583.
## What changes were proposed in this pull request?
Make some cross-cutting code improvements according to static analysis. These are individually up for discussion since they exist in separate commits that can be reverted. The changes are broadly:
- Inner class should be static
- Mismatched hashCode/equals
- Overflow in compareTo
- Unchecked warnings
- Misuse of assert, vs junit.assert
- get(a) + getOrElse(b) -> getOrElse(a,b)
- Array/String .size -> .length (occasionally, -> .isEmpty / .nonEmpty) to avoid implicit conversions
- Dead code
- tailrec
- exists(_ == ) -> contains find + nonEmpty -> exists filter + size -> count
- reduce(_+_) -> sum map + flatten -> map
The most controversial may be .size -> .length simply because of its size. It is intended to avoid implicits that might be expensive in some places.
## How was the this patch tested?
Existing Jenkins unit tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#11292 from srowen/SPARK-13423.
JIRA: https://issues.apache.org/jira/browse/SPARK-13466
## What changes were proposed in this pull request?
With column pruning rule in optimizer, some Project operators will become redundant. We should remove these redundant Projects.
For an example query:
val input = LocalRelation('key.int, 'value.string)
val query =
Project(Seq($"x.key", $"y.key"),
Join(
SubqueryAlias("x", input),
BroadcastHint(SubqueryAlias("y", input)), Inner, None))
After the first run of column pruning, it would like:
Project(Seq($"x.key", $"y.key"),
Join(
Project(Seq($"x.key"), SubqueryAlias("x", input)),
Project(Seq($"y.key"), <-- inserted by the rule
BroadcastHint(SubqueryAlias("y", input))),
Inner, None))
Actually we don't need the outside Project now. This patch will remove it:
Join(
Project(Seq($"x.key"), SubqueryAlias("x", input)),
Project(Seq($"y.key"),
BroadcastHint(SubqueryAlias("y", input))),
Inner, None)
## How was the this patch tested?
Unit test is added into ColumnPruningSuite.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#11341 from viirya/remove-redundant-project.
#### What changes were proposed in this pull request?
This PR is to prune unnecessary columns when the operator is `MapPartitions`. The solution is to add an extra `Project` in the child node.
For the other two operators `AppendColumns` and `MapGroups`, it sounds doable. More discussions are required. The major reason is the current implementation of the `inputPlan` of `groupBy` is based on the child of `AppendColumns`. It might be a bug? Thus, will submit a separate PR.
#### How was this patch tested?
Added a test case in ColumnPruningSuite to verify the rule. Added another test case in DatasetSuite.scala to verify the data.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11460 from gatorsmile/datasetPruningNew.
#### What changes were proposed in this pull request?
After analysis by Analyzer, two operators could have alias. They are `Project` and `Aggregate`. So far, we only rewrite and propagate constraints if `Alias` is defined in `Project`. This PR is to resolve this issue in `Aggregate`.
#### How was this patch tested?
Added a test case for `Aggregate` in `ConstraintPropagationSuite`.
marmbrus sameeragarwal
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11422 from gatorsmile/validConstraintsInUnaryNodes.
## What changes were proposed in this pull request?
Predicates shouldn't be pushed through project with nondeterministic field(s).
See https://github.com/graphframes/graphframes/pull/23 and SPARK-13473 for more details.
This PR targets master, branch-1.6, and branch-1.5.
## How was this patch tested?
A test case is added in `FilterPushdownSuite`. It constructs a query plan where a filter is over a project with a nondeterministic field. Optimized query plan shouldn't change in this case.
Author: Cheng Lian <lian@databricks.com>
Closes#11348 from liancheng/spark-13473-no-ppd-through-nondeterministic-project-field.
## What changes were proposed in this pull request?
This PR mostly rewrite the ColumnPruning rule to support most of the SQL logical plans (except those for Dataset).
This PR also fix a bug in Generate, it should always output UnsafeRow, added an regression test for that.
## How was this patch tested?
This is test by unit tests, also manually test with TPCDS Q78, which could prune all unused columns successfully, improved the performance by 78% (from 22s to 12s).
Author: Davies Liu <davies@databricks.com>
Closes#11354 from davies/fix_column_pruning.
This PR adds a new abstraction called an `ExpressionSet` which attempts to canonicalize expressions to remove cosmetic differences. Deterministic expressions that are in the set after canonicalization will always return the same answer given the same input (i.e. false positives should not be possible). However, it is possible that two canonical expressions that are not equal will in fact return the same answer given any input (i.e. false negatives are possible).
```scala
val set = AttributeSet('a + 1 :: 1 + 'a :: Nil)
set.iterator => Iterator('a + 1)
set.contains('a + 1) => true
set.contains(1 + 'a) => true
set.contains('a + 2) => false
```
Other relevant changes include:
- Since this concept overlaps with the existing `semanticEquals` and `semanticHash`, those functions are also ported to this new infrastructure.
- A memoized `canonicalized` version of the expression is added as a `lazy val` to `Expression` and is used by both `semanticEquals` and `ExpressionSet`.
- A set of unit tests for `ExpressionSet` are added
- Tests which expect `semanticEquals` to be less intelligent than it now is are updated.
As a followup, we should consider auditing the places where we do `O(n)` `semanticEquals` operations and replace them with `ExpressionSet`. We should also consider consolidating `AttributeSet` as a specialized factory for an `ExpressionSet.`
Author: Michael Armbrust <michael@databricks.com>
Closes#11338 from marmbrus/expressionSet.
## What changes were proposed in this pull request?
Reverting SPARK-13376 (d563c8fa01) affects the test added by SPARK-13383. So, I am fixing the test.
Author: Yin Huai <yhuai@databricks.com>
Closes#11355 from yhuai/SPARK-13383-fix-test.
JIRA: https://issues.apache.org/jira/browse/SPARK-13383
## What changes were proposed in this pull request?
When we do column pruning in Optimizer, we put additional Project on top of a logical plan. However, when we already wrap a BroadcastHint on a logical plan, the added Project will hide BroadcastHint after later execution.
We should take care of BroadcastHint when we do column pruning.
## How was the this patch tested?
Unit test is added.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#11260 from viirya/keep-broadcasthint.
## What changes were proposed in this pull request?
This PR mostly rewrite the ColumnPruning rule to support most of the SQL logical plans (except those for Dataset).
## How was the this patch tested?
This is test by unit tests, also manually test with TPCDS Q78, which could prune all unused columns successfully, improved the performance by 78% (from 22s to 12s).
Author: Davies Liu <davies@databricks.com>
Closes#11256 from davies/fix_column_pruning.
The current implementation of statistics of UnaryNode does not considering output (for example, Project may product much less columns than it's child), we should considering it to have a better guess.
We usually only join with few columns from a parquet table, the size of projected plan could be much smaller than the original parquet files. Having a better guess of size help we choose between broadcast join or sort merge join.
After this PR, I saw a few queries choose broadcast join other than sort merge join without turning spark.sql.autoBroadcastJoinThreshold for every query, ended up with about 6-8X improvements on end-to-end time.
We use `defaultSize` of DataType to estimate the size of a column, currently For DecimalType/StringType/BinaryType and UDT, we are over-estimate too much (4096 Bytes), so this PR change them to some more reasonable values. Here are the new defaultSize for them:
DecimalType: 8 or 16 bytes, based on the precision
StringType: 20 bytes
BinaryType: 100 bytes
UDF: default size of SQL type
These numbers are not perfect (hard to have a perfect number for them), but should be better than 4096.
Author: Davies Liu <davies@databricks.com>
Closes#11210 from davies/statics.
The type checking functions of `If` and `UnwrapOption` are fixed to eliminate spurious failures. `UnwrapOption` was checking for an input of `ObjectType` but `ObjectType`'s accept function was hard coded to return `false`. `If`'s type check was returning a false negative in the case that the two options differed only by nullability.
Tests added:
- an end-to-end regression test is added to `DatasetSuite` for the reported failure.
- all the unit tests in `ExpressionEncoderSuite` are augmented to also confirm successful analysis. These tests are actually what pointed out the additional issues with `If` resolution.
Author: Michael Armbrust <michael@databricks.com>
Closes#11316 from marmbrus/datasetOptions.
## What changes were proposed in this pull request?
This PR tries to fix all typos in all markdown files under `docs` module,
and fixes similar typos in other comments, too.
## How was the this patch tested?
manual tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#11300 from dongjoon-hyun/minor_fix_typos.
JIRA: https://issues.apache.org/jira/browse/SPARK-13321
The following SQL can not be parsed with current parser:
SELECT `u_1`.`id` FROM (((SELECT `t0`.`id` FROM `default`.`t0`) UNION ALL (SELECT `t0`.`id` FROM `default`.`t0`)) UNION ALL (SELECT `t0`.`id` FROM `default`.`t0`)) AS u_1
We should fix it.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#11204 from viirya/nested-union.
## What changes were proposed in this pull request?
This is a step towards merging `SQLContext` and `HiveContext`. A new internal Catalog API was introduced in #10982 and extended in #11069. This patch introduces an implementation of this API using `HiveClient`, an existing interface to Hive. It also extends `HiveClient` with additional calls to Hive that are needed to complete the catalog implementation.
*Where should I start reviewing?* The new catalog introduced is `HiveCatalog`. This class is relatively simple because it just calls `HiveClientImpl`, where most of the new logic is. I would not start with `HiveClient`, `HiveQl`, or `HiveMetastoreCatalog`, which are modified mainly because of a refactor.
*Why is this patch so big?* I had to refactor HiveClient to remove an intermediate representation of databases, tables, partitions etc. After this refactor `CatalogTable` convert directly to and from `HiveTable` (etc.). Otherwise we would have to first convert `CatalogTable` to the intermediate representation and then convert that to HiveTable, which is messy.
The new class hierarchy is as follows:
```
org.apache.spark.sql.catalyst.catalog.Catalog
- org.apache.spark.sql.catalyst.catalog.InMemoryCatalog
- org.apache.spark.sql.hive.HiveCatalog
```
Note that, as of this patch, none of these classes are currently used anywhere yet. This will come in the future before the Spark 2.0 release.
## How was the this patch tested?
All existing unit tests, and HiveCatalogSuite that extends CatalogTestCases.
Author: Andrew Or <andrew@databricks.com>
Author: Reynold Xin <rxin@databricks.com>
Closes#11293 from rxin/hive-catalog.
## What changes were proposed in this pull request?
This patch renames logical.Subquery to logical.SubqueryAlias, which is a more appropriate name for this operator (versus subqueries as expressions).
## How was the this patch tested?
Unit tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#11288 from rxin/SPARK-13420.
This PR introduces several major changes:
1. Replacing `Expression.prettyString` with `Expression.sql`
The `prettyString` method is mostly an internal, developer faced facility for debugging purposes, and shouldn't be exposed to users.
1. Using SQL-like representation as column names for selected fields that are not named expression (back-ticks and double quotes should be removed)
Before, we were using `prettyString` as column names when possible, and sometimes the result column names can be weird. Here are several examples:
Expression | `prettyString` | `sql` | Note
------------------ | -------------- | ---------- | ---------------
`a && b` | `a && b` | `a AND b` |
`a.getField("f")` | `a[f]` | `a.f` | `a` is a struct
1. Adding trait `NonSQLExpression` extending from `Expression` for expressions that don't have a SQL representation (e.g. Scala UDF/UDAF and Java/Scala object expressions used for encoders)
`NonSQLExpression.sql` may return an arbitrary user facing string representation of the expression.
Author: Cheng Lian <lian@databricks.com>
Closes#10757 from liancheng/spark-12799.simplify-expression-string-methods.
```scala
// case 1: missing sort columns are resolvable if join is true
sql("SELECT explode(a) AS val, b FROM data WHERE b < 2 order by val, c")
// case 2: missing sort columns are not resolvable if join is false. Thus, issue an error message in this case
sql("SELECT explode(a) AS val FROM data order by val, c")
```
When sort columns are not in `Generate`, we can resolve them when `join` is equal to `true`. Still trying to add more test cases for the other `UnaryNode` types.
Could you review the changes? davies cloud-fan Thanks!
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11198 from gatorsmile/missingInSort.
Conversion of outer joins, if the predicates in filter conditions can restrict the result sets so that all null-supplying rows are eliminated.
- `full outer` -> `inner` if both sides have such predicates
- `left outer` -> `inner` if the right side has such predicates
- `right outer` -> `inner` if the left side has such predicates
- `full outer` -> `left outer` if only the left side has such predicates
- `full outer` -> `right outer` if only the right side has such predicates
If applicable, this can greatly improve the performance, since outer join is much slower than inner join, full outer join is much slower than left/right outer join.
The original PR is https://github.com/apache/spark/pull/10542
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#10567 from gatorsmile/outerJoinEliminationByFilterCond.
This PR adds support for rewriting constraints if there are aliases in the query plan. For e.g., if there is a query of form `SELECT a, a AS b`, any constraints on `a` now also apply to `b`.
JIRA: https://issues.apache.org/jira/browse/SPARK-13091
cc marmbrus
Author: Sameer Agarwal <sameer@databricks.com>
Closes#11144 from sameeragarwal/alias.
JIRA: https://issues.apache.org/jira/browse/SPARK-13384
## What changes were proposed in this pull request?
When we de-duplicate attributes in Analyzer, we create new attributes. However, we don't keep original qualifiers. Some plans will be failed to analysed. We should keep original qualifiers in new attributes.
## How was the this patch tested?
Unit test is added.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#11261 from viirya/keep-attr-qualifiers.
Currently, the columns in projects of Expand that are not used by Aggregate are not pruned, this PR fix that.
Author: Davies Liu <davies@databricks.com>
Closes#11225 from davies/fix_pruning_expand.
This patch adds a new optimizer rule for performing limit pushdown. Limits will now be pushed down in two cases:
- If a limit is on top of a `UNION ALL` operator, then a partition-local limit operator will be pushed to each of the union operator's children.
- If a limit is on top of an `OUTER JOIN` then a partition-local limit will be pushed to one side of the join. For `LEFT OUTER` and `RIGHT OUTER` joins, the limit will be pushed to the left and right side, respectively. For `FULL OUTER` join, we will only push limits when at most one of the inputs is already limited: if one input is limited we will push a smaller limit on top of it and if neither input is limited then we will limit the input which is estimated to be larger.
These optimizations were proposed previously by gatorsmile in #10451 and #10454, but those earlier PRs were closed and deferred for later because at that time Spark's physical `Limit` operator would trigger a full shuffle to perform global limits so there was a chance that pushdowns could actually harm performance by causing additional shuffles/stages. In #7334, we split the `Limit` operator into separate `LocalLimit` and `GlobalLimit` operators, so we can now push down only local limits (which don't require extra shuffles). This patch is based on both of gatorsmile's patches, with changes and simplifications due to partition-local-limiting.
When we push down the limit, we still keep the original limit in place, so we need a mechanism to ensure that the optimizer rule doesn't keep pattern-matching once the limit has been pushed down. In order to handle this, this patch adds a `maxRows` method to `SparkPlan` which returns the maximum number of rows that the plan can compute, then defines the pushdown rules to only push limits to children if the children's maxRows are greater than the limit's maxRows. This idea is carried over from #10451; see that patch for additional discussion.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#11121 from JoshRosen/limit-pushdown-2.
The current implementation of ResolveSortReferences can only push one missing attributes into it's child, it failed to analyze TPCDS Q98, because of there are two missing attributes in that (one from Window, another from Aggregate).
Author: Davies Liu <davies@databricks.com>
Closes#11153 from davies/resolve_sort.
The parser currently parses the following strings without a hitch:
* Table Identifier:
* `a.b.c` should fail, but results in the following table identifier `a.b`
* `table!#` should fail, but results in the following table identifier `table`
* Expression
* `1+2 r+e` should fail, but results in the following expression `1 + 2`
This PR fixes this by adding terminated rules for both expression parsing and table identifier parsing.
cc cloud-fan (we discussed this in https://github.com/apache/spark/pull/10649) jayadevanmurali (this causes your PR https://github.com/apache/spark/pull/11051 to fail)
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#11159 from hvanhovell/SPARK-13276.
Some analysis rules generate aliases or auxiliary attribute references with the same name but different expression IDs. For example, `ResolveAggregateFunctions` introduces `havingCondition` and `aggOrder`, and `DistinctAggregationRewriter` introduces `gid`.
This is OK for normal query execution since these attribute references get expression IDs. However, it's troublesome when converting resolved query plans back to SQL query strings since expression IDs are erased.
Here's an example Spark 1.6.0 snippet for illustration:
```scala
sqlContext.range(10).select('id as 'a, 'id as 'b).registerTempTable("t")
sqlContext.sql("SELECT SUM(a) FROM t GROUP BY a, b ORDER BY COUNT(a), COUNT(b)").explain(true)
```
The above code produces the following resolved plan:
```
== Analyzed Logical Plan ==
_c0: bigint
Project [_c0#101L]
+- Sort [aggOrder#102L ASC,aggOrder#103L ASC], true
+- Aggregate [a#47L,b#48L], [(sum(a#47L),mode=Complete,isDistinct=false) AS _c0#101L,(count(a#47L),mode=Complete,isDistinct=false) AS aggOrder#102L,(count(b#48L),mode=Complete,isDistinct=false) AS aggOrder#103L]
+- Subquery t
+- Project [id#46L AS a#47L,id#46L AS b#48L]
+- LogicalRDD [id#46L], MapPartitionsRDD[44] at range at <console>:26
```
Here we can see that both aggregate expressions in `ORDER BY` are extracted into an `Aggregate` operator, and both of them are named `aggOrder` with different expression IDs.
The solution is to automatically add the expression IDs into the attribute name for the Alias and AttributeReferences that are generated by Analyzer in SQL Generation.
In this PR, it also resolves another issue. Users could use the same name as the internally generated names. The duplicate names should not cause name ambiguity. When resolving the column, Catalyst should not pick the column that is internally generated.
Could you review the solution? marmbrus liancheng
I did not set the newly added flag for all the alias and attribute reference generated by Analyzers. Please let me know if I should do it? Thank you!
Author: gatorsmile <gatorsmile@gmail.com>
Closes#11050 from gatorsmile/namingConflicts.
Adds the benchmark results as comments.
The codegen version is slower than the interpreted version for `simple` case becasue of 3 reasons:
1. codegen version use a more complex hash algorithm than interpreted version, i.e. `Murmur3_x86_32.hashInt` vs [simple multiplication and addition](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala#L153).
2. codegen version will write the hash value to a row first and then read it out. I tried to create a `GenerateHasher` that can generate code to return hash value directly and got about 60% speed up for the `simple` case, does it worth?
3. the row in `simple` case only has one int field, so the runtime reflection may be removed because of branch prediction, which makes the interpreted version faster.
The `array` case is also slow for similar reasons, e.g. array elements are of same type, so interpreted version can probably get rid of runtime reflection by branch prediction.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10917 from cloud-fan/hash-benchmark.
nullability should only be considered as an optimization rather than part of the type system, so instead of failing analysis for mismatch nullability, we should pass analysis and add runtime null check.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#11035 from cloud-fan/ignore-nullability.
Trivial search-and-replace to eliminate deprecation warnings in Scala 2.11.
Also works with 2.10
Author: Jakob Odersky <jakob@odersky.com>
Closes#11085 from jodersky/SPARK-13171.
https://issues.apache.org/jira/browse/SPARK-12939
Now we will catch `ObjectOperator` in `Analyzer` and resolve the `fromRowExpression/deserializer` inside it. Also update the `MapGroups` and `CoGroup` to pass in `dataAttributes`, so that we can correctly resolve value deserializer(the `child.output` contains both groupking key and values, which may mess things up if they have same-name attribtues). End-to-end tests are added.
follow-ups:
* remove encoders from typed aggregate expression.
* completely remove resolve/bind in `ExpressionEncoder`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10852 from cloud-fan/bug.
This patch incorporates review feedback from #11069, which is already merged.
Author: Andrew Or <andrew@databricks.com>
Closes#11080 from andrewor14/catalog-follow-ups.
Spark SQL should collapse adjacent `Repartition` operators and only keep the last one.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#11064 from JoshRosen/collapse-repartition.
This is a small addendum to #10762 to make the code more robust again future changes.
Author: Reynold Xin <rxin@databricks.com>
Closes#11070 from rxin/SPARK-12828-natural-join.
This is a step towards consolidating `SQLContext` and `HiveContext`.
This patch extends the existing Catalog API added in #10982 to include methods for handling table partitions. In particular, a partition is identified by `PartitionSpec`, which is just a `Map[String, String]`. The Catalog is still not used by anything yet, but its API is now more or less complete and an implementation is fully tested.
About 200 lines are test code.
Author: Andrew Or <andrew@databricks.com>
Closes#11069 from andrewor14/catalog.
The ```SparkSqlLexer``` currently swallows characters which have not been defined in the grammar. This causes problems with SQL commands, such as: ```add jar file:///tmp/ab/TestUDTF.jar```. In this example the `````` is swallowed.
This PR adds an extra Lexer rule to handle such input, and makes a tiny modification to the ```ASTNode```.
cc davies liancheng
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#11052 from hvanhovell/SPARK-13157.
Based on the semantics of a query, we can derive a number of data constraints on output of each (logical or physical) operator. For instance, if a filter defines `‘a > 10`, we know that the output data of this filter satisfies 2 constraints:
1. `‘a > 10`
2. `isNotNull(‘a)`
This PR proposes a possible way of keeping track of these constraints and propagating them in the logical plan, which can then help us build more advanced optimizations (such as pruning redundant filters, optimizing joins, among others). We define constraints as a set of (implicitly conjunctive) expressions. For e.g., if a filter operator has constraints = `Set(‘a > 10, ‘b < 100)`, it’s implied that the outputs satisfy both individual constraints (i.e., `‘a > 10` AND `‘b < 100`).
Design Document: https://docs.google.com/a/databricks.com/document/d/1WQRgDurUBV9Y6CWOBS75PQIqJwT-6WftVa18xzm7nCo/edit?usp=sharing
Author: Sameer Agarwal <sameer@databricks.com>
Closes#10844 from sameeragarwal/constraints.
when we generate map, we first randomly pick a length, then create a seq of key value pair with the expected length, and finally call `toMap`. However, `toMap` will remove all duplicated keys, which makes the actual map size much less than we expected.
This PR fixes this problem by put keys in a set first, to guarantee we have enough keys to build a map with expected length.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10930 from cloud-fan/random-generator.
This pull request creates an internal catalog API. The creation of this API is the first step towards consolidating SQLContext and HiveContext. I envision we will have two different implementations in Spark 2.0: (1) a simple in-memory implementation, and (2) an implementation based on the current HiveClient (ClientWrapper).
I took a look at what Hive's internal metastore interface/implementation, and then created this API based on it. I believe this is the minimal set needed in order to achieve all the needed functionality.
Author: Reynold Xin <rxin@databricks.com>
Closes#10982 from rxin/SPARK-13078.
JIRA: https://issues.apache.org/jira/browse/SPARK-12705
**Scope:**
This PR is a general fix for sorting reference resolution when the child's `outputSet` does not have the order-by attributes (called, *missing attributes*):
- UnaryNode support is limited to `Project`, `Window`, `Aggregate`, `Distinct`, `Filter`, `RepartitionByExpression`.
- We will not try to resolve the missing references inside a subquery, unless the outputSet of this subquery contains it.
**General Reference Resolution Rules:**
- Jump over the nodes with the following types: `Distinct`, `Filter`, `RepartitionByExpression`. Do not need to add missing attributes. The reason is their `outputSet` is decided by their `inputSet`, which is the `outputSet` of their children.
- Group-by expressions in `Aggregate`: missing order-by attributes are not allowed to be added into group-by expressions since it will change the query result. Thus, in RDBMS, it is not allowed.
- Aggregate expressions in `Aggregate`: if the group-by expressions in `Aggregate` contains the missing attributes but aggregate expressions do not have it, just add them into the aggregate expressions. This can resolve the analysisExceptions thrown by the three TCPDS queries.
- `Project` and `Window` are special. We just need to add the missing attributes to their `projectList`.
**Implementation:**
1. Traverse the whole tree in a pre-order manner to find all the resolvable missing order-by attributes.
2. Traverse the whole tree in a post-order manner to add the found missing order-by attributes to the node if their `inputSet` contains the attributes.
3. If the origins of the missing order-by attributes are different nodes, each pass only resolves the missing attributes that are from the same node.
**Risk:**
Low. This rule will be trigger iff ```!s.resolved && child.resolved``` is true. Thus, very few cases are affected.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#10678 from gatorsmile/sortWindows.
Our current Intersect physical operator simply delegates to RDD.intersect. We should remove the Intersect physical operator and simply transform a logical intersect into a semi-join with distinct. This way, we can take advantage of all the benefits of join implementations (e.g. managed memory, code generation, broadcast joins).
After a search, I found one of the mainstream RDBMS did the same. In their query explain, Intersect is replaced by Left-semi Join. Left-semi Join could help outer-join elimination in Optimizer, as shown in the PR: https://github.com/apache/spark/pull/10566
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#10630 from gatorsmile/IntersectBySemiJoin.
JIRA: https://issues.apache.org/jira/browse/SPARK-11955
Currently we simply skip pushdowning filters in parquet if we enable schema merging.
However, we can actually mark particular fields in merging schema for safely pushdowning filters in parquet.
Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#9940 from viirya/safe-pushdown-parquet-filters.
This patch adds support for complex types for ColumnarBatch. ColumnarBatch supports structs
and arrays. There is a simple mapping between the richer catalyst types to these two. Strings
are treated as an array of bytes.
ColumnarBatch will contain a column for each node of the schema. Non-complex schemas consists
of just leaf nodes. Structs represent an internal node with one child for each field. Arrays
are internal nodes with one child. Structs just contain nullability. Arrays contain offsets
and lengths into the child array. This structure is able to handle arbitrary nesting. It has
the key property that we maintain columnar throughout and that primitive types are only stored
in the leaf nodes and contiguous across rows. For example, if the schema is
```
array<array<int>>
```
There are three columns in the schema. The internal nodes each have one children. The leaf node contains all the int data stored consecutively.
As part of this, this patch adds append APIs in addition to the Put APIs (e.g. putLong(rowid, v)
vs appendLong(v)). These APIs are necessary when the batch contains variable length elements.
The vectors are not fixed length and will grow as necessary. This should make the usage a lot
simpler for the writer.
Author: Nong Li <nong@databricks.com>
Closes#10820 from nongli/spark-12854.
As we begin to use unsafe row writing framework(`BufferHolder` and `UnsafeRowWriter`) in more and more places(`UnsafeProjection`, `UnsafeRowParquetRecordReader`, `GenerateColumnAccessor`, etc.), we should add more doc to it and make it easier to use.
This PR abstract the technique used in `UnsafeRowParquetRecordReader`: avoid unnecessary operatition as more as possible. For example, do not always point the row to the buffer at the end, we only need to update the size of row. If all fields are of primitive type, we can even save the row size updating. Then we can apply this technique to more places easily.
a local benchmark shows `UnsafeProjection` is up to 1.7x faster after this PR:
**old version**
```
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
unsafe projection: Avg Time(ms) Avg Rate(M/s) Relative Rate
-------------------------------------------------------------------------------
single long 2616.04 102.61 1.00 X
single nullable long 3032.54 88.52 0.86 X
primitive types 9121.05 29.43 0.29 X
nullable primitive types 12410.60 21.63 0.21 X
```
**new version**
```
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
unsafe projection: Avg Time(ms) Avg Rate(M/s) Relative Rate
-------------------------------------------------------------------------------
single long 1533.34 175.07 1.00 X
single nullable long 2306.73 116.37 0.66 X
primitive types 8403.93 31.94 0.18 X
nullable primitive types 12448.39 21.56 0.12 X
```
For single non-nullable long(the best case), we can have about 1.7x speed up. Even it's nullable, we can still have 1.3x speed up. For other cases, it's not such a boost as the saved operations only take a little proportion of the whole process. The benchmark code is included in this PR.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10809 from cloud-fan/unsafe-projection.
This pull request implements strength reduction for comparing integral expressions and decimal literals, which is more common now because we switch to parsing fractional literals as decimal types (rather than doubles). I added the rules to the existing DecimalPrecision rule with some refactoring to simplify the control flow. I also moved DecimalPrecision rule into its own file due to the growing size.
Author: Reynold Xin <rxin@databricks.com>
Closes#10882 from rxin/SPARK-12904-1.
Benchmark it on 4 different schemas, the result:
```
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
Hash For simple: Avg Time(ms) Avg Rate(M/s) Relative Rate
-------------------------------------------------------------------------------
interpreted version 31.47 266.54 1.00 X
codegen version 64.52 130.01 0.49 X
```
```
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
Hash For normal: Avg Time(ms) Avg Rate(M/s) Relative Rate
-------------------------------------------------------------------------------
interpreted version 4068.11 0.26 1.00 X
codegen version 1175.92 0.89 3.46 X
```
```
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
Hash For array: Avg Time(ms) Avg Rate(M/s) Relative Rate
-------------------------------------------------------------------------------
interpreted version 9276.70 0.06 1.00 X
codegen version 14762.23 0.04 0.63 X
```
```
Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz
Hash For map: Avg Time(ms) Avg Rate(M/s) Relative Rate
-------------------------------------------------------------------------------
interpreted version 58869.79 0.01 1.00 X
codegen version 9285.36 0.06 6.34 X
```
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10816 from cloud-fan/hash-benchmark.
The existing `Union` logical operator only supports two children. Thus, adding a new logical operator `Unions` which can have arbitrary number of children to replace the existing one.
`Union` logical plan is a binary node. However, a typical use case for union is to union a very large number of input sources (DataFrames, RDDs, or files). It is not uncommon to union hundreds of thousands of files. In this case, our optimizer can become very slow due to the large number of logical unions. We should change the Union logical plan to support an arbitrary number of children, and add a single rule in the optimizer to collapse all adjacent `Unions` into a single `Unions`. Note that this problem doesn't exist in physical plan, because the physical `Unions` already supports arbitrary number of children.
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#10577 from gatorsmile/unionAllMultiChildren.
Also updated documentation to explain why ComputeCurrentTime and EliminateSubQueries are in the optimizer rather than analyzer.
Author: Reynold Xin <rxin@databricks.com>
Closes#10837 from rxin/optimizer-analyzer-comment.
The three optimization cases are:
1. If the first branch's condition is a true literal, remove the CaseWhen and use the value from that branch.
2. If a branch's condition is a false or null literal, remove that branch.
3. If only the else branch is left, remove the CaseWhen and use the value from the else branch.
Author: Reynold Xin <rxin@databricks.com>
Closes#10827 from rxin/SPARK-12770.
Call `dealias` on local types to fix schema generation for abstract type members, such as
```scala
type KeyValue = (Int, String)
```
Add simple test
Author: Jakob Odersky <jodersky@gmail.com>
Closes#10749 from jodersky/aliased-schema.
I was reading this part of the analyzer code again and got confused by the difference between findWiderTypeForTwo and findTightestCommonTypeOfTwo.
I also simplified WidenSetOperationTypes to make it a lot simpler. The easiest way to review this one is to just read the original code, and the new code. The logic is super simple.
Author: Reynold Xin <rxin@databricks.com>
Closes#10802 from rxin/SPARK-12873.
In this PR the new CatalystQl parser stack reaches grammar parity with the old Parser-Combinator based SQL Parser. This PR also replaces all uses of the old Parser, and removes it from the code base.
Although the existing Hive and SQL parser dialects were mostly the same, some kinks had to be worked out:
- The SQL Parser allowed syntax like ```APPROXIMATE(0.01) COUNT(DISTINCT a)```. In order to make this work we needed to hardcode approximate operators in the parser, or we would have to create an approximate expression. ```APPROXIMATE_COUNT_DISTINCT(a, 0.01)``` would also do the job and is much easier to maintain. So, this PR **removes** this keyword.
- The old SQL Parser supports ```LIMIT``` clauses in nested queries. This is **not supported** anymore. See https://github.com/apache/spark/pull/10689 for the rationale for this.
- Hive has a charset name char set literal combination it supports, for instance the following expression ```_ISO-8859-1 0x4341464562616265``` would yield this string: ```CAFEbabe```. Hive will only allow charset names to start with an underscore. This is quite annoying in spark because as soon as you use a tuple names will start with an underscore. In this PR we **remove** this feature from the parser. It would be quite easy to implement such a feature as an Expression later on.
- Hive and the SQL Parser treat decimal literals differently. Hive will turn any decimal into a ```Double``` whereas the SQL Parser would convert a non-scientific decimal into a ```BigDecimal```, and would turn a scientific decimal into a Double. We follow Hive's behavior here. The new parser supports a big decimal literal, for instance: ```81923801.42BD```, which can be used when a big decimal is needed.
cc rxin viirya marmbrus yhuai cloud-fan
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#10745 from hvanhovell/SPARK-12575-2.
The goal of this PR is to eliminate unnecessary translations when there are back-to-back `MapPartitions` operations. In order to achieve this I also made the following simplifications:
- Operators no longer have hold encoders, instead they have only the expressions that they need. The benefits here are twofold: the expressions are visible to transformations so go through the normal resolution/binding process. now that they are visible we can change them on a case by case basis.
- Operators no longer have type parameters. Since the engine is responsible for its own type checking, having the types visible to the complier was an unnecessary complication. We still leverage the scala compiler in the companion factory when constructing a new operator, but after this the types are discarded.
Deferred to a follow up PR:
- Remove as much of the resolution/binding from Dataset/GroupedDataset as possible. We should still eagerly check resolution and throw an error though in the case of mismatches for an `as` operation.
- Eliminate serializations in more cases by adding more cases to `EliminateSerialization`
Author: Michael Armbrust <michael@databricks.com>
Closes#10747 from marmbrus/encoderExpressions.
This pull request rewrites CaseWhen expression to break the single, monolithic "branches" field into a sequence of tuples (Seq[(condition, value)]) and an explicit optional elseValue field.
Prior to this pull request, each even position in "branches" represents the condition for each branch, and each odd position represents the value for each branch. The use of them have been pretty confusing with a lot sliding windows or grouped(2) calls.
Author: Reynold Xin <rxin@databricks.com>
Closes#10734 from rxin/simplify-case.
Fix the style violation (space before , and :).
This PR is a followup for #10643 and rework of #10685 .
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Closes#10732 from sarutak/SPARK-12692-followup-sql.
This patch removes CaseKeyWhen expression and replaces it with a factory method that generates the equivalent CaseWhen. This reduces the amount of code we'd need to maintain in the future for both code generation and optimizer.
Note that we introduced CaseKeyWhen to avoid duplicate evaluations of the key. This is no longer a problem because we now have common subexpression elimination.
Author: Reynold Xin <rxin@databricks.com>
Closes#10722 from rxin/SPARK-12768.
This pull request does a few small things:
1. Separated if simplification from BooleanSimplification and created a new rule SimplifyConditionals. In the future we can also simplify other conditional expressions here.
2. Added unit test for SimplifyConditionals.
3. Renamed SimplifyCaseConversionExpressionsSuite to SimplifyStringCaseConversionSuite
Author: Reynold Xin <rxin@databricks.com>
Closes#10716 from rxin/SPARK-12762.
Fix the style violation (space before , and :).
This PR is a followup for #10643.
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Closes#10718 from sarutak/SPARK-12692-followup-sql.
The PR allows us to use the new SQL parser to parse SQL expressions such as: ```1 + sin(x*x)```
We enable this functionality in this PR, but we will not start using this actively yet. This will be done as soon as we have reached grammar parity with the existing parser stack.
cc rxin
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#10649 from hvanhovell/SPARK-12576.
This PR tries to enable Spark SQL to convert resolved logical plans back to SQL query strings. For now, the major use case is to canonicalize Spark SQL native view support. The major entry point is `SQLBuilder.toSQL`, which returns an `Option[String]` if the logical plan is recognized.
The current version is still in WIP status, and is quite limited. Known limitations include:
1. The logical plan must be analyzed but not optimized
The optimizer erases `Subquery` operators, which contain necessary scope information for SQL generation. Future versions should be able to recover erased scope information by inserting subqueries when necessary.
1. The logical plan must be created using HiveQL query string
Query plans generated by composing arbitrary DataFrame API combinations are not supported yet. Operators within these query plans need to be rearranged into a canonical form that is more suitable for direct SQL generation. For example, the following query plan
```
Filter (a#1 < 10)
+- MetastoreRelation default, src, None
```
need to be canonicalized into the following form before SQL generation:
```
Project [a#1, b#2, c#3]
+- Filter (a#1 < 10)
+- MetastoreRelation default, src, None
```
Otherwise, the SQL generation process will have to handle a large number of special cases.
1. Only a fraction of expressions and basic logical plan operators are supported in this PR
Currently, 95.7% (1720 out of 1798) query plans in `HiveCompatibilitySuite` can be successfully converted to SQL query strings.
Known unsupported components are:
- Expressions
- Part of math expressions
- Part of string expressions (buggy?)
- Null expressions
- Calendar interval literal
- Part of date time expressions
- Complex type creators
- Special `NOT` expressions, e.g. `NOT LIKE` and `NOT IN`
- Logical plan operators/patterns
- Cube, rollup, and grouping set
- Script transformation
- Generator
- Distinct aggregation patterns that fit `DistinctAggregationRewriter` analysis rule
- Window functions
Support for window functions, generators, and cubes etc. will be added in follow-up PRs.
This PR leverages `HiveCompatibilitySuite` for testing SQL generation in a "round-trip" manner:
* For all select queries, we try to convert it back to SQL
* If the query plan is convertible, we parse the generated SQL into a new logical plan
* Run the new logical plan instead of the original one
If the query plan is inconvertible, the test case simply falls back to the original logic.
TODO
- [x] Fix failed test cases
- [x] Support for more basic expressions and logical plan operators (e.g. distinct aggregation etc.)
- [x] Comments and documentation
Author: Cheng Lian <lian@databricks.com>
Closes#10541 from liancheng/sql-generation.
JIRA: https://issues.apache.org/jira/browse/SPARK-12687
Some queries such as `(select 1 as a) union (select 2 as a)` can't work. This patch fixes it.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#10660 from viirya/fix-union.
Fix most build warnings: mostly deprecated API usages. I'll annotate some of the changes below. CC rxin who is leading the charge to remove the deprecated APIs.
Author: Sean Owen <sowen@cloudera.com>
Closes#10570 from srowen/SPARK-12618.
JIRA: https://issues.apache.org/jira/browse/SPARK-12439
In toCatalystArray, we should look at the data type returned by dataTypeFor instead of silentSchemaFor, to determine if the element is native type. An obvious problem is when the element is Option[Int] class, catalsilentSchemaFor will return Int, then we will wrongly recognize the element is native type.
There is another problem when using Option as array element. When we encode data like Seq(Some(1), Some(2), None) with encoder, we will use MapObjects to construct an array for it later. But in MapObjects, we don't check if the return value of lambdaFunction is null or not. That causes a bug that the decoded data for Seq(Some(1), Some(2), None) would be Seq(1, 2, -1), instead of Seq(1, 2, null).
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#10391 from viirya/fix-catalystarray.
address comments in #10435
This makes the API easier to use if user programmatically generate the call to hash, and they will get analysis exception if the arguments of hash is empty.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10588 from cloud-fan/hash.
just write the arguments into unsafe row and use murmur3 to calculate hash code
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10435 from cloud-fan/hash-expr.
It is currently possible to change the values of the supposedly immutable ```GenericRow``` and ```GenericInternalRow``` classes. This is caused by the fact that scala's ArrayOps ```toArray``` (returned by calling ```toSeq```) will return the backing array instead of a copy. This PR fixes this problem.
This PR was inspired by https://github.com/apache/spark/pull/10374 by apo1.
cc apo1 sarutak marmbrus cloud-fan nongli (everyone in the previous conversation).
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#10553 from hvanhovell/SPARK-12421.
Right now, numFields will be passed in by pointTo(), then bitSetWidthInBytes is calculated, making pointTo() a little bit heavy.
It should be part of constructor of UnsafeRow.
Author: Davies Liu <davies@databricks.com>
Closes#10528 from davies/numFields.
Most of cases we should propagate null when call `NewInstance`, and so far there is only one case we should stop null propagation: create product/java bean. So I think it makes more sense to propagate null by dafault.
This also fixes a bug when encode null array/map, which is firstly discovered in https://github.com/apache/spark/pull/10401
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10443 from cloud-fan/encoder.
Moved (case) classes Strategy, Once, FixedPoint and Batch to the companion object. This is necessary if we want to have the Optimizer easily extendable in the following sense: Usually a user wants to add additional rules, and just take the ones that are already there. However, inner classes made that impossible since the code did not compile
This allows easy extension of existing Optimizers see the DefaultOptimizerExtendableSuite for a corresponding test case.
Author: Stephan Kessler <stephan.kessler@sap.com>
Closes#10174 from stephankessler/SPARK-7727.
When the filter is ```"b in ('1', '2')"```, the filter is not pushed down to Parquet. Thanks!
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#10278 from gatorsmile/parquetFilterNot.
Compare both left and right side of the case expression ignoring nullablity when checking for type equality.
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#10156 from dilipbiswal/spark-12102.
This PR adds a new expression `AssertNotNull` to ensure non-nullable fields of products and case classes don't receive null values at runtime.
Author: Cheng Lian <lian@databricks.com>
Closes#10331 from liancheng/dataset-nullability-check.
This could simplify the generated code for expressions that is not nullable.
This PR fix lots of bugs about nullability.
Author: Davies Liu <davies@databricks.com>
Closes#10333 from davies/skip_nullable.
This PR removes Hive windows functions from Spark and replaces them with (native) Spark ones. The PR is on par with Hive in terms of features.
This has the following advantages:
* Better memory management.
* The ability to use spark UDAFs in Window functions.
cc rxin / yhuai
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#9819 from hvanhovell/SPARK-8641-2.
in https://github.com/apache/spark/pull/10133 we found that, we shoud ensure the children of `TreeNode` are all accessible in the `productIterator`, or the behavior will be very confusing.
In this PR, I try to fix this problem by expsing the `loopVar`.
This also fixes SPARK-12131 which is caused by the hacky `MapObjects`.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10239 from cloud-fan/map-objects.
checked with hive, greatest/least should cast their children to a tightest common type,
i.e. `(int, long) => long`, `(int, string) => error`, `(decimal(10,5), decimal(5, 10)) => error`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#10196 from cloud-fan/type-coercion.
Currently, the order of joins is exactly the same as SQL query, some conditions may not pushed down to the correct join, then those join will become cross product and is extremely slow.
This patch try to re-order the inner joins (which are common in SQL query), pick the joins that have self-contain conditions first, delay those that does not have conditions.
After this patch, the TPCDS query Q64/65 can run hundreds times faster.
cc marmbrus nongli
Author: Davies Liu <davies@databricks.com>
Closes#10073 from davies/reorder_joins.
When \u appears in a comment block (i.e. in /**/), code gen will break. So, in Expression and CodegenFallback, we escape \u to \\u.
yhuai Please review it. I did reproduce it and it works after the fix. Thanks!
Author: gatorsmile <gatorsmile@gmail.com>
Closes#10155 from gatorsmile/escapeU.
We should upgrade to SBT 0.13.9, since this is a requirement in order to use SBT's new Maven-style resolution features (which will be done in a separate patch, because it's blocked by some binary compatibility issues in the POM reader plugin).
I also upgraded Scalastyle to version 0.8.0, which was necessary in order to fix a Scala 2.10.5 compatibility issue (see https://github.com/scalastyle/scalastyle/issues/156). The newer Scalastyle is slightly stricter about whitespace surrounding tokens, so I fixed the new style violations.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#10112 from JoshRosen/upgrade-to-sbt-0.13.9.
create java version of `constructorFor` and `extractorFor` in `JavaTypeInference`
Author: Wenchen Fan <wenchen@databricks.com>
This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>
Closes#9937 from cloud-fan/pojo.
When we build the `fromRowExpression` for an encoder, we set up a lot of "unresolved" stuff and lost the required data type, which may lead to runtime error if the real type doesn't match the encoder's schema.
For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type is `[a: int, b: long]`, then we will hit runtime error and say that we can't construct class `Data` with int and long, because we lost the information that `b` should be a string.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9840 from cloud-fan/err-msg.
In https://github.com/apache/spark/pull/9409 we enabled multi-column counting. The approach taken in that PR introduces a bit of overhead by first creating a row only to check if all of the columns are non-null.
This PR fixes that technical debt. Count now takes multiple columns as its input. In order to make this work I have also added support for multiple columns in the single distinct code path.
cc yhuai
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#10015 from hvanhovell/SPARK-12024.
This is a followup for https://github.com/apache/spark/pull/9959.
I added more documentation and rewrote some monadic code into simpler ifs.
Author: Reynold Xin <rxin@databricks.com>
Closes#9995 from rxin/SPARK-11973.
this is based on https://github.com/apache/spark/pull/9844, with some bug fix and clean up.
The problems is that, normal operator should be resolved based on its child, but `Sort` operator can also be resolved based on its grandchild. So we have 3 rules that can resolve `Sort`: `ResolveReferences`, `ResolveSortReferences`(if grandchild is `Project`) and `ResolveAggregateFunctions`(if grandchild is `Aggregate`).
For example, `select c1 as a , c2 as b from tab group by c1, c2 order by a, c2`, we need to resolve `a` and `c2` for `Sort`. Firstly `a` will be resolved in `ResolveReferences` based on its child, and when we reach `ResolveAggregateFunctions`, we will try to resolve both `a` and `c2` based on its grandchild, but failed because `a` is not a legal aggregate expression.
whoever merge this PR, please give the credit to dilipbiswal
Author: Dilip Biswal <dbiswal@us.ibm.com>
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9961 from cloud-fan/sort.
Currently, filter can't be pushed through aggregation with alias or literals, this patch fix that.
After this patch, the time of TPC-DS query 4 go down to 13 seconds from 141 seconds (10x improvements).
cc nongli yhuai
Author: Davies Liu <davies@databricks.com>
Closes#9959 from davies/push_filter2.
Can someone review my code to make sure I'm not missing anything? Thanks!
Author: Xiu Guo <xguo27@gmail.com>
Author: Xiu Guo <guoxi@us.ibm.com>
Closes#9612 from xguo27/SPARK-11628.
Hive has since changed this behavior as well. https://issues.apache.org/jira/browse/HIVE-3454
Author: Nong Li <nong@databricks.com>
Author: Nong Li <nongli@gmail.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#9685 from nongli/spark-11724.
before this PR, when users try to get an encoder for an un-supported class, they will only get a very simple error message like `Encoder for type xxx is not supported`.
After this PR, the error message become more friendly, for example:
```
No Encoder found for abc.xyz.NonEncodable
- array element class: "abc.xyz.NonEncodable"
- field (class: "scala.Array", name: "arrayField")
- root class: "abc.xyz.AnotherClass"
```
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9810 from cloud-fan/error-message.
JIRA: https://issues.apache.org/jira/browse/SPARK-11817
Instead of return None, we should truncate the fractional seconds to prevent inserting NULL.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#9834 from viirya/truncate-fractional-sec.
After some experiment, I found it's not convenient to have separate encoder builders: `FlatEncoder` and `ProductEncoder`. For example, when create encoders for `ScalaUDF`, we have no idea if the type `T` is flat or not. So I revert the splitting change in https://github.com/apache/spark/pull/9693, while still keeping the bug fixes and tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9726 from cloud-fan/follow.
We currently rely on the optimizer's constant folding to replace current_timestamp and current_date. However, this can still result in different values for different instances of current_timestamp/current_date if the optimizer is not running fast enough.
A better solution is to replace these functions in the analyzer in one shot.
Author: Reynold Xin <rxin@databricks.com>
Closes#9833 from rxin/SPARK-11849.
Also added some nicer error messages for incompatible types (private types and primitive types) for Kryo/Java encoder.
Author: Reynold Xin <rxin@databricks.com>
Closes#9823 from rxin/SPARK-11833.
Before this PR there were two things that would blow up if you called `df.as[MyClass]` if `MyClass` was defined in the REPL:
- [x] Because `classForName` doesn't work on the munged names returned by `tpe.erasure.typeSymbol.asClass.fullName`
- [x] Because we don't have anything to pass into the constructor for the `$outer` pointer.
Note that this PR is just adding the infrastructure for working with inner classes in encoder and is not yet sufficient to make them work in the REPL. Currently, the implementation show in 95cec7d413 is causing a bug that breaks code gen due to some interaction between janino and the `ExecutorClassLoader`. This will be addressed in a follow-up PR.
Author: Michael Armbrust <michael@databricks.com>
Closes#9602 from marmbrus/dataset-replClasses.
This patch refactors the existing Kryo encoder expressions and adds support for Java serialization.
Author: Reynold Xin <rxin@databricks.com>
Closes#9802 from rxin/SPARK-11810.
If user use primitive parameters in UDF, there is no way for him to do the null-check for primitive inputs, so we are assuming the primitive input is null-propagatable for this case and return null if the input is null.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9770 from cloud-fan/udf.
I also found a bug with self-joins returning incorrect results in the Dataset API. Two test cases attached and filed SPARK-11803.
Author: Reynold Xin <rxin@databricks.com>
Closes#9789 from rxin/SPARK-11802.
Based on the comment of cloud-fan in https://github.com/apache/spark/pull/9216, update the AttributeReference's hashCode function by including the hashCode of the other attributes including name, nullable and qualifiers.
Here, I am not 100% sure if we should include name in the hashCode calculation, since the original hashCode calculation does not include it.
marmbrus cloud-fan Please review if the changes are good.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#9761 from gatorsmile/hashCodeNamedExpression.
The randomly generated ArrayData used for the UDT `ExamplePoint` in `RowEncoderSuite` sometimes doesn't have enough elements. In this case, this test will fail. This patch is to fix it.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#9757 from viirya/fix-randomgenerated-udt.
Invocation of getters for type extending AnyVal returns default value (if field value is null) instead of throwing NPE. Please check comments for SPARK-11553 issue for more details.
Author: Bartlomiej Alberski <bartlomiej.alberski@allegrogroup.com>
Closes#9642 from alberskib/bugfix/SPARK-11553.
These 2 are very similar, we can consolidate them into one.
Also add tests for it and fix a bug.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9729 from cloud-fan/tuple.
JIRA: https://issues.apache.org/jira/browse/SPARK-11743
RowEncoder doesn't support UserDefinedType now. We should add the support for it.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#9712 from viirya/rowencoder-udt.
code snippet to reproduce it:
```
TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
val t = Timestamp.valueOf("1900-06-11 12:14:50.789")
val us = fromJavaTimestamp(t)
assert(getSeconds(us) === t.getSeconds)
```
it will be good to add a regression test for it, but the reproducing code need to change the default timezone, and even we change it back, the `lazy val defaultTimeZone` in `DataTimeUtils` is fixed.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9728 from cloud-fan/seconds.
also add more tests for encoders, and fix bugs that I found:
* when convert array to catalyst array, we can only skip element conversion for native types(e.g. int, long, boolean), not `AtomicType`(String is AtomicType but we need to convert it)
* we should also handle scala `BigDecimal` when convert from catalyst `Decimal`.
* complex map type should be supported
other issues that still in investigation:
* encode java `BigDecimal` and decode it back, seems we will loss precision info.
* when encode case class that defined inside a object, `ClassNotFound` exception will be thrown.
I'll remove unused code in a follow-up PR.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9693 from cloud-fan/split.
`to_unix_timestamp` is the deterministic version of `unix_timestamp`, as it accepts at least one parameters.
Since the behavior here is quite similar to `unix_timestamp`, I think the dataframe API is not necessary here.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#9347 from adrian-wang/to_unix_timestamp.
This patch adds the building blocks for codegening subexpr elimination and implements
it end to end for UnsafeProjection. The building blocks can be used to do the same thing
for other operators.
It introduces some utilities to compute common sub expressions. Expressions can be added to
this data structure. The expr and its children will be recursively matched against existing
expressions (ones previously added) and grouped into common groups. This is built using
the existing `semanticEquals`. It does not understand things like commutative or associative
expressions. This can be done as future work.
After building this data structure, the codegen process takes advantage of it by:
1. Generating a helper function in the generated class that computes the common
subexpression. This is done for all common subexpressions that have at least
two occurrences and the expression tree is sufficiently complex.
2. When generating the apply() function, if the helper function exists, call that
instead of regenerating the expression tree. Repeated calls to the helper function
shortcircuit the evaluation logic.
Author: Nong Li <nong@databricks.com>
Author: Nong Li <nongli@gmail.com>
This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>
Closes#9480 from nongli/spark-10371.
https://issues.apache.org/jira/browse/SPARK-9830
This PR contains the following main changes.
* Removing `AggregateExpression1`.
* Removing `Aggregate` operator, which is used to evaluate `AggregateExpression1`.
* Removing planner rule used to plan `Aggregate`.
* Linking `MultipleDistinctRewriter` to analyzer.
* Renaming `AggregateExpression2` to `AggregateExpression` and `AggregateFunction2` to `AggregateFunction`.
* Updating places where we create aggregate expression. The way to create aggregate expressions is `AggregateExpression(aggregateFunction, mode, isDistinct)`.
* Changing `val`s in `DeclarativeAggregate`s that touch children of this function to `lazy val`s (when we create aggregate expression in DataFrame API, children of an aggregate function can be unresolved).
Author: Yin Huai <yhuai@databricks.com>
Closes#9556 from yhuai/removeAgg1.
This PR adds support for multiple column in a single count distinct aggregate to the new aggregation path.
cc yhuai
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#9409 from hvanhovell/SPARK-11451.
https://issues.apache.org/jira/browse/SPARK-10116
This is really trivial, just happened to notice it -- if `XORShiftRandom.hashSeed` is really supposed to have random bits throughout (as the comment implies), it needs to do something for the conversion to `long`.
mengxr mkolod
Author: Imran Rashid <irashid@cloudera.com>
Closes#8314 from squito/SPARK-10116.
A cleanup for https://github.com/apache/spark/pull/9085.
The `DecimalLit` is very similar to `FloatLit`, we can just keep one of them.
Also added low level unit test at `SqlParserSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9482 from cloud-fan/parser.
Currently, if the Timestamp is before epoch (1970/01/01), the hours, minutes and seconds will be negative (also rounding up).
Author: Davies Liu <davies@databricks.com>
Closes#9502 from davies/neg_hour.
Add a rule in optimizer to convert NULL [NOT] IN (expr1,...,expr2) to
Literal(null).
This is a follow up defect to SPARK-8654
cloud-fan Can you please take a look ?
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#9348 from dilipbiswal/spark_11024.
This PR adds a new operation `joinWith` to a `Dataset`, which returns a `Tuple` for each pair where a given `condition` evaluates to true.
```scala
case class ClassData(a: String, b: Int)
val ds1 = Seq(ClassData("a", 1), ClassData("b", 2)).toDS()
val ds2 = Seq(("a", 1), ("b", 2)).toDS()
> ds1.joinWith(ds2, $"_1" === $"a").collect()
res0: Array((ClassData("a", 1), ("a", 1)), (ClassData("b", 2), ("b", 2)))
```
This operation is similar to the relation `join` function with one important difference in the result schema. Since `joinWith` preserves objects present on either side of the join, the result schema is similarly nested into a tuple under the column names `_1` and `_2`.
This type of join can be useful both for preserving type-safety with the original object types as well as working with relational data where either side of the join has column names in common.
## Required Changes to Encoders
In the process of working on this patch, several deficiencies to the way that we were handling encoders were discovered. Specifically, it turned out to be very difficult to `rebind` the non-expression based encoders to extract the nested objects from the results of joins (and also typed selects that return tuples).
As a result the following changes were made.
- `ClassEncoder` has been renamed to `ExpressionEncoder` and has been improved to also handle primitive types. Additionally, it is now possible to take arbitrary expression encoders and rewrite them into a single encoder that returns a tuple.
- All internal operations on `Dataset`s now require an `ExpressionEncoder`. If the users tries to pass a non-`ExpressionEncoder` in, an error will be thrown. We can relax this requirement in the future by constructing a wrapper class that uses expressions to project the row to the expected schema, shielding the users code from the required remapping. This will give us a nice balance where we don't force user encoders to understand attribute references and binding, but still allow our native encoder to leverage runtime code generation to construct specific encoders for a given schema that avoid an extra remapping step.
- Additionally, the semantics for different types of objects are now better defined. As stated in the `ExpressionEncoder` scaladoc:
- Classes will have their sub fields extracted by name using `UnresolvedAttribute` expressions
and `UnresolvedExtractValue` expressions.
- Tuples will have their subfields extracted by position using `BoundReference` expressions.
- Primitives will have their values extracted from the first ordinal with a schema that defaults
to the name `value`.
- Finally, the binding lifecycle for `Encoders` has now been unified across the codebase. Encoders are now `resolved` to the appropriate schema in the constructor of `Dataset`. This process replaces an unresolved expressions with concrete `AttributeReference` expressions. Binding then happens on demand, when an encoder is going to be used to construct an object. This closely mirrors the lifecycle for standard expressions when executing normal SQL or `DataFrame` queries.
Author: Michael Armbrust <michael@databricks.com>
Closes#9300 from marmbrus/datasets-tuples.
I'm new to spark. I was trying out the sort_array function then hit this exception. I looked into the spark source code. I found the root cause is that sort_array does not check for an array of NULLs. It's not meaningful to sort an array of entirely NULLs anyway.
I'm adding a check on the input array type to SortArray. If the array consists of NULLs entirely, there is no need to sort such array. I have also added a test case for this.
Please help to review my fix. Thanks!
Author: Jia Li <jiali@us.ibm.com>
Closes#9247 from jliwork/SPARK-11277.
For nested StructType, the underline buffer could be used for others before, we should zero out the padding bytes for those primitive types that have less than 8 bytes.
cc cloud-fan
Author: Davies Liu <davies@databricks.com>
Closes#9217 from davies/zero_out.
*This PR adds a new experimental API to Spark, tentitively named Datasets.*
A `Dataset` is a strongly-typed collection of objects that can be transformed in parallel using functional or relational operations. Example usage is as follows:
### Functional
```scala
> val ds: Dataset[Int] = Seq(1, 2, 3).toDS()
> ds.filter(_ % 1 == 0).collect()
res1: Array[Int] = Array(1, 2, 3)
```
### Relational
```scala
scala> ds.toDF().show()
+-----+
|value|
+-----+
| 1|
| 2|
| 3|
+-----+
> ds.select(expr("value + 1").as[Int]).collect()
res11: Array[Int] = Array(2, 3, 4)
```
## Comparison to RDDs
A `Dataset` differs from an `RDD` in the following ways:
- The creation of a `Dataset` requires the presence of an explicit `Encoder` that can be
used to serialize the object into a binary format. Encoders are also capable of mapping the
schema of a given object to the Spark SQL type system. In contrast, RDDs rely on runtime
reflection based serialization.
- Internally, a `Dataset` is represented by a Catalyst logical plan and the data is stored
in the encoded form. This representation allows for additional logical operations and
enables many operations (sorting, shuffling, etc.) to be performed without deserializing to
an object.
A `Dataset` can be converted to an `RDD` by calling the `.rdd` method.
## Comparison to DataFrames
A `Dataset` can be thought of as a specialized DataFrame, where the elements map to a specific
JVM object type, instead of to a generic `Row` container. A DataFrame can be transformed into
specific Dataset by calling `df.as[ElementType]`. Similarly you can transform a strongly-typed
`Dataset` to a generic DataFrame by calling `ds.toDF()`.
## Implementation Status and TODOs
This is a rough cut at the least controversial parts of the API. The primary purpose here is to get something committed so that we can better parallelize further work and get early feedback on the API. The following is being deferred to future PRs:
- Joins and Aggregations (prototype here f11f91e6f0)
- Support for Java
Additionally, the responsibility for binding an encoder to a given schema is currently done in a fairly ad-hoc fashion. This is an internal detail, and what we are doing today works for the cases we care about. However, as we add more APIs we'll probably need to do this in a more principled way (i.e. separate resolution from binding as we do in DataFrames).
## COMPATIBILITY NOTE
Long term we plan to make `DataFrame` extend `Dataset[Row]`. However,
making this change to che class hierarchy would break the function signatures for the existing
function operations (map, flatMap, etc). As such, this class should be considered a preview
of the final API. Changes will be made to the interface after Spark 1.6.
Author: Michael Armbrust <michael@databricks.com>
Closes#9190 from marmbrus/dataset-infra.
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.
The fix is to find a common type between LHS and RHS expressions and cast
all the expression to the common type.
Author: Dilip Biswal <dbiswal@us.ibm.com>
This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>
Closes#9036 from dilipbiswal/spark_8654_new.
This PR introduce a new feature to run SQL directly on files without create a table, for example:
```
select id from json.`path/to/json/files` as j
```
Author: Davies Liu <davies@databricks.com>
Closes#9173 from davies/source.
Find out the missing attributes by recursively looking
at the sort order expression and rest of the code
takes care of projecting them out.
Added description from cloud-fan
I wanna explain a bit more about this bug.
When we resolve sort ordering, we will use a special method, which only resolves UnresolvedAttributes and UnresolvedExtractValue. However, for something like Floor('a), even the 'a is resolved, the floor expression may still being unresolved as data type mismatch(for example, 'a is string type and Floor need double type), thus can't pass this filter, and we can't push down this missing attribute 'a
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#9123 from dilipbiswal/SPARK-10534.
Implement encode/decode for external row based on `ClassEncoder`.
TODO:
* code cleanup
* ~~fix corner cases~~
* refactor the encoder interface
* improve test for product codegen, to cover more corner cases.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9184 from cloud-fan/encoder.
Push conjunctive predicates though Aggregate operators when their references are a subset of the groupingExpressions.
Query plan before optimisation :-
Filter ((c#138L = 2) && (a#0 = 3))
Aggregate [a#0], [a#0,count(b#1) AS c#138L]
Project [a#0,b#1]
LocalRelation [a#0,b#1,c#2]
Query plan after optimisation :-
Filter (c#138L = 2)
Aggregate [a#0], [a#0,count(b#1) AS c#138L]
Filter (a#0 = 3)
Project [a#0,b#1]
LocalRelation [a#0,b#1,c#2]
Author: nitin goyal <nitin.goyal@guavus.com>
Author: nitin.goyal <nitin.goyal@guavus.com>
Closes#9167 from nitin2goyal/master.
This PR improve the performance by:
1) Generate an Iterator that take Iterator[CachedBatch] as input, and call accessors (unroll the loop for columns), avoid the expensive Iterator.flatMap.
2) Use Unsafe.getInt/getLong/getFloat/getDouble instead of ByteBuffer.getInt/getLong/getFloat/getDouble, the later one actually read byte by byte.
3) Remove the unnecessary copy() in Coalesce(), which is not related to memory cache, found during benchmark.
The following benchmark showed that we can speedup the columnar cache of int by 2x.
```
path = '/opt/tpcds/store_sales/'
int_cols = ['ss_sold_date_sk', 'ss_sold_time_sk', 'ss_item_sk','ss_customer_sk']
df = sqlContext.read.parquet(path).select(int_cols).cache()
df.count()
t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```
Author: Davies Liu <davies@databricks.com>
Closes#9145 from davies/byte_buffer.
Currently, we use CartesianProduct for join with null-safe-equal condition.
```
scala> sqlContext.sql("select * from t a join t b on (a.i <=> b.i)").explain
== Physical Plan ==
TungstenProject [i#2,j#3,i#7,j#8]
Filter (i#2 <=> i#7)
CartesianProduct
LocalTableScan [i#2,j#3], [[1,1]]
LocalTableScan [i#7,j#8], [[1,1]]
```
Actually, we can have an equal-join condition as `coalesce(i, default) = coalesce(b.i, default)`, then an partitioned join algorithm could be used.
After this PR, the plan will become:
```
>>> sqlContext.sql("select * from a join b ON a.id <=> b.id").explain()
TungstenProject [id#0L,id#1L]
Filter (id#0L <=> id#1L)
SortMergeJoin [coalesce(id#0L,0)], [coalesce(id#1L,0)]
TungstenSort [coalesce(id#0L,0) ASC], false, 0
TungstenExchange hashpartitioning(coalesce(id#0L,0),200)
ConvertToUnsafe
Scan PhysicalRDD[id#0L]
TungstenSort [coalesce(id#1L,0) ASC], false, 0
TungstenExchange hashpartitioning(coalesce(id#1L,0),200)
ConvertToUnsafe
Scan PhysicalRDD[id#1L]
```
Author: Davies Liu <davies@databricks.com>
Closes#9120 from davies/null_safe.
We can't parse `NOT` operator with comparison operations like `SELECT NOT TRUE > TRUE`, this PR fixed it.
Takes over https://github.com/apache/spark/pull/6326.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#8617 from cloud-fan/not.
The purpose of this PR is to keep the unsafe format detail only inside the unsafe class itself, so when we use them(like use unsafe array in unsafe map, use unsafe array and map in columnar cache), we don't need to understand the format before use them.
change list:
* unsafe array's 4-bytes numElements header is now required(was optional), and become a part of unsafe array format.
* w.r.t the previous changing, the `sizeInBytes` of unsafe array now counts the 4-bytes header.
* unsafe map's format was `[numElements] [key array numBytes] [key array content(without numElements header)] [value array content(without numElements header)]` before, which is a little hacky as it makes unsafe array's header optional. I think saving 4 bytes is not a big deal, so the format is now: `[key array numBytes] [unsafe key array] [unsafe value array]`.
* w.r.t the previous changing, the `sizeInBytes` of unsafe map now counts both map's header and array's header.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#9131 from cloud-fan/unsafe.
Actually all of the `UnaryMathExpression` doens't support the Decimal, will create follow ups for supporing it. This is the first PR which will be good to review the approach I am taking.
Author: Cheng Hao <hao.cheng@intel.com>
Closes#9086 from chenghao-intel/ceiling.
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.
Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>
Closes#8453 from cloud-fan/table-name.
This is a first draft of the ability to construct expressions that will take a catalyst internal row and construct a Product (case class or tuple) that has fields with the correct names. Support include:
- Nested classes
- Maps
- Efficiently handling of arrays of primitive types
Not yet supported:
- Case classes that require custom collection types (i.e. List instead of Seq).
Author: Michael Armbrust <michael@databricks.com>
Closes#9100 from marmbrus/productContructor.
This PR is a first cut at code generating an encoder that takes a Scala `Product` type and converts it directly into the tungsten binary format. This is done through the addition of a new set of expression that can be used to invoke methods on raw JVM objects, extracting fields and converting the result into the required format. These can then be used directly in an `UnsafeProjection` allowing us to leverage the existing encoding logic.
According to some simple benchmarks, this can significantly speed up conversion (~4x). However, replacing CatalystConverters is deferred to a later PR to keep this PR at a reasonable size.
```scala
case class SomeInts(a: Int, b: Int, c: Int, d: Int, e: Int)
val data = SomeInts(1, 2, 3, 4, 5)
val encoder = ProductEncoder[SomeInts]
val converter = CatalystTypeConverters.createToCatalystConverter(ScalaReflection.schemaFor[SomeInts].dataType)
(1 to 5).foreach {iter =>
benchmark(s"converter $iter") {
var i = 100000000
while (i > 0) {
val res = converter(data).asInstanceOf[InternalRow]
assert(res.getInt(0) == 1)
assert(res.getInt(1) == 2)
i -= 1
}
}
benchmark(s"encoder $iter") {
var i = 100000000
while (i > 0) {
val res = encoder.toRow(data)
assert(res.getInt(0) == 1)
assert(res.getInt(1) == 2)
i -= 1
}
}
}
```
Results:
```
[info] converter 1: 7170ms
[info] encoder 1: 1888ms
[info] converter 2: 6763ms
[info] encoder 2: 1824ms
[info] converter 3: 6912ms
[info] encoder 3: 1802ms
[info] converter 4: 7131ms
[info] encoder 4: 1798ms
[info] converter 5: 7350ms
[info] encoder 5: 1912ms
```
Author: Michael Armbrust <michael@databricks.com>
Closes#9019 from marmbrus/productEncoder.
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.
The fix is to not generate such a cast if the lhs type is a NullType instead
we translate the expression to Literal(Null).
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#8983 from dilipbiswal/spark_8654.
The created decimal is wrong if using `Decimal(unscaled, precision, scale)` with unscaled > 1e18 and and precision > 18 and scale > 0.
This bug exists since the beginning.
Author: Davies Liu <davies@databricks.com>
Closes#9014 from davies/fix_decimal.
This patch refactors several of the Aggregate2 interfaces in order to improve code clarity.
The biggest change is a refactoring of the `AggregateFunction2` class hierarchy. In the old code, we had a class named `AlgebraicAggregate` that inherited from `AggregateFunction2`, added a new set of methods, then banned the use of the inherited methods. I found this to be fairly confusing because.
If you look carefully at the existing code, you'll see that subclasses of `AggregateFunction2` fall into two disjoint categories: imperative aggregation functions which directly extended `AggregateFunction2` and declarative, expression-based aggregate functions which extended `AlgebraicAggregate`. In order to make this more explicit, this patch refactors things so that `AggregateFunction2` is a sealed abstract class with two subclasses, `ImperativeAggregateFunction` and `ExpressionAggregateFunction`. The superclass, `AggregateFunction2`, now only contains methods and fields that are common to both subclasses.
After making this change, I updated the various AggregationIterator classes to comply with this new naming scheme. I also performed several small renamings in the aggregate interfaces themselves in order to improve clarity and rewrote or expanded a number of comments.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8973 from JoshRosen/tungsten-agg-comments.
This PR is a completely rewritten of GenerateUnsafeProjection, to accomplish the goal of copying data only once. The old code of GenerateUnsafeProjection is still there to reduce review difficulty.
Instead of creating unsafe conversion code for struct, array and map, we create code of writing the content to the global row buffer.
Author: Wenchen Fan <cloud0fan@163.com>
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#8747 from cloud-fan/copy-once.
Floor & Ceiling function should returns Long type, rather than Double.
Verified with MySQL & Hive.
Author: Cheng Hao <hao.cheng@intel.com>
Closes#8933 from chenghao-intel/ceiling.
This is an implementation of Hive's `json_tuple` function using Jackson Streaming.
Author: Nathan Howell <nhowell@godaddy.com>
Closes#7946 from NathanHowell/SPARK-9617.
This PR implements a HyperLogLog based Approximate Count Distinct function using the new UDAF interface.
The implementation is inspired by the ClearSpring HyperLogLog implementation and should produce the same results.
There is still some documentation and testing left to do.
cc yhuai
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#8362 from hvanhovell/SPARK-9741.
Intersect and Except are both set operators and they use the all the columns to compare equality between rows. When pushing their Project parent down, the relations they based on would change, therefore not an equivalent transformation.
JIRA: https://issues.apache.org/jira/browse/SPARK-10539
I added some comments based on the fix of https://github.com/apache/spark/pull/8742.
Author: Yijie Shen <henry.yijieshen@gmail.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#8823 from yhuai/fix_set_optimization.
This fixes https://issues.apache.org/jira/browse/SPARK-9794 by using a real ISO8601 parser. (courtesy of the xml component of the standard java library)
cc: angelini
Author: Kevin Cox <kevincox@kevincox.ca>
Closes#8396 from kevincox/kevincox-sql-time-parsing.
Sometimes we can't push down the whole `Project` though `Sort`, but we still have a chance to push down part of it.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#8644 from cloud-fan/column-prune.
Use these in the optimizer as well:
A and (not(A) or B) => A and B
not(A and B) => not(A) or not(B)
not(A or B) => not(A) and not(B)
Author: Yash Datta <Yash.Datta@guavus.com>
Closes#5700 from saucam/bool_simp.
This PR takes over https://github.com/apache/spark/pull/8389.
This PR improves `checkAnswer` to print the partially analyzed plan in addition to the user friendly error message, in order to aid debugging failing tests.
In doing so, I ran into a conflict with the various ways that we bring a SQLContext into the tests. Depending on the trait we refer to the current context as `sqlContext`, `_sqlContext`, `ctx` or `hiveContext` with access modifiers `public`, `protected` and `private` depending on the defining class.
I propose we refactor as follows:
1. All tests should only refer to a `protected sqlContext` when testing general features, and `protected hiveContext` when it is a method that only exists on a `HiveContext`.
2. All tests should only import `testImplicits._` (i.e., don't import `TestHive.implicits._`)
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#8584 from cloud-fan/cleanupTests.
After this PR, In/InSet/ArrayContain will return null if value is null, instead of false. They also will return null even if there is a null in the set/array.
Author: Davies Liu <davies@databricks.com>
Closes#8492 from davies/fix_in.
In BigDecimal or java.math.BigDecimal, the precision could be smaller than scale, for example, BigDecimal("0.001") has precision = 1 and scale = 3. But DecimalType require that the precision should be larger than scale, so we should use the maximum of precision and scale when inferring the schema from decimal literal.
Author: Davies Liu <davies@databricks.com>
Closes#8428 from davies/smaller_decimal.
We misunderstood the Julian days and nanoseconds of the day in parquet (as TimestampType) from Hive/Impala, they are overlapped, so can't be added together directly.
In order to avoid the confusing rounding when do the converting, we use `2440588` as the Julian Day of epoch of unix timestamp (which should be 2440587.5).
Author: Davies Liu <davies@databricks.com>
Author: Cheng Lian <lian@databricks.com>
Closes#8400 from davies/timestamp_parquet.
This patch adds an analyzer rule to ensure that set operations (union, intersect, and except) are only applied to tables with the same number of columns. Without this rule, there are scenarios where invalid queries can return incorrect results instead of failing with error messages; SPARK-9813 provides one example of this problem. In other cases, the invalid query can crash at runtime with extremely confusing exceptions.
I also performed a bit of cleanup to refactor some of those logical operators' code into a common `SetOperation` base class.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7631 from JoshRosen/SPARK-9293.
This adds a missing null check to the Decimal `toScala` converter in `CatalystTypeConverters`, fixing an NPE.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8401 from JoshRosen/SPARK-10190.
A few minor changes:
1. Improved documentation
2. Rename apply(distinct....) to distinct.
3. Changed MutableAggregationBuffer from a trait to an abstract class.
4. Renamed returnDataType to dataType to be more consistent with other expressions.
And unrelated to UDAFs:
1. Renamed file names in expressions to use suffix "Expressions" to be more consistent.
2. Moved regexp related expressions out to its own file.
3. Renamed StringComparison => StringPredicate.
Author: Reynold Xin <rxin@databricks.com>
Closes#8321 from rxin/SPARK-9242.
create t1 (a decimal(7, 2), b long);
select case when 1=1 then a else 1.0 end from t1;
select case when 1=1 then a else b end from t1;
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#8270 from adrian-wang/casewhenfractional.
We should rounding the result of multiply/division of decimal to expected precision/scale, also check overflow.
Author: Davies Liu <davies@databricks.com>
Closes#8287 from davies/decimal_division.
The type for array of array in Java is slightly different than array of others.
cc cloud-fan
Author: Davies Liu <davies@databricks.com>
Closes#8250 from davies/array_binary.
JIRA: https://issues.apache.org/jira/browse/SPARK-9526
This PR is a follow up of #7830, aiming at utilizing randomized tests to reveal more potential bugs in sql expression.
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7855 from yjshen/property_check.
We should skip unresolved `LogicalPlan`s for `PullOutNondeterministic`, as calling `output` on unresolved `LogicalPlan` will produce confusing error message.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#8203 from cloud-fan/error-msg and squashes the following commits:
1c67ca7 [Wenchen Fan] move test
7593080 [Wenchen Fan] correct error message for aggregate
Also alias the ExtractValue instead of wrapping it with UnresolvedAlias when resolve attribute in LogicalPlan, as this alias will be trimmed if it's unnecessary.
Based on #7957 without the changes to mllib, but instead maintaining earlier behavior when using `withColumn` on expressions that already have metadata.
Author: Wenchen Fan <cloud0fan@outlook.com>
Author: Michael Armbrust <michael@databricks.com>
Closes#8215 from marmbrus/pr/7957.
A fundamental limitation of the existing SQL tests is that *there is simply no way to create your own `SparkContext`*. This is a serious limitation because the user may wish to use a different master or config. As a case in point, `BroadcastJoinSuite` is entirely commented out because there is no way to make it pass with the existing infrastructure.
This patch removes the singletons `TestSQLContext` and `TestData`, and instead introduces a `SharedSQLContext` that starts a context per suite. Unfortunately the singletons were so ingrained in the SQL tests that this patch necessarily needed to touch *all* the SQL test files.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/8111)
<!-- Reviewable:end -->
Author: Andrew Or <andrew@databricks.com>
Closes#8111 from andrewor14/sql-tests-refactor.
PR #7967 enables us to save data source relations to metastore in Hive compatible format when possible. But it fails to persist Parquet relations with decimal column(s) to Hive metastore of versions lower than 1.2.0. This is because `ParquetHiveSerDe` in Hive versions prior to 1.2.0 doesn't support decimal. This PR checks for this case and falls back to Spark SQL specific metastore table format.
Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Lian <lian@databricks.com>
Closes#8130 from liancheng/spark-9757/old-hive-parquet-decimal.
HashPartitioning compatibility is currently defined w.r.t the _set_ of expressions, but the ordering of those expressions matters when computing hash codes; this could lead to incorrect answers if we mistakenly avoided a shuffle based on the assumption that HashPartitionings with the same expressions in different orders will produce equivalent row hashcodes. The first commit adds a regression test which illustrates this problem.
The fix for this is simple: make `HashPartitioning.compatibleWith` and `HashPartitioning.guarantees` sensitive to the expression ordering (i.e. do not perform set comparison).
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8074 from JoshRosen/hashpartitioning-compatiblewith-fixes and squashes the following commits:
b61412f [Josh Rosen] Demonstrate that I haven't cheated in my fix
0b4d7d9 [Josh Rosen] Update so that clusteringSet is only used in satisfies().
dc9c9d7 [Josh Rosen] Add failing regression test for SPARK-9785
PlatformDependent.UNSAFE is way too verbose.
Author: Reynold Xin <rxin@databricks.com>
Closes#8094 from rxin/SPARK-9815 and squashes the following commits:
229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
Currently, generated UnsafeProjection can reach 64k byte code limit of Java. This patch will split the generated expressions into multiple functions, to avoid the limitation.
After this patch, we can work well with table that have up to 64k columns (hit max number of constants limit in Java), it should be enough in practice.
cc rxin
Author: Davies Liu <davies@databricks.com>
Closes#8044 from davies/wider_table and squashes the following commits:
9192e6c [Davies Liu] fix generated safe projection
d1ef81a [Davies Liu] fix failed tests
737b3d3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
ffcd132 [Davies Liu] address comments
1b95be4 [Davies Liu] put the generated class into sql package
77ed72d [Davies Liu] address comments
4518e17 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
75ccd01 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
495e932 [Davies Liu] support wider table with more than 1k columns for generated projections
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#8057 from yjshen/explode_star and squashes the following commits:
eae181d [Yijie Shen] change explaination message
54c9d11 [Yijie Shen] meaning message for * in explode
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#8025 from cloud-fan/analysis and squashes the following commits:
51461b1 [Wenchen Fan] move test file to test folder
ec88ace [Wenchen Fan] Improve Analysis Unit test framework
This re-applies #7955, which was reverted due to a race condition to fix build breaking.
Author: Wenchen Fan <cloud0fan@outlook.com>
Author: Reynold Xin <rxin@databricks.com>
Closes#8002 from rxin/InternalRow-toSeq and squashes the following commits:
332416a [Reynold Xin] Merge pull request #7955 from cloud-fan/toSeq
21665e2 [Wenchen Fan] fix hive again...
4addf29 [Wenchen Fan] fix hive
bc16c59 [Wenchen Fan] minor fix
33d802c [Wenchen Fan] pass data type info to InternalRow.toSeq
3dd033e [Wenchen Fan] move the default special getters implementation from InternalRow to BaseGenericInternalRow
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7955 from cloud-fan/toSeq and squashes the following commits:
21665e2 [Wenchen Fan] fix hive again...
4addf29 [Wenchen Fan] fix hive
bc16c59 [Wenchen Fan] minor fix
33d802c [Wenchen Fan] pass data type info to InternalRow.toSeq
3dd033e [Wenchen Fan] move the default special getters implementation from InternalRow to BaseGenericInternalRow
In order to support update a varlength (actually fixed length) object, the space should be preserved even it's null. And, we can't call setNullAt(i) for it anymore, we because setNullAt(i) will remove the offset of the preserved space, should call setDecimal(i, null, precision) instead.
After this, we can do hash based aggregation on DecimalType with precision > 18. In a tests, this could decrease the end-to-end run time of aggregation query from 37 seconds (sort based) to 24 seconds (hash based).
cc rxin
Author: Davies Liu <davies@databricks.com>
Closes#7978 from davies/update_decimal and squashes the following commits:
bed8100 [Davies Liu] isSettable -> isMutable
923c9eb [Davies Liu] address comments and fix bug
385891d [Davies Liu] Merge branch 'master' of github.com:apache/spark into update_decimal
36a1872 [Davies Liu] fix tests
cd6c524 [Davies Liu] support set decimal with precision > 18
![translate](http://www.w3resource.com/PostgreSQL/postgresql-translate-function.png)
Author: zhichao.li <zhichao.li@intel.com>
Closes#7709 from zhichao-li/translate and squashes the following commits:
9418088 [zhichao.li] refine checking condition
f2ab77a [zhichao.li] clone string
9d88f2d [zhichao.li] fix indent
6aa2962 [zhichao.li] style
e575ead [zhichao.li] add python api
9d4bab0 [zhichao.li] add special case for fodable and refactor unittest
eda7ad6 [zhichao.li] update to use TernaryExpression
cdfd4be [zhichao.li] add function translate
This patches renames `RowOrdering` to `InterpretedOrdering` and updates SortMergeJoin to use the `SparkPlan` methods for constructing its ordering so that it may benefit from codegen.
This is an updated version of #7408.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7973 from JoshRosen/SPARK-9054 and squashes the following commits:
e610655 [Josh Rosen] Add comment RE: Ascending ordering
34b8e0c [Josh Rosen] Import ordering
be19a0f [Josh Rosen] [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in more places.
This continues tarekauel's work in #7778.
Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes#7893 from viirya/codegen_in and squashes the following commits:
81ff97b [Liang-Chi Hsieh] For comments.
47761c6 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
cf4bf41 [Liang-Chi Hsieh] For comments.
f532b3c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
446bbcd [Liang-Chi Hsieh] Fix bug.
b3d0ab4 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
4610eff [Liang-Chi Hsieh] Relax the types of references and update optimizer test.
224f18e [Liang-Chi Hsieh] Beef up the test cases for In and InSet to include all primitive data types.
86dc8aa [Liang-Chi Hsieh] Only convert In to InSet when the number of items in set is more than the threshold.
b7ded7e [Tarek Auel] [SPARK-9403][SQL] codeGen in / inSet
This is a follow-up of https://github.com/apache/spark/pull/7920 to fix comments.
Author: Yin Huai <yhuai@databricks.com>
Closes#7964 from yhuai/SPARK-9141-follow-up and squashes the following commits:
4d0ee80 [Yin Huai] Fix comments.
Currently we collapse successive projections that are added by `withColumn`. However, this optimization violates the constraint that adding nodes to a plan will never change its analyzed form and thus breaks caching. Instead of doing early optimization, in this PR I just fix some low-hanging slowness in the analyzer. In particular, I add a mechanism for skipping already analyzed subplans, `resolveOperators` and `resolveExpression`. Since trees are generally immutable after construction, it's safe to annotate a plan as already analyzed as any transformation will create a new tree with this bit no longer set.
Together these result in a faster analyzer than before, even with added timing instrumentation.
```
Original Code
[info] 3430ms
[info] 2205ms
[info] 1973ms
[info] 1982ms
[info] 1916ms
Without Project Collapsing in DataFrame
[info] 44610ms
[info] 45977ms
[info] 46423ms
[info] 46306ms
[info] 54723ms
With analyzer optimizations
[info] 6394ms
[info] 4630ms
[info] 4388ms
[info] 4093ms
[info] 4113ms
With resolveOperators
[info] 2495ms
[info] 1380ms
[info] 1685ms
[info] 1414ms
[info] 1240ms
```
Author: Michael Armbrust <michael@databricks.com>
Closes#7920 from marmbrus/withColumnCache and squashes the following commits:
2145031 [Michael Armbrust] fix hive udfs tests
5a5a525 [Michael Armbrust] remove wrong comment
7a507d5 [Michael Armbrust] style
b59d710 [Michael Armbrust] revert small change
1fa5949 [Michael Armbrust] move logic into LogicalPlan, add tests
0e2cb43 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into withColumnCache
c926e24 [Michael Armbrust] naming
e593a2d [Michael Armbrust] style
f5a929e [Michael Armbrust] [SPARK-9141][SQL] Remove project collapsing from DataFrame API
38b1c83 [Michael Armbrust] WIP
Let Decimal carry the correct precision and scale with DecimalType.
cc rxin yhuai
Author: Davies Liu <davies@databricks.com>
Closes#7925 from davies/decimal_scale and squashes the following commits:
e19701a [Davies Liu] some tweaks
57d78d2 [Davies Liu] fix tests
5d5bc69 [Davies Liu] match precision and scale with DecimalType
This PR is based on #7580 , thanks to EntilZha
PR for work on https://issues.apache.org/jira/browse/SPARK-8231
Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128
Main points are:
1. If the array is empty, null, or the value is null, return false
2. If there is a type mismatch, throw error
3. If comparison is not supported, throw error
Closes#7580
Author: Pedro Rodriguez <prodriguez@trulia.com>
Author: Pedro Rodriguez <ski.rodriguez@gmail.com>
Author: Davies Liu <davies@databricks.com>
Closes#7949 from davies/array_contains and squashes the following commits:
d3c08bc [Davies Liu] use foreach() to avoid copy
bc3d1fe [Davies Liu] fix array_contains
719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains
e352cf9 [Pedro Rodriguez] fixed diff from master
4d5b0ff [Pedro Rodriguez] added docs and another type check
ffc0591 [Pedro Rodriguez] fixed unit test
7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3
b5ffae8 [Pedro Rodriguez] fixed pyspark test
4e7dce3 [Pedro Rodriguez] added more docs
3082399 [Pedro Rodriguez] fixed unit test
46f9789 [Pedro Rodriguez] reverted change
d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this
8528027 [Pedro Rodriguez] added more tests
686e029 [Pedro Rodriguez] fix scala style
d262e9d [Pedro Rodriguez] reworked type checking code and added more tests
2517a58 [Pedro Rodriguez] removed unused import
28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests
12f8795 [Pedro Rodriguez] fix scala style checks
e8a20a9 [Pedro Rodriguez] added python df (broken atm)
65b562c [Pedro Rodriguez] made array_contains nullable false
33b45aa [Pedro Rodriguez] reordered test
9623c64 [Pedro Rodriguez] fixed test
4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs
72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs
69c46fb [Pedro Rodriguez] added tests and codegen
9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
JIRA: https://issues.apache.org/jira/browse/SPARK-9432
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7933 from yjshen/numeric_ranges and squashes the following commits:
e719f78 [Yijie Shen] proper integral range check
The analysis rule has a bug and we ended up making the sorter still capable of doing evaluation, so lets revert this for now.
Author: Michael Armbrust <michael@databricks.com>
Closes#7906 from marmbrus/revertSortProjection and squashes the following commits:
2da6972 [Michael Armbrust] unrevert unrelated changes
4f2b00c [Michael Armbrust] Revert "[SPARK-9251][SQL] do not order by expressions which still need evaluation"
This is based on #7485 , thanks to NathanHowell
Tests were copied from Hive, but do not seem to be super comprehensive. I've generally replicated Hive's unusual behavior rather than following a JSONPath reference, except for one case (as noted in the comments). I don't know if there is a way of fully replicating Hive's behavior without a slower TreeNode implementation, so I've erred on the side of performance instead.
Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Nathan Howell <nhowell@godaddy.com>
Closes#7901 from davies/get_json_object and squashes the following commits:
3ace9b9 [Davies Liu] Merge branch 'get_json_object' of github.com:davies/spark into get_json_object
98766fc [Davies Liu] Merge branch 'master' of github.com:apache/spark into get_json_object
a7dc6d0 [Davies Liu] Update JsonExpressionsSuite.scala
c818519 [Yin Huai] new results.
18ce26b [Davies Liu] fix tests
6ac29fb [Yin Huai] Golden files.
25eebef [Davies Liu] use HiveQuerySuite
e0ac6ec [Yin Huai] Golden answer files.
940c060 [Davies Liu] tweat code style
44084c5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into get_json_object
9192d09 [Nathan Howell] Match Hive’s behavior for unwrapping arrays of one element
8dab647 [Nathan Howell] [SPARK-8246] [SQL] Implement get_json_object
This PR is based on #7186 (just fix the conflict), thanks to tarekauel .
find_in_set(string str, string strList): int
Returns the first occurance of str in strList where strList is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument contains any commas. For example, find_in_set('ab', 'abc,b,ab,c,def') returns 3.
Only add this to SQL, not DataFrame.
Closes#7186
Author: Tarek Auel <tarek.auel@googlemail.com>
Author: Davies Liu <davies@databricks.com>
Closes#7900 from davies/find_in_set and squashes the following commits:
4334209 [Davies Liu] Merge branch 'master' of github.com:apache/spark into find_in_set
8f00572 [Davies Liu] Merge branch 'master' of github.com:apache/spark into find_in_set
243ede4 [Tarek Auel] [SPARK-8244][SQL] hive compatibility
1aaf64e [Tarek Auel] [SPARK-8244][SQL] unit test fix
e4093a4 [Tarek Auel] [SPARK-8244][SQL] final modifier for COMMA_UTF8
0d05df5 [Tarek Auel] Merge branch 'master' into SPARK-8244
208d710 [Tarek Auel] [SPARK-8244] address comments & bug fix
71b2e69 [Tarek Auel] [SPARK-8244] find_in_set
66c7fda [Tarek Auel] Merge branch 'master' into SPARK-8244
61b8ca2 [Tarek Auel] [SPARK-8224] removed loop and split; use unsafe String comparison
4f75a65 [Tarek Auel] Merge branch 'master' into SPARK-8244
e3b20c8 [Tarek Auel] [SPARK-8244] added type check
1c2bbb7 [Tarek Auel] [SPARK-8244] findInSet
Currently, when copy the bitsets, we didn't consider that the row1 may not sit in the beginning of byte array.
cc rxin
Author: Davies Liu <davies@databricks.com>
Closes#7892 from davies/clean_join and squashes the following commits:
14cce9e [Davies Liu] cleanup generated UnsafeRowJoiner and fix bug
JIRA: https://issues.apache.org/jira/browse/SPARK-9549
This PR fix the following bugs:
1. `UnaryMinus`'s codegen version would fail to compile when the input is `Long.MinValue`
2. `BinaryComparison` would fail to compile in codegen mode when comparing Boolean types.
3. `AddMonth` would fail if passed a huge negative month, which would lead accessing negative index of `monthDays` array.
4. `Nanvl` with different type operands.
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7882 from yjshen/minor_bug_fix and squashes the following commits:
41bbd2c [Yijie Shen] fix bug in Nanvl type coercion
3dee204 [Yijie Shen] address comments
4fa5de0 [Yijie Shen] fix bugs in expressions
This PR adds a UnsafeArrayData, current we encode it in this way:
first 4 bytes is the # elements
then each 4 byte is the start offset of the element, unless it is negative, in which case the element is null.
followed by the elements themselves
an example: [10, 11, 12, 13, null, 14] will be encoded as:
5, 28, 32, 36, 40, -44, 44, 10, 11, 12, 13, 14
Note that, when we read a UnsafeArrayData from bytes, we can read the first 4 bytes as numElements and take the rest(first 4 bytes skipped) as value region.
unsafe map data just use 2 unsafe array data, first 4 bytes is # of elements, second 4 bytes is numBytes of key array, the follows key array data and value array data.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7752 from cloud-fan/unsafe-array and squashes the following commits:
3269bd7 [Wenchen Fan] fix a bug
6445289 [Wenchen Fan] add unit tests
49adf26 [Wenchen Fan] add unsafe map
20d1039 [Wenchen Fan] add comments and unsafe converter
821b8db [Wenchen Fan] add unsafe array
This PR adds an optimization rule, `FilterNullsInJoinKey`, to add `Filter` before join operators to filter out rows having null values for join keys.
This optimization is guarded by a new SQL conf, `spark.sql.advancedOptimization`.
The code in this PR was authored by yhuai; I'm opening this PR to factor out this change from #7685, a larger pull request which contains two other optimizations.
Author: Yin Huai <yhuai@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7768 from JoshRosen/filter-nulls-in-join-key and squashes the following commits:
c02fc3f [Yin Huai] Address Josh's comments.
0a8e096 [Yin Huai] Update comments.
ea7d5a6 [Yin Huai] Make sure we do not keep adding filters.
be88760 [Yin Huai] Make it clear that FilterNullsInJoinKeySuite.scala is used to test FilterNullsInJoinKey.
8bb39ad [Yin Huai] Fix non-deterministic tests.
303236b [Josh Rosen] Revert changes that are unrelated to null join key filtering
40eeece [Josh Rosen] Merge remote-tracking branch 'origin/master' into filter-nulls-in-join-key
c57a954 [Yin Huai] Bug fix.
d3d2e64 [Yin Huai] First round of cleanup.
f9516b0 [Yin Huai] Style
c6667e7 [Yin Huai] Add PartitioningCollection.
e616d3b [Yin Huai] wip
7c2d2d8 [Yin Huai] Bug fix and refactoring.
69bb072 [Yin Huai] Introduce NullSafeHashPartitioning and NullUnsafePartitioning.
d5b84c3 [Yin Huai] Do not add unnessary filters.
2201129 [Yin Huai] Filter out rows that will not be joined in equal joins early.
This PR adds `PartitioningCollection`, which is used to represent the `outputPartitioning` for SparkPlans with multiple children (e.g. `ShuffledHashJoin`). So, a `SparkPlan` can have multiple descriptions of its partitioning schemes. Taking `ShuffledHashJoin` as an example, it has two descriptions of its partitioning schemes, i.e. `left.outputPartitioning` and `right.outputPartitioning`. So when we have a query like `select * from t1 join t2 on (t1.x = t2.x) join t3 on (t2.x = t3.x)` will only have three Exchange operators (when shuffled joins are needed) instead of four.
The code in this PR was authored by yhuai; I'm opening this PR to factor out this change from #7685, a larger pull request which contains two other optimizations.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7773)
<!-- Reviewable:end -->
Author: Yin Huai <yhuai@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7773 from JoshRosen/multi-way-join-planning-improvements and squashes the following commits:
5c45924 [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements
cd8269b [Josh Rosen] Refactor test to use SQLTestUtils
2963857 [Yin Huai] Revert unnecessary SqlConf change.
73913f7 [Yin Huai] Add comments and test. Also, revert the change in ShuffledHashOuterJoin for now.
4a99204 [Josh Rosen] Delete unrelated expression change
884ab95 [Josh Rosen] Carve out only SPARK-2205 changes.
247e5fa [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements
c57a954 [Yin Huai] Bug fix.
d3d2e64 [Yin Huai] First round of cleanup.
f9516b0 [Yin Huai] Style
c6667e7 [Yin Huai] Add PartitioningCollection.
e616d3b [Yin Huai] wip
7c2d2d8 [Yin Huai] Bug fix and refactoring.
69bb072 [Yin Huai] Introduce NullSafeHashPartitioning and NullUnsafePartitioning.
d5b84c3 [Yin Huai] Do not add unnessary filters.
2201129 [Yin Huai] Filter out rows that will not be joined in equal joins early.
This pull request creates two isOrderable functions in RowOrdering that can be used to check whether a data type or a sequence of expressions can be used in sorting.
Author: Reynold Xin <rxin@databricks.com>
Closes#7880 from rxin/SPARK-9546 and squashes the following commits:
f9e322d [Reynold Xin] Fixed tests.
0439b43 [Reynold Xin] [SPARK-9546][SQL] Centralize orderable data type checking.
The detailed approach is documented in UnsafeKVExternalSorterSuite.testKVSorter(), working as follows:
1. Create input by generating data randomly based on the given key/value schema (which is also randomly drawn from a list of candidate types)
2. Run UnsafeKVExternalSorter on the generated data
3. Collect the output from the sorter, and make sure the keys are sorted in ascending order
4. Sort the input by both key and value, and sort the sorter output also by both key and value. Compare the sorted input and sorted output together to make sure all the key/values match.
5. Check memory allocation to make sure there is no memory leak.
There is also a spill flag. When set to true, the sorter will spill probabilistically roughly every 100 records.
Author: Reynold Xin <rxin@databricks.com>
Closes#7873 from rxin/kvsorter-randomized-test and squashes the following commits:
a08c251 [Reynold Xin] Resource cleanup.
0488b5c [Reynold Xin] [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter.
Generate prefix for DecimalType, fix the random generator of decimal
cc JoshRosen
Author: Davies Liu <davies@databricks.com>
Closes#7857 from davies/sort_decimal and squashes the following commits:
2433959 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sort_decimal
de24253 [Davies Liu] fix style
0a54c1a [Davies Liu] sort decimal
When accessing a column in UnsafeRow, it's good to avoid the copy, then we should do deep copy when turn the UnsafeRow into generic Row, this PR brings generated FromUnsafeProjection to do that.
This PR also fix the expressions that cache the UTF8String, which should also copy it.
Author: Davies Liu <davies@databricks.com>
Closes#7840 from davies/avoid_copy and squashes the following commits:
230c8a1 [Davies Liu] address comment
fd797c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into avoid_copy
e095dd0 [Davies Liu] rollback rename
8ef5b0b [Davies Liu] copy String in Columnar
81360b8 [Davies Liu] fix class name
9aecb88 [Davies Liu] use FromUnsafeProjection to do deep copy for UTF8String and struct
This PR is based on #7643 , thanks to adrian-wang
Author: Davies Liu <davies@databricks.com>
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#7847 from davies/datediff and squashes the following commits:
74333d7 [Davies Liu] fix bug
22d8a8c [Davies Liu] optimize
85cdd21 [Davies Liu] remove unnecessary tests
241d90c [Davies Liu] Merge branch 'master' of github.com:apache/spark into datediff
e9dc0f5 [Davies Liu] fix datediff/to_utc_timestamp/from_utc_timestamp
c360447 [Daoyuan Wang] function datediff, to_utc_timestamp, from_utc_timestamp (commits merged)
This PR is based on #7208 , thanks to HuJiayin
Closes#7208
Author: HuJiayin <jiayin.hu@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#7850 from davies/initcap and squashes the following commits:
54472e9 [Davies Liu] fix python test
17ffe51 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
ca46390 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
3a906e4 [Davies Liu] implement title case in UTF8String
8b2506a [HuJiayin] Update functions.py
2cd43e5 [HuJiayin] fix python style check
b616c0e [HuJiayin] add python api
1f5a0ef [HuJiayin] add codegen
7e0c604 [HuJiayin] Merge branch 'master' of https://github.com/apache/spark into initcap
6a0b958 [HuJiayin] add column
c79482d [HuJiayin] support soundex
7ce416b [HuJiayin] support initcap rebase code
This is based on #7641, thanks to zhichao-li
Closes#7641
Author: zhichao.li <zhichao.li@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#7848 from davies/substr and squashes the following commits:
461b709 [Davies Liu] remove bytearry from tests
b45377a [Davies Liu] Merge branch 'master' of github.com:apache/spark into substr
01d795e [zhichao.li] scala style
99aa130 [zhichao.li] add substring to dataframe
4f68bfe [zhichao.li] add binary type support for substring
This PR is based on #7581 , just fix the conflict.
Author: Cheng Hao <hao.cheng@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#7851 from davies/sort_array and squashes the following commits:
a80ef66 [Davies Liu] fix conflict
7cfda65 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sort_array
664c960 [Cheng Hao] update the sort_array by using the ArrayData
276d2d5 [Cheng Hao] add empty line
0edab9c [Cheng Hao] Add asending/descending support for sort_array
80fc0f8 [Cheng Hao] Add type checking
a42b678 [Cheng Hao] Add sort_array support
This PR adds a `MapData` as internal representation of map type in Spark SQL, and provides a default implementation with just 2 `ArrayData`.
After that, we have specialized getters for all internal type, so I removed generic getter in `ArrayData` and added specialized `toArray` for it.
Also did some refactor and cleanup for `InternalRow` and its subclasses.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7799 from cloud-fan/map-data and squashes the following commits:
77d482f [Wenchen Fan] fix python
e8f6682 [Wenchen Fan] skip MapData equality check in HiveInspectorSuite
40cc9db [Wenchen Fan] add toString
6e06ec9 [Wenchen Fan] some more cleanup
a90aca1 [Wenchen Fan] add MapData
Add expression `sort_array` support.
Author: Cheng Hao <hao.cheng@intel.com>
This patch had conflicts when merged, resolved by
Committer: Davies Liu <davies.liu@gmail.com>
Closes#7581 from chenghao-intel/sort_array and squashes the following commits:
664c960 [Cheng Hao] update the sort_array by using the ArrayData
276d2d5 [Cheng Hao] add empty line
0edab9c [Cheng Hao] Add asending/descending support for sort_array
80fc0f8 [Cheng Hao] Add type checking
a42b678 [Cheng Hao] Add sort_array support
JIRA: https://issues.apache.org/jira/browse/SPARK-9415
Following up #7787. We shouldn't use MapType as grouping keys and join keys too.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#7819 from viirya/map_join_groupby and squashes the following commits:
005ee0c [Liang-Chi Hsieh] For comments.
7463398 [Liang-Chi Hsieh] MapType can't be used as join keys, grouping keys.
This PR is based on #7533 , thanks to zhichao-li
Closes#7533
Author: zhichao.li <zhichao.li@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#7843 from davies/str_index and squashes the following commits:
391347b [Davies Liu] add python api
3ce7802 [Davies Liu] fix substringIndex
f2d29a1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into str_index
515519b [zhichao.li] add foldable and remove null checking
9546991 [zhichao.li] scala style
67c253a [zhichao.li] hide some apis and clean code
b19b013 [zhichao.li] add codegen and clean code
ac863e9 [zhichao.li] reduce the calling of numChars
12e108f [zhichao.li] refine unittest
d92951b [zhichao.li] add lastIndexOf
52d7b03 [zhichao.li] add substring_index function
This patch creates a code generated unsafe row concatenator that can be used to concatenate/join two UnsafeRows into a single UnsafeRow.
Since it is inherently hard to test these low level stuff, the test suites employ randomized testing heavily in order to guarantee correctness.
Author: Reynold Xin <rxin@databricks.com>
Closes#7821 from rxin/rowconcat and squashes the following commits:
8717f35 [Reynold Xin] Rebase and code review.
72c5d8e [Reynold Xin] Fixed a bug.
a84ed2e [Reynold Xin] Fixed offset.
40c3fb2 [Reynold Xin] Reset random data generator.
f0913aa [Reynold Xin] Test fixes.
6687b6f [Reynold Xin] Updated documentation.
00354b9 [Reynold Xin] Support concat data as well.
e9a4347 [Reynold Xin] Updated.
6269f96 [Reynold Xin] Fixed a bug .
0f89716 [Reynold Xin] [SPARK-9358][SQL][WIP] Code generation for UnsafeRow concat.
This patch adds support for entries larger than the default page size in BytesToBytesMap. These large rows are handled by allocating special overflow pages to hold individual entries.
In addition, this patch integrates BytesToBytesMap with the ShuffleMemoryManager:
- Move BytesToBytesMap from `unsafe` to `core` so that it can import `ShuffleMemoryManager`.
- Before allocating new data pages, ask the ShuffleMemoryManager to reserve the memory:
- `putNewKey()` now returns a boolean to indicate whether the insert succeeded or failed due to a lack of memory. The caller can use this value to respond to the memory pressure (e.g. by spilling).
- `UnsafeFixedWidthAggregationMap. getAggregationBuffer()` now returns `null` to signal failure due to a lack of memory.
- Updated all uses of these classes to handle these error conditions.
- Added new tests for allocating large records and for allocations which fail due to memory pressure.
- Extended the `afterAll()` test teardown methods to detect ShuffleMemoryManager leaks.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7762 from JoshRosen/large-rows and squashes the following commits:
ae7bc56 [Josh Rosen] Fix compilation
82fc657 [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-rows
34ab943 [Josh Rosen] Remove semi
31a525a [Josh Rosen] Integrate BytesToBytesMap with ShuffleMemoryManager.
626b33c [Josh Rosen] Move code to sql/core and spark/core packages so that ShuffleMemoryManager can be integrated
ec4484c [Josh Rosen] Move BytesToBytesMap from unsafe package to core.
642ed69 [Josh Rosen] Rename size to numElements
bea1152 [Josh Rosen] Add basic test.
2cd3570 [Josh Rosen] Remove accidental duplicated code
07ff9ef [Josh Rosen] Basic support for large rows in BytesToBytesMap.
This PR brings SQL function soundex(), see https://issues.apache.org/jira/browse/HIVE-9738
It's based on #7115 , thanks to HuJiayin
Author: HuJiayin <jiayin.hu@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#7812 from davies/soundex and squashes the following commits:
fa75941 [Davies Liu] Merge branch 'master' of github.com:apache/spark into soundex
a4bd6d8 [Davies Liu] fix soundex
2538908 [HuJiayin] add codegen soundex
d15d329 [HuJiayin] add back ut
ded1a14 [HuJiayin] Merge branch 'master' of https://github.com/apache/spark
e2dec2c [HuJiayin] support soundex rebase code
This PR address the comments in #7805
cc rxin
Author: Davies Liu <davies@databricks.com>
Closes#7817 from davies/trunc and squashes the following commits:
f729d5f [Davies Liu] rollback
cb7f7832 [Davies Liu] genCode() is protected
31e52ef [Davies Liu] fix style
ed1edc7 [Davies Liu] address comments for #7805
This PR is based on #6988 , thanks to adrian-wang .
This brings two SQL functions: to_date() and trunc().
Closes#6988
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#7805 from davies/to_date and squashes the following commits:
2c7beba [Davies Liu] Merge branch 'master' of github.com:apache/spark into to_date
310dd55 [Daoyuan Wang] remove dup test in rebase
980b092 [Daoyuan Wang] resolve rebase conflict
a476c5a [Daoyuan Wang] address comments from davies
d44ea5f [Daoyuan Wang] function to_date, trunc
This PR brings the support of DecimalType in UnsafeRow, for precision <= 18, it's settable, otherwise it's not settable.
Author: Davies Liu <davies@databricks.com>
Closes#7758 from davies/unsafe_decimal and squashes the following commits:
478b1ba [Davies Liu] address comments
536314c [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_decimal
7c2e77a [Davies Liu] fix JoinedRow
76d6fa4 [Davies Liu] fix tests
99d3151 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_decimal
d49c6ae [Davies Liu] support DecimalType in UnsafeRow
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes#7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
unix_timestamp(): long
Gets current Unix timestamp in seconds.
unix_timestamp(string|date): long
Converts time string in format yyyy-MM-dd HH:mm:ss to Unix timestamp (in seconds), using the default timezone and the default locale, return null if fail: unix_timestamp('2009-03-20 11:30:01') = 1237573801
unix_timestamp(string date, string pattern): long
Convert time string with given pattern (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) to Unix time stamp (in seconds), return null if fail: unix_timestamp('2009-03-20', 'yyyy-MM-dd') = 1237532400.
from_unixtime(bigint unixtime[, string format]): string
Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string representing the timestamp of that moment in the current system time zone in the format of "1970-01-01 00:00:00".
Jira:
https://issues.apache.org/jira/browse/SPARK-8174https://issues.apache.org/jira/browse/SPARK-8175
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#7644 from adrian-wang/udfunixtime and squashes the following commits:
2fe20c4 [Daoyuan Wang] util.Date
ea2ec16 [Daoyuan Wang] use util.Date for better performance
a2cf929 [Daoyuan Wang] doc return null instead of 0
f6f070a [Daoyuan Wang] address comments from davies
6a4cbb3 [Daoyuan Wang] temp
56ded53 [Daoyuan Wang] rebase and address comments
14a8b37 [Daoyuan Wang] function unix_timestamp, from_unixtime
Users can now get the file name of the partition being read in. A thread local variable is in `SQLNewHadoopRDD` and is set when the partition is computed. `SQLNewHadoopRDD` is moved to core so that the catalyst package can reach it.
This supports:
`df.select(inputFileName())`
and
`sqlContext.sql("select input_file_name() from table")`
Author: Joseph Batchik <josephbatchik@gmail.com>
Closes#7743 from JDrit/input_file_name and squashes the following commits:
abb8609 [Joseph Batchik] fixed failing test and changed the default value to be an empty string
d2f323d [Joseph Batchik] updates per review
102061f [Joseph Batchik] updates per review
75313f5 [Joseph Batchik] small fixes
c7f7b5a [Joseph Batchik] addeding input file name to Spark SQL
JIRA: https://issues.apache.org/jira/browse/SPARK-9428
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7748 from yjshen/string_cleanup and squashes the following commits:
e0c2b3d [Yijie Shen] update codegen in RegExpExtract and RegExpReplace
26614d2 [Yijie Shen] MathFunctionSuite
a402859 [Yijie Shen] complex_create, conditional and cast
6e4e608 [Yijie Shen] arithmetic and cast
52593c1 [Yijie Shen] null input test cases for StringExpressionSuite
Author: Reynold Xin <rxin@databricks.com>
Closes#7767 from rxin/SPARK-9462 and squashes the following commits:
ef3e2d9 [Reynold Xin] Removed println
713ac3a [Reynold Xin] More unit tests.
bb5c334 [Reynold Xin] [SPARK-9462][SQL] Initialize nondeterministic expressions in code gen fallback mode.
We accidentally moved the list of expressions from the generated code instance to the class wrapper, and as a result, different threads are sharing the same set of expressions, which cause problems for expressions with mutable state.
This pull request fixed that problem, and also added unit tests for all codegen classes, except GeneratedOrdering (which will never need any expressions since sort now only accepts bound references.
Author: Reynold Xin <rxin@databricks.com>
Closes#7759 from rxin/SPARK-9448 and squashes the following commits:
c09b50f [Reynold Xin] [SPARK-9448][SQL] GenerateUnsafeProjection should not share expressions across instances.
We need to make page sizes configurable so we can reduce them in unit tests and increase them in real production workloads. These sizes are now controlled by a new configuration, `spark.buffer.pageSize`. The new default is 64 megabytes.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7741 from JoshRosen/SPARK-9411 and squashes the following commits:
a43c4db [Josh Rosen] Fix pow
2c0eefc [Josh Rosen] Fix MAXIMUM_PAGE_SIZE_BYTES comment + value
bccfb51 [Josh Rosen] Lower page size to 4MB in TestHive
ba54d4b [Josh Rosen] Make UnsafeExternalSorter's page size configurable
0045aa2 [Josh Rosen] Make UnsafeShuffle's page size configurable
bc734f0 [Josh Rosen] Rename configuration
e614858 [Josh Rosen] Makes BytesToBytesMap page size configurable
We want to introduce a new IntervalType in 1.6 that is based on only the number of microseoncds,
so interval can be compared.
Renaming the existing IntervalType to CalendarIntervalType so we can do that in the future.
Author: Reynold Xin <rxin@databricks.com>
Closes#7745 from rxin/calendarintervaltype and squashes the following commits:
99f64e8 [Reynold Xin] One more line ...
13466c8 [Reynold Xin] Fixed tests.
e20f24e [Reynold Xin] [SPARK-9430][SQL] Rename IntervalType to CalendarIntervalType.
Author: Reynold Xin <rxin@databricks.com>
Closes#7747 from rxin/SPARK-9127 and squashes the following commits:
e851418 [Reynold Xin] [SPARK-9127][SQL] Rand/Randn codegen fails with long seed.
as an offline discussion with rxin , it's weird to be computing stuff while doing sorting, we should only order by bound reference during execution.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7593 from cloud-fan/sort and squashes the following commits:
7b1bef7 [Wenchen Fan] add test
daf206d [Wenchen Fan] add more comments
289bee0 [Wenchen Fan] do not order by expressions which still need evaluation
Right now, we use double to parse all the float number in SQL. When it's used in expression together with DecimalType, it will turn the decimal into double as well. Also it will loss some precision when using double.
This PR change to parse float number to decimal or double, based on it's using scientific notation or not, see https://msdn.microsoft.com/en-us/library/ms179899.aspx
This is a break change, should we doc it somewhere?
Author: Davies Liu <davies@databricks.com>
Closes#7642 from davies/parse_decimal and squashes the following commits:
1f576d9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into parse_decimal
5e142b6 [Davies Liu] fix scala style
eca99de [Davies Liu] fix tests
2afe702 [Davies Liu] Merge branch 'master' of github.com:apache/spark into parse_decimal
f4a320b [Davies Liu] Update SqlParser.scala
1c48e34 [Davies Liu] use decimal or double when parsing SQL
We will do local projection for LocalRelation, and thus reuse the same Expression object among multiply evaluations. We should reset the mutable states of Expression before evaluate it.
Fix `PullOutNondeterministic` rule to make it work for `Sort`.
Also got a chance to cleanup the dataframe test suite.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7674 from cloud-fan/show and squashes the following commits:
888934f [Wenchen Fan] fix sort
c0e93e8 [Wenchen Fan] local DataFrame with random columns should return same value when call `show`
UnsafeRow.getDouble and getFloat() return NaN when called on columns that are null, which is inconsistent with the behavior of other row classes (which is to return 0.0).
In addition, the generic get(ordinal, dataType) method should always return null for a null literal, but currently it handles nulls by calling the type-specific accessors.
This patch addresses both of these issues and adds a regression test.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7736 from JoshRosen/unsafe-row-null-fixes and squashes the following commits:
c8eb2ee [Josh Rosen] Fix test in UnsafeRowConverterSuite
6214682 [Josh Rosen] Fixes to null handling in UnsafeRow
Since catalyst package already depends on Spark core, we can move those expressions
into catalyst, and simplify function registry.
This is a followup of #7478.
Author: Reynold Xin <rxin@databricks.com>
Closes#7735 from rxin/SPARK-8003 and squashes the following commits:
2ffbdc3 [Reynold Xin] [SPARK-8003][SQL] Move expressions in sql/core package to catalyst.
The original patch didn't handle nulls correctly for next_day.
Author: Reynold Xin <rxin@databricks.com>
Closes#7718 from rxin/next_day and squashes the following commits:
616a425 [Reynold Xin] Merged DatetimeExpressionsSuite into DateFunctionsSuite.
faa78cf [Reynold Xin] Merged DatetimeFunctionsSuite into DateExpressionsSuite.
6c4fb6a [Reynold Xin] [SPARK-8196][SQL] Fix null handling & documentation for next_day.
Our CodeFormatter currently does not handle parentheses, and as a result in code dump, we see code formatted this way:
```
foo(
a,
b,
c)
```
With this patch, it is formatted this way:
```
foo(
a,
b,
c)
```
Author: Reynold Xin <rxin@databricks.com>
Closes#7712 from rxin/codeformat-parentheses and squashes the following commits:
c2b1c5f [Reynold Xin] Took square bracket out
3cfb174 [Reynold Xin] Code review feedback.
91f5bb1 [Reynold Xin] [SPARK-9394][SQL] Handle parentheses in CodeFormatter.
This pull request updates GenerateUnsafeProjection to support StructType. If an input struct type is backed already by an UnsafeRow, GenerateUnsafeProjection copies the bytes directly into its buffer space without any conversion. However, if the input is not an UnsafeRow, GenerateUnsafeProjection runs the code generated recursively to convert the input into an UnsafeRow and then copies it into the buffer space.
Also create a TungstenProject operator that projects data directly into UnsafeRow. Note that I'm not sure if this is the way we want to structure Unsafe+codegen operators, but we can defer that decision to follow-up pull requests.
Author: Reynold Xin <rxin@databricks.com>
Closes#7689 from rxin/tungsten-struct-type and squashes the following commits:
9162f42 [Reynold Xin] Support IntervalType in UnsafeRow's getter.
be9f377 [Reynold Xin] Fixed tests.
10c4b7c [Reynold Xin] Format generated code.
77e8d0e [Reynold Xin] Fixed NondeterministicSuite.
ac4951d [Reynold Xin] Yay.
ac203bf [Reynold Xin] More comments.
9f36216 [Reynold Xin] Updated comment.
6b781fe [Reynold Xin] Reset the change in DataFrameSuite.
525b95b [Reynold Xin] Merged with master, more documentation & test cases.
321859a [Reynold Xin] [SPARK-9373][SQL] Support StructType in Tungsten projection [WIP]
next_day, returns next certain dayofweek.
last_day, returns the last day of the month which given date belongs to.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#6986 from adrian-wang/udfnlday and squashes the following commits:
ef7e3da [Daoyuan Wang] fix
02b3426 [Daoyuan Wang] address 2 comments
dc69630 [Daoyuan Wang] address comments from rxin
8846086 [Daoyuan Wang] address comments from rxin
d09bcce [Daoyuan Wang] multi fix
1a9de3d [Daoyuan Wang] function next_day and last_day
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7688 from cloud-fan/interval and squashes the following commits:
5b36b17 [Wenchen Fan] fix codegen
a99ed50 [Wenchen Fan] address comment
9e6d319 [Wenchen Fan] Support IntervalType in UnsafeRow
literals in grouping expressions have no effect at all, only make our grouping key bigger, so we should remove them in Optimizer.
I also make old and new aggregation code consistent about literals in grouping here. In old aggregation, actually literals in grouping are already removed but new aggregation is not. So I explicitly make it a rule in Optimizer.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7583 from cloud-fan/minor and squashes the following commits:
471adff [Wenchen Fan] add test
0839925 [Wenchen Fan] use transformDown when rewrite final result expressions
Make this test deterministic, i.e. make sure this test can be passed no matter how many times we run it.
The origin implementation uses a random seed and gives a chance that we may break the null check assertion `assert(Iterator.fill(100)(generator()).contains(null))`.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7691 from cloud-fan/seed and squashes the following commits:
eae7281 [Wenchen Fan] use a seed in RandomDataGeneratorSuite
JIRA: https://issues.apache.org/jira/browse/SPARK-9356
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7671 from yjshen/deprecated_unlimit and squashes the following commits:
c707f56 [Yijie Shen] remove pattern matching in changePrecision
4a1823c [Yijie Shen] remove internal occurrence of Decimal.Unlimited
Currently UnsafeRow cannot support a generic getter. However, if the data type is known, we can support a generic getter.
Author: Reynold Xin <rxin@databricks.com>
Closes#7666 from rxin/generic-getter-with-datatype and squashes the following commits:
ee2874c [Reynold Xin] Add a default implementation for getStruct.
1e109a0 [Reynold Xin] [SPARK-9350][SQL] Introduce an InternalRow generic getter that requires a DataType.
033ee88 [Reynold Xin] Removed getAs in non test code.
Author: Reynold Xin <rxin@databricks.com>
Closes#7665 from rxin/remove-row-apply and squashes the following commits:
0b43001 [Reynold Xin] support getString in UnsafeRow.
176d633 [Reynold Xin] apply -> get.
2941324 [Reynold Xin] [SPARK-9348][SQL] Remove apply method on InternalRow.
Currently nondeterministic expression is broken without a explicit initialization phase.
Let me take `MonotonicallyIncreasingID` as an example. This expression need a mutable state to remember how many times it has been evaluated, so we use `transient var count: Long` there. By being transient, the `count` will be reset to 0 and **only** to 0 when serialize and deserialize it, as deserialize transient variable will result to default value. There is *no way* to use another initial value for `count`, until we add the explicit initialization phase.
Another use case is local execution for `LocalRelation`, there is no serialize and deserialize phase and thus we can't reset mutable states for it.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7535 from cloud-fan/init and squashes the following commits:
6c6f332 [Wenchen Fan] add test
ef68ff4 [Wenchen Fan] fix comments
9eac85e [Wenchen Fan] move init code to interpreted class
bb7d838 [Wenchen Fan] pulls out nondeterministic expressions into a project
b4a4fc7 [Wenchen Fan] revert a refactor
86fee36 [Wenchen Fan] add initialization phase for nondeterministic expression
The two are redundant.
Once this patch is merged, I plan to remove the inbound conversions from unsafe aggregates.
Author: Reynold Xin <rxin@databricks.com>
Closes#7658 from rxin/unsafeconverters and squashes the following commits:
ed19e6c [Reynold Xin] Updated support types.
2a56d7e [Reynold Xin] [SPARK-9334][SQL] Remove UnsafeRowConverter in favor of UnsafeProjection.
The generated expression code can be hard to read since they are not indented well. This patch adds a code formatter that formats code automatically when we output them to the screen.
Author: Reynold Xin <rxin@databricks.com>
Closes#7656 from rxin/codeformatter and squashes the following commits:
5ba0e90 [Reynold Xin] [SPARK-9331][SQL] Add a code formatter to auto-format generated code.
This patch extends CheckAnalysis to throw errors for queries that try to sort on unsupported column types, such as ArrayType.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7633 from JoshRosen/SPARK-9295 and squashes the following commits:
23b2fbf [Josh Rosen] Embed function in foreach
bfe1451 [Josh Rosen] Update to allow sorting by null literals
2f1b802 [Josh Rosen] Add analysis rule to detect sorting on unsupported column types (SPARK-9295)
This patch adds an analysis check to ensure that join conditions' data types are BooleanType. This check is necessary in order to report proper errors for non-boolean DataFrame join conditions.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7630 from JoshRosen/SPARK-9292 and squashes the following commits:
aec6c7b [Josh Rosen] Check condition type in resolved()
75a3ea6 [Josh Rosen] Fix SPARK-9292.
It's a thing used in test cases, but named Row. Pretty annoying because everytime I search for Row, it shows up before the Spark SQL Row, which is what a developer wants most of the time.
Author: Reynold Xin <rxin@databricks.com>
Closes#7638 from rxin/remove-row and squashes the following commits:
aeda52d [Reynold Xin] [SPARK-9305] Rename org.apache.spark.Row to Item.
I also changed InternalRow's size/length function to numFields, to make it more obvious that it is not about bytes, but the number of fields.
Author: Reynold Xin <rxin@databricks.com>
Closes#7626 from rxin/internalRow and squashes the following commits:
e124daf [Reynold Xin] Fixed test case.
805ceb7 [Reynold Xin] Commented out the failed test suite.
f8a9ca5 [Reynold Xin] Fixed more bugs. Still at least one more remaining.
76d9081 [Reynold Xin] Fixed data sources.
7807f70 [Reynold Xin] Fixed DataFrameSuite.
cb60cd2 [Reynold Xin] Code review & small bug fixes.
0a2948b [Reynold Xin] Fixed style.
3280d03 [Reynold Xin] [SPARK-9285][SQL] Remove InternalRow's inheritance from Row.
Address comments for #7605
cc rxin
Author: Davies Liu <davies@databricks.com>
Closes#7634 from davies/decimal_unlimited2 and squashes the following commits:
b2d8b0d [Davies Liu] add doc and test for DecimalType.isWiderThan
65b251c [Davies Liu] fix test
6a91f32 [Davies Liu] fix style
ca9c973 [Davies Liu] address comments
Author: Reynold Xin <rxin@databricks.com>
Closes#7636 from rxin/complex-string-implicit-cast and squashes the following commits:
3e67327 [Reynold Xin] [SPARK-9200][SQL] Don't implicitly cast non-atomic types to string type.
Romove Decimal.Unlimited (change to support precision up to 38, to match with Hive and other databases).
In order to keep backward source compatibility, Decimal.Unlimited is still there, but change to Decimal(38, 18).
If no precision and scale is provide, it's Decimal(10, 0) as before.
Author: Davies Liu <davies@databricks.com>
Closes#7605 from davies/decimal_unlimited and squashes the following commits:
aa3f115 [Davies Liu] fix tests and style
fb0d20d [Davies Liu] address comments
bfaae35 [Davies Liu] fix style
df93657 [Davies Liu] address comments and clean up
06727fd [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_unlimited
4c28969 [Davies Liu] fix tests
8d783cc [Davies Liu] fix tests
788631c [Davies Liu] fix double with decimal in Union/except
1779bde [Davies Liu] fix scala style
c9c7c78 [Davies Liu] remove Decimal.Unlimited
Reverts ObjectPool. As it stands, it has a few problems:
1. ObjectPool doesn't work with spilling and memory accounting.
2. I don't think in the long run the idea of an object pool is what we want to support, since it essentially goes back to unmanaged memory, and creates pressure on GC, and is hard to account for the total in memory size.
3. The ObjectPool patch removed the specialized getters for strings and binary, and as a result, actually introduced branches when reading non primitive data types.
If we do want to support arbitrary user defined types in the future, I think we can just add an object array in UnsafeRow, rather than relying on indirect memory addressing through a pool. We also need to pick execution strategies that are optimized for those, rather than keeping a lot of unserialized JVM objects in memory during aggregation.
This is probably the hardest thing I had to revert in Spark, due to recent patches that also change the same part of the code. Would be great to get a careful look.
Author: Reynold Xin <rxin@databricks.com>
Closes#7591 from rxin/revert-object-pool and squashes the following commits:
01db0bc [Reynold Xin] Scala style.
eda89fc [Reynold Xin] Fixed describe.
2967118 [Reynold Xin] Fixed accessor for JoinedRow.
e3294eb [Reynold Xin] Merge branch 'master' into revert-object-pool
657855f [Reynold Xin] Temp commit.
c20f2c8 [Reynold Xin] Style fix.
fe37079 [Reynold Xin] Revert "[SPARK-8579] [SQL] support arbitrary object in UnsafeRow"
JIRA: https://issues.apache.org/jira/browse/SPARK-8935
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7365 from yjshen/cast_codegen and squashes the following commits:
ef6e8b5 [Yijie Shen] getColumn and setColumn in struct cast, autounboxing in array and map
eaece18 [Yijie Shen] remove null case in cast code gen
fd7eba4 [Yijie Shen] resolve comments
80378a5 [Yijie Shen] the missing self cast
611d66e [Yijie Shen] Bug fix: NullType & primitive object unboxing
6d5c0fe [Yijie Shen] rebase and add Interval codegen
9424b65 [Yijie Shen] tiny style fix
4a1c801 [Yijie Shen] remove CodeHolder class, use function instead.
3f5df88 [Yijie Shen] CodeHolder for complex dataTypes
c286f13 [Yijie Shen] moved all the cast code into class body
4edfd76 [Yijie Shen] [WIP] finished primitive part
JIRA: https://issues.apache.org/jira/browse/SPARK-9165
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7537 from yjshen/array_struct_codegen and squashes the following commits:
3a6dce6 [Yijie Shen] use infix notion in createArray test
5e90f0a [Yijie Shen] resolve comments: classOf
39cefb8 [Yijie Shen] codegen for createArray createStruct & createNamedStruct
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7446 from cloud-fan/filter and squashes the following commits:
330021e [Wenchen Fan] add exists to tree node
2cab68c [Wenchen Fan] more enhance
949be07 [Wenchen Fan] push down part of predicate if possible
3912f84 [Wenchen Fan] address comments
8ce15ca [Wenchen Fan] fix bug
557158e [Wenchen Fan] Filter using non-deterministic expressions should not be pushed down
Also make format_string the canonical form, rather than printf.
Author: Reynold Xin <rxin@databricks.com>
Closes#7579 from rxin/format_strings and squashes the following commits:
53ee54f [Reynold Xin] Fixed unit tests.
52357e1 [Reynold Xin] Add format_string alias.
b40a42a [Reynold Xin] [SPARK-9154][SQL] Rename formatString to format_string.
Jira: https://issues.apache.org/jira/browse/SPARK-9154
fixes bug of #7546
marmbrus I can't reopen the other PR, because I didn't closed it. Can you trigger Jenkins?
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes#7571 from tarekauel/SPARK-9154 and squashes the following commits:
dcae272 [Tarek Auel] [SPARK-9154][SQL] build fix
1487602 [Tarek Auel] Merge remote-tracking branch 'upstream/master' into SPARK-9154
f512c5f [Tarek Auel] [SPARK-9154][SQL] build fix
a943d3e [Tarek Auel] [SPARK-9154] implicit input cast, added tests for null, support for null primitives
10b4de8 [Tarek Auel] [SPARK-9154][SQL] codegen removed fallback trait
cd8322b [Tarek Auel] [SPARK-9154][SQL] codegen string format
086caba [Tarek Auel] [SPARK-9154][SQL] codegen string format
This reverts commit 7f072c3d5e.
Revert #7546
Author: Michael Armbrust <michael@databricks.com>
Closes#7570 from marmbrus/revert9154 and squashes the following commits:
ed2c32a [Michael Armbrust] Revert "[SPARK-9154] [SQL] codegen StringFormat"
JIRA:
https://issues.apache.org/jira/browse/SPARK-9081https://issues.apache.org/jira/browse/SPARK-9168
This PR target at two modifications:
1. Change `isNaN` to return `false` on `null` input
2. Make `dropna` and `fillna` to fill/drop NaN values as well
3. Implement `nanvl`
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7523 from yjshen/fillna_dropna and squashes the following commits:
f0a51db [Yijie Shen] make coalesce untouched and implement nanvl
1d3e35f [Yijie Shen] make Coalesce aware of NaN in order to support fillna
2760cbc [Yijie Shen] change isNaN(null) to false as well as implement dropna
JIRA: https://issues.apache.org/jira/browse/SPARK-9173
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7540 from yjshen/union_pushdown and squashes the following commits:
278510a [Yijie Shen] rename UnionPushDown to SetOperationPushDown
91741c1 [Yijie Shen] Add UnionPushDown support for intersect and except
Pull Request for: https://issues.apache.org/jira/browse/SPARK-8230
Primary issue resolved is to implement array/map size for Spark SQL. Code is ready for review by a committer. Chen Hao is on the JIRA ticket, but I don't know his username on github, rxin is also on JIRA ticket.
Things to review:
1. Where to put added functions namespace wise, they seem to be part of a few operations on collections which includes `sort_array` and `array_contains`. Hence the name given `collectionOperations.scala` and `_collection_functions` in python.
2. In Python code, should it be in a `1.5.0` function array or in a collections array?
3. Are there any missing methods on the `Size` case class? Looks like many of these functions have generated Java code, is that also needed in this case?
4. Something else?
Author: Pedro Rodriguez <ski.rodriguez@gmail.com>
Author: Pedro Rodriguez <prodriguez@trulia.com>
Closes#7462 from EntilZha/SPARK-8230 and squashes the following commits:
9a442ae [Pedro Rodriguez] fixed functions and sorted __all__
9aea3bb [Pedro Rodriguez] removed imports from python docs
15d4bf1 [Pedro Rodriguez] Added null test case and changed to nullSafeCodeGen
d88247c [Pedro Rodriguez] removed python code
bd5f0e4 [Pedro Rodriguez] removed duplicate function from rebase/merge
59931b4 [Pedro Rodriguez] fixed compile bug instroduced when merging
c187175 [Pedro Rodriguez] updated code to add size to __all__ directly and removed redundent pretty print
130839f [Pedro Rodriguez] fixed failing test
aa9bade [Pedro Rodriguez] fix style
e093473 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests
0449377 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations
9a1a2ff [Pedro Rodriguez] added unit tests for map size
2bfbcb6 [Pedro Rodriguez] added unit test for size
20df2b4 [Pedro Rodriguez] Finished working version of size function and added it to python
b503e75 [Pedro Rodriguez] First attempt at implementing size for maps and arrays
99a6a5c [Pedro Rodriguez] fixed failing test
cac75ac [Pedro Rodriguez] fix style
933d843 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests
42bb7d4 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations
f9c3b8a [Pedro Rodriguez] added unit tests for map size
2515d9f [Pedro Rodriguez] added documentation
0e60541 [Pedro Rodriguez] added unit test for size
acf9853 [Pedro Rodriguez] Finished working version of size function and added it to python
84a5d38 [Pedro Rodriguez] First attempt at implementing size for maps and arrays
Add expressions `regex_extract` & `regex_replace`
Author: Cheng Hao <hao.cheng@intel.com>
Closes#7468 from chenghao-intel/regexp and squashes the following commits:
e5ea476 [Cheng Hao] minor update for documentation
ef96fd6 [Cheng Hao] update the code gen
72cf28f [Cheng Hao] Add more log for compilation error
4e11381 [Cheng Hao] Add regexp_replace / regexp_extract support
This patch addresses an issue where queries that sorted float or double columns containing NaN values could fail with "Comparison method violates its general contract!" errors from TimSort. The root of this problem is that `NaN > anything`, `NaN == anything`, and `NaN < anything` all return `false`.
Per the design specified in SPARK-9079, we have decided that `NaN = NaN` should return true and that NaN should appear last when sorting in ascending order (i.e. it is larger than any other numeric value).
In addition to implementing these semantics, this patch also adds canonicalization of NaN values in UnsafeRow, which is necessary in order to be able to do binary equality comparisons on equal NaNs that might have different bit representations (see SPARK-9147).
Author: Josh Rosen <joshrosen@databricks.com>
Closes#7194 from JoshRosen/nan and squashes the following commits:
983d4fc [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
88bd73c [Josh Rosen] Fix Row.equals()
a702e2e [Josh Rosen] normalization -> canonicalization
a7267cf [Josh Rosen] Normalize NaNs in UnsafeRow
fe629ae [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
fbb2a29 [Josh Rosen] Fix NaN comparisons in BinaryComparison expressions
c1fd4fe [Josh Rosen] Fold NaN test into existing test framework
b31eb19 [Josh Rosen] Uncomment failing tests
7fe67af [Josh Rosen] Support NaN == NaN (SPARK-9145)
58bad2c [Josh Rosen] Revert "Compare rows' string representations to work around NaN incomparability."
fc6b4d2 [Josh Rosen] Update CodeGenerator
3998ef2 [Josh Rosen] Remove unused code
a2ba2e7 [Josh Rosen] Fix prefix comparision for NaNs
a30d371 [Josh Rosen] Compare rows' string representations to work around NaN incomparability.
6f03f85 [Josh Rosen] Fix bug in Double / Float ordering
42a1ad5 [Josh Rosen] Stop filtering NaNs in UnsafeExternalSortSuite
bfca524 [Josh Rosen] Change ordering so that NaN is maximum value.
8d7be61 [Josh Rosen] Update randomized test to use ScalaTest's assume()
b20837b [Josh Rosen] Add failing test for new NaN comparision ordering
5b88b2b [Josh Rosen] Fix compilation of CodeGenerationSuite
d907b5b [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
630ebc5 [Josh Rosen] Specify an ordering for NaN values.
9bf195a [Josh Rosen] Re-enable NaNs in CodeGenerationSuite to produce more regression tests
13fc06a [Josh Rosen] Add regression test for NaN sorting issue
f9efbb5 [Josh Rosen] Fix ORDER BY NULL
e7dc4fb [Josh Rosen] Add very generic test for ordering
7d5c13e [Josh Rosen] Add regression test for SPARK-8782 (ORDER BY NULL)
b55875a [Josh Rosen] Generate doubles and floats over entire possible range.
5acdd5c [Josh Rosen] Infinity and NaN are interesting.
ab76cbd [Josh Rosen] Move code to Catalyst package.
d2b4a4a [Josh Rosen] Add random data generator test utilities to Spark SQL.
Jira: https://issues.apache.org/jira/browse/SPARK-9132https://issues.apache.org/jira/browse/SPARK-9163
rxin as you proposed in the Jira ticket, I just moved the logic to a separate object. I haven't changed anything of the logic of `NumberConverter`.
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes#7552 from tarekauel/SPARK-9163 and squashes the following commits:
40dcde9 [Tarek Auel] [SPARK-9132][SPARK-9163][SQL] style fix
fa985bd [Tarek Auel] [SPARK-9132][SPARK-9163][SQL] codegen conv
JIRA: https://issues.apache.org/jira/browse/SPARK-9172
Simply make `DecimalPrecision` support for `Intersect` and `Except` in addition to `Union`.
Besides, add unit test for `DecimalPrecision` as well.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#7511 from viirya/more_decimalprecieion and squashes the following commits:
4d29d10 [Liang-Chi Hsieh] Fix code comment.
9fb0d49 [Liang-Chi Hsieh] Make DecimalPrecision support for Intersect and Except.
I also changed the semantics of concat w.r.t. null back to the same behavior as Hive.
That is to say, concat now returns null if any input is null.
Author: Reynold Xin <rxin@databricks.com>
Closes#7504 from rxin/concat_ws and squashes the following commits:
83fd950 [Reynold Xin] Fixed type casting.
3ae85f7 [Reynold Xin] Write null better.
cdc7be6 [Reynold Xin] Added code generation for pure string mode.
a61c4e4 [Reynold Xin] Updated comments.
2d51406 [Reynold Xin] [SPARK-8241][SQL] string function: concat_ws.
PR #7506 breaks master build because of compilation error. Note that #7506 itself looks good, but it seems that `git merge` did something stupid.
Author: Cheng Lian <lian@databricks.com>
Closes#7510 from liancheng/hotfix-for-pr-7506 and squashes the following commits:
7ea7e89 [Cheng Lian] Fixes compilation error
This pull request fixes some of the problems in #6981.
- Added date functions to `__all__` so they get exposed
- Rename day_of_month -> dayofmonth
- Rename day_in_year -> dayofyear
- Rename week_of_year -> weekofyear
- Removed "day" from Scala/Python API since it is ambiguous. Only leaving the alias in SQL.
Author: Reynold Xin <rxin@databricks.com>
This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>
Closes#7506 from rxin/datetime and squashes the following commits:
0cb24d9 [Reynold Xin] Export all functions in Python.
e44a4a0 [Reynold Xin] Removed day function from Scala and Python.
9c08fdc [Reynold Xin] [SQL] Make date/time functions more consistent with other database systems.
rxin / davies
Sorry for that unnecessary change. And thanks again for all your support!
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes#7505 from tarekauel/SPARK-8199-FollowUp and squashes the following commits:
d09321c [Tarek Auel] [SPARK-8199] follow up; revert change in test
c17397f [Tarek Auel] [SPARK-8199] follow up; revert change in test
67acfe6 [Tarek Auel] [SPARK-8199] follow up; revert change in test
By grouping projection calls into multiple apply function, we are able to push the number of projections codegen can handle from ~1k to ~60k. I have set the unit test to test against 5k as 60k took 15s for the unit test to complete.
Author: Forest Fang <forest.fang@outlook.com>
Closes#7076 from saurfang/codegen_size_limit and squashes the following commits:
b7a7635 [Forest Fang] [SPARK-8443][SQL] Execute and verify split projections in test
adef95a [Forest Fang] [SPARK-8443][SQL] Use safer factor and rewrite splitting code
1b5aa7e [Forest Fang] [SPARK-8443][SQL] inline execution if one block only
9405680 [Forest Fang] [SPARK-8443][SQL] split projection code by size limit
It is very hard to track which expressions have code gen implemented or not. This patch removes the default fallback gencode implementation from Expression, and moves that into a new trait called CodegenFallback. Each concrete expression needs to either implement code generation, or mix in CodegenFallback. This makes it very easy to track which expressions have code generation implemented already.
Additionally, this patch creates an Unevaluable trait that can be used to track expressions that don't support evaluation (e.g. Star).
Author: Reynold Xin <rxin@databricks.com>
Closes#7487 from rxin/codegenfallback and squashes the following commits:
14ebf38 [Reynold Xin] Fixed Conv
6c1c882 [Reynold Xin] Fixed Alias.
b42611b [Reynold Xin] [SPARK-9150][SQL] Create a trait to track code generation for expressions.
cb5c066 [Reynold Xin] Removed extra import.
39cbe40 [Reynold Xin] [SPARK-8240][SQL] string function: concat
JIRA: https://issues.apache.org/jira/browse/SPARK-9055
cc rxin
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7491 from yijieshen/widen and squashes the following commits:
079fa52 [Yijie Shen] widenType support for intersect and expect
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7496 from cloud-fan/tests and squashes the following commits:
0958f90 [Wenchen Fan] improve test for nondeterministic expressions
fix 2 bugs introduced in https://github.com/apache/spark/pull/7353
1. we should use UTC Calendar when cast string to date . Before #7353 , we use `DateTimeUtils.fromJavaDate(Date.valueOf(s.toString))` to cast string to date, and `fromJavaDate` will call `millisToDays` to avoid the time zone issue. Now we use `DateTimeUtils.stringToDate(s)`, we should create a Calendar with UTC in the begging.
2. we should not change the default time zone in test cases. The `threadLocalLocalTimeZone` and `threadLocalTimestampFormat` in `DateTimeUtils` will only be evaluated once for each thread, so we can't set the default time zone back anymore.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7488 from cloud-fan/datetime and squashes the following commits:
9cd6005 [Wenchen Fan] address comments
21ef293 [Wenchen Fan] fix 2 bugs in datetime
Author: Reynold Xin <rxin@databricks.com>
Closes#7490 from rxin/unit-test-null-funcs and squashes the following commits:
7b276f0 [Reynold Xin] Move isNaN.
8307287 [Reynold Xin] [SPARK-9169][SQL] Improve unit test coverage for null expressions.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7452 from cloud-fan/boolean-simplify and squashes the following commits:
2a6e692 [Wenchen Fan] fix style
d3cfd26 [Wenchen Fan] fix BooleanSimplification in case-insensitive
The check was unreachable before, as `case operator: LogicalPlan` catches everything already.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7449 from cloud-fan/tmp and squashes the following commits:
2bb6637 [Wenchen Fan] add test
5493aea [Wenchen Fan] add the check back
27221a7 [Wenchen Fan] remove unnecessary analysis check code for self join
JIRA: https://issues.apache.org/jira/browse/SPARK-9080
cc rxin
Author: Yijie Shen <henry.yijieshen@gmail.com>
Closes#7464 from yijieshen/isNaN and squashes the following commits:
11ae039 [Yijie Shen] add isNaN in functions
666718e [Yijie Shen] add isNaN predicate expression
a follow up of https://github.com/apache/spark/pull/7353
1. we should use `Calendar.HOUR_OF_DAY` instead of `Calendar.HOUR`(this is for AM, PM).
2. we should call `c.set(Calendar.MILLISECOND, 0)` after `Calendar.getInstance`
I'm not sure why the tests didn't fail in jenkins, but I ran latest spark master branch locally and `DateTimeUtilsSuite` failed.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7473 from cloud-fan/datetime and squashes the following commits:
66cdaf2 [Wenchen Fan] fix several bugs in DateTimeUtils.stringToTimestamp
cc chenghao-intel adrian-wang
Author: zhichao.li <zhichao.li@intel.com>
Closes#6872 from zhichao-li/conv and squashes the following commits:
6ef3b37 [zhichao.li] add unittest and comments
78d9836 [zhichao.li] polish dataframe api and add unittest
e2bace3 [zhichao.li] update to use ImplicitCastInputTypes
cbcad3f [zhichao.li] add function conv
instead of return false, throw exception when check equality between external and internal row is better.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7460 from cloud-fan/row-compare and squashes the following commits:
8a20911 [Wenchen Fan] improve equals
402daa8 [Wenchen Fan] throw exception when check equality between external and internal row
Added two projections: GenerateUnsafeProjection and FromUnsafeProjection, which could be used to convert UnsafeRow from/to GenericInternalRow.
They will re-use the buffer during projection, similar to MutableProjection (without all the interface MutableProjection has).
cc rxin JoshRosen
Author: Davies Liu <davies@databricks.com>
Closes#7437 from davies/unsafe_proj2 and squashes the following commits:
dbf538e [Davies Liu] test with all the expression (only for supported types)
dc737b2 [Davies Liu] address comment
e424520 [Davies Liu] fix scala style
70e231c [Davies Liu] address comments
729138d [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_proj2
5a26373 [Davies Liu] unsafe projections
Currently we will stop project collapse when the lower projection has nondeterministic expressions. However it's overkill sometimes, we should be able to optimize `df.select(Rand(10)).select('a)` to `df.select('a)`
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7445 from cloud-fan/non-deterministic and squashes the following commits:
0deaef6 [Wenchen Fan] Improve project collapse with nondeterministic expressions
Jira https://issues.apache.org/jira/browse/SPARK-8995
In PR #6981we noticed that we cannot cast date strings that contains a time, like '2015-03-18 12:39:40' to date. Besides it's not possible to cast a string like '18:03:20' to a timestamp.
If a time is passed without a date, today is inferred as date.
Author: Tarek Auel <tarek.auel@googlemail.com>
Author: Tarek Auel <tarek.auel@gmail.com>
Closes#7353 from tarekauel/SPARK-8995 and squashes the following commits:
14f333b [Tarek Auel] [SPARK-8995] added tests for daylight saving time
ca1ae69 [Tarek Auel] [SPARK-8995] style fix
d20b8b4 [Tarek Auel] [SPARK-8995] bug fix: distinguish between 0 and null
ef05753 [Tarek Auel] [SPARK-8995] added check for year >= 1000
01c9ff3 [Tarek Auel] [SPARK-8995] support for time strings
34ec573 [Tarek Auel] fixed style
71622c0 [Tarek Auel] improved timestamp and date parsing
0e30c0a [Tarek Auel] Hive compatibility
cfbaed7 [Tarek Auel] fixed wrong checks
71f89c1 [Tarek Auel] [SPARK-8995] minor style fix
f7452fa [Tarek Auel] [SPARK-8995] removed old timestamp parsing
30e5aec [Tarek Auel] [SPARK-8995] date and timestamp cast
c1083fb [Tarek Auel] [SPARK-8995] cast date strings like '2015-01-01 12:15:31' to date or timestamp
based on https://github.com/apache/spark/pull/7348
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7420 from cloud-fan/type-check and squashes the following commits:
7633fa9 [Wenchen Fan] revert
fe169b0 [Wenchen Fan] improve test
03b70da [Wenchen Fan] enhance implicit type cast
- `BinaryType` for `Length`
- `FormatNumber`
Author: Cheng Hao <hao.cheng@intel.com>
Closes#7034 from chenghao-intel/expression and squashes the following commits:
e534b87 [Cheng Hao] python api style issue
601bbf5 [Cheng Hao] add python API support
3ebe288 [Cheng Hao] update as feedback
52274f7 [Cheng Hao] add support for udf_format_number and length for binary
https://issues.apache.org/jira/browse/SPARK-8221
One concern is the result would be negative if the divisor is not positive( i.e pmod(7, -3) ), but the behavior is the same as hive.
Author: zhichao.li <zhichao.li@intel.com>
Closes#6783 from zhichao-li/pmod2 and squashes the following commits:
7083eb9 [zhichao.li] update to the latest type checking
d26dba7 [zhichao.li] add pmod
This patch makes the following changes:
1. ExpectsInputTypes only defines expected input types, but does not perform any implicit type casting.
2. ImplicitCastInputTypes is a new trait that defines both expected input types, as well as performs implicit type casting.
3. BinaryOperator has a new abstract function "inputType", which defines the expected input type for both left/right. Concrete BinaryOperator expressions no longer perform any implicit type casting.
4. For BinaryOperators, convert NullType (i.e. null literals) into some accepted type so BinaryOperators don't need to handle NullTypes.
TODOs needed: fix unit tests for error reporting.
I'm intentionally not changing anything in aggregate expressions because yhuai is doing a big refactoring on that right now.
Author: Reynold Xin <rxin@databricks.com>
Closes#7348 from rxin/typecheck and squashes the following commits:
8fcf814 [Reynold Xin] Fixed ordering of cases.
3bb63e7 [Reynold Xin] Style fix.
f45408f [Reynold Xin] Comment update.
aa7790e [Reynold Xin] Moved RemoveNullTypes into ImplicitTypeCasts.
438ea07 [Reynold Xin] space
d55c9e5 [Reynold Xin] Removes NullTypes.
360d124 [Reynold Xin] Fixed the rule.
fb66657 [Reynold Xin] Convert NullType into some accepted type for BinaryOperators.
2e22330 [Reynold Xin] Fixed unit tests.
4932d57 [Reynold Xin] Style fix.
d061691 [Reynold Xin] Rename existing ExpectsInputTypes -> ImplicitCastInputTypes.
e4727cc [Reynold Xin] BinaryOperator should not be doing implicit cast.
d017861 [Reynold Xin] Improve expression type checking.
JIRA: https://issues.apache.org/jira/browse/SPARK-8800
Previously, we turn to Java BigDecimal's divide with specified ROUNDING_MODE to avoid non-terminating decimal expansion problem. However, as JihongMA reported, for the division operation on some specific values, we get inaccurate results.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#7212 from viirya/fix_decimal4 and squashes the following commits:
4205a0a [Liang-Chi Hsieh] Fix inaccuracy precision/scale of Decimal division operation.
This is a follow up of remaining comments from #6851
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#7387 from adrian-wang/udflgfollow and squashes the following commits:
6163e62 [Daoyuan Wang] add skipping null values
e8c2e09 [Daoyuan Wang] use seq
8362966 [Daoyuan Wang] pr6851 follow up
Author: Vinod K C <vinod.kc@huawei.com>
Closes#7040 from vinodkc/fix_CaseKeyWhen_equalNullSafe and squashes the following commits:
be5e641 [Vinod K C] Renamed equalNullSafe to threeValueEquals
aac9f67 [Vinod K C] Updated test suite and genCode method
f2d0b53 [Vinod K C] Fix equalNullSafe comparison
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#7355 from cloud-fan/fromString and squashes the following commits:
3bbb9d6 [Wenchen Fan] fix code gen
7dab957 [Wenchen Fan] naming fix
0fbbe19 [Wenchen Fan] address comments
ac1f3d1 [Wenchen Fan] Support casting between IntervalType and StringType
chenghao-intel zhichao-li qiansl127
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#6851 from adrian-wang/udflg and squashes the following commits:
0f1bff2 [Daoyuan Wang] address comments from davis
7a6bdbb [Daoyuan Wang] add '.' for hex()
c1f6824 [Daoyuan Wang] add codegen, test for all types
ec625b0 [Daoyuan Wang] conditional function: least/greatest