Commit graph

731 commits

Author SHA1 Message Date
Davies Liu 487d409e71 [SPARK-11243][SQL] zero out padding bytes in UnsafeRow
For nested StructType, the underline buffer could be used for others before, we should zero out the padding bytes for those primitive types that have less than 8 bytes.

cc cloud-fan

Author: Davies Liu <davies@databricks.com>

Closes #9217 from davies/zero_out.
2015-10-23 01:33:14 -07:00
Reynold Xin cdea0174e3 [SPARK-11273][SQL] Move ArrayData/MapData/DataTypeParser to catalyst.util package
Author: Reynold Xin <rxin@databricks.com>

Closes #9239 from rxin/types-private.
2015-10-23 00:00:21 -07:00
Michael Armbrust 53e83a3a77 [SPARK-11116][SQL] First Draft of Dataset API
*This PR adds a new experimental API to Spark, tentitively named Datasets.*

A `Dataset` is a strongly-typed collection of objects that can be transformed in parallel using functional or relational operations.  Example usage is as follows:

### Functional
```scala
> val ds: Dataset[Int] = Seq(1, 2, 3).toDS()
> ds.filter(_ % 1 == 0).collect()
res1: Array[Int] = Array(1, 2, 3)
```

### Relational
```scala
scala> ds.toDF().show()
+-----+
|value|
+-----+
|    1|
|    2|
|    3|
+-----+

> ds.select(expr("value + 1").as[Int]).collect()
res11: Array[Int] = Array(2, 3, 4)
```

## Comparison to RDDs
 A `Dataset` differs from an `RDD` in the following ways:
  - The creation of a `Dataset` requires the presence of an explicit `Encoder` that can be
    used to serialize the object into a binary format.  Encoders are also capable of mapping the
    schema of a given object to the Spark SQL type system.  In contrast, RDDs rely on runtime
    reflection based serialization.
  - Internally, a `Dataset` is represented by a Catalyst logical plan and the data is stored
    in the encoded form.  This representation allows for additional logical operations and
    enables many operations (sorting, shuffling, etc.) to be performed without deserializing to
    an object.

A `Dataset` can be converted to an `RDD` by calling the `.rdd` method.

## Comparison to DataFrames

A `Dataset` can be thought of as a specialized DataFrame, where the elements map to a specific
JVM object type, instead of to a generic `Row` container. A DataFrame can be transformed into
specific Dataset by calling `df.as[ElementType]`.  Similarly you can transform a strongly-typed
`Dataset` to a generic DataFrame by calling `ds.toDF()`.

## Implementation Status and TODOs

This is a rough cut at the least controversial parts of the API.  The primary purpose here is to get something committed so that we can better parallelize further work and get early feedback on the API.  The following is being deferred to future PRs:
 - Joins and Aggregations (prototype here f11f91e6f0)
 - Support for Java

Additionally, the responsibility for binding an encoder to a given schema is currently done in a fairly ad-hoc fashion.  This is an internal detail, and what we are doing today works for the cases we care about.  However, as we add more APIs we'll probably need to do this in a more principled way (i.e. separate resolution from binding as we do in DataFrames).

## COMPATIBILITY NOTE
Long term we plan to make `DataFrame` extend `Dataset[Row]`.  However,
making this change to che class hierarchy would break the function signatures for the existing
function operations (map, flatMap, etc).  As such, this class should be considered a preview
of the final API.  Changes will be made to the interface after Spark 1.6.

Author: Michael Armbrust <michael@databricks.com>

Closes #9190 from marmbrus/dataset-infra.
2015-10-22 15:20:17 -07:00
Wenchen Fan 42d225f449 [SPARK-11216][SQL][FOLLOW-UP] add encoder/decoder for external row
address comments in https://github.com/apache/spark/pull/9184

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9212 from cloud-fan/encoder.
2015-10-22 10:53:59 -07:00
Dilip Biswal dce2f8c9d7 [SPARK-8654][SQL] Analysis exception when using NULL IN (...) : invalid cast
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.

The fix is to find a common type between LHS and RHS expressions and cast
all the expression to the common type.

Author: Dilip Biswal <dbiswal@us.ibm.com>

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

Closes #9036 from dilipbiswal/spark_8654_new.
2015-10-21 14:29:03 -07:00
Davies Liu f8c6bec657 [SPARK-11197][SQL] run SQL on files directly
This PR introduce a new feature to run SQL directly on files without create a table, for example:

```
select id from json.`path/to/json/files` as j
```

Author: Davies Liu <davies@databricks.com>

Closes #9173 from davies/source.
2015-10-21 13:38:30 -07:00
Dilip Biswal 49ea0e9d7c [SPARK-10534] [SQL] ORDER BY clause allows only columns that are present in the select projection list
Find out the missing attributes by recursively looking
at the sort order expression and rest of the code
takes care of projecting them out.

Added description from cloud-fan

I wanna explain a bit more about this bug.

