Commit graph

771 commits

Author SHA1 Message Date
Liang-Chi Hsieh eba6a1af4c [SPARK-8945][SQL] Add add and subtract expressions for IntervalType
JIRA: https://issues.apache.org/jira/browse/SPARK-8945

Add add and subtract expressions for IntervalType.

Author: Liang-Chi Hsieh <viirya@appier.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>

Closes #7398 from viirya/interval_add_subtract and squashes the following commits:

acd1f1e [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into interval_add_subtract
5abae28 [Liang-Chi Hsieh] For comments.
6f5b72e [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into interval_add_subtract
dbe3906 [Liang-Chi Hsieh] For comments.
13a2fc5 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into interval_add_subtract
83ec129 [Liang-Chi Hsieh] Remove intervalMethod.
acfe1ab [Liang-Chi Hsieh] Fix scala style.
d3e9d0e [Liang-Chi Hsieh] Add add and subtract expressions for IntervalType.
2015-07-17 09:38:08 -07:00
zhichao.li 305e77cd83 [SPARK-8209[SQL]Add function conv
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
2015-07-17 09:32:27 -07:00
Wenchen Fan 59d24c226a [SPARK-9130][SQL] throw exception when check equality between external and internal row
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
2015-07-17 09:31:13 -07:00
Davies Liu ec8973d124 [SPARK-9022] [SQL] Generated projections for UnsafeRow
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
2015-07-17 01:27:14 -07:00
Wenchen Fan 3f6d28a5ca [SPARK-9102] [SQL] Improve project collapse with nondeterministic expressions
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
2015-07-17 00:59:15 -07:00
Reynold Xin 111c05538d Added inline comment for the canEqual PR by @cloud-fan. 2015-07-16 23:13:06 -07:00
Wenchen Fan f893955b9c [SPARK-8899] [SQL] remove duplicated equals method for Row
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7291 from cloud-fan/row and squashes the following commits:

a11addf [Wenchen Fan] move hashCode back to internal row
2de6180 [Wenchen Fan] making apply() call to get()
fbe1b24 [Wenchen Fan] add null check
ebdf148 [Wenchen Fan] address comments
25ef087 [Wenchen Fan] remove duplicated equals method for Row
2015-07-16 21:41:36 -07:00
Reynold Xin fec10f0c63 [SPARK-9085][SQL] Remove LeafNode, UnaryNode, BinaryNode from TreeNode.
This builds on #7433 but also removes LeafNode/UnaryNode. These are slightly more complicated to remove. I had to change some abstract classes to traits in order for it to work.

The problem with LeafNode/UnaryNode is that they are often mixed in at the end of an Expression, and then the toString function actually gets resolved to the ones defined in TreeNode, rather than in Expression.

Author: Reynold Xin <rxin@databricks.com>

Closes #7434 from rxin/remove-binary-unary-leaf-node and squashes the following commits:

9e8a4de [Reynold Xin] Generator should not be foldable.
3135a8b [Reynold Xin] SortOrder should not be foldable.
9c589cf [Reynold Xin] Fixed one more test case...
2225331 [Reynold Xin] Aggregate expressions should not be foldable.
16b5c90 [Reynold Xin] [SPARK-9085][SQL] Remove LeafNode, UnaryNode, BinaryNode from TreeNode.
2015-07-16 13:58:39 -07:00
Tarek Auel 4ea6480a3b [SPARK-8995] [SQL] cast date strings like '2015-01-01 12:15:31' to date
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
2015-07-16 08:26:39 -07:00
Cheng Hao e27212317c [SPARK-8972] [SQL] Incorrect result for rollup
We don't support the complex expression keys in the rollup/cube, and we even will not report it if we have the complex group by keys, that will cause very confusing/incorrect result.

e.g. `SELECT key%100 FROM src GROUP BY key %100 with ROLLUP`

This PR adds an additional project during the analyzing for the complex GROUP BY keys, and that projection will be the child of `Expand`, so to `Expand`, the GROUP BY KEY are always the simple key(attribute names).

Author: Cheng Hao <hao.cheng@intel.com>

Closes #7343 from chenghao-intel/expand and squashes the following commits:

1ebbb59 [Cheng Hao] update the comment
827873f [Cheng Hao] update as feedback
34def69 [Cheng Hao] Add more unit test and comments
c695760 [Cheng Hao] fix bug of incorrect result for rollup
2015-07-15 23:35:27 -07:00
Wenchen Fan ba33096846 [SPARK-9068][SQL] refactor the implicit type cast code
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
2015-07-15 22:27:39 -07:00
Cheng Hao 42dea3acf9 [SPARK-8245][SQL] FormatNumber/Length Support for Expression
- `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
2015-07-15 21:47:21 -07:00
Yin Huai 9c64a75bfc [SPARK-9060] [SQL] Revert SPARK-8359, SPARK-8800, and SPARK-8677
JIRA: https://issues.apache.org/jira/browse/SPARK-9060

This PR reverts:
* 31bd30687b (SPARK-8359)
* 24fda73811 (SPARK-8677)
* 4b5cfc988f (SPARK-8800)

Author: Yin Huai <yhuai@databricks.com>

Closes #7426 from yhuai/SPARK-9060 and squashes the following commits:

651264d [Yin Huai] Revert "[SPARK-8359] [SQL] Fix incorrect decimal precision after multiplication"
cfda7e4 [Yin Huai] Revert "[SPARK-8677] [SQL] Fix non-terminating decimal expansion for decimal divide operation"
2de9afe [Yin Huai] Revert "[SPARK-8800] [SQL] Fix inaccurate precision/scale of Decimal division operation"
2015-07-15 21:08:30 -07:00
Reynold Xin b0645195d0 [SPARK-9086][SQL] Remove BinaryNode from TreeNode.
These traits are not super useful, and yet cause problems with toString in expressions due to the orders they are mixed in.

Author: Reynold Xin <rxin@databricks.com>

Closes #7433 from rxin/remove-binary-node and squashes the following commits:

1881f78 [Reynold Xin] [SPARK-9086][SQL] Remove BinaryNode from TreeNode.
2015-07-15 17:50:11 -07:00
Reynold Xin affbe329ae [SPARK-9071][SQL] MonotonicallyIncreasingID and SparkPartitionID should be marked as nondeterministic.
I also took the chance to more explicitly define the semantics of deterministic.

Author: Reynold Xin <rxin@databricks.com>

Closes #7428 from rxin/non-deterministic and squashes the following commits:

a760827 [Reynold Xin] [SPARK-9071][SQL] MonotonicallyIncreasingID and SparkPartitionID should be marked as nondeterministic.
2015-07-15 14:52:02 -07:00
zhichao.li a9385271a9 [SPARK-8221][SQL]Add pmod function
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
2015-07-15 10:43:38 -07:00
Wenchen Fan fa4ec3606a [SPARK-9020][SQL] Support mutable state in code gen expressions
We can keep expressions' mutable states in generated class(like `SpecificProjection`) as member variables, so that we can read and modify them inside codegened expressions.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7392 from cloud-fan/mutable-state and squashes the following commits:

eb3a221 [Wenchen Fan] fix order
73144d8 [Wenchen Fan] naming improvement
318f41d [Wenchen Fan] address more comments
d43b65d [Wenchen Fan] address comments
fd45c7a [Wenchen Fan] Support mutable state in code gen expressions
2015-07-15 10:31:39 -07:00
Reynold Xin 14935d846a [HOTFIX][SQL] Unit test breaking. 2015-07-15 00:12:21 -07:00
Yijie Shen f0e129740d [SPARK-8279][SQL]Add math function round
JIRA: https://issues.apache.org/jira/browse/SPARK-8279

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #6938 from yijieshen/udf_round_3 and squashes the following commits:

07a124c [Yijie Shen] remove useless def children
392b65b [Yijie Shen] add negative scale test in DecimalSuite
61760ee [Yijie Shen] address reviews
302a78a [Yijie Shen] Add dataframe function test
31dfe7c [Yijie Shen] refactor round to make it readable
8c7a949 [Yijie Shen] rebase & inputTypes update
9555e35 [Yijie Shen] tiny style fix
d10be4a [Yijie Shen] use TypeCollection to specify wanted input and implicit cast
c3b9839 [Yijie Shen] rely on implict cast to handle string input
b0bff79 [Yijie Shen] make round's inner method's name more meaningful
9bd6930 [Yijie Shen] revert accidental change
e6f44c4 [Yijie Shen] refactor eval and genCode
1b87540 [Yijie Shen] modify checkInputDataTypes using foldable
5486b2d [Yijie Shen] DataFrame API modification
2077888 [Yijie Shen] codegen versioned eval
6cd9a64 [Yijie Shen] refactor Round's constructor
9be894e [Yijie Shen] add round functions in o.a.s.sql.functions
7c83e13 [Yijie Shen] more tests on round
56db4bb [Yijie Shen] Add decimal support to Round
7e163ae [Yijie Shen] style fix
653d047 [Yijie Shen] Add math function round
2015-07-14 23:30:41 -07:00
Reynold Xin f23a721c10 [SPARK-8993][SQL] More comprehensive type checking in expressions.
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.
2015-07-14 22:52:53 -07:00
Josh Rosen e965a798d0 [SPARK-9045] Fix Scala 2.11 build break in UnsafeExternalRowSorter
This fixes a compilation break in under Scala 2.11:

```
[error] /home/jenkins/workspace/Spark-Master-Scala211-Compile/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java:135: error: <anonymous org.apache.spark.sql.execution.UnsafeExternalRowSorter$1> is not abstract and does not override abstract method <B>minBy(Function1<InternalRow,B>,Ordering<B>) in TraversableOnce
[error]       return new AbstractScalaRowIterator() {
[error]                                             ^
[error]   where B,A are type-variables:
[error]     B extends Object declared in method <B>minBy(Function1<A,B>,Ordering<B>)
[error]     A extends Object declared in interface TraversableOnce
[error] 1 error
```

The workaround for this is to make `AbstractScalaRowIterator` into a concrete class.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7405 from JoshRosen/SPARK-9045 and squashes the following commits:

cbcbb4c [Josh Rosen] Forgot that we can't use the ??? operator anymore
577ba60 [Josh Rosen] [SPARK-9045] Fix Scala 2.11 build break in UnsafeExternalRowSorter.
2015-07-14 17:21:48 -07:00
Josh Rosen 11e5c37286 [SPARK-8962] Add Scalastyle rule to ban direct use of Class.forName; fix existing uses
This pull request adds a Scalastyle regex rule which fails the style check if `Class.forName` is used directly.  `Class.forName` always loads classes from the default / system classloader, but in a majority of cases, we should be using Spark's own `Utils.classForName` instead, which tries to load classes from the current thread's context classloader and falls back to the classloader which loaded Spark when the context classloader is not defined.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7350)
<!-- Reviewable:end -->

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7350 from JoshRosen/ban-Class.forName and squashes the following commits:

e3e96f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
c0b7885 [Josh Rosen] Hopefully fix the last two cases
d707ba7 [Josh Rosen] Fix uses of Class.forName that I missed in my first cleanup pass
046470d [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
62882ee [Josh Rosen] Fix uses of Class.forName or add exclusion.
d9abade [Josh Rosen] Add stylechecker rule to ban uses of Class.forName
2015-07-14 16:08:17 -07:00
Liang-Chi Hsieh 4b5cfc988f [SPARK-8800] [SQL] Fix inaccurate precision/scale of Decimal division operation
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.
2015-07-14 14:19:27 -07:00
Wenchen Fan 59d820aa8d [SPARK-9029] [SQL] shortcut CaseKeyWhen if key is null
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7389 from cloud-fan/case-when and squashes the following commits:

ea4b6ba [Wenchen Fan] shortcut for case key when
2015-07-14 10:20:15 -07:00
Daoyuan Wang 257236c3e1 [SPARK-6851] [SQL] function least/greatest follow up
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
2015-07-14 01:09:33 -07:00
Vinod K C 4c797f2b09 [SPARK-8636] [SQL] Fix equalNullSafe comparison
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
2015-07-13 12:51:33 -07:00
Wenchen Fan 6b89943834 [SPARK-8944][SQL] Support casting between IntervalType and StringType
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
2015-07-13 00:49:39 -07:00
Daoyuan Wang 92540d22e4 [SPARK-8203] [SPARK-8204] [SQL] conditional function: least/greatest
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
2015-07-13 00:14:32 -07:00
Wenchen Fan c472eb17ae [SPARK-8970][SQL] remove unnecessary abstraction for ExtractValue
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7339 from cloud-fan/minor and squashes the following commits:

84a2128 [Wenchen Fan] remove unapply
6a37c12 [Wenchen Fan] remove unnecessary abstraction for ExtractValue
2015-07-10 23:25:11 -07:00
Josh Rosen fb8807c9b0 [SPARK-7078] [SPARK-7079] Binary processing sort for Spark SQL
This patch adds a cache-friendly external sorter which operates on serialized bytes and uses this sorter to implement a new sort operator for Spark SQL and DataFrames.

### Overview of the new sorter

The new sorter design is inspired by [Alphasort](http://research.microsoft.com/pubs/68249/alphasort.doc) and implements a key-prefix optimization in order to improve the cache friendliness of the sort.  In naive sort implementations, the sorting algorithm operates on an array of record pointers.  To compare two records for ordering, the sorter must dereference these pointers, which likely involves random memory access, then compare the objects themselves.

![image](https://cloud.githubusercontent.com/assets/50748/8611390/3b1402ae-2675-11e5-8308-1a10bf347e6e.png)

In a key-prefix sort, the sort operates on an array which stores the record pointer alongside a prefix of the record's key. When comparing two records for ordering, the sorter first compares the the stored key prefixes. If the ordering can be determined from the key prefixes (i.e. the prefixes are unequal), then the sort can avoid directly comparing the records, avoiding random memory accesses and full record comparisons. For example, if we're sorting a list of strings then we can store the first 8 bytes of the UTF-8 encoded string as the key-prefix and can perform unsigned byte-at-a-time comparisons to determine the ordering of strings based on their prefixes, only resorting to full comparisons for strings that share a common prefix.  In cases where the sort key can fit entirely in the space allotted for the key prefix (e.g. the sorting key is an integer), we completely avoid direct record comparison.

In this patch's implementation of key-prefix sorting, our sorter's internal array stores a 64-bit long and 64-bit pointer for each record being sorted. The key prefixes are generated by the user when inserting records into the sorter, which uses a user-defined comparison function for comparing them.  The `PrefixComparators` object implements a set of comparators for many common types, including primitive numeric types and UTF-8 strings.

The actual sorting is implemented by `UnsafeInMemorySorter`.  Most consumers will not use this directly, but instead will use `UnsafeExternalSorter`, a class which implements a sort that can spill to disk in response to memory pressure.  Internally, `UnsafeExternalSorter` creates `UnsafeInMemorySorters` to perform sorting and uses `UnsafeSortSpillReader/Writer` to spill and read back runs of sorted records and `UnsafeSortSpillMerger` to merge multiple sorted spills into a single sorted iterator.  This external sorter integrates with Spark's existing ShuffleMemoryManager for controlling spilling.

Many parts of this sorter's design are based on / copied from the more specialized external sort implementation that I designed for the new UnsafeShuffleManager write path; see #5868 for more details on that patch.

### Sorting rows in Spark SQL

For now, `UnsafeExternalSorter` is only used by Spark SQL, which uses it to implement a new sort operator, `UnsafeExternalSort`.  This sort operator uses a SQL-specific class called `UnsafeExternalRowSorter` that configures an `UnsafeExternalSorter` to use prefix generators and comparators that operate on rows encoded in the UnsafeRow format that was designed for Project Tungsten.

I used some interesting unit-testing techniques to test this patch's SQL-specific components.  `UnsafeExternalSortSuite` uses the SQL random data generators introduced in #7176 to test the UnsafeSort operator with all atomic types both with and without nullability and in both ascending and descending sort orders.  `PrefixComparatorsSuite` contains a cool use of ScalaCheck + ScalaTest's `GeneratorDrivenPropertyChecks` in order to test UTF8String prefix comparison.

### Misc. additional improvements made in this patch

This patch made several miscellaneous improvements to related code in Spark SQL:

- The logic for selecting physical sort operator implementations, which was partially duplicated in both `Exchange` and `SparkStrategies, has now been consolidated into a `getSortOperator()` helper function in `SparkStrategies`.
- The `SparkPlanTest` unit testing helper trait has been extended with new methods for comparing the output produced by two different physical plans. This makes it easy to write tests which assert that two physical operator implementations should produce the same output.  I also added a method for disabling the implicit sorting of outputs prior to comparing them, a change which is necessary in order to be able to write proper SparkPlan tests for sort operators.

### Tasks deferred to followup patches

While most of this patch's features are reasonably well-tested and complete, there are a number of tasks that are intentionally being deferred to followup patches:

- Add tests which mock the ShuffleMemoryManager to check that memory pressure properly triggers spilling (there are examples of this type of test in #5868).
- Add tests to ensure that spill files are properly cleaned up after errors.  I'd like to do this in the context of a patch which introduces more general metrics for ensuring proper cleanup of tasks' temporary files; see https://issues.apache.org/jira/browse/SPARK-8966 for more details.
- Metrics integration: there are some open questions regarding how to track / report spill metrics for non-shuffle operations, so I've deferred most of the IO / shuffle metrics integration for now.
- Performance profiling.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/6444)
<!-- Reviewable:end -->

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6444 from JoshRosen/sql-external-sort and squashes the following commits:

6beb467 [Josh Rosen] Remove a bunch of overloaded methods to avoid default args. issue
2bbac9c [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
35dad9f [Josh Rosen] Make sortAnswers = false the default in SparkPlanTest
5135200 [Josh Rosen] Fix spill reading for large rows; add test
2f48777 [Josh Rosen] Add test and fix bug for sorting empty arrays
d1e28bc [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
cd05866 [Josh Rosen] Fix scalastyle
3947fc1 [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
d13ac55 [Josh Rosen] Hacky approach to copying of UnsafeRows for sort followed by limit.
845bea3 [Josh Rosen] Remove unnecessary zeroing of row conversion buffer
c56ec18 [Josh Rosen] Clean up final row copying code.
d31f180 [Josh Rosen] Re-enable NullType sorting test now that SPARK-8868 is fixed
844f4ca [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
293f109 [Josh Rosen] Add missing license header.
f99a612 [Josh Rosen] Fix bugs in string prefix comparison.
9d00afc [Josh Rosen] Clean up prefix comparators for integral types
88aff18 [Josh Rosen] NULL_PREFIX has to be negative infinity for floating point types
613e16f [Josh Rosen] Test with larger data.
1d7ffaa [Josh Rosen] Somewhat hacky fix for descending sorts
08701e7 [Josh Rosen] Fix prefix comparison of null primitives.
b86e684 [Josh Rosen] Set global = true in UnsafeExternalSortSuite.
1c7bad8 [Josh Rosen] Make sorting of answers explicit in SparkPlanTest.checkAnswer().
b81a920 [Josh Rosen] Temporarily enable only the passing sort tests
5d6109d [Josh Rosen] Fix inconsistent handling / encoding of record lengths.
87b6ed9 [Josh Rosen] Fix critical issues in test which led to false negatives.
8d7fbe7 [Josh Rosen] Fixes to multiple spilling-related bugs.
82e21c1 [Josh Rosen] Force spilling in UnsafeExternalSortSuite.
88b72db [Josh Rosen] Test ascending and descending sort orders.
f27be09 [Josh Rosen] Fix tests by binding attributes.
0a79d39 [Josh Rosen] Revert "Undo part of a SparkPlanTest change in #7162 that broke my test."
7c3c864 [Josh Rosen] Undo part of a SparkPlanTest change in #7162 that broke my test.
9969c14 [Josh Rosen] Merge remote-tracking branch 'origin/master' into sql-external-sort
5822e6f [Josh Rosen] Fix test compilation issue
939f824 [Josh Rosen] Remove code gen experiment.
0dfe919 [Josh Rosen] Implement prefix sort for strings (albeit inefficiently).
66a813e [Josh Rosen] Prefix comparators for float and double
b310c88 [Josh Rosen] Integrate prefix comparators for Int and Long (others coming soon)
95058d9 [Josh Rosen] Add missing SortPrefixUtils file
4c37ba6 [Josh Rosen] Add tests for sorting on all primitive types.
6890863 [Josh Rosen] Fix memory leak on empty inputs.
d246e29 [Josh Rosen] Fix consideration of column types when choosing sort implementation.
6b156fb [Josh Rosen] Some WIP work on prefix comparison.
7f875f9 [Josh Rosen] Commit failing test demonstrating bug in handling objects in spills
41b8881 [Josh Rosen] Get UnsafeInMemorySorterSuite to pass (WIP)
90c2b6a [Josh Rosen] Update test name
6d6a1e6 [Josh Rosen] Centralize logic for picking sort operator implementations
9869ec2 [Josh Rosen] Clean up Exchange code a bit
82bb0ec [Josh Rosen] Fix IntelliJ complaint due to negated if condition
1db845a [Josh Rosen] Many more changes to harmonize with shuffle sorter
ebf9eea [Josh Rosen] Harmonization with shuffle's unsafe sorter
206bfa2 [Josh Rosen] Add some missing newlines at the ends of files
26c8931 [Josh Rosen] Back out some Hive changes that aren't needed anymore
62f0bb8 [Josh Rosen] Update to reflect SparkPlanTest changes
21d7d93 [Josh Rosen] Back out of BlockObjectWriter change
7eafecf [Josh Rosen] Port test to SparkPlanTest
d468a88 [Josh Rosen] Update for InternalRow refactoring
269cf86 [Josh Rosen] Back out SMJ operator change; isolate changes to selection of sort op.
1b841ca [Josh Rosen] WIP towards copying
b420a71 [Josh Rosen] Move most of the existing SMJ code into Java.
dfdb93f [Josh Rosen] SparkFunSuite change
73cc761 [Josh Rosen] Fix whitespace
9cc98f5 [Josh Rosen] Move more code to Java; fix bugs in UnsafeRowConverter length type.
c8792de [Josh Rosen] Remove some debug logging
dda6752 [Josh Rosen] Commit some missing code from an old git stash.
58f36d0 [Josh Rosen] Merge in a sketch of a unit test for the new sorter (now failing).
2bd8c9a [Josh Rosen] Import my original tests and get them to pass.
d5d3106 [Josh Rosen] WIP towards external sorter for Spark SQL.
2015-07-10 16:44:51 -07:00
Jonathan Alter e14b545d2d [SPARK-7977] [BUILD] Disallowing println
Author: Jonathan Alter <jonalter@users.noreply.github.com>

Closes #7093 from jonalter/SPARK-7977 and squashes the following commits:

ccd44cc [Jonathan Alter] Changed println to log in ThreadingSuite
7fcac3e [Jonathan Alter] Reverting to println in ThreadingSuite
10724b6 [Jonathan Alter] Changing some printlns to logs in tests
eeec1e7 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
0b1dcb4 [Jonathan Alter] More println cleanup
aedaf80 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
925fd98 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
0c16fa3 [Jonathan Alter] Replacing some printlns with logs
45c7e05 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
5c8e283 [Jonathan Alter] Allowing println in audit-release examples
5b50da1 [Jonathan Alter] Allowing printlns in example files
ca4b477 [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
83ab635 [Jonathan Alter] Fixing new printlns
54b131f [Jonathan Alter] Merge branch 'master' of github.com:apache/spark into SPARK-7977
1cd8a81 [Jonathan Alter] Removing some unnecessary comments and printlns
b837c3a [Jonathan Alter] Disallowing println
2015-07-10 11:34:01 +01:00
Davies Liu c9e2ef52bb [SPARK-7902] [SPARK-6289] [SPARK-8685] [SQL] [PYSPARK] Refactor of serialization for Python DataFrame
This PR fix the long standing issue of serialization between Python RDD and DataFrame, it change to using a customized Pickler for InternalRow to enable customized unpickling (type conversion, especially for UDT), now we can support UDT for UDF, cc mengxr .

There is no generated `Row` anymore.

Author: Davies Liu <davies@databricks.com>

Closes #7301 from davies/sql_ser and squashes the following commits:

81bef71 [Davies Liu] address comments
e9217bd [Davies Liu] add regression tests
db34167 [Davies Liu] Refactor of serialization for Python DataFrame
2015-07-09 14:43:38 -07:00
Cheng Hao 0b0b9ceaf7 [SPARK-8247] [SPARK-8249] [SPARK-8252] [SPARK-8254] [SPARK-8257] [SPARK-8258] [SPARK-8259] [SPARK-8261] [SPARK-8262] [SPARK-8253] [SPARK-8260] [SPARK-8267] [SQL] Add String Expressions
Author: Cheng Hao <hao.cheng@intel.com>

Closes #6762 from chenghao-intel/str_funcs and squashes the following commits:

b09a909 [Cheng Hao] update the code as feedback
7ebbf4c [Cheng Hao] Add more string expressions
2015-07-09 11:11:34 -07:00
Wenchen Fan f6c0bd5c37 [SPARK-8938][SQL] Implement toString for Interval data type
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7315 from cloud-fan/toString and squashes the following commits:

4fc8d80 [Wenchen Fan] Implement toString for Interval data type
2015-07-09 10:04:42 -07:00
Reynold Xin a870a82fb6 [SPARK-8926][SQL] Code review followup.
I merged https://github.com/apache/spark/pull/7303 so it unblocks another PR. This addresses my own code review comment for that PR.

Author: Reynold Xin <rxin@databricks.com>

Closes #7313 from rxin/adt and squashes the following commits:

7ade82b [Reynold Xin] Fixed unit tests.
f8d5533 [Reynold Xin] [SPARK-8926][SQL] Code review followup.
2015-07-09 10:01:33 -07:00
Reynold Xin e204d22bb7 [SPARK-8948][SQL] Remove ExtractValueWithOrdinal abstract class
Also added more documentation for the file.

Author: Reynold Xin <rxin@databricks.com>

Closes #7316 from rxin/extract-value and squashes the following commits:

069cb7e [Reynold Xin] Removed ExtractValueWithOrdinal.
621b705 [Reynold Xin] Reverted a line.
11ebd6c [Reynold Xin] [Minor][SQL] Improve documentation for complex type extractors.
2015-07-09 10:01:01 -07:00
Tarek Auel a1964e9d90 [SPARK-8830] [SQL] native levenshtein distance
Jira: https://issues.apache.org/jira/browse/SPARK-8830

rxin and HuJiayin can you have a look on it.

Author: Tarek Auel <tarek.auel@googlemail.com>

Closes #7236 from tarekauel/native-levenshtein-distance and squashes the following commits:

ee4c4de [Tarek Auel] [SPARK-8830] implemented improvement proposals
c252e71 [Tarek Auel] [SPARK-8830] removed chartAt; use unsafe method for byte array comparison
ddf2222 [Tarek Auel] Merge branch 'master' into native-levenshtein-distance
179920a [Tarek Auel] [SPARK-8830] added description
5e9ed54 [Tarek Auel] [SPARK-8830] removed StringUtils import
dce4308 [Tarek Auel] [SPARK-8830] native levenshtein distance
2015-07-09 09:23:35 -07:00
Wenchen Fan 09cb0d9c2d [SPARK-8942][SQL] use double not decimal when cast double and float to timestamp
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7312 from cloud-fan/minor and squashes the following commits:

a4589fa [Wenchen Fan] use double not decimal when cast double and float to timestamp
2015-07-09 00:26:25 -07:00
Michael Armbrust 768907eb7b [SPARK-8926][SQL] Good errors for ExpectsInputType expressions
For example: `cannot resolve 'testfunction(null)' due to data type mismatch: argument 1 is expected to be of type int, however, null is of type datetype.`

Author: Michael Armbrust <michael@databricks.com>

Closes #7303 from marmbrus/expectsTypeErrors and squashes the following commits:

c654a0e [Michael Armbrust] fix udts and make errors pretty
137160d [Michael Armbrust] style
5428fda [Michael Armbrust] style
10fac82 [Michael Armbrust] [SPARK-8926][SQL] Good errors for ExpectsInputType expressions
2015-07-08 22:05:58 -07:00
Josh Rosen b55499a44a [SPARK-8932] Support copy() for UnsafeRows that do not use ObjectPools
We call Row.copy() in many places throughout SQL but UnsafeRow currently throws UnsupportedOperationException when copy() is called.

Supporting copying when ObjectPool is used may be difficult, since we may need to handle deep-copying of objects in the pool. In addition, this copy() method needs to produce a self-contained row object which may be passed around / buffered by downstream code which does not understand the UnsafeRow format.

In the long run, we'll need to figure out how to handle the ObjectPool corner cases, but this may be unnecessary if other changes are made. Therefore, in order to unblock my sort patch (#6444) I propose that we support copy() for the cases where UnsafeRow does not use an ObjectPool and continue to throw UnsupportedOperationException when an ObjectPool is used.

This patch accomplishes this by modifying UnsafeRow so that it knows the size of the row's backing data in order to be able to copy it into a byte array.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7306 from JoshRosen/SPARK-8932 and squashes the following commits:

338e6bf [Josh Rosen] Support copy for UnsafeRows that do not use ObjectPools.
2015-07-08 20:28:05 -07:00
Yijie Shen a290814877 [SPARK-8866][SQL] use 1us precision for timestamp type
JIRA: https://issues.apache.org/jira/browse/SPARK-8866

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #7283 from yijieshen/micro_timestamp and squashes the following commits:

dc735df [Yijie Shen] update CastSuite to avoid round error
714eaea [Yijie Shen] add timestamp_udf into blacklist due to precision lose
c3ca2f4 [Yijie Shen] fix unhandled case in CurrentTimestamp
8d4aa6b [Yijie Shen] use 1us precision for timestamp type
2015-07-08 20:20:17 -07:00
Cheng Lian 4ffc27caaf [SPARK-6123] [SPARK-6775] [SPARK-6776] [SQL] Refactors Parquet read path for interoperability and backwards-compatibility
This PR is a follow-up of #6617 and is part of [SPARK-6774] [2], which aims to ensure interoperability and backwards-compatibility for Spark SQL Parquet support.  And this one fixes the read path.  Now Spark SQL is expected to be able to read legacy Parquet data files generated by most (if not all) common libraries/tools like parquet-thrift, parquet-avro, and parquet-hive. However, we still need to refactor the write path to write standard Parquet LISTs and MAPs ([SPARK-8848] [4]).

### Major changes

1. `CatalystConverter` class hierarchy refactoring

   - Replaces `CatalystConverter` trait with a much simpler `ParentContainerUpdater`.

     Now instead of extending the original `CatalystConverter` trait, every converter class accepts an updater which is responsible for propagating the converted value to some parent container. For example, appending array elements to a parent array buffer, appending a key-value pairs to a parent mutable map, or setting a converted value to some specific field of a parent row. Root converter doesn't have a parent and thus uses a `NoopUpdater`.

     This simplifies the design since converters don't need to care about details of their parent converters anymore.

   - Unifies `CatalystRootConverter`, `CatalystGroupConverter` and `CatalystPrimitiveRowConverter` into `CatalystRowConverter`

     Specifically, now all row objects are represented by `SpecificMutableRow` during conversion.

   - Refactors `CatalystArrayConverter`, and removes `CatalystArrayContainsNullConverter` and `CatalystNativeArrayConverter`

     `CatalystNativeArrayConverter` was probably designed with the intention of avoiding boxing costs. However, the way it uses Scala generics actually doesn't achieve this goal.

     The new `CatalystArrayConverter` handles both nullable and non-nullable array elements in a consistent way.

   - Implements backwards-compatibility rules in `CatalystArrayConverter`

     When Parquet records are being converted, schema of Parquet files should have already been verified. So we only need to care about the structure rather than field names in the Parquet schema. Since all map objects represented in legacy systems have the same structure as the standard one (see [backwards-compatibility rules for MAP] [1]), we only need to deal with LIST (namely array) in `CatalystArrayConverter`.

2. Requested columns handling

   When specifying requested columns in `RowReadSupport`, we used to use a Parquet `MessageType` converted from a Catalyst `StructType` which contains all requested columns.  This is not preferable when taking compatibility and interoperability into consideration.  Because the actual Parquet file may have different physical structure from the converted schema.

   In this PR, the schema for requested columns is constructed using the following method:

   - For a column that exists in the target Parquet file, we extract the column type by name from the full file schema, and construct a single-field `MessageType` for that column.
   - For a column that doesn't exist in the target Parquet file, we create a single-field `StructType` and convert it to a `MessageType` using `CatalystSchemaConverter`.
   - Unions all single-field `MessageType`s into a full schema containing all requested fields

   With this change, we also fix [SPARK-6123] [3] by validating the global schema against each individual Parquet part-files.

### Testing

This PR also adds compatibility tests for parquet-avro, parquet-thrift, and parquet-hive. Please refer to `README.md` under `sql/core/src/test` for more information about these tests. To avoid build time code generation and adding extra complexity to the build system, Java code generated from testing Thrift schema and Avro IDL is also checked in.

[1]: https://github.com/apache/incubator-parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules-1
[2]: https://issues.apache.org/jira/browse/SPARK-6774
[3]: https://issues.apache.org/jira/browse/SPARK-6123
[4]: https://issues.apache.org/jira/browse/SPARK-8848

Author: Cheng Lian <lian@databricks.com>

Closes #7231 from liancheng/spark-6776 and squashes the following commits:

360fe18 [Cheng Lian] Adds ParquetHiveCompatibilitySuite
c6fbc06 [Cheng Lian] Removes WIP file committed by mistake
b8c1295 [Cheng Lian] Excludes the whole parquet package from MiMa
598c3e8 [Cheng Lian] Adds extra Maven repo for hadoop-lzo, which is a transitive dependency of parquet-thrift
926af87 [Cheng Lian] Simplifies Parquet compatibility test suites
7946ee1 [Cheng Lian] Fixes Scala styling issues
3d7ab36 [Cheng Lian] Fixes .rat-excludes
a8f13bb [Cheng Lian] Using Parquet writer API to do compatibility tests
f2208cd [Cheng Lian] Adds README.md for Thrift/Avro code generation
1d390aa [Cheng Lian] Adds parquet-thrift compatibility test
440f7b3 [Cheng Lian] Adds generated files to .rat-excludes
13b9121 [Cheng Lian] Adds ParquetAvroCompatibilitySuite
06cfe9d [Cheng Lian] Adds comments about TimestampType handling
a099d3e [Cheng Lian] More comments
0cc1b37 [Cheng Lian] Fixes MiMa checks
884d3e6 [Cheng Lian] Fixes styling issue and reverts unnecessary changes
802cbd7 [Cheng Lian] Fixes bugs related to schema merging and empty requested columns
38fe1e7 [Cheng Lian] Adds explicit return type
7fb21f1 [Cheng Lian] Reverts an unnecessary debugging change
1781dff [Cheng Lian] Adds test case for SPARK-8811
6437d4b [Cheng Lian] Assembles requested schema from Parquet file schema
bcac49f [Cheng Lian] Removes the 16-byte restriction of decimals
a74fb2c [Cheng Lian] More comments
0525346 [Cheng Lian] Removes old Parquet record converters
03c3bd9 [Cheng Lian] Refactors Parquet read path to implement backwards-compatibility rules
2015-07-08 15:51:01 -07:00
Keuntae Park f031543782 [SPARK-8783] [SQL] CTAS with WITH clause does not work
Currently, CTESubstitution only handles the case that WITH is on the top of the plan.
I think it SHOULD handle the case that WITH is child of CTAS.
This patch simply changes 'match' to 'transform' for recursive search of WITH in the plan.

Author: Keuntae Park <sirpkt@apache.org>

Closes #7180 from sirpkt/SPARK-8783 and squashes the following commits:

e4428f0 [Keuntae Park] Merge remote-tracking branch 'upstream/master' into CTASwithWITH
1671c77 [Keuntae Park] WITH clause can be inside CTAS
2015-07-08 14:29:52 -07:00
Wenchen Fan 0ba98c04c7 [SPARK-8753][SQL] Create an IntervalType data type
We need a new data type to represent time intervals. Because we can't determine how many days in a month, so we need 2 values for interval: a int `months`, a long `microseconds`.

The interval literal syntax looks like:
`interval 3 years -4 month 4 weeks 3 second`

Because we use number of 100ns as value of `TimestampType`, so it may not makes sense to support nano second unit.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7226 from cloud-fan/interval and squashes the following commits:

632062d [Wenchen Fan] address comments
ac348c3 [Wenchen Fan] use case class
0342d2e [Wenchen Fan] use array byte
df9256c [Wenchen Fan] fix style
fd6f18a [Wenchen Fan] address comments
1856af3 [Wenchen Fan] support interval type
2015-07-08 10:51:32 -07:00
Cheng Hao 351a36d0c5 [SPARK-8883][SQL]Remove the OverrideFunctionRegistry
Remove the `OverrideFunctionRegistry` from the Spark SQL, as the subclasses of `FunctionRegistry` have their own way to the delegate to the right underlying `FunctionRegistry`.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #7260 from chenghao-intel/override and squashes the following commits:

164d093 [Cheng Hao] enable the function registry
2ca8459 [Cheng Hao] remove the OverrideFunctionRegistry
2015-07-08 00:10:24 -07:00
Reynold Xin 61c3cf793d [SPARK-8879][SQL] Remove EmptyRow class.
As a baby step towards no megamorphic InternalRow.

Author: Reynold Xin <rxin@databricks.com>

Closes #7277 from rxin/remove-empty-row and squashes the following commits:

594100e [Reynold Xin] [SPARK-8879][SQL] Remove EmptyRow class.
2015-07-07 22:12:46 -07:00
Reynold Xin 5d603dfe49 [SPARK-8878][SQL] Improve unit test coverage for bitwise expressions.
Author: Reynold Xin <rxin@databricks.com>

Closes #7273 from rxin/bitwise-unittest and squashes the following commits:

60c5667 [Reynold Xin] [SPARK-8878][SQL] Improve unit test coverage for bitwise expressions.
2015-07-07 19:12:40 -07:00
Davies Liu 4ca90935c5 [SPARK-7190] [SPARK-8804] [SPARK-7815] [SQL] unsafe UTF8String
Let UTF8String work with binary buffer. Before we have better idea on manage the lifecycle of UTF8String in Row, we still do the copy when calling `UnsafeRow.get()` for StringType.

cc rxin JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #7197 from davies/unsafe_string and squashes the following commits:

51b0ea0 [Davies Liu] fix test
50c1ebf [Davies Liu] remove optimization for upper/lower case
315d491 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_string
93fce17 [Davies Liu] address comment
e9ff7ba [Davies Liu] clean up
67ec266 [Davies Liu] fix bug
7b74b1f [Davies Liu] fallback to String if local dependent
ab7857c [Davies Liu] address comments
7da92f5 [Davies Liu] handle local in toUpperCase/toLowerCase
59dbb23 [Davies Liu] revert python change
d1e0716 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_string
002e35f [Davies Liu] rollback hashCode change
a87b7a8 [Davies Liu] improve toLowerCase and toUpperCase
76e794a [Davies Liu] fix test
8b2d5ce [Davies Liu] fix tests
fd3f0a6 [Davies Liu] bug fix
c4e9c88 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_string
c45d921 [Davies Liu] address comments
175405f [Davies Liu] unsafe UTF8String
2015-07-07 17:57:17 -07:00
Reynold Xin 770ff1025e [SPARK-8876][SQL] Remove InternalRow type alias in expressions package.
The type alias was there because initially when I moved Row around, I didn't want to do massive changes to the expression code. But now it should be pretty easy to just remove it. One less concept to worry about.

Author: Reynold Xin <rxin@databricks.com>

Closes #7270 from rxin/internalrow and squashes the following commits:

72fc842 [Reynold Xin] [SPARK-8876][SQL] Remove InternalRow type alias in expressions package.
2015-07-07 17:40:14 -07:00
Liang-Chi Hsieh da56c4e728 [SPARK-8794] [SQL] Make PrunedScan work for Sample
JIRA: https://issues.apache.org/jira/browse/SPARK-8794

Currently `PrunedScan` works only when followed by project or filter operations. However, even if there is a `Sample` between these operations and `PrunedScan`, `PrunedScan` should work too.

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #7228 from viirya/sample_prunedscan and squashes the following commits:

ede7cd8 [Liang-Chi Hsieh] Keep PrunedScanSuite untouched.
6f05d30 [Liang-Chi Hsieh] Move unit test to FilterPushdownSuite.
5f32473 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into sample_prunedscan
7e4ba76 [Liang-Chi Hsieh] Use Optimzier for push down projection and filter.
0686830 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into sample_prunedscan
df82785 [Liang-Chi Hsieh] Make PrunedScan work on Sample.
2015-07-07 15:49:22 -07:00
Wenchen Fan c46aaf47f3 [SPARK-8759][SQL] add default eval to binary and unary expression according to default behavior of nullable
We have `nullSafeCodeGen` to provide default code generation for binary and unary expression, and we can do the same thing for `eval`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7157 from cloud-fan/refactor and squashes the following commits:

f3987c6 [Wenchen Fan] refactor Expression
2015-07-06 22:13:50 -07:00
Yin Huai 7b467cc934 [SPARK-8588] [SQL] Regression test
This PR adds regression test for https://issues.apache.org/jira/browse/SPARK-8588 (fixed by 457d07eaa0).

Author: Yin Huai <yhuai@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>

Closes #7103 from yhuai/SPARK-8588-test and squashes the following commits:

eb5f418 [Yin Huai] Add a query test.
c61a173 [Yin Huai] Regression test for SPARK-8588.
2015-07-06 16:28:47 -07:00
Davies Liu 37e4d92142 [SPARK-8784] [SQL] Add Python API for hex and unhex
Add Python API for hex/unhex,  also cleanup Hex/Unhex

Author: Davies Liu <davies@databricks.com>

Closes #7223 from davies/hex and squashes the following commits:

6f1249d [Davies Liu] no explicit rule to cast string into binary
711a6ed [Davies Liu] fix test
f9fe5a3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex
f032fbb [Davies Liu] Merge branch 'hex' of github.com:davies/spark into hex
49e325f [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex
b31fc9a [Davies Liu] Update math.scala
25156b7 [Davies Liu] address comments and fix test
c3af78c [Davies Liu] address commments
1a24082 [Davies Liu] Add Python API for hex and unhex
2015-07-06 13:31:31 -07:00
Wenchen Fan 0e194645f4 [SPARK-8837][SPARK-7114][SQL] support using keyword in column name
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7237 from cloud-fan/parser and squashes the following commits:

e7b49bb [Wenchen Fan] support using keyword in column name
2015-07-06 13:26:46 -07:00
Reynold Xin 86768b7b3b [SPARK-8831][SQL] Support AbstractDataType in TypeCollection.
Otherwise it is impossible to declare an expression supporting DecimalType.

Author: Reynold Xin <rxin@databricks.com>

Closes #7232 from rxin/typecollection-adt and squashes the following commits:

934d3d1 [Reynold Xin] [SPARK-8831][SQL] Support AbstractDataType in TypeCollection.
2015-07-05 23:54:25 -07:00
Cheng Hao 6d0411b4f3 [SQL][Minor] Update the DataFrame API for encode/decode
This is a the follow up of #6843.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #7230 from chenghao-intel/str_funcs2_followup and squashes the following commits:

52cc553 [Cheng Hao] update the code as comment
2015-07-05 21:50:52 -07:00
Reynold Xin c991ef5abb [SPARK-8822][SQL] clean up type checking in math.scala.
Author: Reynold Xin <rxin@databricks.com>

Closes #7220 from rxin/SPARK-8822 and squashes the following commits:

0cda076 [Reynold Xin] Test cases.
22d0463 [Reynold Xin] Fixed type precedence.
beb2a97 [Reynold Xin] [SPARK-8822][SQL] clean up type checking in math.scala.
2015-07-04 11:55:20 -07:00
Reynold Xin 347cab85cd [SQL] More unit tests for implicit type cast & add simpleString to AbstractDataType.
Author: Reynold Xin <rxin@databricks.com>

Closes #7221 from rxin/implicit-cast-tests and squashes the following commits:

64b13bd [Reynold Xin] Fixed a bug ..
489b732 [Reynold Xin] [SQL] More unit tests for implicit type cast & add simpleString to AbstractDataType.
2015-07-04 11:55:04 -07:00
Tarek Auel 6b3574e687 [SPARK-8270][SQL] levenshtein distance
Jira: https://issues.apache.org/jira/browse/SPARK-8270

Info: I can not build the latest master, it stucks during the build process: `[INFO] Dependency-reduced POM written at: /Users/tarek/test/spark/bagel/dependency-reduced-pom.xml`

Author: Tarek Auel <tarek.auel@googlemail.com>

Closes #7214 from tarekauel/SPARK-8270 and squashes the following commits:

ab348b9 [Tarek Auel] Merge branch 'master' into SPARK-8270
a2ad318 [Tarek Auel] [SPARK-8270] changed order of fields
d91b12c [Tarek Auel] [SPARK-8270] python fix
adbd075 [Tarek Auel] [SPARK-8270] fixed typo
23185c9 [Tarek Auel] [SPARK-8270] levenshtein distance
2015-07-04 01:10:52 -07:00
Cheng Hao f35b0c3436 [SPARK-8238][SPARK-8239][SPARK-8242][SPARK-8243][SPARK-8268][SQL]Add ascii/base64/unbase64/encode/decode functions
Add `ascii`,`base64`,`unbase64`,`encode` and `decode` expressions.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #6843 from chenghao-intel/str_funcs2 and squashes the following commits:

78dee7d [Cheng Hao] base 64 -> base64
9d6f9f4 [Cheng Hao] remove the toString method for expressions
ed5c19c [Cheng Hao] update code as comments
96170fc [Cheng Hao] scalastyle issues
e2df768 [Cheng Hao] remove the unused import
491ce7b [Cheng Hao] add ascii/base64/unbase64/encode/decode functions
2015-07-03 23:45:21 -07:00
Josh Rosen f32487b7ca [SPARK-8777] [SQL] Add random data generator test utilities to Spark SQL
This commit adds a set of random data generation utilities to Spark SQL, for use in its own unit tests.

- `RandomDataGenerator.forType(DataType)` returns an `Option[() => Any]` that, if defined, contains a function for generating random values for the given DataType.  The random values use the external representations for the given DataType (for example, for DateType we return `java.sql.Date` instances instead of longs).
- `DateTypeTestUtilities` defines some convenience fields for looping over instances of data types.  For example, `numericTypes` holds `DataType` instances for all supported numeric types.  These constants will help us to raise the level of abstraction in our tests.  For example, it's now very easy to write a test which is parameterized by all common data types.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7176 from JoshRosen/sql-random-data-generators and squashes the following commits:

f71634d [Josh Rosen] Roll back ScalaCheck usage
e0d7d49 [Josh Rosen] Bump ScalaCheck version in LICENSE
89d86b1 [Josh Rosen] Bump ScalaCheck version.
0c20905 [Josh Rosen] Initial attempt at using ScalaCheck.
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.
2015-07-03 23:05:17 -07:00
Daoyuan Wang 9fb6b832bc [SPARK-8192] [SPARK-8193] [SQL] udf current_date, current_timestamp
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #6985 from adrian-wang/udfcurrent and squashes the following commits:

6a20b64 [Daoyuan Wang] remove codegen and add lazy in testsuite
27c9f95 [Daoyuan Wang] refine tests..
e11ae75 [Daoyuan Wang] refine tests
61ed3d5 [Daoyuan Wang] add in functions
98e8550 [Daoyuan Wang] fix sytle
427d9dc [Daoyuan Wang] add tests and codegen
0b69a1f [Daoyuan Wang] udf current
2015-07-03 22:19:43 -07:00
Cheolsoo Park 4a22bce8fc [SPARK-8572] [SQL] Type coercion for ScalaUDFs
Implemented type coercion for udf arguments in Scala. The changes include-
* Add `with ExpectsInputTypes ` to `ScalaUDF` class.
* Pass down argument types info from `UDFRegistration` and `functions`.

With this patch, the example query in [SPARK-8572](https://issues.apache.org/jira/browse/SPARK-8572) no longer throws a type cast error at runtime.

Also added a unit test to `UDFSuite` in which a decimal type is passed to a udf that expects an int.

Author: Cheolsoo Park <cheolsoop@netflix.com>

Closes #7203 from piaozhexiu/SPARK-8572 and squashes the following commits:

2d0ed15 [Cheolsoo Park] Incorporate comments
dce1efd [Cheolsoo Park] Fix unit tests and update the codegen script
066deed [Cheolsoo Park] Type coercion for udf inputs
2015-07-03 22:14:21 -07:00
zhichao.li ab535b9a1d [SPARK-8226] [SQL] Add function shiftrightunsigned
Author: zhichao.li <zhichao.li@intel.com>

Closes #7035 from zhichao-li/shiftRightUnsigned and squashes the following commits:

6bcca5a [zhichao.li] change coding style
3e9f5ae [zhichao.li] python style
d85ae0b [zhichao.li] add shiftrightunsigned
2015-07-03 15:39:16 -07:00
Reynold Xin 2848f4da47 [SPARK-8809][SQL] Remove ConvertNaNs analyzer rule.
"NaN" from string to double is already handled by Cast expression itself.

Author: Reynold Xin <rxin@databricks.com>

Closes #7206 from rxin/convertnans and squashes the following commits:

3d99c33 [Reynold Xin] [SPARK-8809][SQL] Remove ConvertNaNs analyzer rule.
2015-07-03 00:25:02 -07:00
Reynold Xin a59d14f623 [SPARK-8801][SQL] Support TypeCollection in ExpectsInputTypes
This patch adds a new TypeCollection AbstractDataType that can be used by expressions to specify more than one expected input types.

Author: Reynold Xin <rxin@databricks.com>

Closes #7202 from rxin/type-collection and squashes the following commits:

c714ca1 [Reynold Xin] Fixed style.
a0c0d12 [Reynold Xin] Fixed bugs and unit tests.
d8b8ae7 [Reynold Xin] Added TypeCollection.
2015-07-02 21:45:25 -07:00
Reynold Xin dfd8bac8f5 Minor style fix for the previous commit. 2015-07-02 20:47:04 -07:00
zhichao.li 1a7a7d7d57 [SPARK-8213][SQL]Add function factorial
Author: zhichao.li <zhichao.li@intel.com>

Closes #6822 from zhichao-li/factorial and squashes the following commits:

26edf4f [zhichao.li] add factorial
2015-07-02 20:37:31 -07:00
Josh Rosen d9838196ff [SPARK-8782] [SQL] Fix code generation for ORDER BY NULL
This fixes code generation for queries containing `ORDER BY NULL`.  Previously, the generated code would fail to compile.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7179 from JoshRosen/generate-order-fixes and squashes the following commits:

6ef49a6 [Josh Rosen] Fix ORDER BY NULL
0036696 [Josh Rosen] Add regression test for SPARK-8782 (ORDER BY NULL)
2015-07-02 18:07:09 -07:00
Reynold Xin e589e71a29 Revert "[SPARK-8784] [SQL] Add Python API for hex and unhex"
This reverts commit fc7aebd94a.
2015-07-02 16:25:10 -07:00
Davies Liu fc7aebd94a [SPARK-8784] [SQL] Add Python API for hex and unhex
Also improve the performance of hex/unhex

Author: Davies Liu <davies@databricks.com>

Closes #7181 from davies/hex and squashes the following commits:

f032fbb [Davies Liu] Merge branch 'hex' of github.com:davies/spark into hex
49e325f [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex
b31fc9a [Davies Liu] Update math.scala
25156b7 [Davies Liu] address comments and fix test
c3af78c [Davies Liu] address commments
1a24082 [Davies Liu] Add Python API for hex and unhex
2015-07-02 15:43:02 -07:00
Reynold Xin 52508beb65 [SPARK-8772][SQL] Implement implicit type cast for expressions that define input types.
Author: Reynold Xin <rxin@databricks.com>

Closes #7175 from rxin/implicitCast and squashes the following commits:

88080a2 [Reynold Xin] Clearer definition of implicit type cast.
f0ff97f [Reynold Xin] Added missing file.
c65e532 [Reynold Xin] [SPARK-8772][SQL] Implement implicit type cast for expressions that defines input types.
2015-07-02 14:16:14 -07:00
Yijie Shen 52302a8039 [SPARK-8407] [SQL] complex type constructors: struct and named_struct
This is a follow up of [SPARK-8283](https://issues.apache.org/jira/browse/SPARK-8283) ([PR-6828](https://github.com/apache/spark/pull/6828)), to support both `struct` and `named_struct` in Spark SQL.

After [#6725](https://github.com/apache/spark/pull/6828), the semantic of [`CreateStruct`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala#L56) methods have changed a little and do not limited to cols of `NamedExpressions`, it will name non-NamedExpression fields following the hive convention, col1, col2 ...

This PR would both loosen [`struct`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L723) to take children of `Expression` type and add `named_struct` support.

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #6874 from yijieshen/SPARK-8283 and squashes the following commits:

4cd3375ac [Yijie Shen] change struct documentation
d599d0b [Yijie Shen] rebase code
9a7039e [Yijie Shen] fix reviews and regenerate golden answers
b487354 [Yijie Shen] replace assert using checkAnswer
f07e114 [Yijie Shen] tiny fix
9613be9 [Yijie Shen] review fix
7fef712 [Yijie Shen] Fix checkInputTypes' implementation using foldable and nullable
60812a7 [Yijie Shen] Fix type check
828d694 [Yijie Shen] remove unnecessary resolved assertion inside dataType method
fd3cd8e [Yijie Shen] remove type check from eval
7a71255 [Yijie Shen] tiny fix
ccbbd86 [Yijie Shen] Fix reviews
47da332 [Yijie Shen] remove nameStruct API from DataFrame
917e680 [Yijie Shen] Fix reviews
4bd75ad [Yijie Shen] loosen struct method in functions.scala to take Expression children
0acb7be [Yijie Shen] Add CreateNamedStruct in both DataFrame function API and FunctionRegistery
2015-07-02 10:12:25 -07:00
Wenchen Fan afa021e03f [SPARK-8747] [SQL] fix EqualNullSafe for binary type
also improve tests for binary comparison.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7143 from cloud-fan/binary and squashes the following commits:

28a5b76 [Wenchen Fan] improve test
04ef4b0 [Wenchen Fan] fix equalNullSafe
2015-07-02 10:06:38 -07:00
Tarek Auel 5b3338130d [SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224

~~I am aware of #7174 and will update this pr, if it's merged.~~ Done
I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged

rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception).

If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX`

Author: Tarek Auel <tarek.auel@googlemail.com>

Closes #7178 from tarekauel/8223 and squashes the following commits:

8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 10:02:19 -07:00
zhichao.li b285ac5ba8 [SPARK-8227] [SQL] Add function unhex
cc chenghao-intel  adrian-wang

Author: zhichao.li <zhichao.li@intel.com>

Closes #7113 from zhichao-li/unhex and squashes the following commits:

379356e [zhichao.li] remove exception checking
a4ae6dc [zhichao.li] add udf_unhex to whitelist
fe5c14a [zhichao.li] add todigit
607d7a3 [zhichao.li] use checkInputTypes
bffd37f [zhichao.li] change to use Hex in apache common package
cde73f5 [zhichao.li] update to use AutoCastInputTypes
11945c7 [zhichao.li] style
c852d46 [zhichao.li] Add function unhex
2015-07-01 22:19:51 -07:00
Reynold Xin 9fd13d5613 [SPARK-8770][SQL] Create BinaryOperator abstract class.
Our current BinaryExpression abstract class is not for generic binary expressions, i.e. it requires left/right children to have the same type. However, due to its name, contributors build new binary expressions that don't have that assumption (e.g. Sha) and still extend BinaryExpression.

This patch creates a new BinaryOperator abstract class, and update the analyzer o only apply type casting rule there. This patch also adds the notion of "prettyName" to expressions, which defines the user-facing name for the expression.

Author: Reynold Xin <rxin@databricks.com>

Closes #7174 from rxin/binary-opterator and squashes the following commits:

f31900d [Reynold Xin] [SPARK-8770][SQL] Create BinaryOperator abstract class.
fceb216 [Reynold Xin] Merge branch 'master' of github.com:apache/spark into binary-opterator
d8518cf [Reynold Xin] Updated Python tests.
2015-07-01 21:14:13 -07:00
Reynold Xin 3a342dedc0 Revert "[SPARK-8770][SQL] Create BinaryOperator abstract class."
This reverts commit 2727789998.
2015-07-01 16:59:39 -07:00
Reynold Xin 2727789998 [SPARK-8770][SQL] Create BinaryOperator abstract class.
Our current BinaryExpression abstract class is not for generic binary expressions, i.e. it requires left/right children to have the same type. However, due to its name, contributors build new binary expressions that don't have that assumption (e.g. Sha) and still extend BinaryExpression.

This patch creates a new BinaryOperator abstract class, and update the analyzer o only apply type casting rule there. This patch also adds the notion of "prettyName" to expressions, which defines the user-facing name for the expression.

Author: Reynold Xin <rxin@databricks.com>

Closes #7170 from rxin/binaryoperator and squashes the following commits:

51264a5 [Reynold Xin] [SPARK-8770][SQL] Create BinaryOperator abstract class.
2015-07-01 16:56:48 -07:00
Davies Liu 3083e17645 [QUICKFIX] [SQL] fix copy of generated row
copy() of generated Row doesn't check nullability of columns

Author: Davies Liu <davies@databricks.com>

Closes #7163 from davies/fix_copy and squashes the following commits:

661a206 [Davies Liu] fix copy of generated row
2015-07-01 12:39:57 -07:00
Wenchen Fan 31b4a3d7f2 [SPARK-8621] [SQL] support empty string as column name
improve the empty check in `parseAttributeName` so that we can allow empty string as column name.
Close https://github.com/apache/spark/pull/7117

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7149 from cloud-fan/8621 and squashes the following commits:

efa9e3e [Wenchen Fan] support empty string
2015-07-01 10:31:35 -07:00
Reynold Xin 4137f769b8 [SPARK-8752][SQL] Add ExpectsInputTypes trait for defining expected input types.
This patch doesn't actually introduce any code that uses the new ExpectsInputTypes. It just adds the trait so others can use it. Also renamed the old expectsInputTypes function to just inputTypes.

We should add implicit type casting also in the future.

Author: Reynold Xin <rxin@databricks.com>

Closes #7151 from rxin/expects-input-types and squashes the following commits:

16cf07b [Reynold Xin] [SPARK-8752][SQL] Add ExpectsInputTypes trait for defining expected input types.
2015-07-01 10:30:54 -07:00
Reynold Xin fc3a6fe67f [SPARK-8749][SQL] Remove HiveTypeCoercion trait.
Moved all the rules into the companion object.

Author: Reynold Xin <rxin@databricks.com>

Closes #7147 from rxin/SPARK-8749 and squashes the following commits:

c1c6dc0 [Reynold Xin] [SPARK-8749][SQL] Remove HiveTypeCoercion trait.
2015-07-01 00:08:16 -07:00
Reynold Xin 365c14055e [SPARK-8748][SQL] Move castability test out from Cast case class into Cast object.
This patch moved resolve function in Cast case class into the companion object, and renamed it canCast. We can then use this in the analyzer without a Cast expr.

Author: Reynold Xin <rxin@databricks.com>

Closes #7145 from rxin/cast and squashes the following commits:

cd086a9 [Reynold Xin] Whitespace changes.
4d2d989 [Reynold Xin] [SPARK-8748][SQL] Move castability test out from Cast case class into Cast object.
2015-06-30 23:04:54 -07:00
Vinod K C b8e5bb6fc1 [SPARK-8628] [SQL] Race condition in AbstractSparkSQLParser.parse
Made lexical iniatialization as lazy val

Author: Vinod K C <vinod.kc@huawei.com>

Closes #7015 from vinodkc/handle_lexical_initialize_schronization and squashes the following commits:

b6d1c74 [Vinod K C] Avoided repeated lexical  initialization
5863cf7 [Vinod K C] Removed space
e27c66c [Vinod K C] Avoid reinitialization of lexical in parse method
ef4f60f [Vinod K C] Reverted import order
e9fc49a [Vinod K C] handle  synchronization in SqlLexical.initialize
2015-06-30 12:24:47 -07:00
Christian Kadner 1e1f339976 [SPARK-6785] [SQL] fix DateTimeUtils for dates before 1970
Hi Michael,
this Pull-Request is a follow-up to [PR-6242](https://github.com/apache/spark/pull/6242). I removed the two obsolete test cases from the HiveQuerySuite and deleted the corresponding golden answer files.
Thanks for your review!

Author: Christian Kadner <ckadner@us.ibm.com>

Closes #6983 from ckadner/SPARK-6785 and squashes the following commits:

ab1e79b [Christian Kadner] Merge remote-tracking branch 'origin/SPARK-6785' into SPARK-6785
1fed877 [Christian Kadner] [SPARK-6785][SQL] failed Scala style test, remove spaces on empty line DateTimeUtils.scala:61
9d8021d [Christian Kadner] [SPARK-6785][SQL] merge recent changes in DateTimeUtils & MiscFunctionsSuite
b97c3fb [Christian Kadner] [SPARK-6785][SQL] move test case for DateTimeUtils to DateTimeUtilsSuite
a451184 [Christian Kadner] [SPARK-6785][SQL] fix DateTimeUtils.fromJavaDate(java.util.Date) for Dates before 1970
2015-06-30 12:22:34 -07:00
Davies Liu fbb267ed6f [SPARK-8713] Make codegen thread safe
Codegen takes three steps:

1. Take a list of expressions, convert them into Java source code and a list of expressions that don't not support codegen (fallback to interpret mode).
2. Compile the Java source into Java class (bytecode)
3. Using the Java class and the list of expression to build a Projection.

Currently, we cache the whole three steps, the key is a list of expression, result is projection. Because some of expressions (which may not thread-safe, for example, Random) will be hold by the Projection, the projection maybe not thread safe.

This PR change to only cache the second step, then we can build projection using codegen even some expressions are not thread-safe, because the cache will not hold any expression anymore.

cc marmbrus rxin JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #7101 from davies/codegen_safe and squashes the following commits:

7dd41f1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into codegen_safe
847bd08 [Davies Liu] don't use scala.refect
4ddaaed [Davies Liu] Merge branch 'master' of github.com:apache/spark into codegen_safe
1793cf1 [Davies Liu] make codegen thread safe
2015-06-30 10:48:49 -07:00
Shilei 722aa5f48e [SPARK-8236] [SQL] misc functions: crc32
https://issues.apache.org/jira/browse/SPARK-8236

Author: Shilei <shilei.qian@intel.com>

Closes #7108 from qiansl127/Crc32 and squashes the following commits:

5477352 [Shilei] Change to AutoCastInputTypes
5f16e5d [Shilei] Add misc function crc32
2015-06-30 09:49:58 -07:00
Liang-Chi Hsieh a48e619153 [SPARK-8680] [SQL] Slightly improve PropagateTypes
JIRA: https://issues.apache.org/jira/browse/SPARK-8680

This PR slightly improve `PropagateTypes` in `HiveTypeCoercion`. It moves `q.inputSet` outside `q transformExpressions` instead calling `inputSet` multiple times. It also builds a map of attributes for looking attribute easily.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #7087 from viirya/improve_propagatetypes and squashes the following commits:

5c314c1 [Liang-Chi Hsieh] For comments.
913f6ad [Liang-Chi Hsieh] Slightly improve PropagateTypes.
2015-06-30 08:17:24 -07:00
Wenchen Fan 865a834e51 [SPARK-8723] [SQL] improve divide and remainder code gen
We can avoid execution of both left and right expression by null and zero check.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7111 from cloud-fan/cg and squashes the following commits:

d6b12ef [Wenchen Fan] improve divide and remainder code gen
2015-06-30 08:08:15 -07:00
Wenchen Fan 08fab48438 [SPARK-8590] [SQL] add code gen for ExtractValue
TODO:  use array instead of Seq as internal representation for `ArrayType`

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6982 from cloud-fan/extract-value and squashes the following commits:

e203bc1 [Wenchen Fan] address comments
4da0f0b [Wenchen Fan] some clean up
f679969 [Wenchen Fan] fix bug
e64f942 [Wenchen Fan] remove generic
e3f8427 [Wenchen Fan] fix style and address comments
fc694e8 [Wenchen Fan] add code gen for extract value
2015-06-30 07:58:49 -07:00
Reynold Xin f79410c49b [SPARK-8721][SQL] Rename ExpectsInputTypes => AutoCastInputTypes.
Author: Reynold Xin <rxin@databricks.com>

Closes #7109 from rxin/auto-cast and squashes the following commits:

a914cc3 [Reynold Xin] [SPARK-8721][SQL] Rename ExpectsInputTypes => AutoCastInputTypes.
2015-06-29 22:32:43 -07:00
Wenchen Fan 881662e9c9 [SPARK-8589] [SQL] cleanup DateTimeUtils
move date time related operations into `DateTimeUtils` and rename some methods to make it more clear.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6980 from cloud-fan/datetime and squashes the following commits:

9373a9d [Wenchen Fan] cleanup DateTimeUtil
2015-06-29 16:34:50 -07:00
Yin Huai 4b497a724a [SPARK-8710] [SQL] Change ScalaReflection.mirror from a val to a def.
jira: https://issues.apache.org/jira/browse/SPARK-8710

Author: Yin Huai <yhuai@databricks.com>

Closes #7094 from yhuai/SPARK-8710 and squashes the following commits:

c854baa [Yin Huai] Change ScalaReflection.mirror from a val to a def.
2015-06-29 16:26:05 -07:00
Davies Liu ed359de595 [SPARK-8579] [SQL] support arbitrary object in UnsafeRow
This PR brings arbitrary object support in UnsafeRow (both in grouping key and aggregation buffer).

Two object pools will be created to hold those non-primitive objects, and put the index of them into UnsafeRow. In order to compare the grouping key as bytes, the objects in key will be stored in a unique object pool, to make sure same objects will have same index (used as hashCode).

For StringType and BinaryType, we still put them as var-length in UnsafeRow when initializing for better performance. But for update, they will be an object inside object pools (there will be some garbages left in the buffer).

BTW: Will create a JIRA once issue.apache.org is available.

cc JoshRosen rxin

Author: Davies Liu <davies@databricks.com>

Closes #6959 from davies/unsafe_obj and squashes the following commits:

5ce39da [Davies Liu] fix comment
5e797bf [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_obj
5803d64 [Davies Liu] fix conflict
461d304 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_obj
2f41c90 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_obj
b04d69c [Davies Liu] address comments
4859b80 [Davies Liu] fix comments
f38011c [Davies Liu] add a test for grouping by decimal
d2cf7ab [Davies Liu] add more tests for null checking
71983c5 [Davies Liu] add test for timestamp
e8a1649 [Davies Liu] reuse buffer for string
39f09ca [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_obj
035501e [Davies Liu] fix style
236d6de [Davies Liu] support arbitrary object in UnsafeRow
2015-06-29 15:59:20 -07:00
BenFradet 931da5c8ab [SPARK-8478] [SQL] Harmonize UDF-related code to use uniformly UDF instead of Udf
Follow-up of #6902 for being coherent between ```Udf``` and ```UDF```

Author: BenFradet <benjamin.fradet@gmail.com>

Closes #6920 from BenFradet/SPARK-8478 and squashes the following commits:

c500f29 [BenFradet] renamed a few variables in functions to use UDF
8ab0f2d [BenFradet] renamed idUdf to idUDF in SQLQuerySuite
98696c2 [BenFradet] renamed originalUdfs in TestHive to originalUDFs
7738f74 [BenFradet] modified HiveUDFSuite to use only UDF
c52608d [BenFradet] renamed HiveUdfSuite to HiveUDFSuite
e51b9ac [BenFradet] renamed ExtractPythonUdfs to ExtractPythonUDFs
8c756f1 [BenFradet] renamed Hive UDF related code
2a1ca76 [BenFradet] renamed pythonUdfs to pythonUDFs
261e6fb [BenFradet] renamed ScalaUdf to ScalaUDF
2015-06-29 15:27:13 -07:00
Ilya Ganelin f6fc254ec4 [SPARK-8056][SQL] Design an easier way to construct schema for both Scala and Python
I've added functionality to create new StructType similar to how we add parameters to a new SparkContext.

I've also added tests for this type of creation.

Author: Ilya Ganelin <ilya.ganelin@capitalone.com>

Closes #6686 from ilganeli/SPARK-8056B and squashes the following commits:

27c1de1 [Ilya Ganelin] Rename
467d836 [Ilya Ganelin] Removed from_string in favor of _parse_Datatype_json_value
5fef5a4 [Ilya Ganelin] Updates for type parsing
4085489 [Ilya Ganelin] Style errors
3670cf5 [Ilya Ganelin] added string to DataType conversion
8109e00 [Ilya Ganelin] Fixed error in tests
41ab686 [Ilya Ganelin] Fixed style errors
e7ba7e0 [Ilya Ganelin] Moved some python tests to tests.py. Added cleaner handling of null data type and added test for correctness of input format
15868fa [Ilya Ganelin] Fixed python errors
b79b992 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-8056B
a3369fc [Ilya Ganelin] Fixing space errors
e240040 [Ilya Ganelin] Style
bab7823 [Ilya Ganelin] Constructor error
73d4677 [Ilya Ganelin] Style
4ed00d9 [Ilya Ganelin] Fixed default arg
67df57a [Ilya Ganelin] Removed Foo
04cbf0c [Ilya Ganelin] Added comments for single object
0484d7a [Ilya Ganelin] Restored second method
6aeb740 [Ilya Ganelin] Style
689e54d [Ilya Ganelin] Style
f497e9e [Ilya Ganelin] Got rid of old code
e3c7a88 [Ilya Ganelin] Fixed doctest failure
a62ccde [Ilya Ganelin] Style
966ac06 [Ilya Ganelin] style checks
dabb7e6 [Ilya Ganelin] Added Python tests
a3f4152 [Ilya Ganelin] added python bindings and better comments
e6e536c [Ilya Ganelin] Added extra space
7529a2e [Ilya Ganelin] Fixed formatting
d388f86 [Ilya Ganelin] Fixed small bug
c4e3bf5 [Ilya Ganelin] Reverted to using parse. Updated parse to support long
d7634b6 [Ilya Ganelin] Reverted to fromString to properly support types
22c39d5 [Ilya Ganelin] replaced FromString with DataTypeParser.parse. Replaced empty constructor initializing a null to have it instead create a new array to allow appends to it.
faca398 [Ilya Ganelin] [SPARK-8056] Replaced default argument usage. Updated usage and code for DataType.fromString
1acf76e [Ilya Ganelin] Scala style
e31c674 [Ilya Ganelin] Fixed bug in test
8dc0795 [Ilya Ganelin] Added tests for creation of StructType object with new methods
fdf7e9f [Ilya Ganelin] [SPARK-8056] Created add methods to facilitate building new StructType objects.
2015-06-29 14:15:15 -07:00
zhichao.li 637b4eedad [SPARK-8214] [SQL] Add function hex
cc chenghao-intel  adrian-wang

Author: zhichao.li <zhichao.li@intel.com>

Closes #6976 from zhichao-li/hex and squashes the following commits:

e218d1b [zhichao.li] turn off scalastyle for non-ascii
de3f5ea [zhichao.li] non-ascii char
cf9c936 [zhichao.li] give separated buffer for each hex method
967ec90 [zhichao.li] Make 'value' as a feild of Hex
3b2fa13 [zhichao.li] tiny fix
a647641 [zhichao.li] remove duplicate null check
7cab020 [zhichao.li] tiny refactoring
35ecfe5 [zhichao.li] add function hex
2015-06-29 12:25:16 -07:00
Tarek Auel a5c2961caa [SPARK-8235] [SQL] misc function sha / sha1
Jira: https://issues.apache.org/jira/browse/SPARK-8235

I added the support for sha1. If I understood rxin correctly, sha and sha1 should execute the same algorithm, shouldn't they?

Please take a close look on the Python part. This is adopted from #6934

Author: Tarek Auel <tarek.auel@gmail.com>
Author: Tarek Auel <tarek.auel@googlemail.com>

Closes #6963 from tarekauel/SPARK-8235 and squashes the following commits:

f064563 [Tarek Auel] change to shaHex
7ce3cdc [Tarek Auel] rely on automatic cast
a1251d6 [Tarek Auel] Merge remote-tracking branch 'upstream/master' into SPARK-8235
68eb043 [Tarek Auel] added docstring
be5aff1 [Tarek Auel] improved error message
7336c96 [Tarek Auel] added type check
cf23a80 [Tarek Auel] simplified example
ebf75ef [Tarek Auel] [SPARK-8301] updated the python documentation. Removed sha in python and scala
6d6ff0d [Tarek Auel] [SPARK-8233] added docstring
ea191a9 [Tarek Auel] [SPARK-8233] fixed signatureof python function. Added expected type to misc
e3fd7c3 [Tarek Auel] SPARK[8235] added sha to the list of __all__
e5dad4e [Tarek Auel] SPARK[8235] sha / sha1
2015-06-29 11:57:19 -07:00
Wenchen Fan ed413bcc78 [SPARK-8692] [SQL] re-order the case statements that handling catalyst data types
use same order: boolean, byte, short, int, date, long, timestamp, float, double, string, binary, decimal.

Then we can easily check whether some data types are missing by just one glance, and make sure we handle data/timestamp just as int/long.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7073 from cloud-fan/fix-date and squashes the following commits:

463044d [Wenchen Fan] fix style
51cd347 [Wenchen Fan] refactor handling of date and timestmap
2015-06-29 11:41:26 -07:00
Liang-Chi Hsieh 24fda73811 [SPARK-8677] [SQL] Fix non-terminating decimal expansion for decimal divide operation
JIRA: https://issues.apache.org/jira/browse/SPARK-8677

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #7056 from viirya/fix_decimal3 and squashes the following commits:

34d7419 [Liang-Chi Hsieh] Fix Non-terminating decimal expansion for decimal divide operation.
2015-06-28 14:48:44 -07:00
Davies Liu 77da5be6f1 [SPARK-8610] [SQL] Separate Row and InternalRow (part 2)
Currently, we use GenericRow both for Row and InternalRow, which is confusing because it could contain Scala type also Catalyst types.

This PR changes to use GenericInternalRow for InternalRow (contains catalyst types), GenericRow for Row (contains Scala types).

Also fixes some incorrect use of InternalRow or Row.

Author: Davies Liu <davies@databricks.com>

Closes #7003 from davies/internalrow and squashes the following commits:

d05866c [Davies Liu] fix test: rollback changes for pyspark
72878dd [Davies Liu] Merge branch 'master' of github.com:apache/spark into internalrow
efd0b25 [Davies Liu] fix copy of MutableRow
87b13cf [Davies Liu] fix test
d2ebd72 [Davies Liu] fix style
eb4b473 [Davies Liu] mark expensive API as final
bd4e99c [Davies Liu] Merge branch 'master' of github.com:apache/spark into internalrow
bdfb78f [Davies Liu] remove BaseMutableRow
6f99a97 [Davies Liu] fix catalyst test
defe931 [Davies Liu] remove BaseRow
288b31f [Davies Liu] Merge branch 'master' of github.com:apache/spark into internalrow
9d24350 [Davies Liu] separate Row and InternalRow (part 2)
2015-06-28 08:03:58 -07:00
Wenchen Fan 1a79f0eb8d [SPARK-8635] [SQL] improve performance of CatalystTypeConverters
In `CatalystTypeConverters.createToCatalystConverter`, we add special handling for primitive types. We can apply this strategy to more places to improve performance.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7018 from cloud-fan/converter and squashes the following commits:

8b16630 [Wenchen Fan] another fix
326c82c [Wenchen Fan] optimize type converter
2015-06-25 22:44:26 -07:00
Wenchen Fan 40360112c4 [SPARK-8620] [SQL] cleanup CodeGenContext
fix docs, remove nativeTypes , use java type to get boxed type ,default value, etc. to avoid handle `DateType` and `TimestampType` as int and long again and again.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7010 from cloud-fan/cg and squashes the following commits:

aa01cf9 [Wenchen Fan] cleanup CodeGenContext
2015-06-25 22:16:53 -07:00
Liang-Chi Hsieh 47c874babe [SPARK-8237] [SQL] Add misc function sha2
JIRA: https://issues.apache.org/jira/browse/SPARK-8237

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6934 from viirya/expr_sha2 and squashes the following commits:

35e0bb3 [Liang-Chi Hsieh] For comments.
68b5284 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_sha2
8573aff [Liang-Chi Hsieh] Remove unnecessary Product.
ee61e06 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_sha2
59e41aa [Liang-Chi Hsieh] Add misc function: sha2.
2015-06-25 22:07:37 -07:00
Reynold Xin 82f80c1c7d Two minor SQL cleanup (compiler warning & indent).
Author: Reynold Xin <rxin@databricks.com>

Closes #7000 from rxin/minor-cleanup and squashes the following commits:

046044c [Reynold Xin] Two minor SQL cleanup (compiler warning & indent).
2015-06-24 19:34:07 -07:00
Wenchen Fan b71d3254e5 [SPARK-8075] [SQL] apply type check interface to more expressions
a follow up of https://github.com/apache/spark/pull/6405.
Note: It's not a big change, a lot of changing is due to I swap some code in `aggregates.scala` to make aggregate functions right below its corresponding aggregate expressions.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6723 from cloud-fan/type-check and squashes the following commits:

2124301 [Wenchen Fan] fix tests
5a658bb [Wenchen Fan] add tests
287d3bb [Wenchen Fan] apply type check interface to more expressions
2015-06-24 16:26:00 -07:00
Cheng Lian 8ab50765cd [SPARK-6777] [SQL] Implements backwards compatibility rules in CatalystSchemaConverter
This PR introduces `CatalystSchemaConverter` for converting Parquet schema to Spark SQL schema and vice versa.  Original conversion code in `ParquetTypesConverter` is removed. Benefits of the new version are:

1. When converting Spark SQL schemas, it generates standard Parquet schemas conforming to [the most updated Parquet format spec] [1]. Converting to old style Parquet schemas is also supported via feature flag `spark.sql.parquet.followParquetFormatSpec` (which is set to `false` for now, and should be set to `true` after both read and write paths are fixed).

   Note that although this version of Parquet format spec hasn't been officially release yet, Parquet MR 1.7.0 already sticks to it. So it should be safe to follow.

1. It implements backwards-compatibility rules described in the most updated Parquet format spec. Thus can recognize more schema patterns generated by other/legacy systems/tools.
1. Code organization follows convention used in [parquet-mr] [2], which is easier to follow. (Structure of `CatalystSchemaConverter` is similar to `AvroSchemaConverter`).

To fully implement backwards-compatibility rules in both read and write path, we also need to update `CatalystRowConverter` (which is responsible for converting Parquet records to `Row`s), `RowReadSupport`, and `RowWriteSupport`. These would be done in follow-up PRs.

TODO

- [x] More schema conversion test cases for legacy schema patterns.

[1]: ea09522659/LogicalTypes.md
[2]: https://github.com/apache/parquet-mr/

Author: Cheng Lian <lian@databricks.com>

Closes #6617 from liancheng/spark-6777 and squashes the following commits:

2a2062d [Cheng Lian] Don't convert decimals without precision information
b60979b [Cheng Lian] Adds a constructor which accepts a Configuration, and fixes default value of assumeBinaryIsString
743730f [Cheng Lian] Decimal scale shouldn't be larger than precision
a104a9e [Cheng Lian] Fixes Scala style issue
1f71d8d [Cheng Lian] Adds feature flag to allow falling back to old style Parquet schema conversion
ba84f4b [Cheng Lian] Fixes MapType schema conversion bug
13cb8d5 [Cheng Lian] Fixes MiMa failure
81de5b0 [Cheng Lian] Fixes UDT, workaround read path, and add tests
28ef95b [Cheng Lian] More AnalysisExceptions
b10c322 [Cheng Lian] Replaces require() with analysisRequire() which throws AnalysisException
cceaf3f [Cheng Lian] Implements backwards compatibility rules in CatalystSchemaConverter
2015-06-24 15:03:43 -07:00
Wenchen Fan f04b5672c5 [SPARK-7289] handle project -> limit -> sort efficiently
make the `TakeOrdered` strategy and operator more general, such that it can optionally handle a projection when necessary

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6780 from cloud-fan/limit and squashes the following commits:

34aa07b [Wenchen Fan] revert
07d5456 [Wenchen Fan] clean closure
20821ec [Wenchen Fan] fix
3676a82 [Wenchen Fan] address comments
b558549 [Wenchen Fan] address comments
214842b [Wenchen Fan] fix style
2d8be83 [Wenchen Fan] add LimitPushDown
948f740 [Wenchen Fan] fix existing
2015-06-24 13:28:50 -07:00
Santiago M. Mola b84d4b4dfe [SPARK-7088] [SQL] Fix analysis for 3rd party logical plan.
ResolveReferences analysis rule now does not throw when it cannot resolve references in a self-join.

Author: Santiago M. Mola <smola@stratio.com>

Closes #6853 from smola/SPARK-7088 and squashes the following commits:

af71ac7 [Santiago M. Mola] [SPARK-7088] Fix analysis for 3rd party logical plan.
2015-06-24 12:29:07 -07:00
Wenchen Fan 09fcf96b8f [SPARK-8371] [SQL] improve unit test for MaxOf and MinOf and fix bugs
a follow up of https://github.com/apache/spark/pull/6813

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6825 from cloud-fan/cg and squashes the following commits:

43170cc [Wenchen Fan] fix bugs in code gen
2015-06-23 23:11:42 -07:00
Davies Liu 6f4cadf5ee [SPARK-8432] [SQL] fix hashCode() and equals() of BinaryType in Row
Also added more tests in LiteralExpressionSuite

Author: Davies Liu <davies@databricks.com>

Closes #6876 from davies/fix_hashcode and squashes the following commits:

429c2c0 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_hashcode
32d9811 [Davies Liu] fix test
a0626ed [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_hashcode
89c2432 [Davies Liu] fix style
bd20780 [Davies Liu] check with catalyst types
41caec6 [Davies Liu] change for to while
d96929b [Davies Liu] address comment
6ad2a90 [Davies Liu] fix style
5819d33 [Davies Liu] unify equals() and hashCode()
0fff25d [Davies Liu] fix style
53c38b1 [Davies Liu] fix hashCode() and equals() of BinaryType in Row
2015-06-23 11:55:47 -07:00
Cheng Hao 7b1450b666 [SPARK-7235] [SQL] Refactor the grouping sets
The logical plan `Expand` takes the `output` as constructor argument, which break the references chain. We need to refactor the code, as well as the column pruning.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #5780 from chenghao-intel/expand and squashes the following commits:

76e4aa4 [Cheng Hao] revert the change for case insenstive
7c10a83 [Cheng Hao] refactor the grouping sets
2015-06-23 10:52:17 -07:00
Reynold Xin 6ceb169608 [SPARK-8300] DataFrame hint for broadcast join.
Users can now do
```scala
left.join(broadcast(right), "joinKey")
```
to give the query planner a hint that "right" DataFrame is small and should be broadcasted.

Author: Reynold Xin <rxin@databricks.com>

Closes #6751 from rxin/broadcastjoin-hint and squashes the following commits:

953eec2 [Reynold Xin] Code review feedback.
88752d8 [Reynold Xin] Fixed import.
8187b88 [Reynold Xin] [SPARK-8300] DataFrame hint for broadcast join.
2015-06-23 01:50:31 -07:00
Liang-Chi Hsieh 31bd30687b [SPARK-8359] [SQL] Fix incorrect decimal precision after multiplication
JIRA: https://issues.apache.org/jira/browse/SPARK-8359

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6814 from viirya/fix_decimal2 and squashes the following commits:

071a757 [Liang-Chi Hsieh] Remove maximum precision and use MathContext.UNLIMITED.
df217d4 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_decimal2
a43bfc3 [Liang-Chi Hsieh] Add MathContext with maximum supported precision.
72eeb3f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_decimal2
44c9348 [Liang-Chi Hsieh] Fix incorrect decimal precision after multiplication.
2015-06-22 23:11:56 -07:00
Davies Liu 6b7f2ceafd [SPARK-8307] [SQL] improve timestamp from parquet
This PR change to convert julian day to unix timestamp directly (without Calendar and Timestamp).

cc adrian-wang rxin

Author: Davies Liu <davies@databricks.com>

Closes #6759 from davies/improve_ts and squashes the following commits:

849e301 [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
b0e4cad [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
8e2d56f [Davies Liu] address comments
634b9f5 [Davies Liu] fix mima
4891efb [Davies Liu] address comment
bfc437c [Davies Liu] fix build
ae5979c [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
602b969 [Davies Liu] remove jodd
2f2e48c [Davies Liu] fix test
8ace611 [Davies Liu] fix mima
212143b [Davies Liu] fix mina
c834108 [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
a3171b8 [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
5233974 [Davies Liu] fix scala style
361fd62 [Davies Liu] address comments
ea196d4 [Davies Liu] improve timestamp from parquet
2015-06-22 18:03:59 -07:00
Wenchen Fan 860a49ef20 [SPARK-7153] [SQL] support all integral type ordinal in GetArrayItem
first convert `ordinal` to `Number`, then convert to int type.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5706 from cloud-fan/7153 and squashes the following commits:

915db79 [Wenchen Fan] fix 7153
2015-06-22 17:37:35 -07:00
Davies Liu 96aa01378e [SPARK-8492] [SQL] support binaryType in UnsafeRow
Support BinaryType in UnsafeRow, just like StringType.

Also change the layout of StringType and BinaryType in UnsafeRow, by combining offset and size together as Long, which will limit the size of Row to under 2G (given that fact that any single buffer can not be bigger than 2G in JVM).

Author: Davies Liu <davies@databricks.com>

Closes #6911 from davies/unsafe_bin and squashes the following commits:

d68706f [Davies Liu] update comment
519f698 [Davies Liu] address comment
98a964b [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_bin
180b49d [Davies Liu] fix zero-out
22e4c0a [Davies Liu] zero-out padding bytes
6abfe93 [Davies Liu] fix style
447dea0 [Davies Liu] support binaryType in UnsafeRow
2015-06-22 15:22:17 -07:00
Wenchen Fan da7bbb9435 [SPARK-8104] [SQL] auto alias expressions in analyzer
Currently we auto alias expression in parser. However, during parser phase we don't have enough information to do the right alias. For example, Generator that has more than 1 kind of element need MultiAlias, ExtractValue don't need Alias if it's in middle of a ExtractValue chain.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6647 from cloud-fan/alias and squashes the following commits:

552eba4 [Wenchen Fan] fix python
5b5786d [Wenchen Fan] fix agg
73a90cb [Wenchen Fan] fix case-preserve of ExtractValue
4cfd23c [Wenchen Fan] fix order by
d18f401 [Wenchen Fan] refine
9f07359 [Wenchen Fan] address comments
39c1aef [Wenchen Fan] small fix
33640ec [Wenchen Fan] auto alias expressions in analyzer
2015-06-22 12:13:00 -07:00
Tarek Auel 41ab2853f4 [SPARK-8301] [SQL] Improve UTF8String substring/startsWith/endsWith/contains performance
Jira: https://issues.apache.org/jira/browse/SPARK-8301

Added the private method startsWith(prefix, offset) to implement startsWith, endsWith and contains without copying the array

I hope that the component SQL is still correct. I copied it from the Jira ticket.

Author: Tarek Auel <tarek.auel@googlemail.com>
Author: Tarek Auel <tarek.auel@gmail.com>

Closes #6804 from tarekauel/SPARK-8301 and squashes the following commits:

f5d6b9a [Tarek Auel] fixed parentheses and annotation
6d7b068 [Tarek Auel] [SPARK-8301] removed null checks
9ca0473 [Tarek Auel] [SPARK-8301] removed null checks
1c327eb [Tarek Auel] [SPARK-8301] removed new
9f17cc8 [Tarek Auel] [SPARK-8301] fixed conversion byte to string in codegen
3a0040f [Tarek Auel] [SPARK-8301] changed call of UTF8String.set to UTF8String.from
e4530d2 [Tarek Auel] [SPARK-8301] changed call of UTF8String.set to UTF8String.from
a5f853a [Tarek Auel] [SPARK-8301] changed visibility of set to protected. Changed annotation of bytes from Nullable to Nonnull
d2fb05f [Tarek Auel] [SPARK-8301] added additional null checks
79cb55b [Tarek Auel] [SPARK-8301] null check. Added test cases for null check.
b17909e [Tarek Auel] [SPARK-8301] removed unnecessary copying of UTF8String. Added a private function startsWith(prefix, offset) to implement the check for startsWith, endsWith and contains.
2015-06-20 20:03:59 -07:00
Michael Armbrust a333a72e02 [SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings
In earlier versions of Spark SQL we casted `TimestampType` and `DataType` to `StringType` when it was involved in a binary comparison with a `StringType`.  This allowed comparing a timestamp with a partial date as a user would expect.
 - `time > "2014-06-10"`
 - `time > "2014"`

In 1.4.0 we tried to cast the String instead into a Timestamp.  However, since partial dates are not a valid complete timestamp this results in `null` which results in the tuple being filtered.

This PR restores the earlier behavior.  Note that we still special case equality so that these comparisons are not affected by not printing zeros for subsecond precision.

Author: Michael Armbrust <michael@databricks.com>

Closes #6888 from marmbrus/timeCompareString and squashes the following commits:

bdef29c [Michael Armbrust] test partial date
1f09adf [Michael Armbrust] special handling of equality
1172c60 [Michael Armbrust] more test fixing
4dfc412 [Michael Armbrust] fix tests
aaa9508 [Michael Armbrust] newline
04d908f [Michael Armbrust] [SPARK-8420][SQL] Fix comparision of timestamps/dates with strings
2015-06-19 16:54:51 -07:00
Davies Liu e41e2fd6c6 [SPARK-8461] [SQL] fix codegen with REPL class loader
The ExecutorClassLoader for REPL will cause Janino failed to find class for those in java.lang, so switch to use default class loader for Janino, which will also help performance.

cc liancheng yhuai

Author: Davies Liu <davies@databricks.com>

Closes #6898 from davies/fix_class_loader and squashes the following commits:

24276d4 [Davies Liu] add regression test
4ff0457 [Davies Liu] address comment, refactor
7f5ffbe [Davies Liu] fix REPL class loader with codegen
2015-06-19 11:40:04 -07:00
Shilei 0c32fc125c [SPARK-8234][SQL] misc function: md5
Author: Shilei <shilei.qian@intel.com>

Closes #6779 from qiansl127/MD5 and squashes the following commits:

11fcdb2 [Shilei] Fix the indent
04bd27b [Shilei] Add codegen
da60eb3 [Shilei] Remove checkInputDataTypes function
9509ad0 [Shilei] Format code
12c61f4 [Shilei] Accept only BinaryType for Md5
1df0b5b [Shilei] format to scala type
60ccde1 [Shilei] Add more test case
b8c73b4 [Shilei] Rewrite the type check for Md5
c166167 [Shilei] Add md5 function
2015-06-19 10:49:27 -07:00
Liang-Chi Hsieh 2c59d5c12a [SPARK-8207] [SQL] Add math function bin
JIRA: https://issues.apache.org/jira/browse/SPARK-8207

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6721 from viirya/expr_bin and squashes the following commits:

07e1c8f [Liang-Chi Hsieh] Remove AbstractUnaryMathExpression and let BIN inherit UnaryExpression.
0677f1a [Liang-Chi Hsieh] For comments.
cf62b95 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
0cf20f2 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
dea9c12 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
d4f4774 [Liang-Chi Hsieh] Add @ignore_unicode_prefix.
7a0196f [Liang-Chi Hsieh] Fix python style.
ac2bacd [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
a0a2d0f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
4cb764d [Liang-Chi Hsieh] For comments.
0f78682 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
c0c3197 [Liang-Chi Hsieh] Add bin to FunctionRegistry.
824f761 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
50e0c3b [Liang-Chi Hsieh] Add math function bin(a: long): string.
2015-06-19 10:09:31 -07:00
Reynold Xin dc41313899 [SPARK-8218][SQL] Binary log math function update.
Some minor updates based on after merging #6725.

Author: Reynold Xin <rxin@databricks.com>

Closes #6871 from rxin/log and squashes the following commits:

ab51542 [Reynold Xin] Use JVM log
76fc8de [Reynold Xin] Fixed arg.
a7c1522 [Reynold Xin] [SPARK-8218][SQL] Binary log math function update.
2015-06-18 18:41:15 -07:00
Liang-Chi Hsieh 31641128b3 [SPARK-8363][SQL] Move sqrt to math and extend UnaryMathExpression
JIRA: https://issues.apache.org/jira/browse/SPARK-8363

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6823 from viirya/move_sqrt and squashes the following commits:

8977e11 [Liang-Chi Hsieh] Remove unnecessary old tests.
d23e79e [Liang-Chi Hsieh] Explicitly indicate sqrt value sequence.
699f48b [Liang-Chi Hsieh] Use correct @since tag.
8dff6d1 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into move_sqrt
bc2ed77 [Liang-Chi Hsieh] Remove/move arithmetic expression test and expression type checking test. Remove unnecessary Sqrt type rule.
d38492f [Liang-Chi Hsieh] Now sqrt accepts boolean because type casting is handled by HiveTypeCoercion.
297cc90 [Liang-Chi Hsieh] Sqrt only accepts double input.
ef4a21a [Liang-Chi Hsieh] Move sqrt to math.
2015-06-18 13:00:31 -07:00
Yijie Shen e86fbdb1e6 [SPARK-8283][SQL] Resolve udf_struct test failure in HiveCompatibilitySuite
This PR aimed to resolve udf_struct test failure in HiveCompatibilitySuite.

Currently, this is done by loosening CreateStruct's children type from NamedExpression to Expression and automatically generating StructField name for non-NamedExpression children.

The naming convention for unnamed children follows the udf's counterpart in Hive:
`col1, col2, col3, ...`

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #6828 from yijieshen/SPARK-8283 and squashes the following commits:

6052b73 [Yijie Shen] Doc fix
677e0b7 [Yijie Shen] Resolve udf_struct test failure by automatically generate structField name for non-NamedExpression children
2015-06-17 23:46:57 -07:00
Liang-Chi Hsieh fee3438a32 [SPARK-8218][SQL] Add binary log math function
JIRA: https://issues.apache.org/jira/browse/SPARK-8218

Because there is already `log` unary function defined, the binary log function is called `logarithm` for now.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6725 from viirya/expr_binary_log and squashes the following commits:

bf96bd9 [Liang-Chi Hsieh] Compare log result in string.
102070d [Liang-Chi Hsieh] Round log result to better comparing in python test.
fd01863 [Liang-Chi Hsieh] For comments.
beed631 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
6089d11 [Liang-Chi Hsieh] Remove unnecessary override.
8cf37b7 [Liang-Chi Hsieh] For comments.
bc89597 [Liang-Chi Hsieh] For comments.
db7dc38 [Liang-Chi Hsieh] Use ctor instead of companion object.
0634ef7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
1750034 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
3d75bfc [Liang-Chi Hsieh] Fix scala style.
5b39c02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
23c54a3 [Liang-Chi Hsieh] Fix scala style.
ebc9929 [Liang-Chi Hsieh] Let Logarithm accept one parameter too.
605574d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
21c3bfd [Liang-Chi Hsieh] Fix scala style.
c6c187f [Liang-Chi Hsieh] For comments.
c795342 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
f373bac [Liang-Chi Hsieh] Add binary log expression.
2015-06-17 23:31:30 -07:00
Lianhui Wang 9db73ec124 [SPARK-8381][SQL]reuse typeConvert when convert Seq[Row] to catalyst type
reuse-typeConvert when convert Seq[Row] to CatalystType

Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #6831 from lianhuiwang/reuse-typeConvert and squashes the following commits:

1fec395 [Lianhui Wang] remove CatalystTypeConverters.convertToCatalyst
714462d [Lianhui Wang] add package[sql]
9d1fbf3 [Lianhui Wang] address JoshRosen's comments
768956f [Lianhui Wang] update scala style
4498c62 [Lianhui Wang] reuse typeConvert
2015-06-17 22:52:47 -07:00
Wenchen Fan 7f05b1fe69 [SPARK-7067] [SQL] fix bug when use complex nested fields in ORDER BY
This PR is a improvement for https://github.com/apache/spark/pull/5189.

The resolution rule for ORDER BY is: first resolve based on what comes from the select clause and then fall back on its child only when this fails.

There are 2 steps. First, try to resolve `Sort` in `ResolveReferences` based on select clause, and ignore exceptions. Second, try to resolve `Sort` in `ResolveSortReferences` and add missing projection.

However, the way we resolve `SortOrder` is wrong. We just resolve `UnresolvedAttribute` and use the result to indicate if we can resolve `SortOrder`. But `UnresolvedAttribute` is only part of `GetField` chain(broken by `GetItem`), so we need to go through the whole chain to indicate if we can resolve `SortOrder`.

With this change, we can also avoid re-throw GetField exception in `CheckAnalysis` which is little ugly.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5659 from cloud-fan/order-by and squashes the following commits:

cfa79f8 [Wenchen Fan] update test
3245d28 [Wenchen Fan] minor improve
465ee07 [Wenchen Fan] address comment
1fc41a2 [Wenchen Fan] fix SPARK-7067
2015-06-17 14:46:00 -07:00
OopsOutOfMemory 98ee3512b2 [SPARK-8010] [SQL] Promote types to StringType as implicit conversion in non-binary expression of HiveTypeCoercion
1. Given a query
`select coalesce(null, 1, '1') from dual` will cause exception:
java.lang.RuntimeException: Could not determine return type of Coalesce for IntegerType,StringType
2. Given a query:
`select case when true then 1 else '1' end from dual` will cause exception:
java.lang.RuntimeException: Types in CASE WHEN must be the same or coercible to a common type: StringType != IntegerType
I checked the code, the main cause is the HiveTypeCoercion doesn't do implicit convert when there is a IntegerType and StringType.

Numeric types can be promoted to string type

Hive will always do this implicit conversion.

Author: OopsOutOfMemory <victorshengli@126.com>

Closes #6551 from OopsOutOfMemory/pnts and squashes the following commits:

7a209d7 [OopsOutOfMemory] rebase master
6018613 [OopsOutOfMemory] convert function to method
4cd5618 [OopsOutOfMemory] limit the data type to primitive type
df365d2 [OopsOutOfMemory] refine
95cbd58 [OopsOutOfMemory] fix style
403809c [OopsOutOfMemory] promote non-string to string when can not found tighestCommonTypeOfTwo
2015-06-17 13:37:59 -07:00
Michael Davies 0c1b2df043 [SPARK-8077] [SQL] Optimization for TreeNodes with large numbers of children
For example large IN clauses

Large IN clauses are parsed very slowly. For example SQL below (10K items in IN) takes 45-50s.

s"""SELECT * FROM Person WHERE ForeName IN ('${(1 to 10000).map("n" + _).mkString("','")}')"""

This is principally due to TreeNode which repeatedly call contains on children, where children in this case is a List that is 10K long. In effect parsing for large IN clauses is O(N squared).
A lazily initialised Set based on children for contains reduces parse time to around 2.5s

Author: Michael Davies <Michael.BellDavies@gmail.com>

Closes #6673 from MickDavies/SPARK-8077 and squashes the following commits:

38cd425 [Michael Davies] SPARK-8077: Optimization for  TreeNodes with large numbers of children
d80103b [Michael Davies] SPARK-8077: Optimization for  TreeNodes with large numbers of children
e6be8be [Michael Davies] SPARK-8077: Optimization for  TreeNodes with large numbers of children
2015-06-17 12:56:55 -07:00
Liang-Chi Hsieh 104f30c36f [SPARK-7199] [SQL] Add date and timestamp support to UnsafeRow
JIRA: https://issues.apache.org/jira/browse/SPARK-7199

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #5984 from viirya/add_date_timestamp and squashes the following commits:

7f21ce9 [Liang-Chi Hsieh] For comment.
0b89698 [Liang-Chi Hsieh] Add timestamp to settableFieldTypes.
c30d490 [Liang-Chi Hsieh] Use default IntUnsafeColumnWriter and LongUnsafeColumnWriter.
672ef17 [Liang-Chi Hsieh] Remove getter/setter for Date and Timestamp and use Int and Long for them.
9f3e577 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_date_timestamp
281e844 [Liang-Chi Hsieh] Fix scala style.
fb532b5 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_date_timestamp
80af342 [Liang-Chi Hsieh] Fix compiling error.
f4f5de6 [Liang-Chi Hsieh] Fix scala style.
a463e83 [Liang-Chi Hsieh] Use Long to store timestamp for rows.
635388a [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_date_timestamp
46946c6 [Liang-Chi Hsieh] Adapt for moved DateUtils.
b16994e [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_date_timestamp
752251f [Liang-Chi Hsieh] Support setDate. Fix failed test.
fcf8db9 [Liang-Chi Hsieh] Add functions for Date and Timestamp to SpecificRow.
e42a809 [Liang-Chi Hsieh] Fix style.
4c07b57 [Liang-Chi Hsieh] Add date and timestamp support to UnsafeRow.
2015-06-17 09:00:37 -07:00
dragonli bedff7d532 [SPARK-8220][SQL]Add positive identify function
chenghao-intel adrian-wang

Author: dragonli <lisurprise@gmail.com>
Author: zhichao.li <zhichao.li@intel.com>

Closes #6838 from zhichao-li/positive and squashes the following commits:

e1032a0 [dragonli] remove useless import and refactor code
624d438 [zhichao.li] add positive identify function
2015-06-16 23:44:10 -07:00
Davies Liu bc76a0f750 [SPARK-7184] [SQL] enable codegen by default
In order to have better performance out of box, this PR turn on codegen by default, then codegen can be tested by sql/test and hive/test.

This PR also fix some corner cases for codegen.

Before 1.5 release, we should re-visit this, turn it off if it's not stable or causing regressions.

cc rxin JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #6726 from davies/enable_codegen and squashes the following commits:

f3b25a5 [Davies Liu] fix warning
73750ea [Davies Liu] fix long overflow when compare
3017a47 [Davies Liu] Merge branch 'master' of github.com:apache/spark into enable_codegen
a7d75da [Davies Liu] Merge branch 'master' of github.com:apache/spark into enable_codegen
ff5b75a [Davies Liu] Merge branch 'master' of github.com:apache/spark into enable_codegen
f4cf2c2 [Davies Liu] fix style
99fc139 [Davies Liu] Merge branch 'enable_codegen' of github.com:davies/spark into enable_codegen
91fc7a2 [Davies Liu] disable codegen for ScalaUDF
207e339 [Davies Liu] Update CodeGenerator.scala
44573a3 [Davies Liu] check thread safety of expression
f3886fa [Davies Liu] don't inline primitiveTerm for null literal
c8e7cd2 [Davies Liu] address comment
a8618c9 [Davies Liu] enable codegen by default
2015-06-15 23:03:14 -07:00
tedyu 1a62d61696 SPARK-8336 Fix NullPointerException with functions.rand()
This PR fixes the problem reported by Justin Yip in the thread 'NullPointerException with functions.rand()'

Tested using spark-shell and verified that the following works:
sqlContext.createDataFrame(Seq((1,2), (3, 100))).withColumn("index", rand(30)).show()

Author: tedyu <yuzhihong@gmail.com>

Closes #6793 from tedyu/master and squashes the following commits:

62fd97b [tedyu] Create RandomSuite
750f92c [tedyu] Add test for Rand() with seed
a1d66c5 [tedyu] Fix NullPointerException with functions.rand()
2015-06-15 17:00:38 -07:00
Yadong Qi 6ae21a944a [SPARK-6583] [SQL] Support aggregate functions in ORDER BY
Add aggregates in ORDER BY clauses to the `Aggregate` operator beneath.  Project these results away after the Sort.

Based on work by watermen.  Also Closes #5290.

Author: Yadong Qi <qiyadong2010@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #6816 from marmbrus/pr/5290 and squashes the following commits:

3226a97 [Michael Armbrust] consistent ordering
eb8938d [Michael Armbrust] no vars
c8b25c1 [Yadong Qi] move the test data.
7f9b736 [Yadong Qi] delete Substring case
a1e87c1 [Yadong Qi] fix conflict
f119849 [Yadong Qi] order by aggregated function
2015-06-15 12:01:52 -07:00
Reynold Xin 53c16b92a5 [SPARK-8362] [SQL] Add unit tests for +, -, *, /, %
Added unit tests for all supported data types for:
- Add
- Subtract
- Multiply
- Divide
- UnaryMinus
- Remainder

Fixed bugs caught by the unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #6813 from rxin/SPARK-8362 and squashes the following commits:

fb3fe62 [Reynold Xin] Added Remainder.
3b266ba [Reynold Xin] [SPARK-8362] Add unit tests for +, -, *, /.
2015-06-14 11:23:23 -07:00
Michael Armbrust 9073a426e4 [SPARK-8358] [SQL] Wait for child resolution when resolving generators
Author: Michael Armbrust <michael@databricks.com>

Closes #6811 from marmbrus/aliasExplodeStar and squashes the following commits:

fbd2065 [Michael Armbrust] more style
806a373 [Michael Armbrust] fix style
7cbb530 [Michael Armbrust] [SPARK-8358][SQL] Wait for child resolution when resolving generatorsa
2015-06-14 11:21:42 -07:00
Josh Rosen ea7fd2ff64 [SPARK-8354] [SQL] Fix off-by-factor-of-8 error when allocating scratch space in UnsafeFixedWidthAggregationMap
UnsafeFixedWidthAggregationMap contains an off-by-factor-of-8 error when allocating row conversion scratch space: we take a size requirement, measured in bytes, then allocate a long array of that size.  This means that we end up allocating 8x too much conversion space.

This patch fixes this by allocating a `byte[]` array instead.  This doesn't impose any new limitations on the maximum sizes of UnsafeRows, since UnsafeRowConverter already used integers when calculating the size requirements for rows.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6809 from JoshRosen/sql-bytes-vs-words-fix and squashes the following commits:

6520339 [Josh Rosen] Updates to reflect fact that UnsafeRow max size is constrained by max byte[] size
2015-06-14 09:34:35 -07:00
Liang-Chi Hsieh cb7ada1196 [SPARK-8342][SQL] Fix Decimal setOrNull
JIRA: https://issues.apache.org/jira/browse/SPARK-8342

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6797 from viirya/fix_decimal and squashes the following commits:

8a447b1 [Liang-Chi Hsieh] Add unit test.
d67a5ea [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_decimal
ab6d8af [Liang-Chi Hsieh] Fix setOrNull.
2015-06-13 22:42:28 -07:00
Reynold Xin 2d71ba4c8a [SPARK-8349] [SQL] Use expression constructors (rather than apply) in FunctionRegistry
Author: Reynold Xin <rxin@databricks.com>

Closes #6806 from rxin/gs and squashes the following commits:

ed1aebb [Reynold Xin] Fixed style.
c7fc3e6 [Reynold Xin] [SPARK-8349][SQL] Use expression constructors (rather than apply) in FunctionRegistry
2015-06-13 18:22:17 -07:00
Reynold Xin a138953391 [SPARK-8347][SQL] Add unit tests for abs.
Also addressed code review feedback from #6754

Author: Reynold Xin <rxin@databricks.com>

Closes #6803 from rxin/abs and squashes the following commits:

d07beba [Reynold Xin] [SPARK-8347] Add unit tests for abs.
2015-06-13 17:10:13 -07:00
Liang-Chi Hsieh ddec45279e [SPARK-8052] [SQL] Use java.math.BigDecimal for casting String to Decimal instead of using toDouble
JIRA: https://issues.apache.org/jira/browse/SPARK-8052

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #6645 from viirya/cast_string_integraltype and squashes the following commits:

e19c6a3 [Liang-Chi Hsieh] For comment.
c3e472a [Liang-Chi Hsieh] Add test.
7ced9b0 [Liang-Chi Hsieh] Use java.math.BigDecimal for casting String to Decimal instead of using toDouble.
2015-06-13 16:39:52 -07:00
Davies Liu ce1041c38f [SPARK-8346] [SQL] Use InternalRow instread of catalyst.InternalRow
cc rxin marmbrus

Author: Davies Liu <davies@databricks.com>

Closes #6802 from davies/cleanup_internalrow and squashes the following commits:

769d2aa [Davies Liu] remove not needed cast
4acbbe4 [Davies Liu] catalyst.Internal -> InternalRow
2015-06-13 16:13:26 -07:00
Davies Liu d46f8e5d4b [SPARK-7186] [SQL] Decouple internal Row from external Row
Currently, we use o.a.s.sql.Row both internally and externally. The external interface is wider than what the internal needs because it is designed to facilitate end-user programming. This design has proven to be very error prone and cumbersome for internal Row implementations.

As a first step, we create an InternalRow interface in the catalyst module, which is identical to the current Row interface. And we switch all internal operators/expressions to use this InternalRow instead. When we need to expose Row, we convert the InternalRow implementation into Row for users.

For all public API, we use Row (for example, data source APIs), which will be converted into/from InternalRow by CatalystTypeConverters.

For all internal data sources (Json, Parquet, JDBC, Hive), we use InternalRow for better performance, casted into Row in buildScan() (without change the public API). When create a PhysicalRDD, we cast them back to InternalRow.

cc rxin marmbrus JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #6792 from davies/internal_row and squashes the following commits:

f2abd13 [Davies Liu] fix scalastyle
a7e025c [Davies Liu] move InternalRow into catalyst
30db8ba [Davies Liu] Merge branch 'master' of github.com:apache/spark into internal_row
7cbced8 [Davies Liu] separate Row and InternalRow
2015-06-12 23:06:31 -07:00
Wenchen Fan c19c78577a [SQL] [MINOR] correct semanticEquals logic
It's a follow up of https://github.com/apache/spark/pull/6173, for expressions like `Coalesce` that have a `Seq[Expression]`, when we do semantic equal check for it, we need to do semantic equal check for all of its children.
Also we can just use `Seq[(Expression, NamedExpression)]` instead of `Map[Expression, NamedExpression]` as we only search it with `find`.

chenghao-intel, I agree that we probably never knows `semanticEquals` in a general way, but I think we have done that in `TreeNode`, so we can use similar logic. Then we can handle something like `Coalesce(children: Seq[Expression])` correctly.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6261 from cloud-fan/tmp and squashes the following commits:

4daef88 [Wenchen Fan] address comments
dd8fbd9 [Wenchen Fan] correct semanticEquals
2015-06-12 16:38:28 +08:00
Reynold Xin 337c16d57e [SQL] Miscellaneous SQL/DF expression changes.
SPARK-8201 conditional function: if
SPARK-8205 conditional function: nvl
SPARK-8208 math function: ceiling
SPARK-8210 math function: degrees
SPARK-8211 math function: radians
SPARK-8219 math function: negative
SPARK-8216 math function: rename log -> ln
SPARK-8222 math function: alias power / pow
SPARK-8225 math function: alias sign / signum
SPARK-8228 conditional function: isnull
SPARK-8229 conditional function: isnotnull
SPARK-8250 string function: alias lower/lcase
SPARK-8251 string function: alias upper / ucase

Author: Reynold Xin <rxin@databricks.com>

Closes #6754 from rxin/expressions-misc and squashes the following commits:

35fce15 [Reynold Xin] Removed println.
2647067 [Reynold Xin] Promote to string type.
3c32bbc [Reynold Xin] Fixed if.
de827ac [Reynold Xin] Fixed style
b201cd4 [Reynold Xin] Removed if.
6b21a9b [Reynold Xin] [SQL] Miscellaneous SQL/DF expression changes.
2015-06-11 17:06:21 -07:00
Zhongshuai Pei 7914c720bf [SPARK-7824] [SQL] Collapse operator reordering and constant folding into a single batch.
SQL
```
select * from tableA join tableB on (a > 3 and b = d) or (a > 3 and b = e)
```
Plan before modify
```
== Optimized Logical Plan ==
Project [a#293,b#294,c#295,d#296,e#297]
 Join Inner, Some(((a#293 > 3) && ((b#294 = d#296) || (b#294 = e#297))))
  MetastoreRelation default, tablea, None
  MetastoreRelation default, tableb, None
```
Plan after modify
```
== Optimized Logical Plan ==
Project [a#293,b#294,c#295,d#296,e#297]
 Join Inner, Some(((b#294 = d#296) || (b#294 = e#297)))
  Filter (a#293 > 3)
   MetastoreRelation default, tablea, None
  MetastoreRelation default, tableb, None
```

CombineLimits ==> Limit(If(LessThan(ne, le), ne, le), grandChild) and LessThan is in BooleanSimplification ,  so CombineLimits  must before BooleanSimplification and BooleanSimplification must before PushPredicateThroughJoin.

Author: Zhongshuai Pei <799203320@qq.com>
Author: DoingDone9 <799203320@qq.com>

Closes #6351 from DoingDone9/master and squashes the following commits:

20de7be [Zhongshuai Pei] Update Optimizer.scala
7bc7d28 [Zhongshuai Pei] Merge pull request #17 from apache/master
0ba5f42 [Zhongshuai Pei] Update Optimizer.scala
f8b9314 [Zhongshuai Pei] Update FilterPushdownSuite.scala
c529d9f [Zhongshuai Pei] Update FilterPushdownSuite.scala
ae3af6d [Zhongshuai Pei] Update FilterPushdownSuite.scala
a04ffae [Zhongshuai Pei] Update Optimizer.scala
11beb61 [Zhongshuai Pei] Update FilterPushdownSuite.scala
f2ee5fe [Zhongshuai Pei] Update Optimizer.scala
be6b1d5 [Zhongshuai Pei] Update Optimizer.scala
b01e622 [Zhongshuai Pei] Merge pull request #15 from apache/master
8df716a [Zhongshuai Pei] Update FilterPushdownSuite.scala
d98bc35 [Zhongshuai Pei] Update FilterPushdownSuite.scala
fa65718 [Zhongshuai Pei] Update Optimizer.scala
ab8e9a6 [Zhongshuai Pei] Merge pull request #14 from apache/master
14952e2 [Zhongshuai Pei] Merge pull request #13 from apache/master
f03fe7f [Zhongshuai Pei] Merge pull request #12 from apache/master
f12fa50 [Zhongshuai Pei] Merge pull request #10 from apache/master
f61210c [Zhongshuai Pei] Merge pull request #9 from apache/master
34b1a9a [Zhongshuai Pei] Merge pull request #8 from apache/master
802261c [DoingDone9] Merge pull request #7 from apache/master
d00303b [DoingDone9] Merge pull request #6 from apache/master
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-06-11 17:01:07 -07:00
Reynold Xin 7d669a56ff [SPARK-8286] Rewrite UTF8String in Java and move it into unsafe package.
Unit test is still in Scala.

Author: Reynold Xin <rxin@databricks.com>

Closes #6738 from rxin/utf8string-java and squashes the following commits:

562dc6e [Reynold Xin] Flag...
98e600b [Reynold Xin] Another try with encoding setting ..
cfa6bdf [Reynold Xin] Merge branch 'master' into utf8string-java
a3b124d [Reynold Xin] Try different UTF-8 encoded characters.
1ff7c82 [Reynold Xin] Enable UTF-8 encoding.
82d58cc [Reynold Xin] Reset run-tests.
2cb3c69 [Reynold Xin] Use utf-8 encoding in set bytes.
53f8ef4 [Reynold Xin] Hack Jenkins to run one test.
9a48e8d [Reynold Xin] Fixed runtime compilation error.
911c450 [Reynold Xin] Moved unit test also to Java.
4eff7bd [Reynold Xin] Improved unit test coverage.
8e89a3c [Reynold Xin] Fixed tests.
77c64bd [Reynold Xin] Fixed string type codegen.
ffedb62 [Reynold Xin] Code review feedback.
0967ce6 [Reynold Xin] Fixed import ordering.
45a123d [Reynold Xin] [SPARK-8286] Rewrite UTF8String in Java and move it into unsafe package.
2015-06-11 16:07:15 -07:00