When we resolve sort ordering, we will use a special method, which only resolves UnresolvedAttributes and UnresolvedExtractValue. However, for something like Floor('a), even the 'a is resolved, the floor expression may still being unresolved as data type mismatch(for example, 'a is string type and Floor need double type), thus can't pass this filter, and we can't push down this missing attribute 'a

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #9123 from dilipbiswal/SPARK-10534.
2015-10-21 11:10:32 -07:00
Wenchen Fan ccf536f903 [SPARK-11216] [SQL] add encoder/decoder for external row
Implement encode/decode for external row based on `ClassEncoder`.

TODO:
* code cleanup
* ~~fix corner cases~~
* refactor the encoder interface
* improve test for product codegen, to cover more corner cases.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9184 from cloud-fan/encoder.
2015-10-21 11:06:34 -07:00
nitin goyal f62e326088 [SPARK-11179] [SQL] Push filters through aggregate
Push conjunctive predicates though Aggregate operators when their references are a subset of the groupingExpressions.

Query plan before optimisation :-
Filter ((c#138L = 2) && (a#0 = 3))
 Aggregate [a#0], [a#0,count(b#1) AS c#138L]
  Project [a#0,b#1]
   LocalRelation [a#0,b#1,c#2]

Query plan after optimisation :-
Filter (c#138L = 2)
 Aggregate [a#0], [a#0,count(b#1) AS c#138L]
  Filter (a#0 = 3)
   Project [a#0,b#1]
    LocalRelation [a#0,b#1,c#2]

Author: nitin goyal <nitin.goyal@guavus.com>
Author: nitin.goyal <nitin.goyal@guavus.com>

Closes #9167 from nitin2goyal/master.
2015-10-21 10:45:21 -07:00
Davies Liu 06e6b765d0 [SPARK-11149] [SQL] Improve cache performance for primitive types
This PR improve the performance by:

1) Generate an Iterator that take Iterator[CachedBatch] as input, and call accessors (unroll the loop for columns), avoid the expensive Iterator.flatMap.

2) Use Unsafe.getInt/getLong/getFloat/getDouble instead of ByteBuffer.getInt/getLong/getFloat/getDouble, the later one actually read byte by byte.

3) Remove the unnecessary copy() in Coalesce(), which is not related to memory cache, found during benchmark.

The following benchmark showed that we can speedup the columnar cache of int by 2x.

```
path = '/opt/tpcds/store_sales/'
int_cols = ['ss_sold_date_sk', 'ss_sold_time_sk', 'ss_item_sk','ss_customer_sk']
df = sqlContext.read.parquet(path).select(int_cols).cache()
df.count()

t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```

Author: Davies Liu <davies@databricks.com>

Closes #9145 from davies/byte_buffer.
2015-10-20 14:01:53 -07:00
Davies Liu 67d468f8d9 [SPARK-11111] [SQL] fast null-safe join
Currently, we use CartesianProduct for join with null-safe-equal condition.
```
scala> sqlContext.sql("select * from t a join t b on (a.i <=> b.i)").explain
== Physical Plan ==
TungstenProject [i#2,j#3,i#7,j#8]
 Filter (i#2 <=> i#7)
  CartesianProduct
   LocalTableScan [i#2,j#3], [[1,1]]
   LocalTableScan [i#7,j#8], [[1,1]]
```
Actually, we can have an equal-join condition as  `coalesce(i, default) = coalesce(b.i, default)`, then an partitioned join algorithm could be used.

After this PR, the plan will become:
```
>>> sqlContext.sql("select * from a join b ON a.id <=> b.id").explain()
TungstenProject [id#0L,id#1L]
 Filter (id#0L <=> id#1L)
  SortMergeJoin [coalesce(id#0L,0)], [coalesce(id#1L,0)]
   TungstenSort [coalesce(id#0L,0) ASC], false, 0
    TungstenExchange hashpartitioning(coalesce(id#0L,0),200)
     ConvertToUnsafe
      Scan PhysicalRDD[id#0L]
   TungstenSort [coalesce(id#1L,0) ASC], false, 0
    TungstenExchange hashpartitioning(coalesce(id#1L,0),200)
     ConvertToUnsafe
      Scan PhysicalRDD[id#1L]
```

Author: Davies Liu <davies@databricks.com>

Closes #9120 from davies/null_safe.
2015-10-20 13:40:24 -07:00
Wenchen Fan 478c7ce862 [SPARK-6740] [SQL] correctly parse NOT operator with comparison operations
We can't parse `NOT` operator with comparison operations like `SELECT NOT TRUE > TRUE`, this PR fixed it.

Takes over https://github.com/apache/spark/pull/6326.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8617 from cloud-fan/not.
2015-10-20 13:38:25 -07:00
Wenchen Fan 7893cd95db [SPARK-11119] [SQL] cleanup for unsafe array and map
The purpose of this PR is to keep the unsafe format detail only inside the unsafe class itself, so when we use them(like use unsafe array in unsafe map, use unsafe array and map in columnar cache), we don't need to understand the format before use them.

change list:
* unsafe array's 4-bytes numElements header is now required(was optional), and become a part of unsafe array format.
* w.r.t the previous changing, the `sizeInBytes` of unsafe array now counts the 4-bytes header.
* unsafe map's format was `[numElements] [key array numBytes] [key array content(without numElements header)] [value array content(without numElements header)]` before, which is a little hacky as it makes unsafe array's header optional. I think saving 4 bytes is not a big deal, so the format is now: `[key array numBytes] [unsafe key array] [unsafe value array]`.
* w.r.t the previous changing, the `sizeInBytes` of unsafe map now counts both map's header and array's header.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9131 from cloud-fan/unsafe.
2015-10-19 11:02:26 -07:00
Cheng Hao 9808052b5a [SPARK-11076] [SQL] Add decimal support for floor and ceil
Actually all of the `UnaryMathExpression` doens't support the Decimal, will create follow ups for supporing it. This is the first PR which will be good to review the approach I am taking.

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

Closes #9086 from chenghao-intel/ceiling.
2015-10-14 20:56:08 -07:00
Wenchen Fan 56d7da14ab [SPARK-10104] [SQL] Consolidate different forms of table identifiers
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8453 from cloud-fan/table-name.
2015-10-14 16:05:37 -07:00
Michael Armbrust 328d1b3e4b [SPARK-11090] [SQL] Constructor for Product types from InternalRow
This is a first draft of the ability to construct expressions that will take a catalyst internal row and construct a Product (case class or tuple) that has fields with the correct names.  Support include:
 - Nested classes
 - Maps
 - Efficiently handling of arrays of primitive types

Not yet supported:
 - Case classes that require custom collection types (i.e. List instead of Seq).

Author: Michael Armbrust <michael@databricks.com>

Closes #9100 from marmbrus/productContructor.
2015-10-13 17:09:17 -07:00
Michael Armbrust 9e66a53c99 [SPARK-10993] [SQL] Inital code generated encoder for product types
This PR is a first cut at code generating an encoder that takes a Scala `Product` type and converts it directly into the tungsten binary format.  This is done through the addition of a new set of expression that can be used to invoke methods on raw JVM objects, extracting fields and converting the result into the required format.  These can then be used directly in an `UnsafeProjection` allowing us to leverage the existing encoding logic.

According to some simple benchmarks, this can significantly speed up conversion (~4x).  However, replacing CatalystConverters is deferred to a later PR to keep this PR at a reasonable size.

```scala
case class SomeInts(a: Int, b: Int, c: Int, d: Int, e: Int)

val data = SomeInts(1, 2, 3, 4, 5)
val encoder = ProductEncoder[SomeInts]
val converter = CatalystTypeConverters.createToCatalystConverter(ScalaReflection.schemaFor[SomeInts].dataType)

(1 to 5).foreach {iter =>
  benchmark(s"converter $iter") {
    var i = 100000000
    while (i > 0) {
      val res = converter(data).asInstanceOf[InternalRow]
      assert(res.getInt(0) == 1)
      assert(res.getInt(1) == 2)
      i -= 1
    }
  }

  benchmark(s"encoder $iter") {
    var i = 100000000
    while (i > 0) {
      val res = encoder.toRow(data)
      assert(res.getInt(0) == 1)
      assert(res.getInt(1) == 2)
      i -= 1
    }
  }
}
```

Results:
```
[info] converter 1: 7170ms
[info] encoder 1: 1888ms
[info] converter 2: 6763ms
[info] encoder 2: 1824ms
[info] converter 3: 6912ms
[info] encoder 3: 1802ms
[info] converter 4: 7131ms
[info] encoder 4: 1798ms
[info] converter 5: 7350ms
[info] encoder 5: 1912ms
```

Author: Michael Armbrust <michael@databricks.com>

Closes #9019 from marmbrus/productEncoder.
2015-10-08 14:28:14 -07:00
Michael Armbrust a8226a9f14 Revert [SPARK-8654] [SQL] Fix Analysis exception when using NULL IN
This reverts commit dcbd58a929 from #8983

Author: Michael Armbrust <michael@databricks.com>

Closes #9034 from marmbrus/revert8654.
2015-10-08 13:49:10 -07:00
Dilip Biswal dcbd58a929 [SPARK-8654] [SQL] Fix Analysis exception when using NULL IN (...)
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.

The fix is to not generate such a cast if the lhs type is a NullType instead
we translate the expression to Literal(Null).

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #8983 from dilipbiswal/spark_8654.
2015-10-08 10:41:45 -07:00
Davies Liu 37526aca24 [SPARK-10980] [SQL] fix bug in create Decimal
The created decimal is wrong if using `Decimal(unscaled, precision, scale)` with unscaled > 1e18 and and precision > 18 and scale > 0.

This bug exists since the beginning.

Author: Davies Liu <davies@databricks.com>

Closes #9014 from davies/fix_decimal.
2015-10-07 15:51:09 -07:00
Josh Rosen a9ecd06149 [SPARK-10941] [SQL] Refactor AggregateFunction2 and AlgebraicAggregate interfaces to improve code clarity
This patch refactors several of the Aggregate2 interfaces in order to improve code clarity.

The biggest change is a refactoring of the `AggregateFunction2` class hierarchy. In the old code, we had a class named `AlgebraicAggregate` that inherited from `AggregateFunction2`, added a new set of methods, then banned the use of the inherited methods. I found this to be fairly confusing because.

If you look carefully at the existing code, you'll see that subclasses of `AggregateFunction2` fall into two disjoint categories: imperative aggregation functions which directly extended `AggregateFunction2` and declarative, expression-based aggregate functions which extended `AlgebraicAggregate`. In order to make this more explicit, this patch refactors things so that `AggregateFunction2` is a sealed abstract class with two subclasses, `ImperativeAggregateFunction` and `ExpressionAggregateFunction`. The superclass, `AggregateFunction2`, now only contains methods and fields that are common to both subclasses.

After making this change, I updated the various AggregationIterator classes to comply with this new naming scheme. I also performed several small renamings in the aggregate interfaces themselves in order to improve clarity and rewrote or expanded a number of comments.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8973 from JoshRosen/tungsten-agg-comments.
2015-10-07 13:19:49 -07:00
Wenchen Fan c4871369db [SPARK-10585] [SQL] only copy data once when generate unsafe projection
This PR is a completely rewritten of GenerateUnsafeProjection, to accomplish the goal of copying data only once. The old code of GenerateUnsafeProjection is still there to reduce review difficulty.

Instead of creating unsafe conversion code for struct, array and map, we create code of writing the content to the global row buffer.

Author: Wenchen Fan <cloud0fan@163.com>
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8747 from cloud-fan/copy-once.
2015-10-05 13:00:58 -07:00
Cheng Hao 4d8c7c6d1c [SPARK-10865] [SPARK-10866] [SQL] Fix bug of ceil/floor, which should returns long instead of the Double type
Floor & Ceiling function should returns Long type, rather than Double.

Verified with MySQL & Hive.

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

Closes #8933 from chenghao-intel/ceiling.
2015-10-01 11:48:15 -07:00
Nathan Howell 89ea0041ae [SPARK-9617] [SQL] Implement json_tuple
This is an implementation of Hive's `json_tuple` function using Jackson Streaming.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #7946 from NathanHowell/SPARK-9617.
2015-09-30 15:33:12 -07:00
Herman van Hovell 16fd2a2f42 [SPARK-9741] [SQL] Approximate Count Distinct using the new UDAF interface.
This PR implements a HyperLogLog based Approximate Count Distinct function using the new UDAF interface.

The implementation is inspired by the ClearSpring HyperLogLog implementation and should produce the same results.

There is still some documentation and testing left to do.

cc yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #8362 from hvanhovell/SPARK-9741.
2015-09-30 10:12:52 -07:00
Wenchen Fan 5017c685f4 [SPARK-10740] [SQL] handle nondeterministic expressions correctly for set operations
https://issues.apache.org/jira/browse/SPARK-10740

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8858 from cloud-fan/non-deter.
2015-09-22 12:14:59 -07:00
Yijie Shen c6f8135ee5 [SPARK-10539] [SQL] Project should not be pushed down through Intersect or Except #8742
Intersect and Except are both set operators and they use the all the columns to compare equality between rows. When pushing their Project parent down, the relations they based on would change, therefore not an equivalent transformation.

JIRA: https://issues.apache.org/jira/browse/SPARK-10539

I added some comments based on the fix of https://github.com/apache/spark/pull/8742.

Author: Yijie Shen <henry.yijieshen@gmail.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #8823 from yhuai/fix_set_optimization.
2015-09-18 13:20:13 -07:00
Yin Huai aad644fbe2 [SPARK-10639] [SQL] Need to convert UDAF's result from scala to sql type
https://issues.apache.org/jira/browse/SPARK-10639

Author: Yin Huai <yhuai@databricks.com>

Closes #8788 from yhuai/udafConversion.
2015-09-17 11:14:52 -07:00
Kevin Cox d39f15ea2b [SPARK-9794] [SQL] Fix datetime parsing in SparkSQL.
This fixes https://issues.apache.org/jira/browse/SPARK-9794 by using a real ISO8601 parser. (courtesy of the xml component of the standard java library)

cc: angelini

Author: Kevin Cox <kevincox@kevincox.ca>

Closes #8396 from kevincox/kevincox-sql-time-parsing.
2015-09-16 15:30:17 -07:00
Wenchen Fan 31a229aa73 [SPARK-10475] [SQL] improve column prunning for Project on Sort
Sometimes we can't push down the whole `Project` though `Sort`, but we still have a chance to push down part of it.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8644 from cloud-fan/column-prune.
2015-09-15 13:36:52 -07:00
Davies Liu 7e32387ae6 [SPARK-10522] [SQL] Nanoseconds of Timestamp in Parquet should be positive
Or Hive can't read it back correctly.

Thanks vanzin for report this.

Author: Davies Liu <davies@databricks.com>

Closes #8674 from davies/positive_nano.
2015-09-14 14:20:49 -07:00
Wenchen Fan d5d647380f [SPARK-10442] [SQL] fix string to boolean cast
When we cast string to boolean in hive, it returns `true` if the length of string is > 0, and spark SQL follows this behavior.

However, this behavior is very different from other SQL systems:

1. [presto](https://github.com/facebook/presto/blob/master/presto-main/src/main/java/com/facebook/presto/type/VarcharOperators.java#L89-L118) will return `true` for 't' 'true' '1', `false` for 'f' 'false' '0', throw exception for others.
2. [redshift](http://docs.aws.amazon.com/redshift/latest/dg/r_Boolean_type.html) will return `true` for 't' 'true' 'y' 'yes' '1', `false` for 'f' 'false' 'n' 'no' '0', null for others.
3. [postgresql](http://www.postgresql.org/docs/devel/static/datatype-boolean.html) will return `true` for 't' 'true' 'y' 'yes' 'on' '1', `false` for 'f' 'false' 'n' 'no' 'off' '0', throw exception for others.
4. [vertica](https://my.vertica.com/docs/5.0/HTML/Master/2983.htm) will return `true` for 't' 'true' 'y' 'yes' '1', `false` for 'f' 'false' 'n' 'no' '0', null for others.
5. [impala](http://www.cloudera.com/content/cloudera/en/documentation/cloudera-impala/latest/topics/impala_boolean.html) throw exception when try to cast string to boolean.
6. mysql, oracle, sqlserver don't have boolean type

Whether we should change the cast behavior according to other SQL system or not is not decided yet, this PR is a test to see if we changed, how many compatibility tests will fail.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8698 from cloud-fan/string2boolean.
2015-09-11 14:15:16 -07:00
Yash Datta f892d927d7 [SPARK-7142] [SQL] Minor enhancement to BooleanSimplification Optimizer rule
Use these in the optimizer as well:

            A and (not(A) or B) => A and B
            not(A and B) => not(A) or not(B)
            not(A or B) => not(A) and not(B)

Author: Yash Datta <Yash.Datta@guavus.com>

Closes #5700 from saucam/bool_simp.
2015-09-10 10:34:00 -07:00
Yin Huai 7a9dcbc91d [SPARK-10441] [SQL] Save data correctly to json.
https://issues.apache.org/jira/browse/SPARK-10441

Author: Yin Huai <yhuai@databricks.com>

Closes #8597 from yhuai/timestampJson.
2015-09-08 14:10:12 -07:00
Wenchen Fan c3c0e431a6 [SPARK-10176] [SQL] Show partially analyzed plans when checkAnswer fails to analyze
This PR takes over https://github.com/apache/spark/pull/8389.

This PR improves `checkAnswer` to print the partially analyzed plan in addition to the user friendly error message, in order to aid debugging failing tests.

In doing so, I ran into a conflict with the various ways that we bring a SQLContext into the tests. Depending on the trait we refer to the current context as `sqlContext`, `_sqlContext`, `ctx` or `hiveContext` with access modifiers `public`, `protected` and `private` depending on the defining class.

I propose we refactor as follows:

1. All tests should only refer to a `protected sqlContext` when testing general features, and `protected hiveContext` when it is a method that only exists on a `HiveContext`.
2. All tests should only import `testImplicits._` (i.e., don't import `TestHive.implicits._`)

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8584 from cloud-fan/cleanupTests.
2015-09-04 15:17:37 -07:00
Davies Liu bb7f352393 [SPARK-10323] [SQL] fix nullability of In/InSet/ArrayContain
After this PR, In/InSet/ArrayContain will return null if value is null, instead of false. They also will return null even if there is a null in the set/array.

Author: Davies Liu <davies@databricks.com>

Closes #8492 from davies/fix_in.
2015-08-28 14:38:20 -07:00
Davies Liu 7467b52ed0 [SPARK-10215] [SQL] Fix precision of division (follow the rule in Hive)
Follow the rule in Hive for decimal division. see ac755ebe26/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPDivide.java (L113)

cc chenghao-intel

Author: Davies Liu <davies@databricks.com>

Closes #8415 from davies/decimal_div2.
2015-08-25 15:20:24 -07:00
Davies Liu ec89bd840a [SPARK-10245] [SQL] Fix decimal literals with precision < scale
In BigDecimal or java.math.BigDecimal, the precision could be smaller than scale, for example, BigDecimal("0.001") has precision = 1 and scale = 3. But DecimalType require that the precision should be larger than scale, so we should use the maximum of precision and scale when inferring the schema from decimal literal.

Author: Davies Liu <davies@databricks.com>

Closes #8428 from davies/smaller_decimal.
2015-08-25 14:55:34 -07:00
Davies Liu 2f493f7e39 [SPARK-10177] [SQL] fix reading Timestamp in parquet from Hive
We misunderstood the Julian days and nanoseconds of the day in parquet (as TimestampType) from Hive/Impala, they are overlapped, so can't be added together directly.

In order to avoid the confusing rounding when do the converting, we use `2440588` as the Julian Day of epoch of unix timestamp (which should be 2440587.5).

Author: Davies Liu <davies@databricks.com>
Author: Cheng Lian <lian@databricks.com>

Closes #8400 from davies/timestamp_parquet.
2015-08-25 16:00:44 +08:00
Josh Rosen 82268f07ab [SPARK-9293] [SPARK-9813] Analysis should check that set operations are only performed on tables with equal numbers of columns
This patch adds an analyzer rule to ensure that set operations (union, intersect, and except) are only applied to tables with the same number of columns. Without this rule, there are scenarios where invalid queries can return incorrect results instead of failing with error messages; SPARK-9813 provides one example of this problem. In other cases, the invalid query can crash at runtime with extremely confusing exceptions.

I also performed a bit of cleanup to refactor some of those logical operators' code into a common `SetOperation` base class.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7631 from JoshRosen/SPARK-9293.
2015-08-25 00:04:10 -07:00
Josh Rosen d7b4c09527 [SPARK-10190] Fix NPE in CatalystTypeConverters Decimal toScala converter
This adds a missing null check to the Decimal `toScala` converter in `CatalystTypeConverters`, fixing an NPE.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8401 from JoshRosen/SPARK-10190.
2015-08-24 16:17:45 -07:00
Reynold Xin 2f2686a73f [SPARK-9242] [SQL] Audit UDAF interface.
A few minor changes:

1. Improved documentation
2. Rename apply(distinct....) to distinct.
3. Changed MutableAggregationBuffer from a trait to an abstract class.
4. Renamed returnDataType to dataType to be more consistent with other expressions.

And unrelated to UDAFs:

1. Renamed file names in expressions to use suffix "Expressions" to be more consistent.
2. Moved regexp related expressions out to its own file.
3. Renamed StringComparison => StringPredicate.

Author: Reynold Xin <rxin@databricks.com>

Closes #8321 from rxin/SPARK-9242.
2015-08-19 17:35:41 -07:00
Wenchen Fan b0dbaec4f9 [SPARK-6489] [SQL] add column pruning for Generate
This PR takes over https://github.com/apache/spark/pull/5358

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8268 from cloud-fan/6489.
2015-08-19 15:05:06 -07:00
Daoyuan Wang 373a376c04 [SPARK-10083] [SQL] CaseWhen should support type coercion of DecimalType and FractionalType
create t1 (a decimal(7, 2), b long);
select case when 1=1 then a else 1.0 end from t1;
select case when 1=1 then a else b end from t1;

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #8270 from adrian-wang/casewhenfractional.
2015-08-19 14:31:51 -07:00
Davies Liu 1f4c4fe6df [SPARK-10090] [SQL] fix decimal scale of division
We should rounding the result of multiply/division of decimal to expected precision/scale, also check overflow.

Author: Davies Liu <davies@databricks.com>

Closes #8287 from davies/decimal_division.
2015-08-19 14:03:47 -07:00
Davies Liu 5af3838d2e [SPARK-10038] [SQL] fix bug in generated unsafe projection when there is binary in ArrayData
The type for array of array in Java is slightly different than array of others.

cc cloud-fan

Author: Davies Liu <davies@databricks.com>

Closes #8250 from davies/array_binary.
2015-08-17 23:27:55 -07:00
Yijie Shen b265e282b6 [SPARK-9526] [SQL] Utilize randomized tests to reveal potential bugs in sql expressions
JIRA: https://issues.apache.org/jira/browse/SPARK-9526

This PR is a follow up of #7830, aiming at utilizing randomized tests to reveal more potential bugs in sql expression.

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

Closes #7855 from yjshen/property_check.
2015-08-17 14:10:19 -07:00
Wenchen Fan 570567258b [SPARK-9955] [SQL] correct error message for aggregate
We should skip unresolved `LogicalPlan`s for `PullOutNondeterministic`, as calling `output` on unresolved `LogicalPlan` will produce confusing error message.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8203 from cloud-fan/error-msg and squashes the following commits:

1c67ca7 [Wenchen Fan] move test
7593080 [Wenchen Fan] correct error message for aggregate
2015-08-15 14:13:12 -07:00
Wenchen Fan ec29f2034a [SPARK-9634] [SPARK-9323] [SQL] cleanup unnecessary Aliases in LogicalPlan at the end of analysis
Also alias the ExtractValue instead of wrapping it with UnresolvedAlias when resolve attribute in LogicalPlan, as this alias will be trimmed if it's unnecessary.

Based on #7957 without the changes to mllib, but instead maintaining earlier behavior when using `withColumn` on expressions that already have metadata.

Author: Wenchen Fan <cloud0fan@outlook.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #8215 from marmbrus/pr/7957.
2015-08-14 20:59:54 -07:00
Andrew Or 8187b3ae47 [SPARK-9580] [SQL] Replace singletons in SQL tests
A fundamental limitation of the existing SQL tests is that *there is simply no way to create your own `SparkContext`*. This is a serious limitation because the user may wish to use a different master or config. As a case in point, `BroadcastJoinSuite` is entirely commented out because there is no way to make it pass with the existing infrastructure.

This patch removes the singletons `TestSQLContext` and `TestData`, and instead introduces a `SharedSQLContext` that starts a context per suite. Unfortunately the singletons were so ingrained in the SQL tests that this patch necessarily needed to touch *all* the SQL test files.

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

Author: Andrew Or <andrew@databricks.com>

Closes #8111 from andrewor14/sql-tests-refactor.
2015-08-13 17:42:01 -07:00
Cheng Lian 6993031011 [SPARK-9757] [SQL] Fixes persistence of Parquet relation with decimal column
PR #7967 enables us to save data source relations to metastore in Hive compatible format when possible. But it fails to persist Parquet relations with decimal column(s) to Hive metastore of versions lower than 1.2.0. This is because `ParquetHiveSerDe` in Hive versions prior to 1.2.0 doesn't support decimal. This PR checks for this case and falls back to Spark SQL specific metastore table format.

Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Lian <lian@databricks.com>

Closes #8130 from liancheng/spark-9757/old-hive-parquet-decimal.
2015-08-13 16:16:50 +08:00
Josh Rosen dfe347d2ca [SPARK-9785] [SQL] HashPartitioning compatibility should consider expression ordering
HashPartitioning compatibility is currently defined w.r.t the _set_ of expressions, but the ordering of those expressions matters when computing hash codes; this could lead to incorrect answers if we mistakenly avoided a shuffle based on the assumption that HashPartitionings with the same expressions in different orders will produce equivalent row hashcodes. The first commit adds a regression test which illustrates this problem.

The fix for this is simple: make `HashPartitioning.compatibleWith` and `HashPartitioning.guarantees` sensitive to the expression ordering (i.e. do not perform set comparison).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8074 from JoshRosen/hashpartitioning-compatiblewith-fixes and squashes the following commits:

b61412f [Josh Rosen] Demonstrate that I haven't cheated in my fix
0b4d7d9 [Josh Rosen] Update so that clusteringSet is only used in satisfies().
dc9c9d7 [Josh Rosen] Add failing regression test for SPARK-9785
2015-08-11 08:52:15 -07:00
Reynold Xin d378396f86 [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
PlatformDependent.UNSAFE is way too verbose.

Author: Reynold Xin <rxin@databricks.com>

Closes #8094 from rxin/SPARK-9815 and squashes the following commits:

229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
2015-08-11 08:41:06 -07:00
Davies Liu fe2fb7fb71 [SPARK-9620] [SQL] generated UnsafeProjection should support many columns or large exressions
Currently, generated UnsafeProjection can reach 64k byte code limit of Java. This patch will split the generated expressions into multiple functions, to avoid the limitation.

After this patch, we can work well with table that have up to 64k columns (hit max number of constants limit in Java), it should be enough in practice.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #8044 from davies/wider_table and squashes the following commits:

9192e6c [Davies Liu] fix generated safe projection
d1ef81a [Davies Liu] fix failed tests
737b3d3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
ffcd132 [Davies Liu] address comments
1b95be4 [Davies Liu] put the generated class into sql package
77ed72d [Davies Liu] address comments
4518e17 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
75ccd01 [Davies Liu] Merge branch 'master' of github.com:apache/spark into wider_table
495e932 [Davies Liu] support wider table with more than 1k columns for generated projections
2015-08-10 13:52:18 -07:00
Yijie Shen 68ccc6e184 [SPARK-8930] [SQL] Throw a AnalysisException with meaningful messages if DataFrame#explode takes a star in expressions
Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #8057 from yjshen/explode_star and squashes the following commits:

eae181d [Yijie Shen] change explaination message
54c9d11 [Yijie Shen] meaning message for * in explode
2015-08-09 11:44:51 -07:00
Wenchen Fan 2432c2e239 [SPARK-8382] [SQL] Improve Analysis Unit test framework
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8025 from cloud-fan/analysis and squashes the following commits:

51461b1 [Wenchen Fan] move test file to test folder
ec88ace [Wenchen Fan] Improve Analysis Unit test framework
2015-08-07 11:28:43 -07:00
Wenchen Fan 1f62f104c7 [SPARK-9632][SQL] update InternalRow.toSeq to make it accept data type info
This re-applies #7955, which was reverted due to a race condition to fix build breaking.

Author: Wenchen Fan <cloud0fan@outlook.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #8002 from rxin/InternalRow-toSeq and squashes the following commits:

332416a [Reynold Xin] Merge pull request #7955 from cloud-fan/toSeq
21665e2 [Wenchen Fan] fix hive again...
4addf29 [Wenchen Fan] fix hive
bc16c59 [Wenchen Fan] minor fix
33d802c [Wenchen Fan] pass data type info to InternalRow.toSeq
3dd033e [Wenchen Fan] move the default special getters implementation from InternalRow to BaseGenericInternalRow
2015-08-06 13:11:59 -07:00
Davies Liu 2eca46a17a Revert "[SPARK-9632][SQL] update InternalRow.toSeq to make it accept data type info"
This reverts commit 6e009cb9c4.
2015-08-06 11:15:37 -07:00
Wenchen Fan 6e009cb9c4 [SPARK-9632][SQL] update InternalRow.toSeq to make it accept data type info
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7955 from cloud-fan/toSeq and squashes the following commits:

21665e2 [Wenchen Fan] fix hive again...
4addf29 [Wenchen Fan] fix hive
bc16c59 [Wenchen Fan] minor fix
33d802c [Wenchen Fan] pass data type info to InternalRow.toSeq
3dd033e [Wenchen Fan] move the default special getters implementation from InternalRow to BaseGenericInternalRow
2015-08-06 10:40:54 -07:00
Davies Liu 5b965d64ee [SPARK-9644] [SQL] Support update DecimalType with precision > 18 in UnsafeRow
In order to support update a varlength (actually fixed length) object, the space should be preserved even  it's null. And, we can't call setNullAt(i) for it anymore, we because setNullAt(i) will remove the offset of the preserved space, should call setDecimal(i, null, precision) instead.

After this, we can do hash based aggregation on DecimalType with precision > 18. In a tests, this could decrease the end-to-end run time of aggregation query from 37 seconds (sort based) to 24 seconds (hash based).

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7978 from davies/update_decimal and squashes the following commits:

bed8100 [Davies Liu] isSettable -> isMutable
923c9eb [Davies Liu] address comments and fix bug
385891d [Davies Liu] Merge branch 'master' of github.com:apache/spark into update_decimal
36a1872 [Davies Liu] fix tests
cd6c524 [Davies Liu] support set decimal with precision > 18
2015-08-06 09:10:57 -07:00
zhichao.li aead18ffca [SPARK-8266] [SQL] add function translate
![translate](http://www.w3resource.com/PostgreSQL/postgresql-translate-function.png)

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

Closes #7709 from zhichao-li/translate and squashes the following commits:

9418088 [zhichao.li] refine checking condition
f2ab77a [zhichao.li] clone string
9d88f2d [zhichao.li] fix indent
6aa2962 [zhichao.li] style
e575ead [zhichao.li] add python api
9d4bab0 [zhichao.li] add special case for fodable and refactor unittest
eda7ad6 [zhichao.li] update to use TernaryExpression
cdfd4be [zhichao.li] add function translate
2015-08-06 09:02:30 -07:00
Josh Rosen 9c878923db [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in SMJ
This patches renames `RowOrdering` to `InterpretedOrdering` and updates SortMergeJoin to use the `SparkPlan` methods for constructing its ordering so that it may benefit from codegen.

This is an updated version of #7408.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7973 from JoshRosen/SPARK-9054 and squashes the following commits:

e610655 [Josh Rosen] Add comment RE: Ascending ordering
34b8e0c [Josh Rosen] Import ordering
be19a0f [Josh Rosen] [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in more places.
2015-08-05 16:33:42 -07:00
Liang-Chi Hsieh e1e05873fc [SPARK-9403] [SQL] Add codegen support in In and InSet
This continues tarekauel's work in #7778.

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Tarek Auel <tarek.auel@googlemail.com>

Closes #7893 from viirya/codegen_in and squashes the following commits:

81ff97b [Liang-Chi Hsieh] For comments.
47761c6 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
cf4bf41 [Liang-Chi Hsieh] For comments.
f532b3c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
446bbcd [Liang-Chi Hsieh] Fix bug.
b3d0ab4 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
4610eff [Liang-Chi Hsieh] Relax the types of references and update optimizer test.
224f18e [Liang-Chi Hsieh] Beef up the test cases for In and InSet to include all primitive data types.
86dc8aa [Liang-Chi Hsieh] Only convert In to InSet when the number of items in set is more than the threshold.
b7ded7e [Tarek Auel] [SPARK-9403][SQL] codeGen in / inSet
2015-08-05 11:38:56 -07:00
Yin Huai 1f8c364b9c [SPARK-9141] [SQL] [MINOR] Fix comments of PR #7920
This is a follow-up of https://github.com/apache/spark/pull/7920 to fix comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #7964 from yhuai/SPARK-9141-follow-up and squashes the following commits:

4d0ee80 [Yin Huai] Fix comments.
2015-08-05 11:03:02 -07:00
Michael Armbrust 23d982204b [SPARK-9141] [SQL] Remove project collapsing from DataFrame API
Currently we collapse successive projections that are added by `withColumn`.  However, this optimization violates the constraint that adding nodes to a plan will never change its analyzed form and thus breaks caching.  Instead of doing early optimization, in this PR I just fix some low-hanging slowness in the analyzer.  In particular, I add a mechanism for skipping already analyzed subplans, `resolveOperators` and `resolveExpression`.  Since trees are generally immutable after construction, it's safe to annotate a plan as already analyzed as any transformation will create a new tree with this bit no longer set.

Together these result in a faster analyzer than before, even with added timing instrumentation.

```
Original Code
[info] 3430ms
[info] 2205ms
[info] 1973ms
[info] 1982ms
[info] 1916ms

Without Project Collapsing in DataFrame
[info] 44610ms
[info] 45977ms
[info] 46423ms
[info] 46306ms
[info] 54723ms

With analyzer optimizations
[info] 6394ms
[info] 4630ms
[info] 4388ms
[info] 4093ms
[info] 4113ms

With resolveOperators
[info] 2495ms
[info] 1380ms
[info] 1685ms
[info] 1414ms
[info] 1240ms
```

Author: Michael Armbrust <michael@databricks.com>

Closes #7920 from marmbrus/withColumnCache and squashes the following commits:

2145031 [Michael Armbrust] fix hive udfs tests
5a5a525 [Michael Armbrust] remove wrong comment
7a507d5 [Michael Armbrust] style
b59d710 [Michael Armbrust] revert small change
1fa5949 [Michael Armbrust] move logic into LogicalPlan, add tests
0e2cb43 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into withColumnCache
c926e24 [Michael Armbrust] naming
e593a2d [Michael Armbrust] style
f5a929e [Michael Armbrust] [SPARK-9141][SQL] Remove project collapsing from DataFrame API
38b1c83 [Michael Armbrust] WIP
2015-08-05 09:01:45 -07:00
Davies Liu 781c8d71a0 [SPARK-9119] [SPARK-8359] [SQL] match Decimal.precision/scale with DecimalType
Let Decimal carry the correct precision and scale with DecimalType.

cc rxin yhuai

Author: Davies Liu <davies@databricks.com>

Closes #7925 from davies/decimal_scale and squashes the following commits:

e19701a [Davies Liu] some tweaks
57d78d2 [Davies Liu] fix tests
5d5bc69 [Davies Liu] match precision and scale with DecimalType
2015-08-04 23:12:49 -07:00
Pedro Rodriguez d34548587a [SPARK-8231] [SQL] Add array_contains
This PR is based on #7580 , thanks to EntilZha

PR for work on https://issues.apache.org/jira/browse/SPARK-8231

Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128

Main points are:
1. If the array is empty, null, or the value is null, return false
2. If there is a type mismatch, throw error
3. If comparison is not supported, throw error

Closes #7580

Author: Pedro Rodriguez <prodriguez@trulia.com>
Author: Pedro Rodriguez <ski.rodriguez@gmail.com>
Author: Davies Liu <davies@databricks.com>

Closes #7949 from davies/array_contains and squashes the following commits:

d3c08bc [Davies Liu] use foreach() to avoid copy
bc3d1fe [Davies Liu] fix array_contains
719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains
e352cf9 [Pedro Rodriguez] fixed diff from master
4d5b0ff [Pedro Rodriguez] added docs and another type check
ffc0591 [Pedro Rodriguez] fixed unit test
7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3
b5ffae8 [Pedro Rodriguez] fixed pyspark test
4e7dce3 [Pedro Rodriguez] added more docs
3082399 [Pedro Rodriguez] fixed unit test
46f9789 [Pedro Rodriguez] reverted change
d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this
8528027 [Pedro Rodriguez] added more tests
686e029 [Pedro Rodriguez] fix scala style
d262e9d [Pedro Rodriguez] reworked type checking code and added more tests
2517a58 [Pedro Rodriguez] removed unused import
28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests
12f8795 [Pedro Rodriguez] fix scala style checks
e8a20a9 [Pedro Rodriguez] added python df (broken atm)
65b562c [Pedro Rodriguez] made array_contains nullable false
33b45aa [Pedro Rodriguez] reordered test
9623c64 [Pedro Rodriguez] fixed test
4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs
72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs
69c46fb [Pedro Rodriguez] added tests and codegen
9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
2015-08-04 22:34:02 -07:00
Yijie Shen a7fe48f687 [SPARK-9432][SQL] Audit expression unit tests to make sure we pass the proper numeric ranges
JIRA: https://issues.apache.org/jira/browse/SPARK-9432

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

Closes #7933 from yjshen/numeric_ranges and squashes the following commits:

e719f78 [Yijie Shen] proper integral range check
2015-08-04 18:19:26 -07:00
Michael Armbrust 34a0eb2e89 [SPARK-9512][SQL] Revert SPARK-9251, Allow evaluation while sorting
The analysis rule has a bug and we ended up making the sorter still capable of doing evaluation, so lets revert this for now.

Author: Michael Armbrust <michael@databricks.com>

Closes #7906 from marmbrus/revertSortProjection and squashes the following commits:

2da6972 [Michael Armbrust] unrevert unrelated changes
4f2b00c [Michael Armbrust] Revert "[SPARK-9251][SQL] do not order by expressions which still need evaluation"
2015-08-04 10:07:53 -07:00
Davies Liu 73dedb589d [SPARK-8246] [SQL] Implement get_json_object
This is based on #7485 , thanks to NathanHowell

Tests were copied from Hive, but do not seem to be super comprehensive. I've generally replicated Hive's unusual behavior rather than following a JSONPath reference, except for one case (as noted in the comments). I don't know if there is a way of fully replicating Hive's behavior without a slower TreeNode implementation, so I've erred on the side of performance instead.

Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Nathan Howell <nhowell@godaddy.com>

Closes #7901 from davies/get_json_object and squashes the following commits:

3ace9b9 [Davies Liu] Merge branch 'get_json_object' of github.com:davies/spark into get_json_object
98766fc [Davies Liu] Merge branch 'master' of github.com:apache/spark into get_json_object
a7dc6d0 [Davies Liu] Update JsonExpressionsSuite.scala
c818519 [Yin Huai] new results.
18ce26b [Davies Liu] fix tests
6ac29fb [Yin Huai] Golden files.
25eebef [Davies Liu] use HiveQuerySuite
e0ac6ec [Yin Huai] Golden answer files.
940c060 [Davies Liu] tweat code style
44084c5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into get_json_object
9192d09 [Nathan Howell] Match Hive’s behavior for unwrapping arrays of one element
8dab647 [Nathan Howell] [SPARK-8246] [SQL] Implement get_json_object
2015-08-04 09:07:09 -07:00
Tarek Auel b1f88a38d5 [SPARK-8244] [SQL] string function: find in set
This PR is based on #7186 (just fix the conflict), thanks to tarekauel .

find_in_set(string str, string strList): int

Returns the first occurance of str in strList where strList is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument contains any commas. For example, find_in_set('ab', 'abc,b,ab,c,def') returns 3.

Only add this to SQL, not DataFrame.

Closes #7186

Author: Tarek Auel <tarek.auel@googlemail.com>
Author: Davies Liu <davies@databricks.com>

Closes #7900 from davies/find_in_set and squashes the following commits:

4334209 [Davies Liu] Merge branch 'master' of github.com:apache/spark into find_in_set
8f00572 [Davies Liu] Merge branch 'master' of github.com:apache/spark into find_in_set
243ede4 [Tarek Auel] [SPARK-8244][SQL] hive compatibility
1aaf64e [Tarek Auel] [SPARK-8244][SQL] unit test fix
e4093a4 [Tarek Auel] [SPARK-8244][SQL] final modifier for COMMA_UTF8
0d05df5 [Tarek Auel] Merge branch 'master' into SPARK-8244
208d710 [Tarek Auel] [SPARK-8244] address comments & bug fix
71b2e69 [Tarek Auel] [SPARK-8244] find_in_set
66c7fda [Tarek Auel] Merge branch 'master' into SPARK-8244
61b8ca2 [Tarek Auel] [SPARK-8224] removed loop and split; use unsafe String comparison
4f75a65 [Tarek Auel] Merge branch 'master' into SPARK-8244
e3b20c8 [Tarek Auel] [SPARK-8244] added type check
1c2bbb7 [Tarek Auel] [SPARK-8244] findInSet
2015-08-04 08:59:42 -07:00
Reynold Xin b2e4b85d2d Revert "[SPARK-9372] [SQL] Filter nulls in join keys"
This reverts commit 687c8c3715.
2015-08-03 14:51:15 -07:00
Reynold Xin 8be198c869 Two minor comments from code review on 191bf2689. 2015-08-03 04:26:18 -07:00
Davies Liu 191bf2689d [SPARK-9518] [SQL] cleanup generated UnsafeRowJoiner and fix bug
Currently, when copy the bitsets, we didn't consider that the row1 may not sit in the beginning of byte array.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7892 from davies/clean_join and squashes the following commits:

14cce9e [Davies Liu] cleanup generated UnsafeRowJoiner and fix bug
2015-08-03 04:23:26 -07:00
Yijie Shen 98d6d9c7a9 [SPARK-9549][SQL] fix bugs in expressions
JIRA: https://issues.apache.org/jira/browse/SPARK-9549

This PR fix the following bugs:
1.  `UnaryMinus`'s codegen version would fail to compile when the input is `Long.MinValue`
2.  `BinaryComparison` would fail to compile in codegen mode when comparing Boolean types.
3.  `AddMonth` would fail if passed a huge negative month, which would lead accessing negative index of `monthDays` array.
4.  `Nanvl` with different type operands.

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

Closes #7882 from yjshen/minor_bug_fix and squashes the following commits:

41bbd2c [Yijie Shen] fix bug in Nanvl type coercion
3dee204 [Yijie Shen] address comments
4fa5de0 [Yijie Shen] fix bugs in expressions
2015-08-03 00:15:24 -07:00
Wenchen Fan 608353c8e8 [SPARK-9404][SPARK-9542][SQL] unsafe array data and map data
This PR adds a UnsafeArrayData, current we encode it in this way:

first 4 bytes is the # elements
then each 4 byte is the start offset of the element, unless it is negative, in which case the element is null.
followed by the elements themselves

an example:  [10, 11, 12, 13, null, 14] will be encoded as:
5, 28, 32, 36, 40, -44, 44, 10, 11, 12, 13, 14

Note that, when we read a UnsafeArrayData from bytes, we can read the first 4 bytes as numElements and take the rest(first 4 bytes skipped) as value region.

unsafe map data just use 2 unsafe array data, first 4 bytes is # of elements, second 4 bytes is numBytes of key array, the follows key array data and value array data.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7752 from cloud-fan/unsafe-array and squashes the following commits:

3269bd7 [Wenchen Fan] fix a bug
6445289 [Wenchen Fan] add unit tests
49adf26 [Wenchen Fan] add unsafe map
20d1039 [Wenchen Fan] add comments and unsafe converter
821b8db [Wenchen Fan] add unsafe array
2015-08-02 23:41:16 -07:00
Yin Huai 687c8c3715 [SPARK-9372] [SQL] Filter nulls in join keys
This PR adds an optimization rule, `FilterNullsInJoinKey`, to add `Filter` before join operators to filter out rows having null values for join keys.

This optimization is guarded by a new SQL conf, `spark.sql.advancedOptimization`.

The code in this PR was authored by yhuai; I'm opening this PR to factor out this change from #7685, a larger pull request which contains two other optimizations.

Author: Yin Huai <yhuai@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #7768 from JoshRosen/filter-nulls-in-join-key and squashes the following commits:

c02fc3f [Yin Huai] Address Josh's comments.
0a8e096 [Yin Huai] Update comments.
ea7d5a6 [Yin Huai] Make sure we do not keep adding filters.
be88760 [Yin Huai] Make it clear that FilterNullsInJoinKeySuite.scala is used to test FilterNullsInJoinKey.
8bb39ad [Yin Huai] Fix non-deterministic tests.
303236b [Josh Rosen] Revert changes that are unrelated to null join key filtering
40eeece [Josh Rosen] Merge remote-tracking branch 'origin/master' into filter-nulls-in-join-key
c57a954 [Yin Huai] Bug fix.
d3d2e64 [Yin Huai] First round of cleanup.
f9516b0 [Yin Huai] Style
c6667e7 [Yin Huai] Add PartitioningCollection.
e616d3b [Yin Huai] wip
7c2d2d8 [Yin Huai] Bug fix and refactoring.
69bb072 [Yin Huai] Introduce NullSafeHashPartitioning and NullUnsafePartitioning.
d5b84c3 [Yin Huai] Do not add unnessary filters.
2201129 [Yin Huai] Filter out rows that will not be joined in equal joins early.
2015-08-02 23:32:09 -07:00
Yin Huai 114ff926fc [SPARK-2205] [SQL] Avoid unnecessary exchange operators in multi-way joins
This PR adds `PartitioningCollection`, which is used to represent the `outputPartitioning` for SparkPlans with multiple children (e.g. `ShuffledHashJoin`). So, a `SparkPlan` can have multiple descriptions of its partitioning schemes. Taking `ShuffledHashJoin` as an example, it has two descriptions of its partitioning schemes, i.e. `left.outputPartitioning` and `right.outputPartitioning`. So when we have a query like `select * from t1 join t2 on (t1.x = t2.x) join t3 on (t2.x = t3.x)` will only have three Exchange operators (when shuffled joins are needed) instead of four.

The code in this PR was authored by yhuai; I'm opening this PR to factor out this change from #7685, a larger pull request which contains two other optimizations.

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

Author: Yin Huai <yhuai@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #7773 from JoshRosen/multi-way-join-planning-improvements and squashes the following commits:

5c45924 [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements
cd8269b [Josh Rosen] Refactor test to use SQLTestUtils
2963857 [Yin Huai] Revert unnecessary SqlConf change.
73913f7 [Yin Huai] Add comments and test. Also, revert the change in ShuffledHashOuterJoin for now.
4a99204 [Josh Rosen] Delete unrelated expression change
884ab95 [Josh Rosen] Carve out only SPARK-2205 changes.
247e5fa [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements
c57a954 [Yin Huai] Bug fix.
d3d2e64 [Yin Huai] First round of cleanup.
f9516b0 [Yin Huai] Style
c6667e7 [Yin Huai] Add PartitioningCollection.
e616d3b [Yin Huai] wip
7c2d2d8 [Yin Huai] Bug fix and refactoring.
69bb072 [Yin Huai] Introduce NullSafeHashPartitioning and NullUnsafePartitioning.
d5b84c3 [Yin Huai] Do not add unnessary filters.
2201129 [Yin Huai] Filter out rows that will not be joined in equal joins early.
2015-08-02 20:44:23 -07:00
Reynold Xin 30e89111d6 [SPARK-9546][SQL] Centralize orderable data type checking.
This pull request creates two isOrderable functions in RowOrdering that can be used to check whether a data type or a sequence of expressions can be used in sorting.

Author: Reynold Xin <rxin@databricks.com>

Closes #7880 from rxin/SPARK-9546 and squashes the following commits:

f9e322d [Reynold Xin] Fixed tests.
0439b43 [Reynold Xin] [SPARK-9546][SQL] Centralize orderable data type checking.
2015-08-02 20:12:03 -07:00
Reynold Xin 9d03ad910b [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter.
The detailed approach is documented in UnsafeKVExternalSorterSuite.testKVSorter(), working as follows:

1. Create input by generating data randomly based on the given key/value schema (which is also randomly drawn from a list of candidate types)
2. Run UnsafeKVExternalSorter on the generated data
3. Collect the output from the sorter, and make sure the keys are sorted in ascending order
4. Sort the input by both key and value, and sort the sorter output also by both key and value. Compare the sorted input and sorted output together to make sure all the key/values match.
5. Check memory allocation to make sure there is no memory leak.

There is also a spill flag. When set to true, the sorter will spill probabilistically roughly every 100 records.

Author: Reynold Xin <rxin@databricks.com>

Closes #7873 from rxin/kvsorter-randomized-test and squashes the following commits:

a08c251 [Reynold Xin] Resource cleanup.
0488b5c [Reynold Xin] [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter.
2015-08-02 17:54:30 -07:00
Liang-Chi Hsieh 0722f43316 [SPARK-7937][SQL] Support comparison on StructType
This brings #6519 up-to-date with master branch.

Closes #6519.

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #7877 from rxin/sort-struct and squashes the following commits:

4968231 [Reynold Xin] Minor fixes.
2537813 [Reynold Xin] Merge branch 'compare_named_struct' of github.com:viirya/spark-1 into sort-struct
d2ba8ad [Liang-Chi Hsieh] Remove unused import.
3a3f40e [Liang-Chi Hsieh] Don't need to add compare to InternalRow because we can use RowOrdering.
dae6aad [Liang-Chi Hsieh] Fix nested struct.
d5349c7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
43d4354 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
1f66196 [Liang-Chi Hsieh] Reuse RowOrdering and GenerateOrdering.
f8b2e9c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
1187a65 [Liang-Chi Hsieh] Fix scala style.
9d67f68 [Liang-Chi Hsieh] Fix wrongly merging.
8f4d775 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
94b27d5 [Liang-Chi Hsieh] Remove test for error on complex type comparison.
2071693 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into compare_named_struct
3c142e4 [Liang-Chi Hsieh] Fix scala style.
cf58dc3 [Liang-Chi Hsieh] Use checkAnswer.
f651b8d [Liang-Chi Hsieh] Remove Either and move orderings to BinaryComparison to reuse it.
b6e1009 [Liang-Chi Hsieh] Fix scala style.
3922b54 [Liang-Chi Hsieh] Support ordering on named_struct.
2015-08-02 17:53:44 -07:00
Davies Liu 16b928c543 [SPARK-9529] [SQL] improve TungstenSort on DecimalType
Generate prefix for DecimalType, fix the random generator of decimal

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #7857 from davies/sort_decimal and squashes the following commits:

2433959 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sort_decimal
de24253 [Davies Liu] fix style
0a54c1a [Davies Liu] sort decimal
2015-08-01 23:36:06 -07:00
Davies Liu 57084e0c7c [SPARK-9459] [SQL] use generated FromUnsafeProjection to do deep copy for UTF8String and struct
When accessing a column in UnsafeRow, it's good to avoid the copy, then we should do deep copy when turn the UnsafeRow into generic Row, this PR brings generated FromUnsafeProjection to do that.

This PR also fix the expressions that cache the UTF8String, which should also copy it.

Author: Davies Liu <davies@databricks.com>

Closes #7840 from davies/avoid_copy and squashes the following commits:

230c8a1 [Davies Liu] address comment
fd797c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into avoid_copy
e095dd0 [Davies Liu] rollback rename
8ef5b0b [Davies Liu] copy String in Columnar
81360b8 [Davies Liu] fix class name
9aecb88 [Davies Liu] use FromUnsafeProjection to do deep copy for UTF8String and struct
2015-08-01 21:50:42 -07:00
Davies Liu c1b0cbd762 [SPARK-8185] [SPARK-8188] [SPARK-8191] [SQL] function datediff, to_utc_timestamp, from_utc_timestamp
This PR is based on #7643 , thanks to adrian-wang

Author: Davies Liu <davies@databricks.com>
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #7847 from davies/datediff and squashes the following commits:

74333d7 [Davies Liu] fix bug
22d8a8c [Davies Liu] optimize
85cdd21 [Davies Liu] remove unnecessary tests
241d90c [Davies Liu] Merge branch 'master' of github.com:apache/spark into datediff
e9dc0f5 [Davies Liu] fix datediff/to_utc_timestamp/from_utc_timestamp
c360447 [Daoyuan Wang] function datediff, to_utc_timestamp, from_utc_timestamp (commits merged)
2015-08-01 21:46:46 -07:00
HuJiayin 00cd92f32f [SPARK-8269] [SQL] string function: initcap
This PR is based on #7208 , thanks to HuJiayin

Closes #7208

Author: HuJiayin <jiayin.hu@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7850 from davies/initcap and squashes the following commits:

54472e9 [Davies Liu] fix python test
17ffe51 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
ca46390 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
3a906e4 [Davies Liu] implement title case in UTF8String
8b2506a [HuJiayin] Update functions.py
2cd43e5 [HuJiayin] fix python style check
b616c0e [HuJiayin] add python api
1f5a0ef [HuJiayin] add codegen
7e0c604 [HuJiayin] Merge branch 'master' of https://github.com/apache/spark into initcap
6a0b958 [HuJiayin] add column
c79482d [HuJiayin] support soundex
7ce416b [HuJiayin] support initcap rebase code
2015-08-01 21:44:57 -07:00
zhichao.li c5166f7a69 [SPARK-8263] [SQL] substr/substring should also support binary type
This is based on #7641, thanks to zhichao-li

Closes #7641

Author: zhichao.li <zhichao.li@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7848 from davies/substr and squashes the following commits:

461b709 [Davies Liu] remove bytearry from tests
b45377a [Davies Liu] Merge branch 'master' of github.com:apache/spark into substr
01d795e [zhichao.li] scala style
99aa130 [zhichao.li] add substring to dataframe
4f68bfe [zhichao.li] add binary type support for substring
2015-08-01 08:48:46 -07:00
Cheng Hao cf6c9ca32a [SPARK-8232] [SQL] Add sort_array support
This PR is based on #7581 , just fix the conflict.

Author: Cheng Hao <hao.cheng@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7851 from davies/sort_array and squashes the following commits:

a80ef66 [Davies Liu] fix conflict
7cfda65 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sort_array
664c960 [Cheng Hao] update the sort_array by using the ArrayData
276d2d5 [Cheng Hao] add empty line
0edab9c [Cheng Hao] Add asending/descending support for sort_array
80fc0f8 [Cheng Hao] Add type checking
a42b678 [Cheng Hao] Add sort_array support
2015-08-01 08:32:29 -07:00
Davies Liu 60ea7ab4bb Revert "[SPARK-8232] [SQL] Add sort_array support"
This reverts commit 67ad4e21fc.
2015-08-01 00:41:15 -07:00
Wenchen Fan 1d59a4162b [SPARK-9480][SQL] add MapData and cleanup internal row stuff
This PR adds a `MapData` as internal representation of map type in Spark SQL, and provides a default implementation with just 2 `ArrayData`.

After that, we have specialized getters for all internal type, so I removed generic getter in `ArrayData` and added specialized `toArray` for it.
Also did some refactor and cleanup for `InternalRow` and its subclasses.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7799 from cloud-fan/map-data and squashes the following commits:

77d482f [Wenchen Fan] fix python
e8f6682 [Wenchen Fan] skip MapData equality check in HiveInspectorSuite
40cc9db [Wenchen Fan] add toString
6e06ec9 [Wenchen Fan] some more cleanup
a90aca1 [Wenchen Fan] add MapData
2015-08-01 00:17:15 -07:00
Cheng Hao 67ad4e21fc [SPARK-8232] [SQL] Add sort_array support
Add expression `sort_array` support.

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

This patch had conflicts when merged, resolved by
Committer: Davies Liu <davies.liu@gmail.com>

Closes #7581 from chenghao-intel/sort_array and squashes the following commits:

664c960 [Cheng Hao] update the sort_array by using the ArrayData
276d2d5 [Cheng Hao] add empty line
0edab9c [Cheng Hao] Add asending/descending support for sort_array
80fc0f8 [Cheng Hao] Add type checking
a42b678 [Cheng Hao] Add sort_array support
2015-07-31 23:11:22 -07:00
Liang-Chi Hsieh 3320b0ba26 [SPARK-9415][SQL] Throw AnalysisException when using MapType on Join and Aggregate
JIRA: https://issues.apache.org/jira/browse/SPARK-9415

Following up #7787. We shouldn't use MapType as grouping keys and join keys too.

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

Closes #7819 from viirya/map_join_groupby and squashes the following commits:

005ee0c [Liang-Chi Hsieh] For comments.
7463398 [Liang-Chi Hsieh] MapType can't be used as join keys, grouping keys.
2015-07-31 22:26:30 -07:00
zhichao.li 6996bd2e81 [SPARK-8264][SQL]add substring_index function
This PR is based on #7533 , thanks to zhichao-li

Closes #7533

Author: zhichao.li <zhichao.li@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7843 from davies/str_index and squashes the following commits:

391347b [Davies Liu] add python api
3ce7802 [Davies Liu] fix substringIndex
f2d29a1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into str_index
515519b [zhichao.li] add foldable and remove null checking
9546991 [zhichao.li] scala style
67c253a [zhichao.li] hide some apis and clean code
b19b013 [zhichao.li] add codegen and clean code
ac863e9 [zhichao.li] reduce the calling of numChars
12e108f [zhichao.li] refine unittest
d92951b [zhichao.li] add lastIndexOf
52d7b03 [zhichao.li] add substring_index function
2015-07-31 21:18:01 -07:00
Reynold Xin 03377d2522 [SPARK-9358][SQL] Code generation for UnsafeRow joiner.
This patch creates a code generated unsafe row concatenator that can be used to concatenate/join two UnsafeRows into a single UnsafeRow.

Since it is inherently hard to test these low level stuff, the test suites employ randomized testing heavily in order to guarantee correctness.

Author: Reynold Xin <rxin@databricks.com>

Closes #7821 from rxin/rowconcat and squashes the following commits:

8717f35 [Reynold Xin] Rebase and code review.
72c5d8e [Reynold Xin] Fixed a bug.
a84ed2e [Reynold Xin] Fixed offset.
40c3fb2 [Reynold Xin] Reset random data generator.
f0913aa [Reynold Xin] Test fixes.
6687b6f [Reynold Xin] Updated documentation.
00354b9 [Reynold Xin] Support concat data as well.
e9a4347 [Reynold Xin] Updated.
6269f96 [Reynold Xin] Fixed a bug .
0f89716 [Reynold Xin] [SPARK-9358][SQL][WIP] Code generation for UnsafeRow concat.
2015-07-31 21:09:00 -07:00
Josh Rosen 8cb415a4b9 [SPARK-9451] [SQL] Support entries larger than default page size in BytesToBytesMap & integrate with ShuffleMemoryManager
This patch adds support for entries larger than the default page size in BytesToBytesMap.  These large rows are handled by allocating special overflow pages to hold individual entries.

In addition, this patch integrates BytesToBytesMap with the ShuffleMemoryManager:

- Move BytesToBytesMap from `unsafe` to `core` so that it can import `ShuffleMemoryManager`.
- Before allocating new data pages, ask the ShuffleMemoryManager to reserve the memory:
  - `putNewKey()` now returns a boolean to indicate whether the insert succeeded or failed due to a lack of memory.  The caller can use this value to respond to the memory pressure (e.g. by spilling).
- `UnsafeFixedWidthAggregationMap. getAggregationBuffer()` now returns `null` to signal failure due to a lack of memory.
- Updated all uses of these classes to handle these error conditions.
- Added new tests for allocating large records and for allocations which fail due to memory pressure.
- Extended the `afterAll()` test teardown methods to detect ShuffleMemoryManager leaks.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7762 from JoshRosen/large-rows and squashes the following commits:

ae7bc56 [Josh Rosen] Fix compilation
82fc657 [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-rows
34ab943 [Josh Rosen] Remove semi
31a525a [Josh Rosen] Integrate BytesToBytesMap with ShuffleMemoryManager.
626b33c [Josh Rosen] Move code to sql/core and spark/core packages so that ShuffleMemoryManager can be integrated
ec4484c [Josh Rosen] Move BytesToBytesMap from unsafe package to core.
642ed69 [Josh Rosen] Rename size to numElements
bea1152 [Josh Rosen] Add basic test.
2cd3570 [Josh Rosen] Remove accidental duplicated code
07ff9ef [Josh Rosen] Basic support for large rows in BytesToBytesMap.
2015-07-31 19:19:27 -07:00
HuJiayin 4d5a6e7b60 [SPARK-8271][SQL]string function: soundex
This PR brings SQL function soundex(), see https://issues.apache.org/jira/browse/HIVE-9738

It's based on #7115 , thanks to HuJiayin

Author: HuJiayin <jiayin.hu@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7812 from davies/soundex and squashes the following commits:

fa75941 [Davies Liu] Merge branch 'master' of github.com:apache/spark into soundex
a4bd6d8 [Davies Liu] fix soundex
2538908 [HuJiayin] add codegen soundex
d15d329 [HuJiayin] add back ut
ded1a14 [HuJiayin] Merge branch 'master' of https://github.com/apache/spark
e2dec2c [HuJiayin] support soundex rebase code
2015-07-31 16:05:26 -07:00
Davies Liu 0024da9157 [SQL] address comments for to_date/trunc
This PR address the comments in #7805

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7817 from davies/trunc and squashes the following commits:

f729d5f [Davies Liu] rollback
cb7f7832 [Davies Liu] genCode() is protected
31e52ef [Davies Liu] fix style
ed1edc7 [Davies Liu] address comments for #7805
2015-07-31 11:07:34 -07:00
Liang-Chi Hsieh 0244170b66 [SPARK-9152][SQL] Implement code generation for Like and RLike
JIRA: https://issues.apache.org/jira/browse/SPARK-9152

This PR implements code generation for `Like` and `RLike`.

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

Closes #7561 from viirya/like_rlike_codegen and squashes the following commits:

fe5641b [Liang-Chi Hsieh] Add test for NonFoldableLiteral.
ccd1b43 [Liang-Chi Hsieh] For comments.
0086723 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into like_rlike_codegen
50df9a8 [Liang-Chi Hsieh] Use nullSafeCodeGen.
8092a68 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into like_rlike_codegen
696d451 [Liang-Chi Hsieh] Check expression foldable.
48e5536 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into like_rlike_codegen
aea58e0 [Liang-Chi Hsieh] For comments.
46d946f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into like_rlike_codegen
a0fb76e [Liang-Chi Hsieh] For comments.
6cffe3c [Liang-Chi Hsieh] For comments.
69f0fb6 [Liang-Chi Hsieh] Add code generation for Like and RLike.
2015-07-30 23:05:58 -07:00
Daoyuan Wang 83670fc9e6 [SPARK-8176] [SPARK-8197] [SQL] function to_date/ trunc
This PR is based on #6988 , thanks to adrian-wang .

This brings two SQL functions: to_date() and trunc().

Closes #6988

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7805 from davies/to_date and squashes the following commits:

2c7beba [Davies Liu] Merge branch 'master' of github.com:apache/spark into to_date
310dd55 [Daoyuan Wang] remove dup test in rebase
980b092 [Daoyuan Wang] resolve rebase conflict
a476c5a [Daoyuan Wang] address comments from davies
d44ea5f [Daoyuan Wang] function to_date, trunc
2015-07-30 19:22:38 -07:00
Davies Liu 0b1a464b6e [SPARK-9425] [SQL] support DecimalType in UnsafeRow
This PR brings the support of DecimalType in UnsafeRow, for precision <= 18, it's settable, otherwise it's not settable.

Author: Davies Liu <davies@databricks.com>

Closes #7758 from davies/unsafe_decimal and squashes the following commits:

478b1ba [Davies Liu] address comments
536314c [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_decimal
7c2e77a [Davies Liu] fix JoinedRow
76d6fa4 [Davies Liu] fix tests
99d3151 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_decimal
d49c6ae [Davies Liu] support DecimalType in UnsafeRow
2015-07-30 17:18:32 -07:00
Daoyuan Wang 1abf7dc16c [SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang

Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.

Closes #7589

cc rxin

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>

Closes #7754 from davies/date_add and squashes the following commits:

e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 13:21:46 -07:00
Daoyuan Wang 6d94bf6ac1 [SPARK-8174] [SPARK-8175] [SQL] function unix_timestamp, from_unixtime
unix_timestamp(): long
Gets current Unix timestamp in seconds.

unix_timestamp(string|date): long
Converts time string in format yyyy-MM-dd HH:mm:ss to Unix timestamp (in seconds), using the default timezone and the default locale, return null if fail: unix_timestamp('2009-03-20 11:30:01') = 1237573801

unix_timestamp(string date, string pattern): long
Convert time string with given pattern (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) to Unix time stamp (in seconds), return null if fail: unix_timestamp('2009-03-20', 'yyyy-MM-dd') = 1237532400.

from_unixtime(bigint unixtime[, string format]): string
Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string representing the timestamp of that moment in the current system time zone in the format of "1970-01-01 00:00:00".

Jira:
https://issues.apache.org/jira/browse/SPARK-8174
https://issues.apache.org/jira/browse/SPARK-8175

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #7644 from adrian-wang/udfunixtime and squashes the following commits:

2fe20c4 [Daoyuan Wang] util.Date
ea2ec16 [Daoyuan Wang] use util.Date for better performance
a2cf929 [Daoyuan Wang] doc return null instead of 0
f6f070a [Daoyuan Wang] address comments from davies
6a4cbb3 [Daoyuan Wang] temp
56ded53 [Daoyuan Wang] rebase and address comments
14a8b37 [Daoyuan Wang] function unix_timestamp, from_unixtime
2015-07-30 11:14:09 -07:00
Wenchen Fan c0cc0eaec6 [SPARK-9390][SQL] create a wrapper for array type
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7724 from cloud-fan/array-data and squashes the following commits:

d0408a1 [Wenchen Fan] fix python
661e608 [Wenchen Fan] rebase
f39256c [Wenchen Fan] fix hive...
6dbfa6f [Wenchen Fan] fix hive again...
8cb8842 [Wenchen Fan] remove element type parameter from getArray
43e9816 [Wenchen Fan] fix mllib
e719afc [Wenchen Fan] fix hive
4346290 [Wenchen Fan] address comment
d4a38da [Wenchen Fan] remove sizeInBytes and add license
7e283e2 [Wenchen Fan] create a wrapper for array type
2015-07-30 10:04:30 -07:00
Joseph Batchik 1221849f91 [SPARK-8005][SQL] Input file name
Users can now get the file name of the partition being read in. A thread local variable is in `SQLNewHadoopRDD` and is set when the partition is computed. `SQLNewHadoopRDD` is moved to core so that the catalyst package can reach it.

This supports:

`df.select(inputFileName())`

and

`sqlContext.sql("select input_file_name() from table")`

Author: Joseph Batchik <josephbatchik@gmail.com>

Closes #7743 from JDrit/input_file_name and squashes the following commits:

abb8609 [Joseph Batchik] fixed failing test and changed the default value to be an empty string
d2f323d [Joseph Batchik] updates per review
102061f [Joseph Batchik] updates per review
75313f5 [Joseph Batchik] small fixes
c7f7b5a [Joseph Batchik] addeding input file name to Spark SQL
2015-07-29 23:35:55 -07:00
Yijie Shen e127ec34d5 [SPARK-9428] [SQL] Add test cases for null inputs for expression unit tests
JIRA: https://issues.apache.org/jira/browse/SPARK-9428

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

Closes #7748 from yjshen/string_cleanup and squashes the following commits:

e0c2b3d [Yijie Shen] update codegen in RegExpExtract and RegExpReplace
26614d2 [Yijie Shen] MathFunctionSuite
a402859 [Yijie Shen] complex_create, conditional and cast
6e4e608 [Yijie Shen] arithmetic and cast
52593c1 [Yijie Shen] null input test cases for StringExpressionSuite
2015-07-29 23:24:20 -07:00
Reynold Xin 27850af525 [SPARK-9462][SQL] Initialize nondeterministic expressions in code gen fallback mode.
Author: Reynold Xin <rxin@databricks.com>

Closes #7767 from rxin/SPARK-9462 and squashes the following commits:

ef3e2d9 [Reynold Xin] Removed println
713ac3a [Reynold Xin] More unit tests.
bb5c334 [Reynold Xin] [SPARK-9462][SQL] Initialize nondeterministic expressions in code gen fallback mode.
2015-07-29 21:24:47 -07:00
Reynold Xin 86505962e6 [SPARK-9448][SQL] GenerateUnsafeProjection should not share expressions across instances.
We accidentally moved the list of expressions from the generated code instance to the class wrapper, and as a result, different threads are sharing the same set of expressions, which cause problems for expressions with mutable state.

This pull request fixed that problem, and also added unit tests for all codegen classes, except GeneratedOrdering (which will never need any expressions since sort now only accepts bound references.

Author: Reynold Xin <rxin@databricks.com>

Closes #7759 from rxin/SPARK-9448 and squashes the following commits:

c09b50f [Reynold Xin] [SPARK-9448][SQL] GenerateUnsafeProjection should not share expressions across instances.
2015-07-29 16:49:02 -07:00
Josh Rosen 1b0099fc62 [SPARK-9411] [SQL] Make Tungsten page sizes configurable
We need to make page sizes configurable so we can reduce them in unit tests and increase them in real production workloads.  These sizes are now controlled by a new configuration, `spark.buffer.pageSize`.  The new default is 64 megabytes.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7741 from JoshRosen/SPARK-9411 and squashes the following commits:

a43c4db [Josh Rosen] Fix pow
2c0eefc [Josh Rosen] Fix MAXIMUM_PAGE_SIZE_BYTES comment + value
bccfb51 [Josh Rosen] Lower page size to 4MB in TestHive
ba54d4b [Josh Rosen] Make UnsafeExternalSorter's page size configurable
0045aa2 [Josh Rosen] Make UnsafeShuffle's page size configurable
bc734f0 [Josh Rosen] Rename configuration
e614858 [Josh Rosen] Makes BytesToBytesMap page size configurable
2015-07-29 16:00:30 -07:00
Reynold Xin 5340dfaf94 [SPARK-9430][SQL] Rename IntervalType to CalendarIntervalType.
We want to introduce a new IntervalType in 1.6 that is based on only the number of microseoncds,
so interval can be compared.

Renaming the existing IntervalType to CalendarIntervalType so we can do that in the future.

Author: Reynold Xin <rxin@databricks.com>

Closes #7745 from rxin/calendarintervaltype and squashes the following commits:

99f64e8 [Reynold Xin] One more line ...
13466c8 [Reynold Xin] Fixed tests.
e20f24e [Reynold Xin] [SPARK-9430][SQL] Rename IntervalType to CalendarIntervalType.
2015-07-29 13:49:22 -07:00
Reynold Xin 97906944e1 [SPARK-9127][SQL] Rand/Randn codegen fails with long seed.
Author: Reynold Xin <rxin@databricks.com>

Closes #7747 from rxin/SPARK-9127 and squashes the following commits:

e851418 [Reynold Xin] [SPARK-9127][SQL] Rand/Randn codegen fails with long seed.
2015-07-29 09:36:22 -07:00
Wenchen Fan 708794e8aa [SPARK-9251][SQL] do not order by expressions which still need evaluation
as an offline discussion with rxin , it's weird to be computing stuff while doing sorting, we should only order by bound reference during execution.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7593 from cloud-fan/sort and squashes the following commits:

7b1bef7 [Wenchen Fan] add test
daf206d [Wenchen Fan] add more comments
289bee0 [Wenchen Fan] do not order by expressions which still need evaluation
2015-07-29 00:08:45 -07:00
Davies Liu 15667a0afa [SPARK-9281] [SQL] use decimal or double when parsing SQL
Right now, we use double to parse all the float number in SQL. When it's used in expression together with DecimalType, it will turn the decimal into double as well. Also it will loss some precision when using double.

This PR change to parse float number to decimal or double, based on it's  using scientific notation or not, see https://msdn.microsoft.com/en-us/library/ms179899.aspx

This is a break change, should we doc it somewhere?

Author: Davies Liu <davies@databricks.com>

Closes #7642 from davies/parse_decimal and squashes the following commits:

1f576d9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into parse_decimal
5e142b6 [Davies Liu] fix scala style
eca99de [Davies Liu] fix tests
2afe702 [Davies Liu] Merge branch 'master' of github.com:apache/spark into parse_decimal
f4a320b [Davies Liu] Update SqlParser.scala
1c48e34 [Davies Liu] use decimal or double when parsing SQL
2015-07-28 22:51:08 -07:00
Yijie Shen 6309b93467 [SPARK-9398] [SQL] Datetime cleanup
JIRA: https://issues.apache.org/jira/browse/SPARK-9398

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

Closes #7725 from yjshen/date_null_check and squashes the following commits:

b4eade1 [Yijie Shen] inline daysToMonthEnd
d09acc1 [Yijie Shen] implement getLastDayOfMonth to avoid repeated evaluation
d857ec3 [Yijie Shen] add null check in DateExpressionSuite
2015-07-28 22:38:28 -07:00
Wenchen Fan 429b2f0df4 [SPARK-8608][SPARK-8609][SPARK-9083][SQL] reset mutable states of nondeterministic expression before evaluation and fix PullOutNondeterministic
We will do local projection for LocalRelation, and thus reuse the same Expression object among multiply evaluations. We should reset the mutable states of Expression before evaluate it.

Fix `PullOutNondeterministic` rule to make it work for `Sort`.

Also got a chance to cleanup the dataframe test suite.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7674 from cloud-fan/show and squashes the following commits:

888934f [Wenchen Fan] fix sort
c0e93e8 [Wenchen Fan] local DataFrame with random columns should return same value when call `show`
2015-07-28 21:37:50 -07:00
Josh Rosen e78ec1a8fa [SPARK-9421] Fix null-handling bugs in UnsafeRow.getDouble, getFloat(), and get(ordinal, dataType)
UnsafeRow.getDouble and getFloat() return NaN when called on columns that are null, which is inconsistent with the behavior of other row classes (which is to return 0.0).

In addition, the generic get(ordinal, dataType) method should always return null for a null literal, but currently it handles nulls by calling the type-specific accessors.

This patch addresses both of these issues and adds a regression test.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7736 from JoshRosen/unsafe-row-null-fixes and squashes the following commits:

c8eb2ee [Josh Rosen] Fix test in UnsafeRowConverterSuite
6214682 [Josh Rosen] Fixes to null handling in UnsafeRow
2015-07-28 17:51:58 -07:00
Reynold Xin b7f54119f8 [SPARK-9420][SQL] Move expressions in sql/core package to catalyst.
Since catalyst package already depends on Spark core, we can move those expressions
into catalyst, and simplify function registry.

This is a followup of #7478.

Author: Reynold Xin <rxin@databricks.com>

Closes #7735 from rxin/SPARK-8003 and squashes the following commits:

2ffbdc3 [Reynold Xin] [SPARK-8003][SQL] Move expressions in sql/core package to catalyst.
2015-07-28 17:03:59 -07:00
Reynold Xin 9bbe0171cb [SPARK-8196][SQL] Fix null handling & documentation for next_day.
The original patch didn't handle nulls correctly for next_day.

Author: Reynold Xin <rxin@databricks.com>

Closes #7718 from rxin/next_day and squashes the following commits:

616a425 [Reynold Xin] Merged DatetimeExpressionsSuite into DateFunctionsSuite.
faa78cf [Reynold Xin] Merged DatetimeFunctionsSuite into DateExpressionsSuite.
6c4fb6a [Reynold Xin] [SPARK-8196][SQL] Fix null handling & documentation for next_day.
2015-07-28 09:43:39 -07:00
Reynold Xin 15724fac56 [SPARK-9394][SQL] Handle parentheses in CodeFormatter.
Our CodeFormatter currently does not handle parentheses, and as a result in code dump, we see code formatted this way:

```
foo(
a,
b,
c)
```

With this patch, it is formatted this way:
```
foo(
  a,
  b,
  c)
```

Author: Reynold Xin <rxin@databricks.com>

Closes #7712 from rxin/codeformat-parentheses and squashes the following commits:

c2b1c5f [Reynold Xin] Took square bracket out
3cfb174 [Reynold Xin] Code review feedback.
91f5bb1 [Reynold Xin] [SPARK-9394][SQL] Handle parentheses in CodeFormatter.
2015-07-28 00:52:26 -07:00
Reynold Xin 60f08c7c87 [SPARK-9373][SQL] Support StructType in Tungsten projection
This pull request updates GenerateUnsafeProjection to support StructType. If an input struct type is backed already by an UnsafeRow, GenerateUnsafeProjection copies the bytes directly into its buffer space without any conversion. However, if the input is not an UnsafeRow, GenerateUnsafeProjection runs the code generated recursively to convert the input into an UnsafeRow and then copies it into the buffer space.

Also create a TungstenProject operator that projects data directly into UnsafeRow. Note that I'm not sure if this is the way we want to structure Unsafe+codegen operators, but we can defer that decision to follow-up pull requests.

Author: Reynold Xin <rxin@databricks.com>

Closes #7689 from rxin/tungsten-struct-type and squashes the following commits:

9162f42 [Reynold Xin] Support IntervalType in UnsafeRow's getter.
be9f377 [Reynold Xin] Fixed tests.
10c4b7c [Reynold Xin] Format generated code.
77e8d0e [Reynold Xin] Fixed NondeterministicSuite.
ac4951d [Reynold Xin] Yay.
ac203bf [Reynold Xin] More comments.
9f36216 [Reynold Xin] Updated comment.
6b781fe [Reynold Xin] Reset the change in DataFrameSuite.
525b95b [Reynold Xin] Merged with master, more documentation & test cases.
321859a [Reynold Xin] [SPARK-9373][SQL] Support StructType in Tungsten projection [WIP]
2015-07-27 22:51:15 -07:00
Daoyuan Wang 2e7f99a004 [SPARK-8195] [SPARK-8196] [SQL] udf next_day last_day
next_day, returns next certain dayofweek.
last_day, returns the last day of the month which given date belongs to.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #6986 from adrian-wang/udfnlday and squashes the following commits:

ef7e3da [Daoyuan Wang] fix
02b3426 [Daoyuan Wang] address 2 comments
dc69630 [Daoyuan Wang] address comments from rxin
8846086 [Daoyuan Wang] address comments from rxin
d09bcce [Daoyuan Wang] multi fix
1a9de3d [Daoyuan Wang] function next_day and last_day
2015-07-27 21:08:56 -07:00
Wenchen Fan 75438422c2 [SPARK-9369][SQL] Support IntervalType in UnsafeRow
Author: Wenchen Fan <cloud0fan@outlook.com>

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

5b36b17 [Wenchen Fan] fix codegen
a99ed50 [Wenchen Fan] address comment
9e6d319 [Wenchen Fan] Support IntervalType in UnsafeRow
2015-07-27 11:28:22 -07:00
Wenchen Fan dd9ae7945a [SPARK-9351] [SQL] remove literals from grouping expressions in Aggregate
literals in grouping expressions have no effect at all, only make our grouping key bigger, so we should remove them in Optimizer.

I also make old and new aggregation code consistent about literals in grouping here. In old aggregation, actually literals in grouping are already removed but new aggregation is not. So I explicitly make it a rule in Optimizer.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

471adff [Wenchen Fan] add test
0839925 [Wenchen Fan] use transformDown when rewrite final result expressions
2015-07-27 11:23:29 -07:00
Wenchen Fan e2f38167f8 [SPARK-9376] [SQL] use a seed in RandomDataGeneratorSuite
Make this test deterministic, i.e. make sure this test can be passed no matter how many times we run it.

The origin implementation uses a random seed and gives a chance that we may break the null check assertion `assert(Iterator.fill(100)(generator()).contains(null))`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7691 from cloud-fan/seed and squashes the following commits:

eae7281 [Wenchen Fan] use a seed in RandomDataGeneratorSuite
2015-07-27 11:02:16 -07:00
Yijie Shen fb5d43fb25 [SPARK-9356][SQL]Remove the internal use of DecimalType.Unlimited
JIRA: https://issues.apache.org/jira/browse/SPARK-9356

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

Closes #7671 from yjshen/deprecated_unlimit and squashes the following commits:

c707f56 [Yijie Shen] remove pattern matching in changePrecision
4a1823c [Yijie Shen] remove internal occurrence of Decimal.Unlimited
2015-07-26 10:29:22 -07:00
Reynold Xin 4a01bfc2a2 [SPARK-9350][SQL] Introduce an InternalRow generic getter that requires a DataType
Currently UnsafeRow cannot support a generic getter. However, if the data type is known, we can support a generic getter.

Author: Reynold Xin <rxin@databricks.com>

Closes #7666 from rxin/generic-getter-with-datatype and squashes the following commits:

ee2874c [Reynold Xin] Add a default implementation for getStruct.
1e109a0 [Reynold Xin] [SPARK-9350][SQL] Introduce an InternalRow generic getter that requires a DataType.
033ee88 [Reynold Xin] Removed getAs in non test code.
2015-07-25 23:52:37 -07:00
Reynold Xin b1f4b4abfd [SPARK-9348][SQL] Remove apply method on InternalRow.
Author: Reynold Xin <rxin@databricks.com>

Closes #7665 from rxin/remove-row-apply and squashes the following commits:

0b43001 [Reynold Xin] support getString in UnsafeRow.
176d633 [Reynold Xin] apply -> get.
2941324 [Reynold Xin] [SPARK-9348][SQL] Remove apply method on InternalRow.
2015-07-25 18:41:51 -07:00
Wenchen Fan 2c94d0f24a [SPARK-9192][SQL] add initialization phase for nondeterministic expression
Currently nondeterministic expression is broken without a explicit initialization phase.

Let me take `MonotonicallyIncreasingID` as an example. This expression need a mutable state to remember how many times it has been evaluated, so we use `transient var count: Long` there. By being transient, the `count` will be reset to 0 and **only** to 0 when serialize and deserialize it, as deserialize transient variable will result to default value. There is *no way* to use another initial value for `count`, until we add the explicit initialization phase.

Another use case is local execution for `LocalRelation`, there is no serialize and deserialize phase and thus we can't reset mutable states for it.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7535 from cloud-fan/init and squashes the following commits:

6c6f332 [Wenchen Fan] add test
ef68ff4 [Wenchen Fan] fix comments
9eac85e [Wenchen Fan] move init code to interpreted class
bb7d838 [Wenchen Fan] pulls out nondeterministic expressions into a project
b4a4fc7 [Wenchen Fan] revert a refactor
86fee36 [Wenchen Fan] add initialization phase for nondeterministic expression
2015-07-25 12:10:02 -07:00
Reynold Xin 215713e199 [SPARK-9334][SQL] Remove UnsafeRowConverter in favor of UnsafeProjection.
The two are redundant.

Once this patch is merged, I plan to remove the inbound conversions from unsafe aggregates.

Author: Reynold Xin <rxin@databricks.com>

Closes #7658 from rxin/unsafeconverters and squashes the following commits:

ed19e6c [Reynold Xin] Updated support types.
2a56d7e [Reynold Xin] [SPARK-9334][SQL] Remove UnsafeRowConverter in favor of UnsafeProjection.
2015-07-25 01:37:41 -07:00
Reynold Xin c84acd4aa4 [SPARK-9331][SQL] Add a code formatter to auto-format generated code.
The generated expression code can be hard to read since they are not indented well. This patch adds a code formatter that formats code automatically when we output them to the screen.

Author: Reynold Xin <rxin@databricks.com>

Closes #7656 from rxin/codeformatter and squashes the following commits:

5ba0e90 [Reynold Xin] [SPARK-9331][SQL] Add a code formatter to auto-format generated code.
2015-07-24 19:35:24 -07:00
Josh Rosen 6aceaf3d62 [SPARK-9295] Analysis should detect sorting on unsupported column types
This patch extends CheckAnalysis to throw errors for queries that try to sort on unsupported column types, such as ArrayType.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7633 from JoshRosen/SPARK-9295 and squashes the following commits:

23b2fbf [Josh Rosen] Embed function in foreach
bfe1451 [Josh Rosen] Update to allow sorting by null literals
2f1b802 [Josh Rosen] Add analysis rule to detect sorting on unsupported column types (SPARK-9295)
2015-07-24 11:34:23 -07:00
Josh Rosen c2b50d693e [SPARK-9292] Analysis should check that join conditions' data types are BooleanType
This patch adds an analysis check to ensure that join conditions' data types are BooleanType. This check is necessary in order to report proper errors for non-boolean DataFrame join conditions.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7630 from JoshRosen/SPARK-9292 and squashes the following commits:

aec6c7b [Josh Rosen] Check condition type in resolved()
75a3ea6 [Josh Rosen] Fix SPARK-9292.
2015-07-24 09:49:50 -07:00
Reynold Xin c8d71a4183 [SPARK-9305] Rename org.apache.spark.Row to Item.
It's a thing used in test cases, but named Row. Pretty annoying because everytime I search for Row, it shows up before the Spark SQL Row, which is what a developer wants most of the time.

Author: Reynold Xin <rxin@databricks.com>

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

aeda52d [Reynold Xin] [SPARK-9305] Rename org.apache.spark.Row to Item.
2015-07-24 09:38:13 -07:00
Reynold Xin 431ca39be5 [SPARK-9285][SQL] Remove InternalRow's inheritance from Row.
I also changed InternalRow's size/length function to numFields, to make it more obvious that it is not about bytes, but the number of fields.

Author: Reynold Xin <rxin@databricks.com>

Closes #7626 from rxin/internalRow and squashes the following commits:

e124daf [Reynold Xin] Fixed test case.
805ceb7 [Reynold Xin] Commented out the failed test suite.
f8a9ca5 [Reynold Xin] Fixed more bugs. Still at least one more remaining.
76d9081 [Reynold Xin] Fixed data sources.
7807f70 [Reynold Xin] Fixed DataFrameSuite.
cb60cd2 [Reynold Xin] Code review & small bug fixes.
0a2948b [Reynold Xin] Fixed style.
3280d03 [Reynold Xin] [SPARK-9285][SQL] Remove InternalRow's inheritance from Row.
2015-07-24 09:37:36 -07:00
Davies Liu dfb18be036 [SPARK-9069] [SQL] follow up
Address comments for #7605

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7634 from davies/decimal_unlimited2 and squashes the following commits:

b2d8b0d [Davies Liu] add doc and test for DecimalType.isWiderThan
65b251c [Davies Liu] fix test
6a91f32 [Davies Liu] fix style
ca9c973 [Davies Liu] address comments
2015-07-24 08:24:13 -07:00
Reynold Xin cb8c241f05 [SPARK-9200][SQL] Don't implicitly cast non-atomic types to string type.
Author: Reynold Xin <rxin@databricks.com>

Closes #7636 from rxin/complex-string-implicit-cast and squashes the following commits:

3e67327 [Reynold Xin] [SPARK-9200][SQL] Don't implicitly cast non-atomic types to string type.
2015-07-24 01:18:43 -07:00
Davies Liu 8a94eb23d5 [SPARK-9069] [SPARK-9264] [SQL] remove unlimited precision support for DecimalType
Romove Decimal.Unlimited (change to support precision up to 38, to match with Hive and other databases).

In order to keep backward source compatibility, Decimal.Unlimited is still there, but change to Decimal(38, 18).

If no precision and scale is provide, it's Decimal(10, 0) as before.

Author: Davies Liu <davies@databricks.com>

Closes #7605 from davies/decimal_unlimited and squashes the following commits:

aa3f115 [Davies Liu] fix tests and style
fb0d20d [Davies Liu] address comments
bfaae35 [Davies Liu] fix style
df93657 [Davies Liu] address comments and clean up
06727fd [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_unlimited
4c28969 [Davies Liu] fix tests
8d783cc [Davies Liu] fix tests
788631c [Davies Liu] fix double with decimal in Union/except
1779bde [Davies Liu] fix scala style
c9c7c78 [Davies Liu] remove Decimal.Unlimited
2015-07-23 18:31:13 -07:00
Reynold Xin fb36397b3c Revert "[SPARK-8579] [SQL] support arbitrary object in UnsafeRow"
Reverts ObjectPool. As it stands, it has a few problems:

1. ObjectPool doesn't work with spilling and memory accounting.
2. I don't think in the long run the idea of an object pool is what we want to support, since it essentially goes back to unmanaged memory, and creates pressure on GC, and is hard to account for the total in memory size.
3. The ObjectPool patch removed the specialized getters for strings and binary, and as a result, actually introduced branches when reading non primitive data types.

If we do want to support arbitrary user defined types in the future, I think we can just add an object array in UnsafeRow, rather than relying on indirect memory addressing through a pool. We also need to pick execution strategies that are optimized for those, rather than keeping a lot of unserialized JVM objects in memory during aggregation.

This is probably the hardest thing I had to revert in Spark, due to recent patches that also change the same part of the code. Would be great to get a careful look.

Author: Reynold Xin <rxin@databricks.com>

Closes #7591 from rxin/revert-object-pool and squashes the following commits:

01db0bc [Reynold Xin] Scala style.
eda89fc [Reynold Xin] Fixed describe.
2967118 [Reynold Xin] Fixed accessor for JoinedRow.
e3294eb [Reynold Xin] Merge branch 'master' into revert-object-pool
657855f [Reynold Xin] Temp commit.
c20f2c8 [Reynold Xin] Style fix.
fe37079 [Reynold Xin] Revert "[SPARK-8579] [SQL] support arbitrary object in UnsafeRow"
2015-07-23 01:51:34 -07:00
Yijie Shen 6d0d8b4069 [SPARK-8935] [SQL] Implement code generation for all casts
JIRA: https://issues.apache.org/jira/browse/SPARK-8935

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

Closes #7365 from yjshen/cast_codegen and squashes the following commits:

ef6e8b5 [Yijie Shen] getColumn and setColumn in struct cast, autounboxing in array and map
eaece18 [Yijie Shen] remove null case in cast code gen
fd7eba4 [Yijie Shen] resolve comments
80378a5 [Yijie Shen] the missing self cast
611d66e [Yijie Shen] Bug fix: NullType & primitive object unboxing
6d5c0fe [Yijie Shen] rebase and add Interval codegen
9424b65 [Yijie Shen] tiny style fix
4a1c801 [Yijie Shen] remove CodeHolder class, use function instead.
3f5df88 [Yijie Shen] CodeHolder for complex dataTypes
c286f13 [Yijie Shen] moved all the cast code into class body
4edfd76 [Yijie Shen] [WIP] finished primitive part
2015-07-22 23:44:08 -07:00
Yijie Shen 86f80e2b47 [SPARK-9165] [SQL] codegen for CreateArray, CreateStruct and CreateNamedStruct
JIRA: https://issues.apache.org/jira/browse/SPARK-9165

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

Closes #7537 from yjshen/array_struct_codegen and squashes the following commits:

3a6dce6 [Yijie Shen] use infix notion in createArray test
5e90f0a [Yijie Shen] resolve comments: classOf
39cefb8 [Yijie Shen] codegen for createArray createStruct & createNamedStruct
2015-07-22 12:19:59 -07:00
Wenchen Fan 76520955fd [SPARK-9082] [SQL] Filter using non-deterministic expressions should not be pushed down
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7446 from cloud-fan/filter and squashes the following commits:

330021e [Wenchen Fan] add exists to tree node
2cab68c [Wenchen Fan] more enhance
949be07 [Wenchen Fan] push down part of predicate if possible
3912f84 [Wenchen Fan] address comments
8ce15ca [Wenchen Fan] fix bug
557158e [Wenchen Fan] Filter using non-deterministic expressions should not be pushed down
2015-07-22 11:45:51 -07:00
Reynold Xin a4c83cb1e4 [SPARK-9154][SQL] Rename formatString to format_string.
Also make format_string the canonical form, rather than printf.

Author: Reynold Xin <rxin@databricks.com>

Closes #7579 from rxin/format_strings and squashes the following commits:

53ee54f [Reynold Xin] Fixed unit tests.
52357e1 [Reynold Xin] Add format_string alias.
b40a42a [Reynold Xin] [SPARK-9154][SQL] Rename formatString to format_string.
2015-07-21 19:14:07 -07:00
Tarek Auel d4c7a7a364 [SPARK-9154] [SQL] codegen StringFormat
Jira: https://issues.apache.org/jira/browse/SPARK-9154

fixes bug of #7546

marmbrus I can't reopen the other PR, because I didn't closed it. Can you trigger Jenkins?

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

Closes #7571 from tarekauel/SPARK-9154 and squashes the following commits:

dcae272 [Tarek Auel] [SPARK-9154][SQL] build fix
1487602 [Tarek Auel] Merge remote-tracking branch 'upstream/master' into SPARK-9154
f512c5f [Tarek Auel] [SPARK-9154][SQL] build fix
a943d3e [Tarek Auel] [SPARK-9154] implicit input cast, added tests for null, support for null primitives
10b4de8 [Tarek Auel] [SPARK-9154][SQL] codegen removed fallback trait
cd8322b [Tarek Auel] [SPARK-9154][SQL] codegen string format
086caba [Tarek Auel] [SPARK-9154][SQL] codegen string format
2015-07-21 15:47:40 -07:00
Michael Armbrust 87d890cc10 Revert "[SPARK-9154] [SQL] codegen StringFormat"
This reverts commit 7f072c3d5e.

Revert #7546

Author: Michael Armbrust <michael@databricks.com>

Closes #7570 from marmbrus/revert9154 and squashes the following commits:

ed2c32a [Michael Armbrust] Revert "[SPARK-9154] [SQL] codegen StringFormat"
2015-07-21 11:18:39 -07:00
Tarek Auel 7f072c3d5e [SPARK-9154] [SQL] codegen StringFormat
Jira: https://issues.apache.org/jira/browse/SPARK-9154

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

Closes #7546 from tarekauel/SPARK-9154 and squashes the following commits:

a943d3e [Tarek Auel] [SPARK-9154] implicit input cast, added tests for null, support for null primitives
10b4de8 [Tarek Auel] [SPARK-9154][SQL] codegen removed fallback trait
cd8322b [Tarek Auel] [SPARK-9154][SQL] codegen string format
086caba [Tarek Auel] [SPARK-9154][SQL] codegen string format
2015-07-21 09:58:16 -07:00
Yijie Shen be5c5d3741 [SPARK-9081] [SPARK-9168] [SQL] nanvl & dropna/fillna supporting nan as well
JIRA:
https://issues.apache.org/jira/browse/SPARK-9081
https://issues.apache.org/jira/browse/SPARK-9168

This PR target at two modifications:
1.  Change `isNaN` to return `false` on `null` input
2.  Make `dropna` and `fillna` to fill/drop NaN values as well
3.  Implement `nanvl`

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

Closes #7523 from yjshen/fillna_dropna and squashes the following commits:

f0a51db [Yijie Shen] make coalesce untouched and implement nanvl
1d3e35f [Yijie Shen] make Coalesce aware of NaN in order to support fillna
2760cbc [Yijie Shen] change isNaN(null) to false as well as implement dropna
2015-07-21 08:25:50 -07:00
Yijie Shen ae230596b8 [SPARK-9173][SQL]UnionPushDown should also support Intersect and Except
JIRA: https://issues.apache.org/jira/browse/SPARK-9173

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

Closes #7540 from yjshen/union_pushdown and squashes the following commits:

278510a [Yijie Shen] rename UnionPushDown to SetOperationPushDown
91741c1 [Yijie Shen] Add UnionPushDown support for intersect and except
2015-07-21 00:56:57 -07:00
Pedro Rodriguez 560c658a74 [SPARK-8230][SQL] Add array/map size method
Pull Request for: https://issues.apache.org/jira/browse/SPARK-8230

Primary issue resolved is to implement array/map size for Spark SQL. Code is ready for review by a committer. Chen Hao is on the JIRA ticket, but I don't know his username on github, rxin is also on JIRA ticket.

Things to review:
1. Where to put added functions namespace wise, they seem to be part of a few operations on collections which includes `sort_array` and `array_contains`. Hence the name given `collectionOperations.scala` and `_collection_functions` in python.
2. In Python code, should it be in a `1.5.0` function array or in a collections array?
3. Are there any missing methods on the `Size` case class? Looks like many of these functions have generated Java code, is that also needed in this case?
4. Something else?

Author: Pedro Rodriguez <ski.rodriguez@gmail.com>
Author: Pedro Rodriguez <prodriguez@trulia.com>

Closes #7462 from EntilZha/SPARK-8230 and squashes the following commits:

9a442ae [Pedro Rodriguez] fixed functions and sorted __all__
9aea3bb [Pedro Rodriguez] removed imports from python docs
15d4bf1 [Pedro Rodriguez] Added null test case and changed to nullSafeCodeGen
d88247c [Pedro Rodriguez] removed python code
bd5f0e4 [Pedro Rodriguez] removed duplicate function from rebase/merge
59931b4 [Pedro Rodriguez] fixed compile bug instroduced when merging
c187175 [Pedro Rodriguez] updated code to add size to __all__ directly and removed redundent pretty print
130839f [Pedro Rodriguez] fixed failing test
aa9bade [Pedro Rodriguez] fix style
e093473 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests
0449377 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations
9a1a2ff [Pedro Rodriguez] added unit tests for map size
2bfbcb6 [Pedro Rodriguez] added unit test for size
20df2b4 [Pedro Rodriguez] Finished working version of size function and added it to python
b503e75 [Pedro Rodriguez] First attempt at implementing size for maps and arrays
99a6a5c [Pedro Rodriguez] fixed failing test
cac75ac [Pedro Rodriguez] fix style
933d843 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests
42bb7d4 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations
f9c3b8a [Pedro Rodriguez] added unit tests for map size
2515d9f [Pedro Rodriguez] added documentation
0e60541 [Pedro Rodriguez] added unit test for size
acf9853 [Pedro Rodriguez] Finished working version of size function and added it to python
84a5d38 [Pedro Rodriguez] First attempt at implementing size for maps and arrays
2015-07-21 00:53:20 -07:00
Cheng Hao 8c8f0ef59e [SPARK-8255] [SPARK-8256] [SQL] Add regex_extract/regex_replace
Add expressions `regex_extract` & `regex_replace`

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

Closes #7468 from chenghao-intel/regexp and squashes the following commits:

e5ea476 [Cheng Hao] minor update for documentation
ef96fd6 [Cheng Hao] update the code gen
72cf28f [Cheng Hao] Add more log for compilation error
4e11381 [Cheng Hao] Add regexp_replace / regexp_extract support
2015-07-21 00:48:07 -07:00
Josh Rosen c032b0bf92 [SPARK-8797] [SPARK-9146] [SPARK-9145] [SPARK-9147] Support NaN ordering and equality comparisons in Spark SQL
This patch addresses an issue where queries that sorted float or double columns containing NaN values could fail with "Comparison method violates its general contract!" errors from TimSort.  The root of this problem is that `NaN > anything`, `NaN == anything`, and `NaN < anything` all return `false`.

Per the design specified in SPARK-9079, we have decided that `NaN = NaN` should return true and that NaN should appear last when sorting in ascending order (i.e. it is larger than any other numeric value).

In addition to implementing these semantics, this patch also adds canonicalization of NaN values in UnsafeRow, which is necessary in order to be able to do binary equality comparisons on equal NaNs that might have different bit representations (see SPARK-9147).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7194 from JoshRosen/nan and squashes the following commits:

983d4fc [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
88bd73c [Josh Rosen] Fix Row.equals()
a702e2e [Josh Rosen] normalization -> canonicalization
a7267cf [Josh Rosen] Normalize NaNs in UnsafeRow
fe629ae [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
fbb2a29 [Josh Rosen] Fix NaN comparisons in BinaryComparison expressions
c1fd4fe [Josh Rosen] Fold NaN test into existing test framework
b31eb19 [Josh Rosen] Uncomment failing tests
7fe67af [Josh Rosen] Support NaN == NaN (SPARK-9145)
58bad2c [Josh Rosen] Revert "Compare rows' string representations to work around NaN incomparability."
fc6b4d2 [Josh Rosen] Update CodeGenerator
3998ef2 [Josh Rosen] Remove unused code
a2ba2e7 [Josh Rosen] Fix prefix comparision for NaNs
a30d371 [Josh Rosen] Compare rows' string representations to work around NaN incomparability.
6f03f85 [Josh Rosen] Fix bug in Double / Float ordering
42a1ad5 [Josh Rosen] Stop filtering NaNs in UnsafeExternalSortSuite
bfca524 [Josh Rosen] Change ordering so that NaN is maximum value.
8d7be61 [Josh Rosen] Update randomized test to use ScalaTest's assume()
b20837b [Josh Rosen] Add failing test for new NaN comparision ordering
5b88b2b [Josh Rosen] Fix compilation of CodeGenerationSuite
d907b5b [Josh Rosen] Merge remote-tracking branch 'origin/master' into nan
630ebc5 [Josh Rosen] Specify an ordering for NaN values.
9bf195a [Josh Rosen] Re-enable NaNs in CodeGenerationSuite to produce more regression tests
13fc06a [Josh Rosen] Add regression test for NaN sorting issue
f9efbb5 [Josh Rosen] Fix ORDER BY NULL
e7dc4fb [Josh Rosen] Add very generic test for ordering
7d5c13e [Josh Rosen] Add regression test for SPARK-8782 (ORDER BY NULL)
b55875a [Josh Rosen] Generate doubles and floats over entire possible range.
5acdd5c [Josh Rosen] Infinity and NaN are interesting.
ab76cbd [Josh Rosen] Move code to Catalyst package.
d2b4a4a [Josh Rosen] Add random data generator test utilities to Spark SQL.
2015-07-20 22:38:05 -07:00
Tarek Auel a3c7a3ce32 [SPARK-9132][SPARK-9163][SQL] codegen conv
Jira: https://issues.apache.org/jira/browse/SPARK-9132
https://issues.apache.org/jira/browse/SPARK-9163

rxin as you proposed in the Jira ticket, I just moved the logic to a separate object. I haven't changed anything of the logic of `NumberConverter`.

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

Closes #7552 from tarekauel/SPARK-9163 and squashes the following commits:

40dcde9 [Tarek Auel] [SPARK-9132][SPARK-9163][SQL] style fix
fa985bd [Tarek Auel] [SPARK-9132][SPARK-9163][SQL] codegen conv
2015-07-20 22:08:12 -07:00
Tarek Auel c9db8eaa42 [SPARK-9159][SQL] codegen ascii, base64, unbase64
Jira: https://issues.apache.org/jira/browse/SPARK-9159

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

Closes #7542 from tarekauel/SPARK-9159 and squashes the following commits:

772e6bc [Tarek Auel] [SPARK-9159][SQL] codegen ascii, base64, unbase64
2015-07-20 15:32:46 -07:00
Tarek Auel 5112b7f58b [SPARK-9153][SQL] codegen StringLPad/StringRPad
Jira: https://issues.apache.org/jira/browse/SPARK-9153

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

Closes #7527 from tarekauel/SPARK-9153 and squashes the following commits:

3840c6b [Tarek Auel] [SPARK-9153] removed codegen fallback
92b6a5d [Tarek Auel] [SPARK-9153] codegen lpad/rpad
2015-07-20 09:35:45 -07:00
Liang-Chi Hsieh d743bec645 [SPARK-9172][SQL] Make DecimalPrecision support for Intersect and Except
JIRA: https://issues.apache.org/jira/browse/SPARK-9172

Simply make `DecimalPrecision` support for `Intersect` and `Except` in addition to `Union`.

Besides, add unit test for `DecimalPrecision` as well.

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

Closes #7511 from viirya/more_decimalprecieion and squashes the following commits:

4d29d10 [Liang-Chi Hsieh] Fix code comment.
9fb0d49 [Liang-Chi Hsieh] Make DecimalPrecision support for Intersect and Except.
2015-07-19 20:53:18 -07:00
Reynold Xin 163e3f1df9 [SPARK-8241][SQL] string function: concat_ws.
I also changed the semantics of concat w.r.t. null back to the same behavior as Hive.
That is to say, concat now returns null if any input is null.

Author: Reynold Xin <rxin@databricks.com>

Closes #7504 from rxin/concat_ws and squashes the following commits:

83fd950 [Reynold Xin] Fixed type casting.
3ae85f7 [Reynold Xin] Write null better.
cdc7be6 [Reynold Xin] Added code generation for pure string mode.
a61c4e4 [Reynold Xin] Updated comments.
2d51406 [Reynold Xin] [SPARK-8241][SQL] string function: concat_ws.
2015-07-19 16:48:47 -07:00
Cheng Lian 34ed82bb44 [HOTFIX] [SQL] Fixes compilation error introduced by PR #7506
PR #7506 breaks master build because of compilation error. Note that #7506 itself looks good, but it seems that `git merge` did something stupid.

Author: Cheng Lian <lian@databricks.com>

Closes #7510 from liancheng/hotfix-for-pr-7506 and squashes the following commits:

7ea7e89 [Cheng Lian] Fixes compilation error
2015-07-19 18:58:19 +08:00
Reynold Xin 3427937ea2 [SQL] Make date/time functions more consistent with other database systems.
This pull request fixes some of the problems in #6981.

- Added date functions to `__all__` so they get exposed
- Rename day_of_month -> dayofmonth
- Rename day_in_year -> dayofyear
- Rename week_of_year -> weekofyear
- Removed "day" from Scala/Python API since it is ambiguous. Only leaving the alias in SQL.

Author: Reynold Xin <rxin@databricks.com>

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

Closes #7506 from rxin/datetime and squashes the following commits:

0cb24d9 [Reynold Xin] Export all functions in Python.
e44a4a0 [Reynold Xin] Removed day function from Scala and Python.
9c08fdc [Reynold Xin] [SQL] Make date/time functions more consistent with other database systems.
2015-07-19 01:17:22 -07:00
Tarek Auel a53d13f7aa [SPARK-8199][SQL] follow up; revert change in test
rxin / davies

Sorry for that unnecessary change. And thanks again for all your support!

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

Closes #7505 from tarekauel/SPARK-8199-FollowUp and squashes the following commits:

d09321c [Tarek Auel] [SPARK-8199] follow up; revert change in test
c17397f [Tarek Auel] [SPARK-8199] follow up; revert change in test
67acfe6 [Tarek Auel] [SPARK-8199] follow up; revert change in test
2015-07-19 01:16:01 -07:00
Reynold Xin 04c1b49f5e Fixed test cases. 2015-07-18 22:50:34 -07:00
Tarek Auel 83b682beec [SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions
Jira:
https://issues.apache.org/jira/browse/SPARK-8199
https://issues.apache.org/jira/browse/SPARK-8184
https://issues.apache.org/jira/browse/SPARK-8183
https://issues.apache.org/jira/browse/SPARK-8182
https://issues.apache.org/jira/browse/SPARK-8181
https://issues.apache.org/jira/browse/SPARK-8180
https://issues.apache.org/jira/browse/SPARK-8179
https://issues.apache.org/jira/browse/SPARK-8177
https://issues.apache.org/jira/browse/SPARK-8179
https://issues.apache.org/jira/browse/SPARK-9115

Regarding `day`and `dayofmonth` are both necessary?

~~I am going to add `Quarter` to this PR as well.~~ Done.

~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done

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

Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits:

f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests
bb567b6 [Tarek Auel] [SPARK-8199] fixed test
3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix
256c357 [Tarek Auel] [SPARK-8199] code cleanup
5983dcc [Tarek Auel] [SPARK-8199] whitespace fix
6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488
4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling
ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master
70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199
3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search
fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix
cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix
746b80a [Tarek Auel] [SPARK-8199] build fix
0ad6db8 [Tarek Auel] [SPARK-8199] minor fix
523542d [Tarek Auel] [SPARK-8199] address comments
2259299 [Tarek Auel] [SPARK-8199] day_of_month alias
d01b977 [Tarek Auel] [SPARK-8199] python underscore
56c4a92 [Tarek Auel] [SPARK-8199] update python docu
e223bc0 [Tarek Auel] [SPARK-8199] refactoring
d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility
b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it
1b2e540 [Tarek Auel] [SPARK-8119] style fix
0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts
ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring
1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199
740af0e [Tarek Auel] implement date function using a calculation based on days
4fb66da [Tarek Auel] WIP: date functions on calculation only
1a436c9 [Tarek Auel] wip
f775f39 [Tarek Auel] fixed return type
ad17e96 [Tarek Auel] improved implementation
c42b444 [Tarek Auel] Removed merge conflict file
ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues
10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast
7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue
f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite
6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval
d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track
7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199
5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged
eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199
f120415 [Tarek Auel] improved runtime
a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat
5fe74e1 [Tarek Auel] fixed python style
3bfac90 [Tarek Auel] fixed style
356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation
02efc5d [Tarek Auel] removed doubled code
a5ea120 [Tarek Auel] added python api; changed test to be more meaningful
b680db6 [Tarek Auel] added codegeneration to all functions
c739788 [Tarek Auel] added support for quarter SPARK-8178
849fb41 [Tarek Auel] fixed stupid test
638596f [Tarek Auel] improved codegen
4d8049b [Tarek Auel] fixed tests and added type check
5ebb235 [Tarek Auel] resolved naming conflict
d0e2f99 [Tarek Auel] date functions
2015-07-18 22:48:05 -07:00
Forest Fang 6cb6096c01 [SPARK-8443][SQL] Split GenerateMutableProjection Codegen due to JVM Code Size Limits
By grouping projection calls into multiple apply function, we are able to push the number of projections codegen can handle from ~1k to ~60k. I have set the unit test to test against 5k as 60k took 15s for the unit test to complete.

Author: Forest Fang <forest.fang@outlook.com>

Closes #7076 from saurfang/codegen_size_limit and squashes the following commits:

b7a7635 [Forest Fang] [SPARK-8443][SQL] Execute and verify split projections in test
adef95a [Forest Fang] [SPARK-8443][SQL] Use safer factor and rewrite splitting code
1b5aa7e [Forest Fang] [SPARK-8443][SQL] inline execution if one block only
9405680 [Forest Fang] [SPARK-8443][SQL] split projection code by size limit
2015-07-18 21:05:44 -07:00
Reynold Xin 9914b1b2c5 [SPARK-9150][SQL] Create CodegenFallback and Unevaluable trait
It is very hard to track which expressions have code gen implemented or not. This patch removes the default fallback gencode implementation from Expression, and moves that into a new trait called CodegenFallback. Each concrete expression needs to either implement code generation, or mix in CodegenFallback. This makes it very easy to track which expressions have code generation implemented already.

Additionally, this patch creates an Unevaluable trait that can be used to track expressions that don't support evaluation (e.g. Star).

Author: Reynold Xin <rxin@databricks.com>

Closes #7487 from rxin/codegenfallback and squashes the following commits:

14ebf38 [Reynold Xin] Fixed Conv
6c1c882 [Reynold Xin] Fixed Alias.
b42611b [Reynold Xin] [SPARK-9150][SQL] Create a trait to track code generation for expressions.
cb5c066 [Reynold Xin] Removed extra import.
39cbe40 [Reynold Xin] [SPARK-8240][SQL] string function: concat
2015-07-18 18:18:19 -07:00
Reynold Xin 6e1e2eba69 [SPARK-8240][SQL] string function: concat
Author: Reynold Xin <rxin@databricks.com>

Closes #7486 from rxin/concat and squashes the following commits:

5217d6e [Reynold Xin] Removed Hive's concat test.
f5cb7a3 [Reynold Xin] Concat is never nullable.
ae4e61f [Reynold Xin] Removed extra import.
fddcbbd [Reynold Xin] Fixed NPE.
22e831c [Reynold Xin] Added missing file.
57a2352 [Reynold Xin] [SPARK-8240][SQL] string function: concat
2015-07-18 14:07:56 -07:00
Yijie Shen 3d2134fc0d [SPARK-9055][SQL] WidenTypes should also support Intersect and Except
JIRA: https://issues.apache.org/jira/browse/SPARK-9055

cc rxin

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

Closes #7491 from yijieshen/widen and squashes the following commits:

079fa52 [Yijie Shen] widenType support for intersect and expect
2015-07-18 12:57:53 -07:00
Wenchen Fan 86c50bf72c [SPARK-9171][SQL] add and improve tests for nondeterministic expressions
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7496 from cloud-fan/tests and squashes the following commits:

0958f90 [Wenchen Fan] improve test for nondeterministic expressions
2015-07-18 11:58:53 -07:00
Wenchen Fan 692378c01d [SPARK-9167][SQL] use UTC Calendar in stringToDate
fix 2 bugs introduced in https://github.com/apache/spark/pull/7353

1. we should use UTC Calendar when cast string to date . Before #7353 , we use `DateTimeUtils.fromJavaDate(Date.valueOf(s.toString))` to cast string to date, and `fromJavaDate` will call `millisToDays` to avoid the time zone issue. Now we use `DateTimeUtils.stringToDate(s)`, we should create a Calendar with UTC in the begging.
2. we should not change the default time zone in test cases. The `threadLocalLocalTimeZone` and `threadLocalTimestampFormat` in `DateTimeUtils` will only be evaluated once for each thread, so we can't set the default time zone back anymore.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

9cd6005 [Wenchen Fan] address comments
21ef293 [Wenchen Fan] fix 2 bugs in datetime
2015-07-18 11:25:16 -07:00
Reynold Xin fba3f5ba85 [SPARK-9169][SQL] Improve unit test coverage for null expressions.
Author: Reynold Xin <rxin@databricks.com>

Closes #7490 from rxin/unit-test-null-funcs and squashes the following commits:

7b276f0 [Reynold Xin] Move isNaN.
8307287 [Reynold Xin] [SPARK-9169][SQL] Improve unit test coverage for null expressions.
2015-07-18 11:06:46 -07:00
Yijie Shen 529a2c2d92 [SPARK-8280][SPARK-8281][SQL]Handle NaN, null and Infinity in math
JIRA:
https://issues.apache.org/jira/browse/SPARK-8280
https://issues.apache.org/jira/browse/SPARK-8281

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

Closes #7451 from yijieshen/nan_null2 and squashes the following commits:

47a529d [Yijie Shen] style fix
63dee44 [Yijie Shen] handle log expressions similar to Hive
188be51 [Yijie Shen] null to nan in Math Expression
2015-07-17 17:33:19 -07:00
Wenchen Fan bd903ee89f [SPARK-9117] [SQL] fix BooleanSimplification in case-insensitive
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7452 from cloud-fan/boolean-simplify and squashes the following commits:

2a6e692 [Wenchen Fan] fix style
d3cfd26 [Wenchen Fan] fix BooleanSimplification in case-insensitive
2015-07-17 16:28:24 -07:00
Wenchen Fan fd6b3101fb [SPARK-9113] [SQL] enable analysis check code for self join
The check was unreachable before, as `case operator: LogicalPlan` catches everything already.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

2bb6637 [Wenchen Fan] add test
5493aea [Wenchen Fan] add the check back
27221a7 [Wenchen Fan] remove unnecessary analysis check code for self join
2015-07-17 16:03:33 -07:00
Yijie Shen 15fc2ffe55 [SPARK-9080][SQL] add isNaN predicate expression
JIRA: https://issues.apache.org/jira/browse/SPARK-9080

cc rxin

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

Closes #7464 from yijieshen/isNaN and squashes the following commits:

11ae039 [Yijie Shen] add isNaN in functions
666718e [Yijie Shen] add isNaN predicate expression
2015-07-17 15:49:31 -07:00
Wenchen Fan 074085d678 [SPARK-9136] [SQL] fix several bugs in DateTimeUtils.stringToTimestamp
a follow up of https://github.com/apache/spark/pull/7353

1. we should use `Calendar.HOUR_OF_DAY` instead of `Calendar.HOUR`(this is for AM, PM).
2. we should call `c.set(Calendar.MILLISECOND, 0)` after `Calendar.getInstance`

I'm not sure why the tests didn't fail in jenkins, but I ran latest spark master branch locally and `DateTimeUtilsSuite` failed.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

66cdaf2 [Wenchen Fan] fix several bugs in DateTimeUtils.stringToTimestamp
2015-07-17 13:57:31 -07:00
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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 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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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 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
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
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
Davies Liu 1191c3efc6 [SPARK-8305] [SPARK-8190] [SQL] improve codegen
This PR fix a few small issues about codgen:

1. cast decimal to boolean
2. do not inline literal with null
3. improve SpecificRow.equals()
4. test expressions with optimized express
5. fix compare with BinaryType

cc rxin chenghao-intel

Author: Davies Liu <davies@databricks.com>

Closes #6755 from davies/fix_codegen and squashes the following commits:

ef27343 [Davies Liu] address comments
6617ea6 [Davies Liu] fix scala tyle
70b7dda [Davies Liu] improve codegen
2015-06-11 12:57:33 -07:00
Daoyuan Wang 2758ff0a96 [SPARK-8217] [SQL] math function log2
Author: Daoyuan Wang <daoyuan.wang@intel.com>

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

Closes #6718 from adrian-wang/udflog2 and squashes the following commits:

3909f48 [Daoyuan Wang] math function: log2
2015-06-10 20:22:32 -07:00
Cheng Hao 9fe3adccef [SPARK-8248][SQL] string function: length
Author: Cheng Hao <hao.cheng@intel.com>

Closes #6724 from chenghao-intel/length and squashes the following commits:

aaa3c31 [Cheng Hao] revert the additional change
97148a9 [Cheng Hao] remove the codegen testing temporally
ae08003 [Cheng Hao] update the comments
1eb1fd1 [Cheng Hao] simplify the code as commented
3e92d32 [Cheng Hao] use the selectExpr in unit test intead of SQLQuery
3c729aa [Cheng Hao] fix bug for constant null value in codegen
3641f06 [Cheng Hao] keep the length() method for registered function
8e30171 [Cheng Hao] update the code as comment
db604ae [Cheng Hao] Add code gen support
548d2ef [Cheng Hao] register the length()
09a0738 [Cheng Hao] add length support
2015-06-10 19:55:10 -07:00
Wenchen Fan 4e42842e82 [SPARK-8164] transformExpressions should support nested expression sequence
Currently we only support `Seq[Expression]`, we should handle cases like `Seq[Seq[Expression]]` so that we can remove the unnecessary `GroupExpression`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6706 from cloud-fan/clean and squashes the following commits:

60a1193 [Wenchen Fan] support nested expression sequence and remove GroupExpression
2015-06-10 18:22:47 -07:00
Davies Liu 37719e0cd0 [SPARK-8189] [SQL] use Long for TimestampType in SQL
This PR change to use Long as internal type for TimestampType for efficiency, which means it will the precision below 100ns.

Author: Davies Liu <davies@databricks.com>

Closes #6733 from davies/timestamp and squashes the following commits:

d9565fa [Davies Liu] remove print
65cf2f1 [Davies Liu] fix Timestamp in SparkR
86fecfb [Davies Liu] disable two timestamp tests
8f77ee0 [Davies Liu] fix scala style
246ee74 [Davies Liu] address comments
309d2e1 [Davies Liu] use Long for TimestampType in SQL
2015-06-10 16:55:39 -07:00
Daoyuan Wang c6ba7cca33 [SPARK-8215] [SPARK-8212] [SQL] add leaf math expression for e and pi
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #6716 from adrian-wang/epi and squashes the following commits:

e2e8dbd [Daoyuan Wang] move tests
11b351c [Daoyuan Wang] add tests and remove pu
db331c9 [Daoyuan Wang] py style
599ddd8 [Daoyuan Wang] add py
e6783ef [Daoyuan Wang] register function
82d426e [Daoyuan Wang] add function entry
dbf3ab5 [Daoyuan Wang] add PI and E
2015-06-10 09:45:45 -07:00
Reynold Xin f74be744d4 [SPARK-8149][SQL] Break ExpressionEvaluationSuite down to multiple files
Also moved a few files in expressions package around to match test suites.

Author: Reynold Xin <rxin@databricks.com>

Closes #6693 from rxin/expr-refactoring and squashes the following commits:

857599f [Reynold Xin] Fixed style violation.
c0eb74b [Reynold Xin] Fixed compilation.
b3a40f8 [Reynold Xin] Refactored expression test suites.
2015-06-07 18:45:24 -07:00
Davies Liu 5e7b6b67be [SPARK-8117] [SQL] Push codegen implementation into each Expression
This PR move codegen implementation of expressions into Expression class itself, make it easy to manage.

It introduces two APIs in Expression:
```
def gen(ctx: CodeGenContext): GeneratedExpressionCode
def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code
```

gen(ctx) will call genSource(ctx, ev) to generate Java source code for the current expression. A expression needs to override genSource().

Here are the types:
```
type Term String
type Code String

/**
 * Java source for evaluating an [[Expression]] given a [[Row]] of input.
 */
case class GeneratedExpressionCode(var code: Code,
                               nullTerm: Term,
                               primitiveTerm: Term,
                               objectTerm: Term)
/**
 * A context for codegen, which is used to bookkeeping the expressions those are not supported
 * by codegen, then they are evaluated directly. The unsupported expression is appended at the
 * end of `references`, the position of it is kept in the code, used to access and evaluate it.
 */
class CodeGenContext {
  /**
   * Holding all the expressions those do not support codegen, will be evaluated directly.
   */
  val references: Seq[Expression] = new mutable.ArrayBuffer[Expression]()
}
```

This is basically #6660, but fixed style violation and compilation failure.

Author: Davies Liu <davies@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #6690 from rxin/codegen and squashes the following commits:

e1368c2 [Reynold Xin] Fixed tests.
73db80e [Reynold Xin] Fixed compilation failure.
19d6435 [Reynold Xin] Fixed style violation.
9adaeaf [Davies Liu] address comments
f42c732 [Davies Liu] improve coverage and tests
bad6828 [Davies Liu] address comments
e03edaa [Davies Liu] consts fold
86fac2c [Davies Liu] fix style
02262c9 [Davies Liu] address comments
b5d3617 [Davies Liu] Merge pull request #5 from rxin/codegen
48c454f [Reynold Xin] Some code gen update.
2344bc0 [Davies Liu] fix test
12ff88a [Davies Liu] fix build
c5fb514 [Davies Liu] rename
8c6d82d [Davies Liu] update docs
b145047 [Davies Liu] fix style
e57959d [Davies Liu] add type alias
3ff25f8 [Davies Liu] refactor
593d617 [Davies Liu] pushing codegen into Expression
2015-06-07 14:11:20 -07:00
Wenchen Fan db81b9d89f [SPARK-7952][SQL] use internal Decimal instead of java.math.BigDecimal
This PR fixes a bug introduced in https://github.com/apache/spark/pull/6505.
Decimal literal's value is not `java.math.BigDecimal`, but Spark SQL internal type: `Decimal`.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

b0e3549 [Wenchen Fan] rename to BooleanEquality
1987b37 [Wenchen Fan] use Decimal instead of java.math.BigDecimal
f93c420 [Wenchen Fan] compare literal
2015-06-07 11:07:19 -07:00
Reynold Xin 2bcdf8c239 [SPARK-7440][SQL] Remove physical Distinct operator in favor of Aggregate
This patch replaces Distinct with Aggregate in the optimizer, so Distinct will become
more efficient over time as we optimize Aggregate (via Tungsten).

Author: Reynold Xin <rxin@databricks.com>

Closes #6637 from rxin/replace-distinct and squashes the following commits:

b3cc50e [Reynold Xin] Mima excludes.
93d6117 [Reynold Xin] Code review feedback.
87e4741 [Reynold Xin] [SPARK-7440][SQL] Remove physical Distinct operator in favor of Aggregate.
2015-06-04 13:52:53 -07:00
Davies Liu c8709dcfd1 [SPARK-7956] [SQL] Use Janino to compile SQL expressions into bytecode
In order to reduce the overhead of codegen, this PR switch to use Janino to compile SQL expressions into bytecode.

After this, the time used to compile a SQL expression is decreased from 100ms to 5ms, which is necessary to turn on codegen for general workload, also tests.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #6479 from davies/janino and squashes the following commits:

cc689f5 [Davies Liu] remove globalLock
262d848 [Davies Liu] Merge branch 'master' of github.com:apache/spark into janino
eec3a33 [Davies Liu] address comments from Josh
f37c8c3 [Davies Liu] fix DecimalType and cast to String
202298b [Davies Liu] Merge branch 'master' of github.com:apache/spark into janino
a21e968 [Davies Liu] fix style
0ed3dc6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into janino
551a851 [Davies Liu] fix tests
c3bdffa [Davies Liu] remove print
6089ce5 [Davies Liu] change logging level
7e46ac3 [Davies Liu] fix style
d8f0f6c [Davies Liu] Merge branch 'master' of github.com:apache/spark into janino
da4926a [Davies Liu] fix tests
03660f3 [Davies Liu] WIP: use Janino to compile Java source
f2629cd [Davies Liu] Merge branch 'master' of github.com:apache/spark into janino
f7d66cf [Davies Liu] use template based string for codegen
2015-06-04 10:28:59 -07:00
Wenchen Fan d38cf217e0 [SPARK-7562][SPARK-6444][SQL] Improve error reporting for expression data type mismatch
It seems hard to find a common pattern of checking types in `Expression`. Sometimes we know what input types we need(like `And`, we know we need two booleans), sometimes we just have some rules(like `Add`, we need 2 numeric types which are equal). So I defined a general interface `checkInputDataTypes` in `Expression` which returns a `TypeCheckResult`. `TypeCheckResult` can tell whether this expression passes the type checking or what the type mismatch is.

This PR mainly works on apply input types checking for arithmetic and predicate expressions.

TODO: apply type checking interface to more expressions.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6405 from cloud-fan/6444 and squashes the following commits:

b5ff31b [Wenchen Fan] address comments
b917275 [Wenchen Fan] rebase
39929d9 [Wenchen Fan] add todo
0808fd2 [Wenchen Fan] make constrcutor of TypeCheckResult private
3bee157 [Wenchen Fan] and decimal type coercion rule for binary comparison
8883025 [Wenchen Fan] apply type check interface to CaseWhen
cffb67c [Wenchen Fan] to have resolved call the data type check function
6eaadff [Wenchen Fan] add equal type constraint to EqualTo
3affbd8 [Wenchen Fan] more fixes
654d46a [Wenchen Fan] improve tests
e0a3628 [Wenchen Fan] improve error message
1524ff6 [Wenchen Fan] fix style
69ca3fe [Wenchen Fan] add error message and tests
c71d02c [Wenchen Fan] fix hive tests
6491721 [Wenchen Fan] use value class TypeCheckResult
7ae76b9 [Wenchen Fan] address comments
cb77e4f [Wenchen Fan] Improve error reporting for expression data type mismatch
2015-06-03 00:47:52 -07:00
Josh Rosen cafd5056e1 [SPARK-7691] [SQL] Refactor CatalystTypeConverter to use type-specific row accessors
This patch significantly refactors CatalystTypeConverters to both clean up the code and enable these conversions to work with future Project Tungsten features.

At a high level, I've reorganized the code so that all functions dealing with the same type are grouped together into type-specific subclasses of `CatalystTypeConveter`.  In addition, I've added new methods that allow the Catalyst Row -> Scala Row conversions to access the Catalyst row's fields through type-specific `getTYPE()` methods rather than the generic `get()` / `Row.apply` methods.  This refactoring is a blocker to being able to unit test new operators that I'm developing as part of Project Tungsten, since those operators may output `UnsafeRow` instances which don't support the generic `get()`.

The stricter type usage of types here has uncovered some bugs in other parts of Spark SQL:

- #6217: DescribeCommand is assigned wrong output attributes in SparkStrategies
- #6218: DataFrame.describe() should cast all aggregates to String
- #6400: Use output schema, not relation schema, for data source input conversion

Spark SQL current has undefined behavior for what happens when you try to create a DataFrame from user-specified rows whose values don't match the declared schema.  According to the `createDataFrame()` Scaladoc:

>  It is important to make sure that the structure of every [[Row]] of the provided RDD matches the provided schema. Otherwise, there will be runtime exception.

Given this, it sounds like it's technically not a break of our API contract to fail-fast when the data types don't match. However, there appear to be many cases where we don't fail even though the types don't match. For example, `JavaHashingTFSuite.hasingTF` passes a column of integers values for a "label" column which is supposed to contain floats.  This column isn't actually read or modified as part of query processing, so its actual concrete type doesn't seem to matter. In other cases, there could be situations where we have generic numeric aggregates that tolerate being called with different numeric types than the schema specified, but this can be okay due to numeric conversions.

In the long run, we will probably want to come up with precise semantics for implicit type conversions / widening when converting Java / Scala rows to Catalyst rows.  Until then, though, I think that failing fast with a ClassCastException is a reasonable behavior; this is the approach taken in this patch.  Note that certain optimizations in the inbound conversion functions for primitive types mean that we'll probably preserve the old undefined behavior in a majority of cases.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6222 from JoshRosen/catalyst-converters-refactoring and squashes the following commits:

740341b [Josh Rosen] Optimize method dispatch for primitive type conversions
befc613 [Josh Rosen] Add tests to document Option-handling behavior.
5989593 [Josh Rosen] Use new SparkFunSuite base in CatalystTypeConvertersSuite
6edf7f8 [Josh Rosen] Re-add convertToScala(), since a Hive test still needs it
3f7b2d8 [Josh Rosen] Initialize converters lazily so that the attributes are resolved first
6ad0ebb [Josh Rosen] Fix JavaHashingTFSuite ClassCastException
677ff27 [Josh Rosen] Fix null handling bug; add tests.
8033d4c [Josh Rosen] Fix serialization error in UserDefinedGenerator.
85bba9d [Josh Rosen] Fix wrong input data in InMemoryColumnarQuerySuite
9c0e4e1 [Josh Rosen] Remove last use of convertToScala().
ae3278d [Josh Rosen] Throw ClassCastException errors during inbound conversions.
7ca7fcb [Josh Rosen] Comments and cleanup
1e87a45 [Josh Rosen] WIP refactoring of CatalystTypeConverters
2015-06-02 22:11:03 -07:00
Yin Huai 0f80990bfa [SPARK-8023][SQL] Add "deterministic" attribute to Expression to avoid collapsing nondeterministic projects.
This closes #6570.

Author: Yin Huai <yhuai@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #6573 from rxin/deterministic and squashes the following commits:

356cd22 [Reynold Xin] Added unit test for the optimizer.
da3fde1 [Reynold Xin] Merge pull request #6570 from yhuai/SPARK-8023
da56200 [Yin Huai] Comments.
e38f264 [Yin Huai] Comment.
f9d6a73 [Yin Huai] Add a deterministic method to Expression.
2015-06-02 00:20:52 -07:00
Wenchen Fan a0e46a0d2a [SPARK-7952][SPARK-7984][SQL] equality check between boolean type and numeric type is broken.
The origin code has several problems:
* `true <=> 1` will return false as we didn't set a rule to handle it.
* `true = a` where `a` is not `Literal` and its value is 1, will return false as we only handle literal values.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6505 from cloud-fan/tmp1 and squashes the following commits:

77f0f39 [Wenchen Fan] minor fix
b6401ba [Wenchen Fan] add type coercion for CaseKeyWhen and address comments
ebc8c61 [Wenchen Fan] use SQLTestUtils and If
625973c [Wenchen Fan] improve
9ba2130 [Wenchen Fan] address comments
fc0d741 [Wenchen Fan] fix style
2846a04 [Wenchen Fan] fix 7952
2015-05-31 21:01:46 -07:00
Reynold Xin 63a50be13d [SPARK-3850] Trim trailing spaces for SQL.
Author: Reynold Xin <rxin@databricks.com>

Closes #6535 from rxin/whitespace-sql and squashes the following commits:

de50316 [Reynold Xin] [SPARK-3850] Trim trailing spaces for SQL.
2015-05-31 00:48:49 -07:00
Wenchen Fan 0978aec9cd [SPARK-7964][SQL] remove unnecessary type coercion rule
We have defined these logics in `Cast` already, I think we should remove this rule.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6516 from cloud-fan/tmp2 and squashes the following commits:

d5035a4 [Wenchen Fan] remove useless rule
2015-05-30 00:26:46 -07:00
Andrew Or 9eb222c139 [SPARK-7558] Demarcate tests in unit-tests.log
Right now `unit-tests.log` are not of much value because we can't tell where the test boundaries are easily. This patch adds log statements before and after each test to outline the test boundaries, e.g.:

```
===== TEST OUTPUT FOR o.a.s.serializer.KryoSerializerSuite: 'kryo with parallelize for primitive arrays' =====

15/05/27 12:36:39.596 pool-1-thread-1-ScalaTest-running-KryoSerializerSuite INFO SparkContext: Starting job: count at KryoSerializerSuite.scala:230
15/05/27 12:36:39.596 dag-scheduler-event-loop INFO DAGScheduler: Got job 3 (count at KryoSerializerSuite.scala:230) with 4 output partitions (allowLocal=false)
15/05/27 12:36:39.596 dag-scheduler-event-loop INFO DAGScheduler: Final stage: ResultStage 3(count at KryoSerializerSuite.scala:230)
15/05/27 12:36:39.596 dag-scheduler-event-loop INFO DAGScheduler: Parents of final stage: List()
15/05/27 12:36:39.597 dag-scheduler-event-loop INFO DAGScheduler: Missing parents: List()
15/05/27 12:36:39.597 dag-scheduler-event-loop INFO DAGScheduler: Submitting ResultStage 3 (ParallelCollectionRDD[5] at parallelize at KryoSerializerSuite.scala:230), which has no missing parents

...

15/05/27 12:36:39.624 pool-1-thread-1-ScalaTest-running-KryoSerializerSuite INFO DAGScheduler: Job 3 finished: count at KryoSerializerSuite.scala:230, took 0.028563 s
15/05/27 12:36:39.625 pool-1-thread-1-ScalaTest-running-KryoSerializerSuite INFO KryoSerializerSuite:

***** FINISHED o.a.s.serializer.KryoSerializerSuite: 'kryo with parallelize for primitive arrays' *****

...
```

Author: Andrew Or <andrew@databricks.com>

Closes #6441 from andrewor14/demarcate-tests and squashes the following commits:

879b060 [Andrew Or] Fix compile after rebase
d622af7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into demarcate-tests
017c8ba [Andrew Or] Merge branch 'master' of github.com:apache/spark into demarcate-tests
7790b6c [Andrew Or] Fix tests after logical merge conflict
c7460c0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into demarcate-tests
c43ffc4 [Andrew Or] Fix tests?
8882581 [Andrew Or] Fix tests
ee22cda [Andrew Or] Fix log message
fa9450e [Andrew Or] Merge branch 'master' of github.com:apache/spark into demarcate-tests
12d1e1b [Andrew Or] Various whitespace changes (minor)
69cbb24 [Andrew Or] Make all test suites extend SparkFunSuite instead of FunSuite
bbce12e [Andrew Or] Fix manual things that cannot be covered through automation
da0b12f [Andrew Or] Add core tests as dependencies in all modules
f7d29ce [Andrew Or] Introduce base abstract class for all test suites
2015-05-29 14:03:12 -07:00
Reynold Xin 8da560d7de [SPARK-7927] whitespace fixes for Catalyst module.
So we can enable a whitespace enforcement rule in the style checker to save code review time.

Author: Reynold Xin <rxin@databricks.com>

Closes #6476 from rxin/whitespace-catalyst and squashes the following commits:

650409d [Reynold Xin] Fixed tests.
51a9e5d [Reynold Xin] [SPARK-7927] whitespace fixes for Catalyst module.
2015-05-28 20:11:57 -07:00
Reynold Xin 9f48bf6b37 [SPARK-7887][SQL] Remove EvaluatedType from SQL Expression.
This type is not really used. Might as well remove it.

Author: Reynold Xin <rxin@databricks.com>

Closes #6427 from rxin/evalutedType and squashes the following commits:

51a319a [Reynold Xin] [SPARK-7887][SQL] Remove EvaluatedType from SQL Expression.
2015-05-27 01:12:59 -07:00
rowan 03668348e2 [SPARK-7637] [SQL] O(N) merge implementation for StructType merge
Contribution is my original work and I license the work to the project under the projects open source license.

Author: rowan <rowan.chattaway@googlemail.com>

Closes #6259 from rowan000/SPARK-7637 and squashes the following commits:

c479df4 [rowan] SPARK-7637: rename mapFields to fieldsMap as per comments on github.
8d2e419 [rowan] SPARK-7637: fix up whitespace changes
0e9d662 [rowan] SPARK-7637: O(N) merge implementatio for StructType merge
2015-05-26 18:17:16 -07:00
Michael Armbrust 3c1305107a [SPARK-7834] [SQL] Better window error messages
Author: Michael Armbrust <michael@databricks.com>

Closes #6363 from marmbrus/windowErrors and squashes the following commits:

516b02d [Michael Armbrust] [SPARK-7834] [SQL] Better window error messages
2015-05-22 17:23:12 -07:00
Michael Armbrust 6d0633e3ec [SPARK-7548] [SQL] Add explode function for DataFrames
Add an `explode` function for dataframes and modify the analyzer so that single table generating functions can be present in a select clause along with other expressions.   There are currently the following restrictions:
 - only top level TGFs are allowed (i.e. no `select(explode('list) + 1)`)
 - only one may be present in a single select to avoid potentially confusing implicit Cartesian products.

TODO:
 - [ ] Python

Author: Michael Armbrust <michael@databricks.com>

Closes #6107 from marmbrus/explodeFunction and squashes the following commits:

7ee2c87 [Michael Armbrust] whitespace
6f80ba3 [Michael Armbrust] Update dataframe.py
c176c89 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explodeFunction
81b5da3 [Michael Armbrust] style
d3faa05 [Michael Armbrust] fix self join case
f9e1e3e [Michael Armbrust] fix python, add since
4f0d0a9 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explodeFunction
e710fe4 [Michael Armbrust] add java and python
52ca0dc [Michael Armbrust] [SPARK-7548][SQL] Add explode function for dataframes.
2015-05-14 19:49:44 -07:00
Reynold Xin e683182c3e [SQL] Move some classes into packages that are more appropriate.
JavaTypeInference into catalyst
types.DateUtils into catalyst
CacheManager into execution
DefaultParserDialect into catalyst

Author: Reynold Xin <rxin@databricks.com>

Closes #6108 from rxin/sql-rename and squashes the following commits:

3fc9613 [Reynold Xin] Fixed import ordering.
83d9ff4 [Reynold Xin] Fixed codegen tests.
e271e86 [Reynold Xin] mima
f4e24a6 [Reynold Xin] [SQL] Move some classes into packages that are more appropriate.
2015-05-13 16:15:31 -07:00
scwf 59250fe514 [SPARK-7303] [SQL] push down project if possible when the child is sort
Optimize the case of `project(_, sort)` , a example is:

`select key from (select * from testData order by key) t`

before this PR:
```
== Parsed Logical Plan ==
'Project ['key]
 'Subquery t
  'Sort ['key ASC], true
   'Project [*]
    'UnresolvedRelation [testData], None

== Analyzed Logical Plan ==
Project [key#0]
 Subquery t
  Sort [key#0 ASC], true
   Project [key#0,value#1]
    Subquery testData
     LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Optimized Logical Plan ==
Project [key#0]
 Sort [key#0 ASC], true
  LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Physical Plan ==
Project [key#0]
 Sort [key#0 ASC], true
  Exchange (RangePartitioning [key#0 ASC], 5), []
   PhysicalRDD [key#0,value#1], MapPartitionsRDD[1]
```

after this PR
```
== Parsed Logical Plan ==
'Project ['key]
 'Subquery t
  'Sort ['key ASC], true
   'Project [*]
    'UnresolvedRelation [testData], None

== Analyzed Logical Plan ==
Project [key#0]
 Subquery t
  Sort [key#0 ASC], true
   Project [key#0,value#1]
    Subquery testData
     LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Optimized Logical Plan ==
Sort [key#0 ASC], true
 Project [key#0]
  LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Physical Plan ==
Sort [key#0 ASC], true
 Exchange (RangePartitioning [key#0 ASC], 5), []
  Project [key#0]
   PhysicalRDD [key#0,value#1], MapPartitionsRDD[1]
```

with this rule we will first do column pruning on the table and then do sorting.

Author: scwf <wangfei1@huawei.com>

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

Closes #5838 from scwf/pruning and squashes the following commits:

b00d833 [scwf] address michael's comment
e230155 [scwf] fix tests failure
b09b895 [scwf] improve column pruning
2015-05-13 16:13:48 -07:00
Wenchen Fan 4e290522c2 [SPARK-7276] [DATAFRAME] speed up DataFrame.select by collapsing Project
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5831 from cloud-fan/7276 and squashes the following commits:

ee4a1e1 [Wenchen Fan] fix rebase mistake
a3b565d [Wenchen Fan] refactor
99deb5d [Wenchen Fan] add test
f1f67ad [Wenchen Fan] fix 7276
2015-05-12 11:51:55 -07:00
Zhongshuai Pei 4b5e1fe94c [SPARK-7437] [SQL] Fold "literal in (item1, item2, ..., literal, ...)" into true or false directly
SQL
```
select key from src where 3 in (4, 5);
```
Before
```
== Optimized Logical Plan ==
Project [key#12]
 Filter 3 INSET (5,4)
  MetastoreRelation default, src, None
```

After
```
== Optimized Logical Plan ==
LocalRelation [key#228], []
```

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

Closes #5972 from DoingDone9/InToFalse and squashes the following commits:

4c722a2 [Zhongshuai Pei] Update predicates.scala
abe2bbb [Zhongshuai Pei] Update Optimizer.scala
fa461a5 [Zhongshuai Pei] Update Optimizer.scala
e34c28a [Zhongshuai Pei] Update predicates.scala
24739bd [Zhongshuai Pei] Update ConstantFoldingSuite.scala
f4dbf50 [Zhongshuai Pei] Update ConstantFoldingSuite.scala
35ceb7a [Zhongshuai Pei] Update Optimizer.scala
36c194e [Zhongshuai Pei] Update Optimizer.scala
2e8f6ca [Zhongshuai Pei] Update Optimizer.scala
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-05-11 19:22:44 -07:00
Jacky Li 6dad76e5eb [SPARK-4699] [SQL] Make caseSensitive configurable in spark sql analyzer
based on #3558

Author: Jacky Li <jacky.likun@huawei.com>
Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #5806 from scwf/case and squashes the following commits:

cd51712 [wangfei] fix compile
d4b724f [wangfei] address michael's comment
af512c7 [wangfei] fix conflicts
4ef1be7 [wangfei] fix conflicts
269cf21 [scwf] fix conflicts
b73df6c [scwf] style issue
9e11752 [scwf] improve SimpleCatalystConf
b35529e [scwf] minor style
a3f7659 [scwf] remove unsed imports
2a56515 [scwf] fix conflicts
6db4bf5 [scwf] also fix for HiveContext
7fc4a98 [scwf] fix test case
d5a9933 [wangfei] fix style
eee75ba [wangfei] fix EmptyConf
6ef31cf [wangfei] revert pom changes
5d7c456 [wangfei] set CASE_SENSITIVE false in TestHive
966e719 [wangfei] set CASE_SENSITIVE false in hivecontext
fd30e25 [wangfei] added override
69b3b70 [wangfei] fix AnalysisSuite
5472b08 [wangfei] fix compile issue
56034ca [wangfei] fix conflicts and improve for catalystconf
664d1e9 [Jacky Li] Merge branch 'master' of https://github.com/apache/spark into case
12eca9a [Jacky Li] solve conflict with master
39e369c [Jacky Li] fix confilct after DataFrame PR
dee56e9 [Jacky Li] fix test case failure
05b09a3 [Jacky Li] fix conflict base on the latest master branch
73c16b1 [Jacky Li] fix bug in sql/hive
9bf4cc7 [Jacky Li] fix bug in catalyst
005c56d [Jacky Li] make SQLContext caseSensitivity configurable
6332e0f [Jacky Li] fix bug
fcbf0d9 [Jacky Li] fix scalastyle check
e7bca31 [Jacky Li] make caseSensitive configuration in Analyzer and Catalog
91b1b96 [Jacky Li] make caseSensitive configurable in Analyzer
f57f15c [Jacky Li] add testcase
578d167 [Jacky Li] make caseSensitive configurable
2015-05-08 15:25:54 -07:00
Wenchen Fan 2d05f325dc [SPARK-7133] [SQL] Implement struct, array, and map field accessor
It's the first step: generalize UnresolvedGetField to support all map, struct, and array
TODO: add `apply` in Scala and `__getitem__` in Python, and unify the `getItem` and `getField` methods to one single API(or should we keep them for compatibility?).

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5744 from cloud-fan/generalize and squashes the following commits:

715c589 [Wenchen Fan] address comments
7ea5b31 [Wenchen Fan] fix python test
4f0833a [Wenchen Fan] add python test
f515d69 [Wenchen Fan] add apply method and test cases
8df6199 [Wenchen Fan] fix python test
239730c [Wenchen Fan] fix test compile
2a70526 [Wenchen Fan] use _bin_op in dataframe.py
6bf72bc [Wenchen Fan] address comments
3f880c3 [Wenchen Fan] add java doc
ab35ab5 [Wenchen Fan] fix python test
b5961a9 [Wenchen Fan] fix style
c9d85f5 [Wenchen Fan] generalize UnresolvedGetField to support all map, struct, and array
2015-05-08 11:49:38 -07:00
Michael Armbrust cd1d4110cf [SPARK-6908] [SQL] Use isolated Hive client
This PR switches Spark SQL's Hive support to use the isolated hive client interface introduced by #5851, instead of directly interacting with the client.  By using this isolated client we can now allow users to dynamically configure the version of Hive that they are connecting to by setting `spark.sql.hive.metastore.version` without the need recompile.  This also greatly reduces the surface area for our interaction with the hive libraries, hopefully making it easier to support other versions in the future.

Jars for the desired hive version can be configured using `spark.sql.hive.metastore.jars`, which accepts the following options:
 - a colon-separated list of jar files or directories for hive and hadoop.
 - `builtin` - attempt to discover the jars that were used to load Spark SQL and use those. This
            option is only valid when using the execution version of Hive.
 - `maven` - download the correct version of hive on demand from maven.

By default, `builtin` is used for Hive 13.

This PR also removes the test step for building against Hive 12, as this will no longer be required to talk to Hive 12 metastores.  However, the full removal of the Shim is deferred until a later PR.

Remaining TODOs:
 - Remove the Hive Shims and inline code for Hive 13.
 - Several HiveCompatibility tests are not yet passing.
  - `nullformatCTAS` - As detailed below, we now are handling CTAS parsing ourselves instead of hacking into the Hive semantic analyzer.  However, we currently only handle the common cases and not things like CTAS where the null format is specified.
  - `combine1` now leaks state about compression somehow, breaking all subsequent tests.  As such we currently add it to the blacklist
  - `part_inherit_tbl_props` and `part_inherit_tbl_props_with_star` do not work anymore.  We are correctly propagating the information
  - "load_dyn_part14.*" - These tests pass when run on their own, but fail when run with all other tests.  It seems our `RESET` mechanism may not be as robust as it used to be?

Other required changes:
 -  `CreateTableAsSelect` no longer carries parts of the HiveQL AST with it through the query execution pipeline.  Instead, we parse CTAS during the HiveQL conversion and construct a `HiveTable`.  The full parsing here is not yet complete as detailed above in the remaining TODOs.  Since the operator is Hive specific, it is moved to the hive package.
 - `Command` is simplified to be a trait that simply acts as a marker for a LogicalPlan that should be eagerly evaluated.

Author: Michael Armbrust <michael@databricks.com>

Closes #5876 from marmbrus/useIsolatedClient and squashes the following commits:

258d000 [Michael Armbrust] really really correct path handling
e56fd4a [Michael Armbrust] getAbsolutePath
5a259f5 [Michael Armbrust] fix typos
81bb366 [Michael Armbrust] comments from vanzin
5f3945e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient
4b5cd41 [Michael Armbrust] yin's comments
f5de7de [Michael Armbrust] cleanup
11e9c72 [Michael Armbrust] better coverage in versions suite
7e8f010 [Michael Armbrust] better error messages and jar handling
e7b3941 [Michael Armbrust] more permisive checking for function registration
da91ba7 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient
5fe5894 [Michael Armbrust] fix serialization suite
81711c4 [Michael Armbrust] Initial support for running without maven
1d8ae44 [Michael Armbrust] fix final tests?
1c50813 [Michael Armbrust] more comments
a3bee70 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient
a6f5df1 [Michael Armbrust] style
ab07f7e [Michael Armbrust] WIP
4d8bf02 [Michael Armbrust] Remove hive 12 compilation
8843a25 [Michael Armbrust] [SPARK-6908] [SQL] Use isolated Hive client
2015-05-07 19:36:24 -07:00
Wenchen Fan 35f0173b8f [SPARK-2155] [SQL] [WHEN D THEN E] [ELSE F] add CaseKeyWhen for "CASE a WHEN b THEN c * END"
Avoid translating to CaseWhen and evaluate the key expression many times.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5979 from cloud-fan/condition and squashes the following commits:

3ce54e1 [Wenchen Fan] add CaseKeyWhen
2015-05-07 16:26:49 -07:00
Yin Huai 5784c8d955 [SPARK-1442] [SQL] [FOLLOW-UP] Address minor comments in Window Function PR (#5604).
Address marmbrus and scwf's comments in #5604.

Author: Yin Huai <yhuai@databricks.com>

Closes #5945 from yhuai/windowFollowup and squashes the following commits:

0ef879d [Yin Huai] Add collectFirst to TreeNode.
2373968 [Yin Huai] wip
4a16df9 [Yin Huai] Address minor comments for [SPARK-1442].
2015-05-07 11:46:49 -07:00
Yin Huai f2c47082c3 [SPARK-1442] [SQL] Window Function Support for Spark SQL
Adding more information about the implementation...

This PR is adding the support of window functions to Spark SQL (specifically OVER and WINDOW clause). For every expression having a OVER clause, we use a WindowExpression as the container of a WindowFunction and the corresponding WindowSpecDefinition (the definition of a window frame, i.e. partition specification, order specification, and frame specification appearing in a OVER clause).
# Implementation #
The high level work flow of the implementation is described as follows.

*	Query parsing: In the query parse process, all WindowExpressions are originally placed in the projectList of a Project operator or the aggregateExpressions of an Aggregate operator. It makes our changes to simple and keep all of parsing rules for window functions at a single place (nodesToWindowSpecification). For the WINDOWclause in a query, we use a WithWindowDefinition as the container as the mapping from the name of a window specification to a WindowSpecDefinition. This changes is similar with our common table expression support.

*	Analysis: The query analysis process has three steps for window functions.

 *	Resolve all WindowSpecReferences by replacing them with WindowSpecReferences according to the mapping table stored in the node of WithWindowDefinition.
 *	Resolve WindowFunctions in the projectList of a Project operator or the aggregateExpressions of an Aggregate operator. For this PR, we use Hive's functions for window functions because we will have a major refactoring of our internal UDAFs and it is better to switch our UDAFs after that refactoring work.
 *	Once we have resolved all WindowFunctions, we will use ResolveWindowFunction to extract WindowExpressions from projectList and aggregateExpressions and then create a Window operator for every distinct WindowSpecDefinition. With this choice, at the execution time, we can rely on the Exchange operator to do all of work on reorganizing the table and we do not need to worry about it in the physical Window operator. An example analyzed plan is shown as follows

```
sql("""
SELECT
  year, country, product, sales,
  avg(sales) over(partition by product) avg_product,
  sum(sales) over(partition by country) sum_country
FROM sales
ORDER BY year, country, product
""").explain(true)

== Analyzed Logical Plan ==
Sort [year#34 ASC,country#35 ASC,product#36 ASC], true
 Project [year#34,country#35,product#36,sales#37,avg_product#27,sum_country#28]
  Window [year#34,country#35,product#36,sales#37,avg_product#27], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum(sales#37) WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_country#28], WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
   Window [year#34,country#35,product#36,sales#37], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage(sales#37) WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS avg_product#27], WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
    Project [year#34,country#35,product#36,sales#37]
     MetastoreRelation default, sales, None
```

*	Query planning: In the process of query planning, we simple generate the physical Window operator based on the logical Window operator. Then, to prepare the executedPlan, the EnsureRequirements rule will add Exchange and Sort operators if necessary. The EnsureRequirements rule will analyze the data properties and try to not add unnecessary shuffle and sort. The physical plan for the above example query is shown below.

```
== Physical Plan ==
Sort [year#34 ASC,country#35 ASC,product#36 ASC], true
 Exchange (RangePartitioning [year#34 ASC,country#35 ASC,product#36 ASC], 200), []
  Window [year#34,country#35,product#36,sales#37,avg_product#27], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum(sales#37) WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_country#28], WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
   Exchange (HashPartitioning [country#35], 200), [country#35 ASC]
    Window [year#34,country#35,product#36,sales#37], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage(sales#37) WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS avg_product#27], WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
     Exchange (HashPartitioning [product#36], 200), [product#36 ASC]
      HiveTableScan [year#34,country#35,product#36,sales#37], (MetastoreRelation default, sales, None), None
```

*	Execution time: At execution time, a physical Window operator buffers all rows in a partition specified in the partition spec of a OVER clause. If necessary, it also maintains a sliding window frame. The current implementation tries to buffer the input parameters of a window function according to the window frame to avoid evaluating a row multiple times.

# Future work #

Here are three improvements that are not hard to add:
*	Taking advantage of the window frame specification to reduce the number of rows buffered in the physical Window operator. For some cases, we only need to buffer the rows appearing in the sliding window. But for other cases, we will not be able to reduce the number of rows buffered (e.g. ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING).

*	When aRAGEN frame is used, for <value> PRECEDING and <value> FOLLOWING, it will be great if the <value> part is an expression (we can start with Literal). So, when the data type of ORDER BY expression is a FractionalType, we can support FractionalType as the type <value> (<value> still needs to be evaluated as a positive value).

*	When aRAGEN frame is used, we need to support DateType and TimestampType as the data type of the expression appearing in the order specification. Then, the <value> part of <value> PRECEDING and <value> FOLLOWING can support interval types (once we support them).

This is a joint work with guowei2 and yhuai
Thanks hbutani hvanhovell for his comments
Thanks scwf for his comments and unit tests

Author: Yin Huai <yhuai@databricks.com>

Closes #5604 from guowei2/windowImplement and squashes the following commits:

76fe1c8 [Yin Huai] Implementation.
aa2b0ae [Yin Huai] Tests.
2015-05-06 10:43:00 -07:00
Burak Yavuz ba2b56614d [SPARK-7358][SQL] Move DataFrame mathfunctions into functions
After a discussion on the user mailing list, it was decided to put all UDF's under `o.a.s.sql.functions`

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5923 from brkyvz/move-math-funcs and squashes the following commits:

a8dc3f7 [Burak Yavuz] address comments
cf7a7bb [Burak Yavuz] [SPARK-7358] Move DataFrame mathfunctions into functions
2015-05-05 22:56:01 -07:00
Burak Yavuz b5347a4664 [SPARK-7248] implemented random number generators for DataFrames
Adds the functions `rand` (Uniform Dist) and `randn` (Normal Dist.) as expressions to DataFrames.

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5819 from brkyvz/df-rng and squashes the following commits:

50d69d4 [Burak Yavuz] add seed for test that failed
4234c3a [Burak Yavuz] fix Rand expression
13cad5c [Burak Yavuz] couple fixes
7d53953 [Burak Yavuz] waiting for hive tests
b453716 [Burak Yavuz] move radn with seed down
03637f0 [Burak Yavuz] fix broken hive func
c5909eb [Burak Yavuz] deleted old implementation of Rand
6d43895 [Burak Yavuz] implemented random generators
2015-04-30 21:56:03 -07:00
wangfei a0d8a61ab1 [SPARK-7109] [SQL] Push down left side filter for left semi join
Now in spark sql optimizer we only push down right side filter for left semi join, actually we can push down left side filter because left semi join is doing filter on left table essentially.

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #5677 from scwf/leftsemi and squashes the following commits:

483d205 [wangfei] update with master to fix compile issue
82df0e1 [wangfei] Merge branch 'master' of https://github.com/apache/spark into leftsemi
d68a053 [wangfei] added apply
8f48a3d [scwf] added test
ebadaa9 [wangfei] left filter push down for left semi join
2015-04-30 18:18:54 -07:00
Zhongshuai Pei 77cc25fb74 [SPARK-7267][SQL]Push down Project when it's child is Limit
SQL
```
select key from (select key,value from t1 limit 100) t2 limit 10
```
Optimized Logical Plan before modifying
```
== Optimized Logical Plan ==
Limit 10
  Project key#228
    Limit 100
      MetastoreRelation default, t1, None
```
Optimized Logical Plan after modifying
```
== Optimized Logical Plan ==
Limit 10
  Limit 100
    Project key#228
      MetastoreRelation default, t1, None
```
After this, we can combine limits

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

Closes #5797 from DoingDone9/ProjectLimit and squashes the following commits:

70d0fca [Zhongshuai Pei] Update FilterPushdownSuite.scala
dc83ae9 [Zhongshuai Pei] Update FilterPushdownSuite.scala
485c61c [Zhongshuai Pei] Update Optimizer.scala
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-04-30 15:22:13 -07:00
Zhongshuai Pei 4459514497 [SPARK-7225][SQL] CombineLimits optimizer does not work
SQL
```
select key from (select key from src limit 100) t2 limit 10
```
Optimized Logical Plan before modifying
```
== Optimized Logical Plan ==
Limit 10
Limit 100
Project key#3
MetastoreRelation default, src, None
```
Optimized Logical Plan after modifying
```
== Optimized Logical Plan ==
Limit 10
 Project [key#1]
  MetastoreRelation default, src, None
```

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

Closes #5770 from DoingDone9/limitOptimizer and squashes the following commits:

c68eaa7 [Zhongshuai Pei] Update CombiningLimitsSuite.scala
97e18cf [Zhongshuai Pei] Update Optimizer.scala
19ab875 [Zhongshuai Pei] Update CombiningLimitsSuite.scala
7db4566 [Zhongshuai Pei] Update CombiningLimitsSuite.scala
e2a491d [Zhongshuai Pei] Update Optimizer.scala
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-04-29 22:44:14 -07:00
Josh Rosen f49284b5bf [SPARK-7076][SPARK-7077][SPARK-7080][SQL] Use managed memory for aggregations
This patch adds managed-memory-based aggregation to Spark SQL / DataFrames. Instead of working with Java objects, this new aggregation path uses `sun.misc.Unsafe` to manipulate raw memory.  This reduces the memory footprint for aggregations, resulting in fewer spills, OutOfMemoryErrors, and garbage collection pauses.  As a result, this allows for higher memory utilization.  It can also result in better cache locality since objects will be stored closer together in memory.

This feature can be eanbled by setting `spark.sql.unsafe.enabled=true`.  For now, this feature is only supported when codegen is enabled and only supports aggregations for which the grouping columns are primitive numeric types or strings and aggregated values are numeric.

### Managing memory with sun.misc.Unsafe

This patch supports both on- and off-heap managed memory.

- In on-heap mode, memory addresses are identified by the combination of a base Object and an offset within that object.
- In off-heap mode, memory is addressed directly with 64-bit long addresses.

To support both modes, functions that manipulate memory accept both `baseObject` and `baseOffset` fields.  In off-heap mode, we simply pass `null` as `baseObject`.

We allocate memory in large chunks, so memory fragmentation and allocation speed are not significant bottlenecks.

By default, we use on-heap mode.  To enable off-heap mode, set `spark.unsafe.offHeap=true`.

To track allocated memory, this patch extends `SparkEnv` with an `ExecutorMemoryManager` and supplies each `TaskContext` with a `TaskMemoryManager`.  These classes work together to track allocations and detect memory leaks.

### Compact tuple format

This patch introduces `UnsafeRow`, a compact row layout.  In this format, each tuple has three parts: a null bit set, fixed length values, and variable-length values:

![image](https://cloud.githubusercontent.com/assets/50748/7328538/2fdb65ce-ea8b-11e4-9743-6c0f02bb7d1f.png)

- Rows are always 8-byte word aligned (so their sizes will always be a multiple of 8 bytes)
- The bit set is used for null tracking:
	- Position _i_ is set if and only if field _i_ is null
	- The bit set is aligned to an 8-byte word boundary.
- Every field appears as an 8-byte word in the fixed-length values part:
	- If a field is null, we zero out the values.
	- If a field is variable-length, the word stores a relative offset (w.r.t. the base of the tuple) that points to the beginning of the field's data in the variable-length part.
- Each variable-length data type can have its own encoding:
	- For strings, the first word stores the length of the string and is followed by UTF-8 encoded bytes.  If necessary, the end of the string is padded with empty bytes in order to ensure word-alignment.

For example, a tuple that consists 3 fields of type (int, string, string), with value (null, “data”, “bricks”) would look like this:

![image](https://cloud.githubusercontent.com/assets/50748/7328526/1e21959c-ea8b-11e4-9a28-a4350fe4a7b5.png)

This format allows us to compare tuples for equality by directly comparing their raw bytes.  This also enables fast hashing of tuples.

### Hash map for performing aggregations

This patch introduces `UnsafeFixedWidthAggregationMap`, a hash map for performing aggregations where the aggregation result columns are fixed-with.  This map's keys and values are `Row` objects. `UnsafeFixedWidthAggregationMap` is implemented on top of `BytesToBytesMap`, an append-only map which supports byte-array keys and values.

`BytesToBytesMap` stores pointers to key and value tuples.  For each record with a new key, we copy the key and create the aggregation value buffer for that key and put them in a buffer. The hash table then simply stores pointers to the key and value. For each record with an existing key, we simply run the aggregation function to update the values in place.

This map is implemented using open hashing with triangular sequence probing.  Each entry stores two words in a long array: the first word stores the address of the key and the second word stores the relative offset from the key tuple to the value tuple, as well as the key's 32-bit hashcode.  By storing the full hashcode, we reduce the number of equality checks that need to be performed to handle position collisions ()since the chance of hashcode collision is much lower than position collision).

`UnsafeFixedWidthAggregationMap` allows regular Spark SQL `Row` objects to be used when probing the map.  Internally, it encodes these rows into `UnsafeRow` format using `UnsafeRowConverter`.  This conversion has a small overhead that can be eliminated in the future once we use UnsafeRows in other operators.

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5725 from JoshRosen/unsafe and squashes the following commits:

eeee512 [Josh Rosen] Add converters for Null, Boolean, Byte, and Short columns.
81f34f8 [Josh Rosen] Follow 'place children last' convention for GeneratedAggregate
1bc36cc [Josh Rosen] Refactor UnsafeRowConverter to avoid unnecessary boxing.
017b2dc [Josh Rosen] Remove BytesToBytesMap.finalize()
50e9671 [Josh Rosen] Throw memory leak warning even in case of error; add warning about code duplication
70a39e4 [Josh Rosen] Split MemoryManager into ExecutorMemoryManager and TaskMemoryManager:
6e4b192 [Josh Rosen] Remove an unused method from ByteArrayMethods.
de5e001 [Josh Rosen] Fix debug vs. trace in logging message.
a19e066 [Josh Rosen] Rename unsafe Java test suites to match Scala test naming convention.
78a5b84 [Josh Rosen] Add logging to MemoryManager
ce3c565 [Josh Rosen] More comments, formatting, and code cleanup.
529e571 [Josh Rosen] Measure timeSpentResizing in nanoseconds instead of milliseconds.
3ca84b2 [Josh Rosen] Only zero the used portion of groupingKeyConversionScratchSpace
162caf7 [Josh Rosen] Fix test compilation
b45f070 [Josh Rosen] Don't redundantly store the offset from key to value, since we can compute this from the key size.
a8e4a3f [Josh Rosen] Introduce MemoryManager interface; add to SparkEnv.
0925847 [Josh Rosen] Disable MiMa checks for new unsafe module
cde4132 [Josh Rosen] Add missing pom.xml
9c19fc0 [Josh Rosen] Add configuration options for heap vs. offheap
6ffdaa1 [Josh Rosen] Null handling improvements in UnsafeRow.
31eaabc [Josh Rosen] Lots of TODO and doc cleanup.
a95291e [Josh Rosen] Cleanups to string handling code
afe8dca [Josh Rosen] Some Javadoc cleanup
f3dcbfe [Josh Rosen] More mod replacement
854201a [Josh Rosen] Import and comment cleanup
06e929d [Josh Rosen] More warning cleanup
ef6b3d3 [Josh Rosen] Fix a bunch of FindBugs and IntelliJ inspections
29a7575 [Josh Rosen] Remove debug logging
49aed30 [Josh Rosen] More long -> int conversion.
b26f1d3 [Josh Rosen] Fix bug in murmur hash implementation.
765243d [Josh Rosen] Enable optional performance metrics for hash map.
23a440a [Josh Rosen] Bump up default hash map size
628f936 [Josh Rosen] Use ints intead of longs for indexing.
92d5a06 [Josh Rosen] Address a number of minor code review comments.
1f4b716 [Josh Rosen] Merge Unsafe code into the regular GeneratedAggregate, guarded by a configuration flag; integrate planner support and re-enable all tests.
d85eeff [Josh Rosen] Add basic sanity test for UnsafeFixedWidthAggregationMap
bade966 [Josh Rosen] Comment update (bumping to refresh GitHub cache...)
b3eaccd [Josh Rosen] Extract aggregation map into its own class.
d2bb986 [Josh Rosen] Update to implement new Row methods added upstream
58ac393 [Josh Rosen] Use UNSAFE allocator in GeneratedAggregate (TODO: make this configurable)
7df6008 [Josh Rosen] Optimizations related to zeroing out memory:
c1b3813 [Josh Rosen] Fix bug in UnsafeMemoryAllocator.free():
738fa33 [Josh Rosen] Add feature flag to guard UnsafeGeneratedAggregate
c55bf66 [Josh Rosen] Free buffer once iterator has been fully consumed.
62ab054 [Josh Rosen] Optimize for fact that get() is only called on String columns.
c7f0b56 [Josh Rosen] Reuse UnsafeRow pointer in UnsafeRowConverter
ae39694 [Josh Rosen] Add finalizer as "cleanup method of last resort"
c754ae1 [Josh Rosen] Now that the store*() contract has been stregthened, we can remove an extra lookup
f764d13 [Josh Rosen] Simplify address + length calculation in Location.
079f1bf [Josh Rosen] Some clarification of the BytesToBytesMap.lookup() / set() contract.
1a483c5 [Josh Rosen] First version that passes some aggregation tests:
fc4c3a8 [Josh Rosen] Sketch how the converters will be used in UnsafeGeneratedAggregate
53ba9b7 [Josh Rosen] Start prototyping Java Row -> UnsafeRow converters
1ff814d [Josh Rosen] Add reminder to free memory on iterator completion
8a8f9df [Josh Rosen] Add skeleton for GeneratedAggregate integration.
5d55cef [Josh Rosen] Add skeleton for Row implementation.
f03e9c1 [Josh Rosen] Play around with Unsafe implementations of more string methods.
ab68e08 [Josh Rosen] Begin merging the UTF8String implementations.
480a74a [Josh Rosen] Initial import of code from Databricks unsafe utils repo.
2015-04-29 01:07:26 -07:00
Burak Yavuz fe917f5ec9 [SPARK-7188] added python support for math DataFrame functions
Adds support for the math functions for DataFrames in PySpark.

rxin I love Davies.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5750 from brkyvz/python-math-udfs and squashes the following commits:

7c4f563 [Burak Yavuz] removed is_math
3c4adde [Burak Yavuz] cleanup imports
d5dca3f [Burak Yavuz] moved math functions to mathfunctions
25e6534 [Burak Yavuz] addressed comments v2.0
d3f7e0f [Burak Yavuz] addressed comments and added tests
7b7d7c4 [Burak Yavuz] remove tests for removed methods
33c2c15 [Burak Yavuz] fixed python style
3ee0c05 [Burak Yavuz] added python functions
2015-04-29 00:09:24 -07:00
Burak Yavuz 29576e7860 [SPARK-6829] Added math functions for DataFrames
Implemented almost all math functions found in scala.math (max, min and abs were already present).

cc mengxr marmbrus

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5616 from brkyvz/math-udfs and squashes the following commits:

fb27153 [Burak Yavuz] reverted exception message
836a098 [Burak Yavuz] fixed test and addressed small comment
e5f0d13 [Burak Yavuz] addressed code review v2.2
b26c5fb [Burak Yavuz] addressed review v2.1
2761f08 [Burak Yavuz] addressed review v2
6588a5b [Burak Yavuz] fixed merge conflicts
b084e10 [Burak Yavuz] Addressed code review
029e739 [Burak Yavuz] fixed atan2 test
534cc11 [Burak Yavuz] added more tests, addressed comments
fa68dbe [Burak Yavuz] added double specific test data
937d5a5 [Burak Yavuz] use doubles instead of ints
8e28fff [Burak Yavuz] Added apache header
7ec8f7f [Burak Yavuz] Added math functions for DataFrames
2015-04-27 23:10:14 -07:00
wangfei d188b8bad8 [SQL][Minor] rename DataTypeParser.apply to DataTypeParser.parse
rename DataTypeParser.apply to DataTypeParser.parse to make it more clear and readable.
/cc rxin

Author: wangfei <wangfei1@huawei.com>

Closes #5710 from scwf/apply and squashes the following commits:

c319977 [wangfei] rename apply to parse
2015-04-26 21:08:47 -07:00
Reynold Xin cdf0328684 [SQL] Rename some apply functions.
I was looking at the code gen code and got confused by a few of use cases of apply, in particular apply on objects. So I went ahead and changed a few of them. Hopefully slightly more clear with a proper verb.

Author: Reynold Xin <rxin@databricks.com>

Closes #5624 from rxin/apply-rename and squashes the following commits:

ee45034 [Reynold Xin] [SQL] Rename some apply functions.
2015-04-22 11:18:01 -07:00
Cheng Hao 7662ec23bb [SPARK-5817] [SQL] Fix bug of udtf with column names
It's a bug while do query like:
```sql
select d from (select explode(array(1,1)) d from src limit 1) t
```
And it will throws exception like:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'd' given input columns _c0; line 1 pos 7
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:48)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:45)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:50)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:249)
at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$transformExpressionUp$1(QueryPlan.scala:103)
at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2$$anonfun$apply$2.apply(QueryPlan.scala:117)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
at scala.collection.AbstractTraversable.map(Traversable.scala:105)
at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2.apply(QueryPlan.scala:116)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
```

To solve the bug, it requires code refactoring for UDTF
The major changes are about:
* Simplifying the UDTF development, UDTF will manage the output attribute names any more, instead, the `logical.Generate` will handle that properly.
* UDTF will be asked for the output schema (data types) during the logical plan analyzing.

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

Closes #4602 from chenghao-intel/explode_bug and squashes the following commits:

c2a5132 [Cheng Hao] add back resolved for Alias
556e982 [Cheng Hao] revert the unncessary change
002c361 [Cheng Hao] change the rule of resolved for Generate
04ae500 [Cheng Hao] add qualifier only for generator output
5ee5d2c [Cheng Hao] prepend the new qualifier
d2e8b43 [Cheng Hao] Update the code as feedback
ca5e7f4 [Cheng Hao] shrink the commits
2015-04-21 15:11:15 -07:00
vidmantas zemleris 2e8c6ca47d [SPARK-6994] Allow to fetch field values by name in sql.Row
It looked weird that up to now there was no way in Spark's Scala API to access fields of `DataFrame/sql.Row` by name, only by their index.

This tries to solve this issue.

Author: vidmantas zemleris <vidmantas@vinted.com>

Closes #5573 from vidma/features/row-with-named-fields and squashes the following commits:

6145ae3 [vidmantas zemleris] [SPARK-6994][SQL] Allow to fetch field values by name on Row
9564ebb [vidmantas zemleris] [SPARK-6994][SQL] Add fieldIndex to schema (StructType)
2015-04-21 14:47:09 -07:00
scwf d96608674f [SQL][Minor] Fix foreachUp of treenode
`foreachUp` should runs the given function recursively on [[children]] then on this node(just like transformUp). The current implementation does not follow this.

This will leads to checkanalysis do not check from bottom of logical tree.

Author: scwf <wangfei1@huawei.com>
Author: Fei Wang <wangfei1@huawei.com>

Closes #5518 from scwf/patch-1 and squashes the following commits:

18e28b2 [scwf] added a test case
1ccbfa8 [Fei Wang] fix foreachUp
2015-04-16 17:35:51 -07:00
Wenchen Fan 4754e16f47 [SPARK-6898][SQL] completely support special chars in column names
Even if we wrap column names in backticks like `` `a#$b.c` ``,  we still handle the "." inside column name specially. I think it's fragile to use a special char to split name parts, why not put name parts in `UnresolvedAttribute` directly?

Author: Wenchen Fan <cloud0fan@outlook.com>

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

Closes #5511 from cloud-fan/6898 and squashes the following commits:

48e3e57 [Wenchen Fan] more style fix
820dc45 [Wenchen Fan] do not ignore newName in UnresolvedAttribute
d81ad43 [Wenchen Fan] fix style
11699d6 [Wenchen Fan] completely support special chars in column names
2015-04-15 13:39:12 -07:00
Davies Liu 85842760dc [SPARK-6638] [SQL] Improve performance of StringType in SQL
This PR change the internal representation for StringType from java.lang.String to UTF8String, which is implemented use ArrayByte.

This PR should not break any public API, Row.getString() will still return java.lang.String.

This is the first step of improve the performance of String in SQL.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #5350 from davies/string and squashes the following commits:

3b7bfa8 [Davies Liu] fix schema of AddJar
2772f0d [Davies Liu] fix new test failure
6d776a9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
59025c8 [Davies Liu] address comments from @marmbrus
341ec2c [Davies Liu] turn off scala style check in UTF8StringSuite
744788f [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
b04a19c [Davies Liu] add comment for getString/setString
08d897b [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
5116b43 [Davies Liu] rollback unrelated changes
1314a37 [Davies Liu] address comments from Yin
867bf50 [Davies Liu] fix String filter push down
13d9d42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
2089d24 [Davies Liu] add hashcode check back
ac18ae6 [Davies Liu] address comment
fd11364 [Davies Liu] optimize UTF8String
8d17f21 [Davies Liu] fix hive compatibility tests
e5fa5b8 [Davies Liu] remove clone in UTF8String
28f3d81 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
28d6f32 [Davies Liu] refactor
537631c [Davies Liu] some comment about Date
9f4c194 [Davies Liu] convert data type for data source
956b0a4 [Davies Liu] fix hive tests
73e4363 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
9dc32d1 [Davies Liu] fix some hive tests
23a766c [Davies Liu] refactor
8b45864 [Davies Liu] fix codegen with UTF8String
bb52e44 [Davies Liu] fix scala style
c7dd4d2 [Davies Liu] fix some catalyst tests
38c303e [Davies Liu] fix python sql tests
5f9e120 [Davies Liu] fix sql tests
6b499ac [Davies Liu] fix style
a85fb27 [Davies Liu] refactor
d32abd1 [Davies Liu] fix utf8 for python api
4699c3a [Davies Liu] use Array[Byte] in UTF8String
21f67c6 [Davies Liu] cleanup
685fd07 [Davies Liu] use UTF8String instead of String for StringType
2015-04-15 13:06:38 -07:00
Liang-Chi Hsieh 4898dfa464 [SPARK-6877][SQL] Add code generation support for Min
Currently `min` is not supported in code generation. This pr adds the support for it.

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

Closes #5487 from viirya/add_min_codegen and squashes the following commits:

0ddec23 [Liang-Chi Hsieh] Add code generation support for Min.
2015-04-13 18:16:33 -07:00
Santiago M. Mola 5f7b7cdab4 [SPARK-6611][SQL] Add support for INTEGER as synonym of INT.
https://issues.apache.org/jira/browse/SPARK-6611

Author: Santiago M. Mola <santiago.mola@sap.com>

Closes #5271 from smola/features/integer-parse and squashes the following commits:

f5c1c64 [Santiago M. Mola] [SPARK-6611] Add support for INTEGER as synonym of INT.
2015-04-11 14:52:49 -07:00
Volodymyr Lyubinets 67d06880e4 [SQL] [SPARK-6620] Speed up toDF() and rdd() functions by constructing converters in ScalaReflection
cc marmbrus

Author: Volodymyr Lyubinets <vlyubin@gmail.com>

Closes #5279 from vlyubin/speedup and squashes the following commits:

e75a387 [Volodymyr Lyubinets] Changes to ScalaUDF
11a20ec [Volodymyr Lyubinets] Avoid creating a tuple
c327bc9 [Volodymyr Lyubinets] Moved the only remaining function from DataTypeConversions to DateUtils
dec6802 [Volodymyr Lyubinets] Addresed review feedback
74301fa [Volodymyr Lyubinets] Addressed review comments
afa3aa5 [Volodymyr Lyubinets] Minor refactoring, added license, removed debug output
881dc60 [Volodymyr Lyubinets] Moved to a separate module; addressed review comments; one extra place of usage; changed behaviour for Java
8cad6e2 [Volodymyr Lyubinets] Addressed review commments
41b2aa9 [Volodymyr Lyubinets] Creating converters for ScalaReflection stuff, and more
2015-04-10 16:27:56 -07:00
WangTaoTheTonic 7d92db342e [SPARK-6758]block the right jetty package in log
https://issues.apache.org/jira/browse/SPARK-6758

I am not sure if it is ok to block them in test resources too (as we shade jetty in assembly?).

Author: WangTaoTheTonic <wangtao111@huawei.com>

Closes #5406 from WangTaoTheTonic/SPARK-6758 and squashes the following commits:

e09605b [WangTaoTheTonic] block the right jetty package
2015-04-09 17:44:08 -04:00
Reynold Xin 1b2aab8d5b [SPARK-6765] Fix test code style for SQL
So we can turn style checker on for test code.

Author: Reynold Xin <rxin@databricks.com>

Closes #5412 from rxin/test-style-sql and squashes the following commits:

9098a31 [Reynold Xin] One more compilation error ...
8c7250a [Reynold Xin] Fix compilation.
82d0944 [Reynold Xin] Indentation.
0b03fbb [Reynold Xin] code review.
f2f4348 [Reynold Xin] oops.
ef4ec48 [Reynold Xin] Hive module.
7e0db5e [Reynold Xin] sql module
04ec7ac [Reynold Xin] catalyst module
2015-04-08 20:35:29 -07:00
guowei2 c23ba81b8c [SPARK-5203][SQL] fix union with different decimal type
When union non-decimal types with decimals, we use the following rules:
      - FIRST `intTypeToFixed`, then fixed union decimals with precision/scale p1/s2 and p2/s2  will be promoted to
      DecimalType(max(p1, p2), max(s1, s2))
      - FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive,
      but note that unlimited decimals are considered bigger than doubles in WidenTypes)

Author: guowei2 <guowei2@asiainfo.com>

Closes #4004 from guowei2/SPARK-5203 and squashes the following commits:

ff50f5f [guowei2] fix code style
11df1bf [guowei2] fix decimal union with double, double->Decimal(15,15)
0f345f9 [guowei2] fix structType merge with decimal
101ed4d [guowei2] fix build error after rebase
0b196e4 [guowei2] code style
fe2c2ca [guowei2] handle union decimal precision in 'DecimalPrecision'
421d840 [guowei2] fix union types for decimal precision
ef2c661 [guowei2] fix union with different decimal type
2015-04-04 02:02:30 +08:00
Davies Liu 40df5d49bb [SPARK-6663] [SQL] use Literal.create instread of constructor
In order to do inbound checking and type conversion, we should use Literal.create() instead of  constructor.

Author: Davies Liu <davies@databricks.com>

Closes #5320 from davies/literal and squashes the following commits:

1667604 [Davies Liu] fix style and add comment
5f8c0fd [Davies Liu] use Literal.create instread of constructor
2015-04-01 23:11:38 -07:00
Michael Armbrust cd48ca5012 [SPARK-6145][SQL] fix ORDER BY on nested fields
This PR is based on work by cloud-fan in #4904, but with two differences:
 - We isolate the logic for Sort's special handling into `ResolveSortReferences`
 - We avoid creating UnresolvedGetField expressions during resolution.  Instead we either resolve GetField or we return None.  This avoids us going down the wrong path early on.

Author: Michael Armbrust <michael@databricks.com>

Closes #5189 from marmbrus/nestedOrderBy and squashes the following commits:

b8cae45 [Michael Armbrust] fix another test
0f36a11 [Michael Armbrust] WIP
91820cd [Michael Armbrust] Fix bug.
2015-03-31 11:23:18 -07:00
Xiangrui Meng a05835b89f [SPARK-6542][SQL] add CreateStruct
Similar to `CreateArray`, we can add `CreateStruct` to create nested columns. marmbrus

Author: Xiangrui Meng <meng@databricks.com>

Closes #5195 from mengxr/SPARK-6542 and squashes the following commits:

3795c57 [Xiangrui Meng] update error message
ae7ac3e [Xiangrui Meng] move unit test to a separate suite
85dd559 [Xiangrui Meng] use NamedExpr
c78e31a [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-6542
85f3106 [Xiangrui Meng] add CreateStruct
2015-03-31 17:05:23 +08:00
Reynold Xin 3af7334304 [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 row, not 1 row
Author: Reynold Xin <rxin@databricks.com>

Closes #5226 from rxin/empty-df and squashes the following commits:

1306d88 [Reynold Xin] Proper fix.
e135bb9 [Reynold Xin] [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 rows, not 1 row.
2015-03-27 14:56:57 -07:00
Michael Armbrust 276ef1c3cf [SPARK-6463][SQL] AttributeSet.equal should compare size
Previously this could result in sets compare equals when in fact the right was a subset of the left.

Based on #5133 by sisihj

Author: sisihj <jun.hejun@huawei.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #5194 from marmbrus/pr/5133 and squashes the following commits:

5ed4615 [Michael Armbrust] fix imports
d4cbbc0 [Michael Armbrust] Add test cases
0a0834f [sisihj]  AttributeSet.equal should compare size
2015-03-25 19:22:05 -07:00
Michael Armbrust cbeaf9ebab [SPARK-6376][SQL] Avoid eliminating subqueries until optimization
Previously it was okay to throw away subqueries after analysis, as we would never try to use that tree for resolution again.  However, with eager analysis in `DataFrame`s this can cause errors for queries such as:

```scala
val df = Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str")
df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count()
```

As a result, in this PR we defer the elimination of subqueries until the optimization phase.

Author: Michael Armbrust <michael@databricks.com>

Closes #5160 from marmbrus/subqueriesInDfs and squashes the following commits:

a9bb262 [Michael Armbrust] Update Optimizer.scala
27d25bf [Michael Armbrust] fix hive tests
9137e03 [Michael Armbrust] add type
81cd597 [Michael Armbrust] Avoid eliminating subqueries until optimization
2015-03-24 14:08:20 -07:00
Cheng Lian 1afcf773d0 [SPARK-6452] [SQL] Checks for missing attributes and unresolved operator for all types of operator
In `CheckAnalysis`, `Filter` and `Aggregate` are checked in separate case clauses, thus never hit those clauses for unresolved operators and missing input attributes.

This PR also removes the `prettyString` call when generating error message for missing input attributes. Because result of `prettyString` doesn't contain expression ID, and may give confusing messages like

> resolved attributes a missing from a

cc rxin

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

Author: Cheng Lian <lian@databricks.com>

Closes #5129 from liancheng/spark-6452 and squashes the following commits:

52cdc69 [Cheng Lian] Addresses comments
029f9bd [Cheng Lian] Checks for missing attributes and unresolved operator for all types of operator
2015-03-24 01:12:11 -07:00
Yin Huai 94a102acb8 [SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser.
This PR creates a trait `DataTypeParser` used to parse data types. This trait aims to be single place to provide the functionality of parsing data types' string representation. It is currently mixed in with `DDLParser` and `SqlParser`. It is also used to parse the data type for `DataFrame.cast` and to convert Hive metastore's data type string back to a `DataType`.

JIRA: https://issues.apache.org/jira/browse/SPARK-6250

Author: Yin Huai <yhuai@databricks.com>

Closes #5078 from yhuai/ddlKeywords and squashes the following commits:

0e66097 [Yin Huai] Special handle struct<>.
fea6012 [Yin Huai] Style.
c9733fb [Yin Huai] Create a trait to parse data types.
2015-03-21 13:27:53 -07:00
Michael Armbrust 3579003115 [SPARK-6247][SQL] Fix resolution of ambiguous joins caused by new aliases
We need to handle ambiguous `exprId`s that are produced by new aliases as well as those caused by leaf nodes (`MultiInstanceRelation`).

Attempting to fix this revealed a bug in `equals` for `Alias` as these objects were comparing equal even when the expression ids did not match. Additionally, `LocalRelation` did not correctly provide statistics, and some tests in `catalyst` and `hive` were not using the helper functions for comparing plans.

Based on #4991 by chenghao-intel

Author: Michael Armbrust <michael@databricks.com>

Closes #5062 from marmbrus/selfJoins and squashes the following commits:

8e9b84b [Michael Armbrust] check qualifier too
8038a36 [Michael Armbrust] handle aggs too
0b9c687 [Michael Armbrust] fix more tests
c3c574b [Michael Armbrust] revert change.
725f1ab [Michael Armbrust] add statistics
a925d08 [Michael Armbrust] check for conflicting attributes in join resolution
b022ef7 [Michael Armbrust] Handle project aliases.
d8caa40 [Michael Armbrust] test case: SPARK-6247
f9c67c2 [Michael Armbrust] Check for duplicate attributes in join resolution.
898af73 [Michael Armbrust] Fix Alias equality.
2015-03-17 19:47:51 -07:00
Yin Huai 12599942e6 [SPARK-5950][SQL]Insert array into a metastore table saved as parquet should work when using datasource api
This PR contains the following changes:
1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However,  the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values).
2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types.
3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings.
4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust.
5. Update the equality check of JSON relation. Since JSON does not really cares nullability,  `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables.

JIRA: https://issues.apache.org/jira/browse/SPARK-5950

Thanks viirya for the initial work in #4729.

cc marmbrus liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #4826 from yhuai/insertNullabilityCheck and squashes the following commits:

3b61a04 [Yin Huai] Revert change on equals.
80e487e [Yin Huai] asNullable in UDT.
587d88b [Yin Huai] Make methods private.
0cb7ea2 [Yin Huai] marmbrus's comments.
3cec464 [Yin Huai] Cheng's comments.
486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
d3747d1 [Yin Huai] Remove unnecessary change.
8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check.
0eb5578 [Yin Huai] Fix tests.
f6ed813 [Yin Huai] Update old parquet path.
e4f397c [Yin Huai] Unit tests.
b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check.
8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data.
bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data.
0a703e7 [Yin Huai] Test failed again since we cannot read correct content.
9a26611 [Yin Huai] Make InsertIntoTable happy.
8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability
4ec17fd [Yin Huai] Failed test.
2015-03-02 19:31:55 -08:00
Michael Armbrust 1ed57086d4 [SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution
Author: Michael Armbrust <michael@databricks.com>

Closes #4684 from marmbrus/explainAnalysis and squashes the following commits:

afbaa19 [Michael Armbrust] fix python
d93278c [Michael Armbrust] fix hive
e5fa0a4 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis
52119f2 [Michael Armbrust] more tests
82a5431 [Michael Armbrust] fix tests
25753d2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis
aee1e6a [Michael Armbrust] fix hive
b23a844 [Michael Armbrust] newline
de8dc51 [Michael Armbrust] more comments
acf620a [Michael Armbrust] [SPARK-5873][SQL] Show partially analyzed plans in query execution
2015-02-23 17:34:54 -08:00
Yin Huai d5f12bfe8f [SPARK-5875][SQL]logical.Project should not be resolved if it contains aggregates or generators
https://issues.apache.org/jira/browse/SPARK-5875 has a case to reproduce the bug and explain the root cause.

Author: Yin Huai <yhuai@databricks.com>

Closes #4663 from yhuai/projectResolved and squashes the following commits:

472f7b6 [Yin Huai] If a logical.Project has any AggregateExpression or Generator, it's resolved field should be false.
2015-02-17 17:50:39 -08:00
kai cb6c48c874 [SQL] Optimize arithmetic and predicate operators
Existing implementation of arithmetic operators and BinaryComparison operators have redundant type checking codes, e.g.:
Expression.n2 is used by Add/Subtract/Multiply.
(1) n2 always checks left.dataType == right.dataType. However, this checking should be done once when we resolve expression types;
(2) n2 requires dataType is a NumericType. This can be done once.

This PR optimizes arithmetic and predicate operators by removing such redundant type-checking codes.

Some preliminary benchmarking on 10G TPC-H data over 5 r3.2xlarge EC2 machines shows that this PR can reduce the query time by 5.5% to 11%.
The benchmark queries follow the template below, where OP is plus/minus/times/divide/remainder/bitwise and/bitwise or/bitwise xor.

SELECT l_returnflag,  l_linestatus, SUM(l_quantity OP cnt1), SUM(l_quantity OP cnt2), ...., SUM(l_quantity OP cnt700)
FROM (
    SELECT l_returnflag, l_linestatus, l_quantity, 1 AS cnt1, 2 AS cnt2, ..., 700 AS cnt700
    FROM lineitem
    WHERE l_shipdate <= '1998-09-01'
)
GROUP BY l_returnflag,  l_linestatus;

Author: kai <kaizeng@eecs.berkeley.edu>

Closes #4472 from kai-zeng/arithmetic-optimize and squashes the following commits:

fef0cf1 [kai] Merge branch 'master' of github.com:apache/spark into arithmetic-optimize
4b3a1bb [kai] chmod a-x
5a41e49 [kai] chmod a-x Expression.scala
cb37c94 [kai] rebase onto spark master
7f6e968 [kai] chmod 100755 -> 100644
6cddb46 [kai] format
7490dbc [kai] fix unresolved-expression exception for EqualTo
9c40bc0 [kai] fix bitwisenot
3cbd363 [kai] clean up test code
ca47801 [kai] override evalInternal for bitwise ops
8fa84a1 [kai] add bitwise or and xor
6892fc4 [kai] revert override evalInternal
f8eba24 [kai] override evalInternal
31ccdd4 [kai] rewrite all bitwise op and remove evalInternal
86297e2 [kai] generalized
cb92ae1 [kai] bitwise-and: override eval
97a7d6c [kai] bitwise-and: override evalInternal using and func
0906c39 [kai] add bitwise test
62abbbc [kai] clean up predicate and arithmetic
b34d58d [kai] add caching and benmark option
12c5b32 [kai] override eval
1cd7571 [kai] fix sqrt and maxof
03fd0c3 [kai] fix predicate
16fd84c [kai] optimize + - * / % -(unary) abs < > <= >=
fd95823 [kai] remove unnecessary type checking
24d062f [kai] test suite
2015-02-16 15:58:05 -08:00
Michael Armbrust 104b2c4580 [SQL] Initial support for reporting location of error in sql string
Author: Michael Armbrust <michael@databricks.com>

Closes #4587 from marmbrus/position and squashes the following commits:

0810052 [Michael Armbrust] fix tests
395c019 [Michael Armbrust] Merge remote-tracking branch 'marmbrus/position' into position
e155dce [Michael Armbrust] more errors
f3efa51 [Michael Armbrust] Update AnalysisException.scala
d45ff60 [Michael Armbrust] [SQL] Initial support for reporting location of error in sql string
2015-02-16 12:32:56 -08:00
Daoyuan Wang 2cbb3e433a [SPARK-5642] [SQL] Apply column pruning on unused aggregation fields
select k from (select key k, max(value) v from src group by k) t

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #4415 from adrian-wang/groupprune and squashes the following commits:

5d2d8a3 [Daoyuan Wang] address Michael's comments
61f8ef7 [Daoyuan Wang] add a unit test
80ddcc6 [Daoyuan Wang] keep project
b69d385 [Daoyuan Wang] add a prune rule for grouping set
2015-02-13 13:48:39 -08:00
tianyi 1c8633f3fe [SPARK-3365][SQL]Wrong schema generated for List type
This PR fix the issue SPARK-3365.
The reason is Spark generated wrong schema for the type `List` in `ScalaReflection.scala`
for example:

the generated schema for type `Seq[String]` is:
```
{"name":"x","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}`
```

the generated schema for type `List[String]` is:
```
{"name":"x","type":{"type":"struct","fields":[]},"nullable":true,"metadata":{}}`
```

Author: tianyi <tianyi.asiainfo@gmail.com>

Closes #4581 from tianyi/SPARK-3365 and squashes the following commits:

a097e86 [tianyi] change the order of resolution in ScalaReflection.scala
2015-02-12 22:18:39 -08:00
Michael Armbrust aa4ca8b873 [SQL] Improve error messages
Author: Michael Armbrust <michael@databricks.com>
Author: wangfei <wangfei1@huawei.com>

Closes #4558 from marmbrus/errorMessages and squashes the following commits:

5e5ab50 [Michael Armbrust] Merge pull request #15 from scwf/errorMessages
fa38881 [wangfei] fix for grouping__id
f279a71 [wangfei] make right references for ScriptTransformation
d29fbde [Michael Armbrust] extra case
1a797b4 [Michael Armbrust] comments
d4e9015 [Michael Armbrust] add comment
af9e668 [Michael Armbrust] no braces
34eb3a4 [Michael Armbrust] more work
6197cd5 [Michael Armbrust] [SQL] Better error messages for analysis failures
2015-02-12 13:11:28 -08:00
Michael Armbrust a60d2b70ad [SPARK-5454] More robust handling of self joins
Also I fix a bunch of bad output in test cases.

Author: Michael Armbrust <michael@databricks.com>

Closes #4520 from marmbrus/selfJoin and squashes the following commits:

4f4a85c [Michael Armbrust] comments
49c8e26 [Michael Armbrust] fix tests
6fc38de [Michael Armbrust] fix style
55d64b3 [Michael Armbrust] fix dataframe selfjoins
2015-02-11 12:31:56 -08:00
Michael Armbrust 6195e2473b [SQL] Add an exception for analysis errors.
Also start from the bottom so we show the first error instead of the top error.

Author: Michael Armbrust <michael@databricks.com>

Closes #4439 from marmbrus/analysisException and squashes the following commits:

45862a0 [Michael Armbrust] fix hive test
a773bba [Michael Armbrust] Merge remote-tracking branch 'origin/master' into analysisException
f88079f [Michael Armbrust] update more cases
fede90a [Michael Armbrust] newline
fbf4bc3 [Michael Armbrust] move to sql
6235db4 [Michael Armbrust] [SQL] Add an exception for analysis errors.
2015-02-10 17:32:42 -08:00
Wenchen Fan 0ee53ebce9 [SPARK-2096][SQL] support dot notation on array of struct
~~The rule is simple: If you want `a.b` work, then `a` must be some level of nested array of struct(level 0 means just a StructType). And the result of `a.b` is same level of nested array of b-type.
An optimization is: the resolve chain looks like `Attribute -> GetItem -> GetField -> GetField ...`, so we could transmit the nested array information between `GetItem` and `GetField` to avoid repeated computation of `innerDataType` and `containsNullList` of that nested array.~~
marmbrus Could you take a look?

to evaluate `a.b`, if `a` is array of struct, then `a.b` means get field `b` on each element of `a`, and return a result of array.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #2405 from cloud-fan/nested-array-dot and squashes the following commits:

08a228a [Wenchen Fan] support dot notation on array of struct
2015-02-09 16:39:34 -08:00
Lu Yan 2a36292534 [SPARK-5614][SQL] Predicate pushdown through Generate.
Now in Catalyst's rules, predicates can not be pushed through "Generate" nodes. Further more, partition pruning in HiveTableScan can not be applied on those queries involves "Generate". This makes such queries very inefficient. In practice, it finds patterns like

```scala
Filter(predicate, Generate(generator, _, _, _, grandChild))
```

and splits the predicate into 2 parts by referencing the generated column from Generate node or not. And a new Filter will be created for those conjuncts can be pushed beneath Generate node. If nothing left for the original Filter, it will be removed.
For example, physical plan for query
```sql
select len, bk
from s_server lateral view explode(len_arr) len_table as len
where len > 5 and day = '20150102';
```
where 'day' is a partition column in metastore is like this in current version of Spark SQL:

> Project [len, bk]
>
> Filter ((len > "5") && "(day = "20150102")")
>
> Generate explode(len_arr), true, false
>
> HiveTableScan [bk, len_arr, day], (MetastoreRelation default, s_server, None), None

But theoretically the plan should be like this

> Project [len, bk]
>
> Filter (len > "5")
>
> Generate explode(len_arr), true, false
>
> HiveTableScan [bk, len_arr, day], (MetastoreRelation default, s_server, None), Some(day = "20150102")

Where partition pruning predicates can be pushed to HiveTableScan nodes.

Author: Lu Yan <luyan02@baidu.com>

Closes #4394 from ianluyan/ppd and squashes the following commits:

a67dce9 [Lu Yan] Fix English grammar.
7cea911 [Lu Yan] Revised based on @marmbrus's opinions
ffc59fc [Lu Yan] [SPARK-5614][SQL] Predicate pushdown through Generate.
2015-02-09 16:25:38 -08:00
Reynold Xin a052ed4250 [SPARK-5643][SQL] Add a show method to print the content of a DataFrame in tabular format.
An example:
```
year  month AVG('Adj Close) MAX('Adj Close)
1980  12    0.503218        0.595103
1981  01    0.523289        0.570307
1982  02    0.436504        0.475256
1983  03    0.410516        0.442194
1984  04    0.450090        0.483521
```

Author: Reynold Xin <rxin@databricks.com>

Closes #4416 from rxin/SPARK-5643 and squashes the following commits:

d0e0d6e [Reynold Xin] [SQL] Minor update to data source and statistics documentation.
269da83 [Reynold Xin] Updated isLocal comment.
2cf3c27 [Reynold Xin] Moved logic into optimizer.
1a04d8b [Reynold Xin] [SPARK-5643][SQL] Add a show method to print the content of a DataFrame in columnar format.
2015-02-08 18:56:51 -08:00
Wenchen Fan 4793c8402a [SPARK-5278][SQL] Introduce UnresolvedGetField and complete the check of ambiguous reference to fields
When the `GetField` chain(`a.b.c.d.....`) is interrupted by `GetItem` like `a.b[0].c.d....`, then the check of ambiguous reference to fields is broken.
The reason is that: for something like `a.b[0].c.d`, we first parse it to `GetField(GetField(GetItem(Unresolved("a.b"), 0), "c"), "d")`. Then in `LogicalPlan#resolve`, we resolve `"a.b"` and build a `GetField` chain from bottom(the relation). But for the 2 outer `GetFiled`, we have to resolve them in `Analyzer` or do it in `GetField` lazily, check data type of child, search needed field, etc. which is similar to what we have done in `LogicalPlan#resolve`.
So in this PR, the fix is just copy the same logic in `LogicalPlan#resolve` to `Analyzer`, which is simple and quick, but I do suggest introduce `UnresolvedGetFiled` like I explained in https://github.com/apache/spark/pull/2405.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #4068 from cloud-fan/simple and squashes the following commits:

a6857b5 [Wenchen Fan] fix import order
8411c40 [Wenchen Fan] use UnresolvedGetField
2015-02-06 13:08:09 -08:00
Yin Huai 0d81645f77 [SQL] Correct the default size of TimestampType and expose NumericType
Author: Yin Huai <yhuai@databricks.com>

Closes #4314 from yhuai/minor and squashes the following commits:

d3870a7 [Yin Huai] Update test.
6e4b0c0 [Yin Huai] Two minor changes.
2015-02-04 15:14:49 -08:00
Daoyuan Wang db821ed2ed [SPARK-4508] [SQL] build native date type to conform behavior to Hive
The previous #3732 is reverted due to some test failure.
Have fixed that.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4325 from adrian-wang/datenative and squashes the following commits:

096e20d [Daoyuan Wang] fix for mixed timezone
0ed0fdc [Daoyuan Wang] fix test data
a2fdd4e [Daoyuan Wang] getDate
c37832b [Daoyuan Wang] row to catalyst
f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion
024c9a6 [Daoyuan Wang] clean some import order
d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally
374abd5 [Daoyuan Wang] spark native date type support
2015-02-03 12:21:45 -08:00
Patrick Wendell eccb9fbb2d Revert "[SPARK-4508] [SQL] build native date type to conform behavior to Hive"
This reverts commit 1646f89d96.
2015-02-02 17:52:17 -08:00
Daoyuan Wang 1646f89d96 [SPARK-4508] [SQL] build native date type to conform behavior to Hive
Store daysSinceEpoch as an Int value(4 bytes) to represent DateType, instead of using java.sql.Date(8 bytes as Long) in catalyst row. This ensures the same comparison behavior of Hive and Catalyst.
Subsumes #3381
I thinks there are already some tests in JavaSQLSuite, and for python it will not affect python's datetime class.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #3732 from adrian-wang/datenative and squashes the following commits:

0ed0fdc [Daoyuan Wang] fix test data
a2fdd4e [Daoyuan Wang] getDate
c37832b [Daoyuan Wang] row to catalyst
f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion
024c9a6 [Daoyuan Wang] clean some import order
d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally
374abd5 [Daoyuan Wang] spark native date type support
2015-02-02 15:49:22 -08:00
Daoyuan Wang 8cf4a1f02e [SPARK-5262] [SPARK-5244] [SQL] add coalesce in SQLParser and widen types for parameters of coalesce
I'll add test case in #4040

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4057 from adrian-wang/coal and squashes the following commits:

4d0111a [Daoyuan Wang] address Yin's comments
c393e18 [Daoyuan Wang] fix rebase conflicts
e47c03a [Daoyuan Wang] add coalesce in parser
c74828d [Daoyuan Wang] cast types for coalesce
2015-02-01 18:51:38 -08:00
Joseph K. Bradley e643de42a7 [SPARK-5504] [sql] convertToCatalyst should support nested arrays
After the recent refactoring, convertToCatalyst in ScalaReflection does not recurse on Arrays. It should.

The test suite modification made the test fail before the fix in ScalaReflection.  The fix makes the test suite succeed.

CC: marmbrus

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #4295 from jkbradley/SPARK-5504 and squashes the following commits:

6b7276d [Joseph K. Bradley] Fixed issue in ScalaReflection.convertToCatalyst with Arrays with non-primitive types. Modified test suite so it failed before the fix and works after the fix.
2015-01-30 15:40:14 -08:00
Reynold Xin 80def9deb3 [SQL] Support df("*") to select all columns in a data frame.
This PR makes Star a trait, and provides two implementations: UnresolvedStar (used for *, tblName.*) and ResolvedStar (used for df("*")).

Author: Reynold Xin <rxin@databricks.com>

Closes #4283 from rxin/df-star and squashes the following commits:

c9cba3e [Reynold Xin] Removed mapFunction in UnresolvedStar.
1a3a1d7 [Reynold Xin] [SQL] Support df("*") to select all columns in a data frame.
2015-01-29 19:09:08 -08:00
Reynold Xin 5ad78f6205 [SQL] Various DataFrame DSL update.
1. Added foreach, foreachPartition, flatMap to DataFrame.
2. Added col() in dsl.
3. Support renaming columns in toDataFrame.
4. Support type inference on arrays (in addition to Seq).
5. Updated mllib to use the new DSL.

Author: Reynold Xin <rxin@databricks.com>

Closes #4260 from rxin/sql-dsl-update and squashes the following commits:

73466c1 [Reynold Xin] Fixed LogisticRegression. Also added better error message for resolve.
fab3ccc [Reynold Xin] Bug fix.
d31fcd2 [Reynold Xin] Style fix.
62608c4 [Reynold Xin] [SQL] Various DataFrame DSL update.
2015-01-29 00:01:10 -08:00
Cheng Hao 8361078efa [SPARK-5009] [SQL] Long keyword support in SQL Parsers
* The `SqlLexical.allCaseVersions` will cause `StackOverflowException` if the key word is too long, the patch will fix that by normalizing all of the keywords in `SqlLexical`.
* And make a unified SparkSQLParser for sharing the common code.

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

Closes #3926 from chenghao-intel/long_keyword and squashes the following commits:

686660f [Cheng Hao] Support Long Keyword and Refactor the SQLParsers
2015-01-21 13:05:56 -08:00
Reynold Xin d181c2a1fc [SPARK-5323][SQL] Remove Row's Seq inheritance.
Author: Reynold Xin <rxin@databricks.com>

Closes #4115 from rxin/row-seq and squashes the following commits:

e33abd8 [Reynold Xin] Fixed compilation error.
cceb650 [Reynold Xin] Python test fixes, and removal of WrapDynamic.
0334a52 [Reynold Xin] mkString.
9cdeb7d [Reynold Xin] Hive tests.
15681c2 [Reynold Xin] Fix more test cases.
ea9023a [Reynold Xin] Fixed a catalyst test.
c5e2cb5 [Reynold Xin] Minor patch up.
b9cab7c [Reynold Xin] [SPARK-5323][SQL] Remove Row's Seq inheritance.
2015-01-20 15:16:14 -08:00
Yin Huai bc20a52b34 [SPARK-5287][SQL] Add defaultSizeOf to every data type.
JIRA: https://issues.apache.org/jira/browse/SPARK-5287

This PR only add `defaultSizeOf` to data types and make those internal type classes `protected[sql]`. I will use another PR to cleanup the type hierarchy of data types.

Author: Yin Huai <yhuai@databricks.com>

Closes #4081 from yhuai/SPARK-5287 and squashes the following commits:

90cec75 [Yin Huai] Update unit test.
e1c600c [Yin Huai] Make internal classes protected[sql].
7eaba68 [Yin Huai] Add `defaultSize` method to data types.
fd425e0 [Yin Huai] Add all native types to NativeType.defaultSizeOf.
2015-01-20 13:26:36 -08:00
Cheng Lian 8140802786 [SQL][Minor] Refactors deeply nested FP style code in BooleanSimplification
This is a follow-up of #4090. The original deeply nested `reduceOption` code is hard to grasp.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4091 from liancheng/refactor-boolean-simplification and squashes the following commits:

cd8860b [Cheng Lian] Improves `compareConditions` to handle more subtle cases
1bf3258 [Cheng Lian] Avoids converting predicate sets to lists
e833ca4 [Cheng Lian] Refactors deeply nested FP style code
2015-01-20 11:20:14 -08:00
Reynold Xin debc031953 [SQL][minor] Add a log4j file for catalyst test.
Author: Reynold Xin <rxin@databricks.com>

Closes #4117 from rxin/catalyst-test-log4j and squashes the following commits:

8ad610b [Reynold Xin] [SQL][minor] Add a log4j file for catalyst test.
2015-01-20 00:55:25 -08:00
Reynold Xin 1727e0841c [SPARK-5279][SQL] Use java.math.BigDecimal as the exposed Decimal type.
Author: Reynold Xin <rxin@databricks.com>

Closes #4092 from rxin/bigdecimal and squashes the following commits:

27b08c9 [Reynold Xin] Fixed test.
10cb496 [Reynold Xin] [SPARK-5279][SQL] Use java.math.BigDecimal as the exposed Decimal type.
2015-01-18 11:01:42 -08:00
scwf ee1c1f3a04 [SPARK-4937][SQL] Adding optimization to simplify the And, Or condition in spark sql
Adding optimization to simplify the And/Or condition in spark sql.

There are two kinds of Optimization
1 Numeric condition optimization, such as:
a < 3 && a > 5 ---- False
a < 1 || a > 0 ---- True
a > 3 && a > 5 => a > 5
(a < 2 || b > 5) && a < 2 => a < 2

2 optimizing the some query from a cartesian product into equi-join, such as this sql (one of hive-testbench):
```
select
sum(l_extendedprice* (1 - l_discount)) as revenue
from
lineitem,
part
where
(
p_partkey = l_partkey
and p_brand = 'Brand#32'
and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')
and l_quantity >= 7 and l_quantity <= 7 + 10
and p_size between 1 and 5
and l_shipmode in ('AIR', 'AIR REG')
and l_shipinstruct = 'DELIVER IN PERSON'
)
or
(
p_partkey = l_partkey
and p_brand = 'Brand#35'
and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')
and l_quantity >= 15 and l_quantity <= 15 + 10
and p_size between 1 and 10
and l_shipmode in ('AIR', 'AIR REG')
and l_shipinstruct = 'DELIVER IN PERSON'
)
or
(
p_partkey = l_partkey
and p_brand = 'Brand#24'
and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')
and l_quantity >= 26 and l_quantity <= 26 + 10
and p_size between 1 and 15
and l_shipmode in ('AIR', 'AIR REG')
and l_shipinstruct = 'DELIVER IN PERSON'
)
```
It has a repeated expression in Or, so we can optimize it by ``` (a && b) || (a && c) = a && (b || c)```
Before optimization, this sql hang in my locally test, and the physical plan is:
![image](https://cloud.githubusercontent.com/assets/7018048/5539175/31cf38e8-8af9-11e4-95e3-336f9b3da4a4.png)

After optimization, this sql run successfully in 20+ seconds, and its physical plan is:
![image](https://cloud.githubusercontent.com/assets/7018048/5539176/39a558e0-8af9-11e4-912b-93de94b20075.png)

This PR focus on the second optimization and some simple ones of the first. For complex Numeric condition optimization, I will make a follow up PR.

Author: scwf <wangfei1@huawei.com>
Author: wangfei <wangfei1@huawei.com>

Closes #3778 from scwf/filter1 and squashes the following commits:

58bcbc2 [scwf] minor format fix
9570211 [scwf] conflicts fix
527e6ce [scwf] minor comment improvements
5c6f134 [scwf] remove numeric optimizations and move to BooleanSimplification
546a82b [wangfei] style fix
825fa69 [wangfei] adding more tests
a001e8c [wangfei] revert pom changes
32a595b [scwf] improvement and test fix
e99a26c [wangfei] refactory And/Or optimization to make it more readable and clean
2015-01-16 14:01:22 -08:00
Daoyuan Wang a3f7421b42 [SPARK-5248] [SQL] move sql.types.decimal.Decimal to sql.types.Decimal
rxin follow up of #3732

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4041 from adrian-wang/decimal and squashes the following commits:

aa3d738 [Daoyuan Wang] fix auto refactor
7777a58 [Daoyuan Wang] move sql.types.decimal.Decimal to sql.types.Decimal
2015-01-14 09:36:59 -08:00
Reynold Xin f9969098c8 [SPARK-5123][SQL] Reconcile Java/Scala API for data types.
Having two versions of the data type APIs (one for Java, one for Scala) requires downstream libraries to also have two versions of the APIs if the library wants to support both Java and Scala. I took a look at the Scala version of the data type APIs - it can actually work out pretty well for Java out of the box.

As part of the PR, I created a sql.types package and moved all type definitions there. I then removed the Java specific data type API along with a lot of the conversion code.

This subsumes https://github.com/apache/spark/pull/3925

Author: Reynold Xin <rxin@databricks.com>

Closes #3958 from rxin/SPARK-5123-datatype-2 and squashes the following commits:

66505cc [Reynold Xin] [SPARK-5123] Expose only one version of the data type APIs (i.e. remove the Java-specific API).
2015-01-13 17:16:41 -08:00
Alex Liu 4b39fd1e63 [SPARK-4943][SQL] Allow table name having dot for db/catalog
The pull only fixes the parsing error and changes API to use tableIdentifier. Joining different catalog datasource related change is not done in this pull.

Author: Alex Liu <alex_liu68@yahoo.com>

Closes #3941 from alexliu68/SPARK-SQL-4943-3 and squashes the following commits:

343ae27 [Alex Liu] [SPARK-4943][SQL] refactoring according to review
29e5e55 [Alex Liu] [SPARK-4943][SQL] fix failed Hive CTAS tests
6ae77ce [Alex Liu] [SPARK-4943][SQL] fix TestHive matching error
3652997 [Alex Liu] [SPARK-4943][SQL] Allow table name having dot to support db/catalog ...
2015-01-10 13:23:09 -08:00
Cheng Lian 61a99f6a11 [SPARK-4937][SQL] Normalizes conjunctions and disjunctions to eliminate common predicates
This PR is a simplified version of several filter optimization rules introduced in #3778 authored by scwf. Newly introduced optimizations include:

1. `a && a` => `a`
2. `a || a` => `a`
3. `(a || b || c || ...) && (a || b || d || ...)` => `a && b && (c || d || ...)`

The 3rd rule is particularly useful for optimizing the following query, which is planned into a cartesian product

```sql
SELECT *
  FROM t1, t2
 WHERE (t1.key = t2.key AND t1.value > 10)
    OR (t1.key = t2.key AND t2.value < 20)
```

to the following one, which is planned into an equi-join:

```sql
SELECT *
  FROM t1, t2
 WHERE t1.key = t2.key
   AND (t1.value > 10 OR t2.value < 20)
```

The example above is quite artificial, but common predicates are likely to appear in real life complex queries (like the one mentioned in #3778).

A difference between this PR and #3778 is that these optimizations are not limited to `Filter`, but are generalized to all logical plan nodes. Thanks to scwf for bringing up these optimizations, and chenghao-intel for the generalization suggestion.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3784 from liancheng/normalize-filters and squashes the following commits:

caca560 [Cheng Lian] Moves filter normalization into BooleanSimplification rule
4ab3a58 [Cheng Lian] Fixes test failure, adds more tests
5d54349 [Cheng Lian] Fixes typo in comment
2abbf8e [Cheng Lian] Forgot our sacred Apache licence header...
cf95639 [Cheng Lian] Adds an optimization rule for filter normalization
2014-12-30 13:38:27 -08:00