Commit graph

724 commits

Author SHA1 Message Date
Cheng Hao f728e0fe7e [SPARK-2663] [SQL] Support the Grouping Set
Add support for `GROUPING SETS`, `ROLLUP`, `CUBE` and the the virtual column `GROUPING__ID`.

More details on how to use the `GROUPING SETS" can be found at: https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation,+Cube,+Grouping+and+Rollup
https://issues.apache.org/jira/secure/attachment/12676811/grouping_set.pdf

The generic idea of the implementations are :
1 Replace the `ROLLUP`, `CUBE` with `GROUPING SETS`
2 Explode each of the input row, and then feed them to `Aggregate`
  * Each grouping set are represented as the bit mask for the `GroupBy Expression List`, for each bit, `1` means the expression is selected, otherwise `0` (left is the lower bit, and right is the higher bit in the `GroupBy Expression List`)
  * Several of projections are constructed according to the grouping sets, and within each projection(Seq[Expression), we replace those expressions with `Literal(null)` if it's not selected in the grouping set (based on the bit mask)
  * Output Schema of `Explode` is `child.output :+ grouping__id`
  * GroupBy Expressions of `Aggregate` is `GroupBy Expression List :+ grouping__id`
  * Keep the `Aggregation expressions` the same for the `Aggregate`

The expressions substitutions happen in Logic Plan analyzing, so we will benefit from the Logical Plan optimization (e.g. expression constant folding, and map side aggregation etc.), Only an `Explosive` operator added for Physical Plan, which will explode the rows according the pre-set projections.

A known issue will be done in the follow up PR:
* Optimization `ColumnPruning` is not supported yet for `Explosive` node.

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

Closes #1567 from chenghao-intel/grouping_sets and squashes the following commits:

fe65fcc [Cheng Hao] Remove the extra space
3547056 [Cheng Hao] Add more doc and Simplify the Expand
a7c869d [Cheng Hao] update code as feedbacks
d23c672 [Cheng Hao] Add GroupingExpression to replace the Seq[Expression]
414b165 [Cheng Hao] revert the unnecessary changes
ec276c6 [Cheng Hao] Support Rollup/Cube/GroupingSets
2014-12-18 18:58:29 -08:00
Venkata Ramana Gollamudi f33d550464 [SPARK-3891][SQL] Add array support to percentile, percentile_approx and constant inspectors support
Supported passing array to percentile and percentile_approx UDAFs
To support percentile_approx,  constant inspectors are supported for GenericUDAF
Constant folding support added to CreateArray expression
Avoided constant udf expression re-evaluation

Author: Venkata Ramana G <ramana.gollamudihuawei.com>

Author: Venkata Ramana Gollamudi <ramana.gollamudi@huawei.com>

Closes #2802 from gvramana/percentile_array_support and squashes the following commits:

a0182e5 [Venkata Ramana Gollamudi] fixed review comment
a18f917 [Venkata Ramana Gollamudi] avoid constant udf expression re-evaluation - fixes failure due to return iterator and value type mismatch
c46db0f [Venkata Ramana Gollamudi] Removed TestHive reset
4d39105 [Venkata Ramana Gollamudi] Unified inspector creation, style check fixes
f37fd69 [Venkata Ramana Gollamudi] Fixed review comments
47f6365 [Venkata Ramana Gollamudi] fixed test
cb7c61e [Venkata Ramana Gollamudi] Supported ConstantInspector for UDAF Fixed HiveUdaf wrap object issue.
7f94aff [Venkata Ramana Gollamudi] Added foldable support to CreateArray
2014-12-17 15:41:35 -08:00
Cheng Hao 8d0d2a65eb [SPARK-4856] [SQL] NullType instead of StringType when sampling against empty string or nul...
```
TestSQLContext.sparkContext.parallelize(
  """{"ip":"27.31.100.29","headers":{"Host":"1.abc.com","Charset":"UTF-8"}}""" ::
  """{"ip":"27.31.100.29","headers":{}}""" ::
  """{"ip":"27.31.100.29","headers":""}""" :: Nil)
```
As empty string (the "headers") will be considered as String in the beginning (in line 2 and 3), it ignores the real nested data type (struct type "headers" in line 1), and also take the line 1 (the "headers") as String Type, which is not our expected.

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

Closes #3708 from chenghao-intel/json and squashes the following commits:

e7a72e9 [Cheng Hao] add more concise unit test
853de51 [Cheng Hao] NullType instead of StringType when sampling against empty string or null value
2014-12-17 15:01:59 -08:00
Michael Armbrust 19c0faad6d [HOTFIX][SQL] Fix parquet filter suite
Author: Michael Armbrust <michael@databricks.com>

Closes #3727 from marmbrus/parquetNotEq and squashes the following commits:

2157bfc [Michael Armbrust] Fix parquet filter suite
2014-12-17 14:27:02 -08:00
Cheng Hao 636d9fc450 [SPARK-3739] [SQL] Update the split num base on block size for table scanning
In local mode, Hadoop/Hive will ignore the "mapred.map.tasks", hence for small table file, it's always a single input split, however, SparkSQL doesn't honor that in table scanning, and we will get different result when do the Hive Compatibility test. This PR will fix that.

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

Closes #2589 from chenghao-intel/source_split and squashes the following commits:

dff38e7 [Cheng Hao] Remove the extra blank line
160a2b6 [Cheng Hao] fix the compiling bug
04d67f7 [Cheng Hao] Keep 1 split for small file in table scanning
2014-12-17 13:39:36 -08:00
Daoyuan Wang 902e4d54ac [SPARK-4755] [SQL] sqrt(negative value) should return null
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #3616 from adrian-wang/sqrt and squashes the following commits:

d877439 [Daoyuan Wang] fix NULLTYPE
3effa2c [Daoyuan Wang] sqrt(negative value) should return null
2014-12-17 12:51:27 -08:00
Cheng Lian 6277135376 [SPARK-4493][SQL] Don't pushdown Eq, NotEq, Lt, LtEq, Gt and GtEq predicates with nulls for Parquet
Predicates like `a = NULL` and `a < NULL` can't be pushed down since Parquet `Lt`, `LtEq`, `Gt`, `GtEq` doesn't accept null value. Note that `Eq` and `NotEq` can only be used with `null` to represent predicates like `a IS NULL` and `a IS NOT NULL`.

However, normally this issue doesn't cause NPE because any value compared to `NULL` results `NULL`, and Spark SQL automatically optimizes out `NULL` predicate in the `SimplifyFilters` rule. Only testing code that intentionally disables the optimizer may trigger this issue. (That's why this issue is not marked as blocker and I do **NOT** think we need to backport this to branch-1.1

This PR restricts `Lt`, `LtEq`, `Gt` and `GtEq` to non-null values only, and only uses `Eq` with null value to pushdown `IsNull` and `IsNotNull`. Also, added support for Parquet `NotEq` filter for completeness and (tiny) performance gain, it's also used to pushdown `IsNotNull`.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3367 from liancheng/filters-with-null and squashes the following commits:

cc41281 [Cheng Lian] Fixes several styling issues
de7de28 [Cheng Lian] Adds stricter rules for Parquet filters with null
2014-12-17 12:48:04 -08:00
Michael Armbrust 7ad579ee97 [SPARK-3698][SQL] Fix case insensitive resolution of GetField.
Based on #2543.

Author: Michael Armbrust <michael@databricks.com>

Closes #3724 from marmbrus/resolveGetField and squashes the following commits:

0a47aae [Michael Armbrust] Fix case insensitive resolution of GetField.
2014-12-17 12:43:51 -08:00
carlmartin 4782def094 [SPARK-4694]Fix HiveThriftServer2 cann't stop In Yarn HA mode.
HiveThriftServer2 can not exit automactic when changing the standy resource manager in Yarn HA mode.
The scheduler backend was aware of the AM had been exited so it call sc.stop to exit the driver process but there was a user thread(HiveThriftServer2 ) which was still alive and cause this problem.
To fix it, make a demo thread to detect the sparkContext is null or not.If the sc is stopped, call the ThriftServer.stop to stop the user thread.

Author: carlmartin <carlmartinmax@gmail.com>

Closes #3576 from SaintBacchus/ThriftServer2ExitBug and squashes the following commits:

2890b4a [carlmartin] Use SparkListener instead of the demo thread to stop the hive server.
c15da0e [carlmartin] HiveThriftServer2 can not exit automactic when changing the standy resource manager in Yarn HA mode
2014-12-17 12:24:03 -08:00
Cheng Hao 5fdcbdc0c9 [SPARK-4625] [SQL] Add sort by for DSL & SimpleSqlParser
Add `sort by` support for both DSL & SqlParser.

This PR is relevant with #3386, either one merged, will cause the other rebased.

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

Closes #3481 from chenghao-intel/sortby and squashes the following commits:

041004f [Cheng Hao] Add sort by for DSL & SimpleSqlParser
2014-12-17 12:01:57 -08:00
scwf 60698801eb [SPARK-4618][SQL] Make foreign DDL commands options case-insensitive
Using lowercase for ```options``` key to make it case-insensitive, then we should use lower case to get value from parameters.
So flowing cmd work
```
      create temporary table normal_parquet
      USING org.apache.spark.sql.parquet
      OPTIONS (
        PATH '/xxx/data'
      )
```

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

Closes #3470 from scwf/ddl-ulcase and squashes the following commits:

ae78509 [scwf] address comments
8f4f585 [wangfei] address comments
3c132ef [scwf] minor fix
a0fc20b [scwf] Merge branch 'master' of https://github.com/apache/spark into ddl-ulcase
4f86401 [scwf] adding CaseInsensitiveMap
e244e8d [wangfei] using lower case in json
e0cb017 [wangfei] make options in-casesensitive
2014-12-16 21:26:36 -08:00
Davies Liu ec5c4279ed [SPARK-4866] support StructType as key in MapType
This PR brings support of using StructType(and other hashable types) as key in MapType.

Author: Davies Liu <davies@databricks.com>

Closes #3714 from davies/fix_struct_in_map and squashes the following commits:

68585d7 [Davies Liu] fix primitive types in MapType
9601534 [Davies Liu] support StructType as key in MapType
2014-12-16 21:23:28 -08:00
Cheng Hao 770d8153a5 [SPARK-4375] [SQL] Add 0 argument support for udf
Author: Cheng Hao <hao.cheng@intel.com>

Closes #3595 from chenghao-intel/udf0 and squashes the following commits:

a858973 [Cheng Hao] Add 0 arguments support for udf
2014-12-16 21:21:11 -08:00
Takuya UESHIN ddc7ba31cb [SPARK-4720][SQL] Remainder should also return null if the divider is 0.
This is a follow-up of SPARK-4593 (#3443).

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #3581 from ueshin/issues/SPARK-4720 and squashes the following commits:

c3959d4 [Takuya UESHIN] Make Remainder return null if the divider is 0.
2014-12-16 21:19:57 -08:00
Cheng Hao 0aa834adea [SPARK-4744] [SQL] Short circuit evaluation for AND & OR in CodeGen
Author: Cheng Hao <hao.cheng@intel.com>

Closes #3606 from chenghao-intel/codegen_short_circuit and squashes the following commits:

f466303 [Cheng Hao] short circuit for AND & OR
2014-12-16 21:18:39 -08:00
Cheng Lian 3b395e1051 [SPARK-4798][SQL] A new set of Parquet testing API and test suites
This PR provides a set Parquet testing API (see trait `ParquetTest`) that enables developers to write more concise test cases. A new set of Parquet test suites built upon this API  are added and aim to replace the old `ParquetQuerySuite`. To avoid potential merge conflicts, old testing code are not removed yet. The following classes can be safely removed after most Parquet related PRs are handled:

- `ParquetQuerySuite`
- `ParquetTestData`

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

Author: Cheng Lian <lian@databricks.com>

Closes #3644 from liancheng/parquet-tests and squashes the following commits:

800e745 [Cheng Lian] Enforces ordering of test output
3bb8731 [Cheng Lian] Refactors HiveParquetSuite
aa2cb2e [Cheng Lian] Decouples ParquetTest and TestSQLContext
7b43a68 [Cheng Lian] Updates ParquetTest Scaladoc
7f07af0 [Cheng Lian] Adds a new set of Parquet test suites
2014-12-16 21:16:03 -08:00
Jacky Li fa66ef6c97 [SPARK-4269][SQL] make wait time configurable in BroadcastHashJoin
In BroadcastHashJoin, currently it is using a hard coded value (5 minutes) to wait for the execution and broadcast of the small table.
In my opinion, it should be a configurable value since broadcast may exceed 5 minutes in some case, like in a busy/congested network environment.

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

Closes #3133 from jackylk/timeout-config and squashes the following commits:

733ac08 [Jacky Li] add spark.sql.broadcastTimeout in SQLConf.scala
557acd4 [Jacky Li] switch to sqlContext.getConf
81a5e20 [Jacky Li] make wait time configurable in BroadcastHashJoin
2014-12-16 15:34:59 -08:00
Michael Armbrust a66c23e134 [SPARK-4827][SQL] Fix resolution of deeply nested Project(attr, Project(Star,...)).
Since `AttributeReference` resolution and `*` expansion are currently in separate rules, each pair requires a full iteration instead of being able to resolve in a single pass.  Since its pretty easy to construct queries that have many of these in a row, I combine them into a single rule in this PR.

Author: Michael Armbrust <michael@databricks.com>

Closes #3674 from marmbrus/projectStars and squashes the following commits:

d83d6a1 [Michael Armbrust] Fix resolution of deeply nested Project(attr, Project(Star,...)).
2014-12-16 15:31:19 -08:00
tianyi 30f6b85c81 [SPARK-4483][SQL]Optimization about reduce memory costs during the HashOuterJoin
In `HashOuterJoin.scala`, spark read data from both side of join operation before zip them together. It is a waste for memory. We are trying to read data from only one side, put them into a hashmap, and then generate the `JoinedRow` with data from other side one by one.
Currently, we could only do this optimization for `left outer join` and `right outer join`. For `full outer join`, we will do something in another issue.

for
table test_csv contains 1 million records
table dim_csv contains 10 thousand records

SQL:
`select * from test_csv a left outer join dim_csv b on a.key = b.key`

the result is:
master:
```
CSV: 12671 ms
CSV: 9021 ms
CSV: 9200 ms
Current Mem Usage:787788984
```
after patch:
```
CSV: 10382 ms
CSV: 7543 ms
CSV: 7469 ms
Current Mem Usage:208145728
```

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

Closes #3375 from tianyi/SPARK-4483 and squashes the following commits:

72a8aec [tianyi] avoid having mutable state stored inside of the task
99c5c97 [tianyi] performance optimization
d2f94d7 [tianyi] fix bug: missing output when the join-key is null.
2be45d1 [tianyi] fix spell bug
1f2c6f1 [tianyi] remove commented codes
a676de6 [tianyi] optimize some codes
9e7d5b5 [tianyi] remove commented old codes
838707d [tianyi] Optimization about reduce memory costs during the HashOuterJoin
2014-12-16 15:22:29 -08:00
wangxiaojing ea1315e3e2 [SPARK-4527][SQl]Add BroadcastNestedLoopJoin operator selection testsuite
In `JoinSuite` add BroadcastNestedLoopJoin operator selection testsuite

Author: wangxiaojing <u9jing@gmail.com>

Closes #3395 from wangxiaojing/SPARK-4527 and squashes the following commits:

ea0e495 [wangxiaojing] change style
53c3952 [wangxiaojing] Add BroadcastNestedLoopJoin operator selection testsuite
2014-12-16 14:45:56 -08:00
zsxwing 6530243a52 [SPARK-4812][SQL] Fix the initialization issue of 'codegenEnabled'
The problem is `codegenEnabled` is `val`, but it uses a `val` `sqlContext`, which can be override by subclasses. Here is a simple example to show this issue.

```Scala
scala> :paste
// Entering paste mode (ctrl-D to finish)

abstract class Foo {

  protected val sqlContext = "Foo"

  val codegenEnabled: Boolean = {
    println(sqlContext) // it will call subclass's `sqlContext` which has not yet been initialized.
    if (sqlContext != null) {
      true
    } else {
      false
    }
  }
}

class Bar extends Foo {
  override val sqlContext = "Bar"
}

println(new Bar().codegenEnabled)

// Exiting paste mode, now interpreting.

null
false
defined class Foo
defined class Bar
```

We should make `sqlContext` `final` to prevent subclasses from overriding it incorrectly.

Author: zsxwing <zsxwing@gmail.com>

Closes #3660 from zsxwing/SPARK-4812 and squashes the following commits:

1cbb623 [zsxwing] Make `sqlContext` final to prevent subclasses from overriding it incorrectly
2014-12-16 14:13:40 -08:00
jerryshao dc8280dcca [SPARK-4847][SQL]Fix "extraStrategies cannot take effect in SQLContext" issue
Author: jerryshao <saisai.shao@intel.com>

Closes #3698 from jerryshao/SPARK-4847 and squashes the following commits:

4741130 [jerryshao] Make later added extraStrategies effect when calling strategies
2014-12-16 14:08:28 -08:00
Judy Nash 17688d1429 [SQL] SPARK-4700: Add HTTP protocol spark thrift server
Add HTTP protocol support and test cases to spark thrift server, so users can deploy thrift server in both TCP and http mode.

Author: Judy Nash <judynash@microsoft.com>
Author: judynash <judynash@microsoft.com>

Closes #3672 from judynash/master and squashes the following commits:

526315d [Judy Nash] correct spacing on startThriftServer method
31a6520 [Judy Nash] fix code style issues and update sql programming guide format issue
47bf87e [Judy Nash] modify withJdbcStatement method definition to meet less than 100 line length
2e9c11c [Judy Nash] add thrift server in http mode documentation on sql programming guide
1cbd305 [Judy Nash] Merge remote-tracking branch 'upstream/master'
2b1d312 [Judy Nash] updated http thrift server support based on feedback
377532c [judynash] add HTTP protocol spark thrift server
2014-12-16 12:37:26 -08:00
Sean Owen 81112e4b57 SPARK-4814 [CORE] Enable assertions in SBT, Maven tests / AssertionError from Hive's LazyBinaryInteger
This enables assertions for the Maven and SBT build, but overrides the Hive module to not enable assertions.

Author: Sean Owen <sowen@cloudera.com>

Closes #3692 from srowen/SPARK-4814 and squashes the following commits:

caca704 [Sean Owen] Disable assertions just for Hive
f71e783 [Sean Owen] Enable assertions for SBT and Maven build
2014-12-15 17:12:05 -08:00
Daoyuan Wang 41a3f93438 [SPARK-4829] [SQL] add rule to fold count(expr) if expr is not null
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #3676 from adrian-wang/countexpr and squashes the following commits:

dc5765b [Daoyuan Wang] add rule to fold count(expr) if expr is not null
2014-12-11 22:56:42 -08:00
Sasaki Toru 8091dd62ea [SPARK-4742][SQL] The name of Parquet File generated by AppendingParquetOutputFormat should be zero padded
When I use Parquet File as a output file using ParquetOutputFormat#getDefaultWorkFile, the file name is not zero padded while RDD#saveAsText does zero padding.

Author: Sasaki Toru <sasakitoa@nttdata.co.jp>

Closes #3602 from sasakitoa/parquet-zeroPadding and squashes the following commits:

6b0e58f [Sasaki Toru] Merge branch 'master' of git://github.com/apache/spark into parquet-zeroPadding
20dc79d [Sasaki Toru] Fixed the name of Parquet File generated by AppendingParquetOutputFormat
2014-12-11 22:54:21 -08:00
Cheng Hao 0abbff2862 [SPARK-4825] [SQL] CTAS fails to resolve when created using saveAsTable
Fix bug when query like:
```
  test("save join to table") {
    val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString))
    sql("CREATE TABLE test1 (key INT, value STRING)")
    testData.insertInto("test1")
    sql("CREATE TABLE test2 (key INT, value STRING)")
    testData.insertInto("test2")
    testData.insertInto("test2")
    sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").saveAsTable("test")
    checkAnswer(
      table("test"),
      sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq)
  }
```

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

Closes #3673 from chenghao-intel/spark_4825 and squashes the following commits:

e8cbd56 [Cheng Hao] alternate the pattern matching order for logical plan:CTAS
e004895 [Cheng Hao] fix bug
2014-12-11 22:51:49 -08:00
Daoyuan Wang cbb634ae69 [SQL] enable empty aggr test case
This is fixed by SPARK-4318 #3184

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

Closes #3445 from adrian-wang/emptyaggr and squashes the following commits:

982575e [Daoyuan Wang] enable empty aggr test case
2014-12-11 22:50:18 -08:00
Daoyuan Wang acb3be6bc5 [SPARK-4828] [SQL] sum and avg on empty table should always return null
So the optimizations are not valid. Also I think the optimization here is rarely encounter, so removing them will not have influence on performance.

Can we merge #3445 before I add a comparison test case from this?

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

Closes #3675 from adrian-wang/sumempty and squashes the following commits:

42df763 [Daoyuan Wang] sum and avg on empty table should always return null
2014-12-11 22:49:27 -08:00
scwf d8cf678589 [SQL] Remove unnecessary case in HiveContext.toHiveString
a follow up of #3547
/cc marmbrus

Author: scwf <wangfei1@huawei.com>

Closes #3563 from scwf/rnc and squashes the following commits:

9395661 [scwf] remove unnecessary condition
2014-12-11 22:48:03 -08:00
Takuya UESHIN 334480362b [SPARK-4293][SQL] Make Cast be able to handle complex types.
Inserting data of type including `ArrayType.containsNull == false` or `MapType.valueContainsNull == false` or `StructType.fields.exists(_.nullable == false)` into Hive table will fail because `Cast` inserted by `HiveMetastoreCatalog.PreInsertionCasts` rule of `Analyzer` can't handle these types correctly.

Complex type cast rule proposal:

- Cast for non-complex types should be able to cast the same as before.
- Cast for `ArrayType` can evaluate if
  - Element type can cast
  - Nullability rule doesn't break
- Cast for `MapType` can evaluate if
  - Key type can cast
  - Nullability for casted key type is `false`
  - Value type can cast
  - Nullability rule for value type doesn't break
- Cast for `StructType` can evaluate if
  - The field size is the same
  - Each field can cast
  - Nullability rule for each field doesn't break
- The nested structure should be the same.

Nullability rule:

- If the casted type is `nullable == true`, the target nullability should be `true`

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #3150 from ueshin/issues/SPARK-4293 and squashes the following commits:

e935939 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4293
ba14003 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4293
8999868 [Takuya UESHIN] Fix a test title.
f677c30 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4293
287f410 [Takuya UESHIN] Add tests to insert data of types ArrayType / MapType / StructType with nullability is false into Hive table.
4f71bb8 [Takuya UESHIN] Make Cast be able to handle complex types.
2014-12-11 22:45:25 -08:00
Jacky Li c152dde78f [SPARK-4639] [SQL] Pass maxIterations in as a parameter in Analyzer
fix a TODO in Analyzer:
// TODO: pass this in as a parameter
val fixedPoint = FixedPoint(100)

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

Closes #3499 from jackylk/config and squashes the following commits:

4c1252c [Jacky Li] fix scalastyle
820f460 [Jacky Li] pass maxIterations in as a parameter
2014-12-11 22:44:27 -08:00
Cheng Hao a7f07f511c [SPARK-4662] [SQL] Whitelist more unittest
Whitelist more hive unit test:

"create_like_tbl_props"
"udf5"
"udf_java_method"
"decimal_1"
"udf_pmod"
"udf_to_double"
"udf_to_float"
"udf7" (this will fail in Hive 0.12)

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

Closes #3522 from chenghao-intel/unittest and squashes the following commits:

f54e4c7 [Cheng Hao] work around to clean up the hive.table.parameters.default in reset
16fee22 [Cheng Hao] Whitelist more unittest
2014-12-11 22:43:02 -08:00
Cheng Hao bf40cf89e3 [SPARK-4713] [SQL] SchemaRDD.unpersist() should not raise exception if it is not persisted
Unpersist a uncached RDD, will not raise exception, for example:
```
val data = Array(1, 2, 3, 4, 5)
val distData = sc.parallelize(data)
distData.unpersist(true)
```

But the `SchemaRDD` will raise exception if the `SchemaRDD` is not cached. Since `SchemaRDD` is the subclasses of the `RDD`, we should follow the same behavior.

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

Closes #3572 from chenghao-intel/try_uncache and squashes the following commits:

50a7a89 [Cheng Hao] SchemaRDD.unpersist() should not raise exception if it is not persisted
2014-12-11 22:41:36 -08:00
Joseph K. Bradley 2a5b5fd4cc [SPARK-4791] [sql] Infer schema from case class with multiple constructors
Modified ScalaReflection.schemaFor to take primary constructor of Product when there are multiple constructors.  Added test to suite which failed before but works now.

Needed for [https://github.com/apache/spark/pull/3637]

CC: marmbrus

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

Closes #3646 from jkbradley/sql-reflection and squashes the following commits:

796b2e4 [Joseph K. Bradley] Modified ScalaReflection.schemaFor to take primary constructor of Product when there are multiple constructors.  Added test to suite which failed before but works now.
2014-12-10 23:41:15 -08:00
Cheng Hao 383c5555c9 [SPARK-4785][SQL] Initilize Hive UDFs on the driver and serialize them with a wrapper
Different from Hive 0.12.0, in Hive 0.13.1 UDF/UDAF/UDTF (aka Hive function) objects should only be initialized once on the driver side and then serialized to executors. However, not all function objects are serializable (e.g. GenericUDF doesn't implement Serializable). Hive 0.13.1 solves this issue with Kryo or XML serializer. Several utility ser/de methods are provided in class o.a.h.h.q.e.Utilities for this purpose. In this PR we chose Kryo for efficiency. The Kryo serializer used here is created in Hive. Spark Kryo serializer wasn't used because there's no available SparkConf instance.

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

Closes #3640 from chenghao-intel/udf_serde and squashes the following commits:

8e13756 [Cheng Hao] Update the comment
74466a3 [Cheng Hao] refactor as feedbacks
396c0e1 [Cheng Hao] avoid Simple UDF to be serialized
e9c3212 [Cheng Hao] update the comment
19cbd46 [Cheng Hao] support udf instance ser/de after initialization
2014-12-09 10:28:33 -08:00
Cheng Hao 51b1fe1426 [SPARK-4769] [SQL] CTAS does not work when reading from temporary tables
This is the code refactor and follow ups for #2570

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

Closes #3336 from chenghao-intel/createtbl and squashes the following commits:

3563142 [Cheng Hao] remove the unused variable
e215187 [Cheng Hao] eliminate the compiling warning
4f97f14 [Cheng Hao] fix bug in unittest
5d58812 [Cheng Hao] revert the API changes
b85b620 [Cheng Hao] fix the regression of temp tabl not found in CTAS
2014-12-08 17:39:12 -08:00
Jacky Li 944384363d [SQL] remove unnecessary import in spark-sql
Author: Jacky Li <jacky.likun@huawei.com>

Closes #3630 from jackylk/remove and squashes the following commits:

150e7e0 [Jacky Li] remove unnecessary import
2014-12-08 17:27:46 -08:00
Cheng Lian 6f61e1f961 [SPARK-4761][SQL] Enables Kryo by default in Spark SQL Thrift server
Enables Kryo and disables reference tracking by default in Spark SQL Thrift server. Configurations explicitly defined by users in `spark-defaults.conf` are respected (the Thrift server is started by `spark-submit`, which handles configuration properties properly).

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

Author: Cheng Lian <lian@databricks.com>

Closes #3621 from liancheng/kryo-by-default and squashes the following commits:

70c2775 [Cheng Lian] Enables Kryo by default in Spark SQL Thrift server
2014-12-05 10:27:40 -08:00
Michael Armbrust f5801e813f [SPARK-4753][SQL] Use catalyst for partition pruning in newParquet.
Author: Michael Armbrust <michael@databricks.com>

Closes #3613 from marmbrus/parquetPartitionPruning and squashes the following commits:

4f138f8 [Michael Armbrust] Use catalyst for partition pruning in newParquet.
2014-12-04 22:25:21 -08:00
Aaron Davidson c6c7165e7e [SQL] Minor: Avoid calling Seq#size in a loop
Just found this instance while doing some jstack-based profiling of a Spark SQL job. It is very unlikely that this is causing much of a perf issue anywhere, but it is unnecessarily suboptimal.

Author: Aaron Davidson <aaron@databricks.com>

Closes #3593 from aarondav/seq-opt and squashes the following commits:

962cdfc [Aaron Davidson] [SQL] Minor: Avoid calling Seq#size in a loop
2014-12-04 00:58:42 -08:00
Jacky Li ed88db4cb2 [SQL] remove unnecessary import
Author: Jacky Li <jacky.likun@huawei.com>

Closes #3585 from jackylk/remove and squashes the following commits:

045423d [Jacky Li] remove unnecessary import
2014-12-04 00:43:55 -08:00
Michael Armbrust 513ef82e85 [SPARK-4552][SQL] Avoid exception when reading empty parquet data through Hive
This is a very small fix that catches one specific exception and returns an empty table.  #3441 will address this in a more principled way.

Author: Michael Armbrust <michael@databricks.com>

Closes #3586 from marmbrus/fixEmptyParquet and squashes the following commits:

2781d9f [Michael Armbrust] Handle empty lists for newParquet
04dd376 [Michael Armbrust] Avoid exception when reading empty parquet data through Hive
2014-12-03 14:13:35 -08:00
wangfei 3ae0cda83c [SPARK-4695][SQL] Get result using executeCollect
Using ```executeCollect``` to collect the result, because executeCollect is a custom implementation of collect in spark sql which better than rdd's collect

Author: wangfei <wangfei1@huawei.com>

Closes #3547 from scwf/executeCollect and squashes the following commits:

a5ab68e [wangfei] Revert "adding debug info"
a60d680 [wangfei] fix test failure
0db7ce8 [wangfei] adding debug info
184c594 [wangfei] using executeCollect instead collect
2014-12-02 14:30:44 -08:00
Daoyuan Wang 1f5ddf17e8 [SPARK-4670] [SQL] wrong symbol for bitwise not
We should use `~` instead of `-` for bitwise NOT.

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

Closes #3528 from adrian-wang/symbol and squashes the following commits:

affd4ad [Daoyuan Wang] fix code gen test case
56efb79 [Daoyuan Wang] ensure bitwise NOT over byte and short persist data type
f55fbae [Daoyuan Wang] wrong symbol for bitwise not
2014-12-02 14:25:12 -08:00
Daoyuan Wang f6df609dcc [SPARK-4593][SQL] Return null when denominator is 0
SELECT max(1/0) FROM src
would return a very large number, which is obviously not right.
For hive-0.12, hive would return `Infinity` for 1/0, while for hive-0.13.1, it is `NULL` for 1/0.
I think it is better to keep our behavior with newer Hive version.
This PR ensures that when the divider is 0, the result of expression should be NULL, same with hive-0.13.1

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

Closes #3443 from adrian-wang/div and squashes the following commits:

2e98677 [Daoyuan Wang] fix code gen for divide 0
85c28ba [Daoyuan Wang] temp
36236a5 [Daoyuan Wang] add test cases
6f5716f [Daoyuan Wang] fix comments
cee92bd [Daoyuan Wang] avoid evaluation 2 times
22ecd9a [Daoyuan Wang] fix style
cf28c58 [Daoyuan Wang] divide fix
2dfe50f [Daoyuan Wang] return null when divider is 0 of Double type
2014-12-02 14:21:47 -08:00
YanTangZhai 1066427600 [SPARK-4676][SQL] JavaSchemaRDD.schema may throw NullType MatchError if sql has null
val jsc = new org.apache.spark.api.java.JavaSparkContext(sc)
val jhc = new org.apache.spark.sql.hive.api.java.JavaHiveContext(jsc)
val nrdd = jhc.hql("select null from spark_test.for_test")
println(nrdd.schema)
Then the error is thrown as follows:
scala.MatchError: NullType (of class org.apache.spark.sql.catalyst.types.NullType$)
at org.apache.spark.sql.types.util.DataTypeConversions$.asJavaDataType(DataTypeConversions.scala:43)

Author: YanTangZhai <hakeemzhai@tencent.com>
Author: yantangzhai <tyz0303@163.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #3538 from YanTangZhai/MatchNullType and squashes the following commits:

e052dff [yantangzhai] [SPARK-4676] [SQL] JavaSchemaRDD.schema may throw NullType MatchError if sql has null
4b4bb34 [yantangzhai] [SPARK-4676] [SQL] JavaSchemaRDD.schema may throw NullType MatchError if sql has null
896c7b7 [yantangzhai] fix NullType MatchError in JavaSchemaRDD when sql has null
6e643f8 [YanTangZhai] Merge pull request #11 from apache/master
e249846 [YanTangZhai] Merge pull request #10 from apache/master
d26d982 [YanTangZhai] Merge pull request #9 from apache/master
76d4027 [YanTangZhai] Merge pull request #8 from apache/master
03b62b0 [YanTangZhai] Merge pull request #7 from apache/master
8a00106 [YanTangZhai] Merge pull request #6 from apache/master
cbcba66 [YanTangZhai] Merge pull request #3 from apache/master
cdef539 [YanTangZhai] Merge pull request #1 from apache/master
2014-12-02 14:15:12 -08:00
baishuo 69b6fed206 [SPARK-4663][sql]add finally to avoid resource leak
Author: baishuo <vc_java@hotmail.com>

Closes #3526 from baishuo/master-trycatch and squashes the following commits:

d446e14 [baishuo] correct the code style
b36bf96 [baishuo] correct the code style
ae0e447 [baishuo] add finally to avoid resource leak
2014-12-02 12:12:03 -08:00
Kousuke Saruta e75e04f980 [SPARK-4536][SQL] Add sqrt and abs to Spark SQL DSL
Spark SQL has embeded sqrt and abs but DSL doesn't support those functions.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #3401 from sarutak/dsl-missing-operator and squashes the following commits:

07700cf [Kousuke Saruta] Modified Literal(null, NullType) to Literal(null) in DslQuerySuite
8f366f8 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into dsl-missing-operator
1b88e2e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into dsl-missing-operator
0396f89 [Kousuke Saruta] Added sqrt and abs to Spark SQL DSL
2014-12-02 12:07:52 -08:00
Reynold Xin b1f8fe316a Indent license header properly for interfaces.scala.
A very small nit update.

Author: Reynold Xin <rxin@databricks.com>

Closes #3552 from rxin/license-header and squashes the following commits:

df8d1a4 [Reynold Xin] Indent license header properly for interfaces.scala.
2014-12-02 11:59:15 -08:00
zsxwing d3e02dddf0 [SPARK-4268][SQL] Use #::: to get benefit from Stream in SqlLexical.allCaseVersions
In addition, using `s.isEmpty` to eliminate the string comparison.

Author: zsxwing <zsxwing@gmail.com>

Closes #3132 from zsxwing/SPARK-4268 and squashes the following commits:

358e235 [zsxwing] Improvement of allCaseVersions
2014-12-01 16:39:54 -08:00
Daoyuan Wang 4df60a8cbc [SPARK-4529] [SQL] support view with column alias
Support view definition like

CREATE VIEW view3(valoo)
TBLPROPERTIES ("fear" = "factor")
AS SELECT upper(value) FROM src WHERE key=86;

[valoo as the alias of upper(value)]. This is missing part of SPARK-4239, for a fully view support.

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

Closes #3396 from adrian-wang/viewcolumn and squashes the following commits:

4d001d0 [Daoyuan Wang] support view with column alias
2014-12-01 16:08:51 -08:00
wangfei 7b79957879 [SQL] Minor fix for doc and comment
Author: wangfei <wangfei1@huawei.com>

Closes #3533 from scwf/sql-doc1 and squashes the following commits:

962910b [wangfei] doc and comment fix
2014-12-01 14:02:02 -08:00
ravipesala bc353819cc [SPARK-4658][SQL] Code documentation issue in DDL of datasource API
Author: ravipesala <ravindra.pesala@huawei.com>

Closes #3516 from ravipesala/ddl_doc and squashes the following commits:

d101fdf [ravipesala] Style issues fixed
d2238cd [ravipesala] Corrected documentation
2014-12-01 13:31:27 -08:00
ravipesala 6a9ff19dc0 [SPARK-4650][SQL] Supporting multi column support in countDistinct function like count(distinct c1,c2..) in Spark SQL
Supporting multi column support in countDistinct function like count(distinct c1,c2..) in Spark SQL

Author: ravipesala <ravindra.pesala@huawei.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #3511 from ravipesala/countdistinct and squashes the following commits:

cc4dbb1 [ravipesala] style
070e12a [ravipesala] Supporting multi column support in count(distinct c1,c2..) in Spark SQL
2014-12-01 13:28:04 -08:00
Liang-Chi Hsieh b57365a1ec [SPARK-4358][SQL] Let BigDecimal do checking type compatibility
Remove hardcoding max and min values for types. Let BigDecimal do checking type compatibility.

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

Closes #3208 from viirya/more_numericLit and squashes the following commits:

e9834b4 [Liang-Chi Hsieh] Remove byte and short types for number literal.
1bd1825 [Liang-Chi Hsieh] Fix Indentation and make the modification clearer.
cf1a997 [Liang-Chi Hsieh] Modified for comment to add a rule of analysis that adds a cast.
91fe489 [Liang-Chi Hsieh] add Byte and Short.
1bdc69d [Liang-Chi Hsieh] Let BigDecimal do checking type compatibility.
2014-12-01 13:17:56 -08:00
Jacky Li bafee67eba [SQL] add @group tab in limit() and count()
group tab is missing for scaladoc

Author: Jacky Li <jacky.likun@gmail.com>

Closes #3458 from jackylk/patch-7 and squashes the following commits:

0121a70 [Jacky Li] add @group tab in limit() and count()
2014-12-01 13:12:30 -08:00
zsxwing 30a86acdef [SPARK-4661][Core] Minor code and docs cleanup
Author: zsxwing <zsxwing@gmail.com>

Closes #3521 from zsxwing/SPARK-4661 and squashes the following commits:

03cbe3f [zsxwing] Minor code and docs cleanup
2014-12-01 00:35:01 -08:00
Cheng Lian 5b99bf243e [SPARK-4645][SQL] Disables asynchronous execution in Hive 0.13.1 HiveThriftServer2
This PR disables HiveThriftServer2 asynchronous execution by setting `runInBackground` argument in `ExecuteStatementOperation` to `false`, and reverting `SparkExecuteStatementOperation.run` in Hive 13 shim to Hive 12 version. This change makes Simba ODBC driver v1.0.0.1000 work.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3506 from liancheng/disable-async-exec and squashes the following commits:

593804d [Cheng Lian] Disables asynchronous execution in Hive 0.13.1 HiveThriftServer2
2014-11-28 11:42:40 -05:00
w00228970 723be60e23 [SQL] Compute timeTaken correctly
```timeTaken``` should not count the time of printing result.

Author: w00228970 <wangfei1@huawei.com>

Closes #3423 from scwf/time-taken-bug and squashes the following commits:

da7e102 [w00228970] compute time taken correctly
2014-11-24 21:17:24 -08:00
Davies Liu 6cf507685e [SPARK-4548] []SPARK-4517] improve performance of python broadcast
Re-implement the Python broadcast using file:

1) serialize the python object using cPickle, write into disks.
2) Create a wrapper in JVM (for the dumped file), it read data from during serialization
3) Using TorrentBroadcast or HttpBroadcast to transfer the data (compressed) into executors
4) During deserialization, writing the data into disk.
5) Passing the path into Python worker, read data from disk and unpickle it into python object, until the first access.

It fixes the performance regression introduced in #2659, has similar performance as 1.1, but support object larger than 2G, also improve the memory efficiency (only one compressed copy in driver and executor).

Testing with a 500M broadcast and 4 tasks (excluding the benefit from reused worker in 1.2):

         name |   1.1   | 1.2 with this patch |  improvement
---------|--------|---------|--------
      python-broadcast-w-bytes  |	25.20  |	9.33   |	170.13% |
        python-broadcast-w-set	  |     4.13	   |    4.50  |	-8.35%  |

Testing with 100 tasks (16 CPUs):

         name |   1.1   | 1.2 with this patch |  improvement
---------|--------|---------|--------
     python-broadcast-w-bytes	| 38.16	| 8.40	 | 353.98%
        python-broadcast-w-set	| 23.29	| 9.59 |	142.80%

Author: Davies Liu <davies@databricks.com>

Closes #3417 from davies/pybroadcast and squashes the following commits:

50a58e0 [Davies Liu] address comments
b98de1d [Davies Liu] disable gc while unpickle
e5ee6b9 [Davies Liu] support large string
09303b8 [Davies Liu] read all data into memory
dde02dd [Davies Liu] improve performance of python broadcast
2014-11-24 17:17:03 -08:00
Kousuke Saruta dd1c9cb36c [SPARK-4487][SQL] Fix attribute reference resolution error when using ORDER BY.
When we use ORDER BY clause, at first, attributes referenced by projection are resolved (1).
And then, attributes referenced at ORDER BY clause are resolved (2).
 But when resolving attributes referenced at ORDER BY clause, the resolution result generated in (1) is discarded so for example, following query fails.

    SELECT c1 + c2 FROM mytable ORDER BY c1;

The query above fails because when resolving the attribute reference 'c1', the resolution result of 'c2' is discarded.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #3363 from sarutak/SPARK-4487 and squashes the following commits:

fd314f3 [Kousuke Saruta] Fixed attribute resolution logic in Analyzer
6e60c20 [Kousuke Saruta] Fixed conflicts
cb5b7e9 [Kousuke Saruta] Added test case for SPARK-4487
282d529 [Kousuke Saruta] Fixed attributes reference resolution error
b6123e6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into concat-feature
317b7fb [Kousuke Saruta] WIP
2014-11-24 12:54:37 -08:00
Daniel Darabos d5834f0732 [SQL] Fix comment in HiveShim
This file is for Hive 0.13.1 I think.

Author: Daniel Darabos <darabos.daniel@gmail.com>

Closes #3432 from darabos/patch-2 and squashes the following commits:

4fd22ed [Daniel Darabos] Fix comment. This file is for Hive 0.13.1.
2014-11-24 12:45:12 -08:00
Cheng Lian a6d7b61f92 [SPARK-4479][SQL] Avoids unnecessary defensive copies when sort based shuffle is on
This PR is a workaround for SPARK-4479. Two changes are introduced: when merge sort is bypassed in `ExternalSorter`,

1. also bypass RDD elements buffering as buffering is the reason that `MutableRow` backed row objects must be copied, and
2. avoids defensive copies in `Exchange` operator

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

Author: Cheng Lian <lian@databricks.com>

Closes #3422 from liancheng/avoids-defensive-copies and squashes the following commits:

591f2e9 [Cheng Lian] Passes all shuffle suites
0c3c91e [Cheng Lian] Fixes shuffle write metrics when merge sort is bypassed
ed5df3c [Cheng Lian] Fixes styling changes
f75089b [Cheng Lian] Avoids unnecessary defensive copies when sort based shuffle is on
2014-11-24 12:43:45 -08:00
Michael Armbrust 90a6a46bd1 [SPARK-4522][SQL] Parse schema with missing metadata.
This is just a quick fix for 1.2.  SPARK-4523 describes a more complete solution.

Author: Michael Armbrust <michael@databricks.com>

Closes #3392 from marmbrus/parquetMetadata and squashes the following commits:

bcc6626 [Michael Armbrust] Parse schema with missing metadata.
2014-11-20 20:34:43 -08:00
Michael Armbrust 02ec058efe [SPARK-4413][SQL] Parquet support through datasource API
Goals:
 - Support for accessing parquet using SQL but not requiring Hive (thus allowing support of parquet tables with decimal columns)
 - Support for folder based partitioning with automatic discovery of available partitions
 - Caching of file metadata

See scaladoc of `ParquetRelation2` for more details.

Author: Michael Armbrust <michael@databricks.com>

Closes #3269 from marmbrus/newParquet and squashes the following commits:

1dd75f1 [Michael Armbrust] Pass all paths for FileInputFormat at once.
645768b [Michael Armbrust] Review comments.
abd8e2f [Michael Armbrust] Alternative implementation of parquet based on the datasources API.
938019e [Michael Armbrust] Add an experimental interface to data sources that exposes catalyst expressions.
e9d2641 [Michael Armbrust] logging / formatting improvements.
2014-11-20 18:31:02 -08:00
Cheng Hao 84d79ee9ec [SPARK-4244] [SQL] Support Hive Generic UDFs with constant object inspector parameters
Query `SELECT named_struct(lower("AA"), "12", lower("Bb"), "13") FROM src LIMIT 1` will throw exception, some of the Hive Generic UDF/UDAF requires the input object inspector is `ConstantObjectInspector`, however, we won't get that before the expression optimization executed. (Constant Folding).

This PR is a work around to fix this. (As ideally, the `output` of LogicalPlan should be identical before and after Optimization).

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

Closes #3109 from chenghao-intel/optimized and squashes the following commits:

487ff79 [Cheng Hao] rebase to the latest master & update the unittest
2014-11-20 16:50:59 -08:00
Jacky Li ad5f1f3ca2 [SQL] fix function description mistake
Sample code in the description of SchemaRDD.where is not correct

Author: Jacky Li <jacky.likun@gmail.com>

Closes #3344 from jackylk/patch-6 and squashes the following commits:

62cd126 [Jacky Li] [SQL] fix function description mistake
2014-11-20 15:48:36 -08:00
Cheng Hao 6aa0fc9f4d [SPARK-2918] [SQL] Support the CTAS in EXPLAIN command
Hive supports the `explain` the CTAS, which was supported by Spark SQL previously, however, seems it was reverted after the code refactoring in HiveQL.

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

Closes #3357 from chenghao-intel/explain and squashes the following commits:

7aace63 [Cheng Hao] Support the CTAS in EXPLAIN command
2014-11-20 15:46:00 -08:00
Takuya UESHIN 2c2e7a44db [SPARK-4318][SQL] Fix empty sum distinct.
Executing sum distinct for empty table throws `java.lang.UnsupportedOperationException: empty.reduceLeft`.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #3184 from ueshin/issues/SPARK-4318 and squashes the following commits:

8168c42 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4318
66fdb0a [Takuya UESHIN] Re-refine aggregate functions.
6186eb4 [Takuya UESHIN] Fix Sum of GeneratedAggregate.
d2975f6 [Takuya UESHIN] Refine Sum and Average of GeneratedAggregate.
1bba675 [Takuya UESHIN] Refine Sum, SumDistinct and Average functions.
917e533 [Takuya UESHIN] Use aggregate instead of groupBy().
1a5f874 [Takuya UESHIN] Add tests to be executed as non-partial aggregation.
a5a57d2 [Takuya UESHIN] Fix empty Average.
22799dc [Takuya UESHIN] Fix empty Sum and SumDistinct.
65b7dd2 [Takuya UESHIN] Fix empty sum distinct.
2014-11-20 15:41:24 -08:00
ravipesala 98e9419784 [SPARK-4513][SQL] Support relational operator '<=>' in Spark SQL
The relational operator '<=>' is not working in Spark SQL. Same works in Spark HiveQL

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #3387 from ravipesala/<=> and squashes the following commits:

7198e90 [ravipesala] Supporting relational operator '<=>' in Spark SQL
2014-11-20 15:34:03 -08:00
Dan McClary b8e6886fb8 [SPARK-4228][SQL] SchemaRDD to JSON
Here's a simple fix for SchemaRDD to JSON.

Author: Dan McClary <dan.mcclary@gmail.com>

Closes #3213 from dwmclary/SPARK-4228 and squashes the following commits:

d714e1d [Dan McClary] fixed PEP 8 error
cac2879 [Dan McClary] move pyspark comment and doctest to correct location
f9471d3 [Dan McClary] added pyspark doc and doctest
6598cee [Dan McClary] adding complex type queries
1a5fd30 [Dan McClary] removing SPARK-4228 from SQLQuerySuite
4a651f0 [Dan McClary] cleaned PEP and Scala style failures.  Moved tests to JsonSuite
47ceff6 [Dan McClary] cleaned up scala style issues
2ee1e70 [Dan McClary] moved rowToJSON to JsonRDD
4387dd5 [Dan McClary] Added UserDefinedType, cleaned up case formatting
8f7bfb6 [Dan McClary] Map type added to SchemaRDD.toJSON
1b11980 [Dan McClary] Map and UserDefinedTypes partially done
11d2016 [Dan McClary] formatting and unicode deserialization default fixed
6af72d1 [Dan McClary] deleted extaneous comment
4d11c0c [Dan McClary] JsonFactory rewrite of toJSON for SchemaRDD
149dafd [Dan McClary] wrapped scala toJSON in sql.py
5e5eb1b [Dan McClary] switched to Jackson for JSON processing
6c94a54 [Dan McClary] added toJSON to pyspark SchemaRDD
aaeba58 [Dan McClary] added toJSON to pyspark SchemaRDD
1d171aa [Dan McClary] upated missing brace on if statement
319e3ba [Dan McClary] updated to upstream master with merged SPARK-4228
424f130 [Dan McClary] tests pass, ready for pull and PR
626a5b1 [Dan McClary] added toJSON to SchemaRDD
f7d166a [Dan McClary] added toJSON method
5d34e37 [Dan McClary] merge resolved
d6d19e9 [Dan McClary] pr example
2014-11-20 13:44:19 -08:00
Cheng Lian abf29187f0 [SPARK-3938][SQL] Names in-memory columnar RDD with corresponding table name
This PR enables the Web UI storage tab to show the in-memory table name instead of the mysterious query plan string as the name of the in-memory columnar RDD.

Note that after #2501, a single columnar RDD can be shared by multiple in-memory tables, as long as their query results are the same. In this case, only the first cached table name is shown. For example:

```sql
CACHE TABLE first AS SELECT * FROM src;
CACHE TABLE second AS SELECT * FROM src;
```

The Web UI only shows "In-memory table first".

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

Author: Cheng Lian <lian@databricks.com>

Closes #3383 from liancheng/columnar-rdd-name and squashes the following commits:

071907f [Cheng Lian] Fixes tests
12ddfa6 [Cheng Lian] Names in-memory columnar RDD with corresponding table name
2014-11-20 13:12:24 -08:00
Marcelo Vanzin 397d3aae5b Bumping version to 1.3.0-SNAPSHOT.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3277 from vanzin/version-1.3 and squashes the following commits:

7c3c396 [Marcelo Vanzin] Added temp repo to sbt build.
5f404ff [Marcelo Vanzin] Add another exclusion.
19457e7 [Marcelo Vanzin] Update old version to 1.2, add temporary 1.2 repo.
3c8d705 [Marcelo Vanzin] Workaround for MIMA checks.
e940810 [Marcelo Vanzin] Bumping version to 1.3.0-SNAPSHOT.
2014-11-18 21:24:18 -08:00
Cheng Lian 423baea953 [SPARK-4468][SQL] Fixes Parquet filter creation for inequality predicates with literals on the left hand side
For expressions like `10 < someVar`, we should create an `Operators.Gt` filter, but right now an `Operators.Lt` is created. This issue affects all inequality predicates with literals on the left hand side.

(This bug existed before #3317 and affects branch-1.1. #3338 was opened to backport this to branch-1.1.)

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

Author: Cheng Lian <lian@databricks.com>

Closes #3334 from liancheng/fix-parquet-comp-filter and squashes the following commits:

0130897 [Cheng Lian] Fixes Parquet comparison filter generation
2014-11-18 17:41:54 -08:00
Davies Liu 4a377aff2d [SPARK-3721] [PySpark] broadcast objects larger than 2G
This patch will bring support for broadcasting objects larger than 2G.

pickle, zlib, FrameSerializer and Array[Byte] all can not support objects larger than 2G, so this patch introduce LargeObjectSerializer to serialize broadcast objects, the object will be serialized and compressed into small chunks, it also change the type of Broadcast[Array[Byte]]] into Broadcast[Array[Array[Byte]]]].

Testing for support broadcast objects larger than 2G is slow and memory hungry, so this is tested manually, could be added into SparkPerf.

Author: Davies Liu <davies@databricks.com>
Author: Davies Liu <davies.liu@gmail.com>

Closes #2659 from davies/huge and squashes the following commits:

7b57a14 [Davies Liu] add more tests for broadcast
28acff9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
a2f6a02 [Davies Liu] bug fix
4820613 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
5875c73 [Davies Liu] address comments
10a349b [Davies Liu] address comments
0c33016 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
6182c8f [Davies Liu] Merge branch 'master' into huge
d94b68f [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
2514848 [Davies Liu] address comments
fda395b [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge
1c2d928 [Davies Liu] fix scala style
091b107 [Davies Liu] broadcast objects larger than 2G
2014-11-18 16:17:51 -08:00
Michael Armbrust 90d72ec850 [SQL] Support partitioned parquet tables that have the key in both the directory and the file
Author: Michael Armbrust <michael@databricks.com>

Closes #3272 from marmbrus/keyInPartitionedTable and squashes the following commits:

447f08c [Michael Armbrust] Support partitioned parquet tables that have the key in both the directory and the file
2014-11-18 12:13:23 -08:00
Cheng Lian 36b0956a3e [SPARK-4453][SPARK-4213][SQL] Simplifies Parquet filter generation code
While reviewing PR #3083 and #3161, I noticed that Parquet record filter generation code can be simplified significantly according to the clue stated in [SPARK-4453](https://issues.apache.org/jira/browse/SPARK-4213). This PR addresses both SPARK-4453 and SPARK-4213 with this simplification.

While generating `ParquetTableScan` operator, we need to remove all Catalyst predicates that have already been pushed down to Parquet. Originally, we first generate the record filter, and then call `findExpression` to traverse the generated filter to find out all pushed down predicates [[1](64c6b9bad5/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala (L213-L228))]. In this way, we have to introduce the `CatalystFilter` class hierarchy to bind the Catalyst predicates together with their generated Parquet filter, and complicate the code base a lot.

The basic idea of this PR is that, we don't need `findExpression` after filter generation, because we already know a predicate can be pushed down if we can successfully generate its corresponding Parquet filter. SPARK-4213 is fixed by returning `None` for any unsupported predicate type.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3317 from liancheng/simplify-parquet-filters and squashes the following commits:

d6a9499 [Cheng Lian] Fixes import styling issue
43760e8 [Cheng Lian] Simplifies Parquet filter generation logic
2014-11-17 16:55:12 -08:00
Cheng Hao ef7c464eff [SPARK-4448] [SQL] unwrap for the ConstantObjectInspector
Author: Cheng Hao <hao.cheng@intel.com>

Closes #3308 from chenghao-intel/unwrap_constant_oi and squashes the following commits:

156b500 [Cheng Hao] rebase the master
c5b20ab [Cheng Hao] unwrap for the ConstantObjectInspector
2014-11-17 16:35:49 -08:00
w00228970 42389b1780 [SPARK-4443][SQL] Fix statistics for external table in spark sql hive
The `totalSize` of external table  is always zero, which will influence join strategy(always use broadcast join for external table).

Author: w00228970 <wangfei1@huawei.com>

Closes #3304 from scwf/statistics and squashes the following commits:

568f321 [w00228970] fix statistics for external table
2014-11-17 16:33:50 -08:00
Cheng Lian 6b7f2f753d [SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types
This PR is exactly the same as #3178 except it reverts the `FileStatus.isDir` to `FileStatus.isDirectory` change, since it doesn't compile with Hadoop 1.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3298 from liancheng/date-for-thriftserver and squashes the following commits:

866037e [Cheng Lian] Revers isDirectory to isDir (it breaks Hadoop 1 profile)
6f71d0b [Cheng Lian] Makes toHiveString static
26fa955 [Cheng Lian] Fixes complex type support in Hive 0.13.1 shim
a92882a [Cheng Lian] Updates HiveShim for 0.13.1
73f442b [Cheng Lian] Adds Date support for HiveThriftServer2 (Hive 0.12.0)
2014-11-17 16:31:05 -08:00
Cheng Hao 69e858cc77 [SQL] Construct the MutableRow from an Array
Author: Cheng Hao <hao.cheng@intel.com>

Closes #3217 from chenghao-intel/mutablerow and squashes the following commits:

e8a10bd [Cheng Hao] revert the change of Row object
4681aea [Cheng Hao] Add toMutableRow method in object Row
a751838 [Cheng Hao] Construct the MutableRow from an existed row
2014-11-17 16:29:52 -08:00
Takuya UESHIN 566c791931 [SPARK-4425][SQL] Handle NaN or Infinity cast to Timestamp correctly.
`Cast` from `NaN` or `Infinity` of `Double` or `Float` to `TimestampType` throws `NumberFormatException`.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #3283 from ueshin/issues/SPARK-4425 and squashes the following commits:

14def0c [Takuya UESHIN] Fix Cast to be able to handle NaN or Infinity to TimestampType.
2014-11-17 16:28:07 -08:00
Takuya UESHIN 3a81a1c9e0 [SPARK-4420][SQL] Change nullability of Cast from DoubleType/FloatType to DecimalType.
This is follow-up of [SPARK-4390](https://issues.apache.org/jira/browse/SPARK-4390) (#3256).

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #3278 from ueshin/issues/SPARK-4420 and squashes the following commits:

7fea558 [Takuya UESHIN] Add some tests.
cb2301a [Takuya UESHIN] Fix tests.
133bad5 [Takuya UESHIN] Change nullability of Cast from DoubleType/FloatType to DecimalType.
2014-11-17 16:26:48 -08:00
Cheng Lian 5ce7dae859 [SQL] Makes conjunction pushdown more aggressive for in-memory table
This is inspired by the [Parquet record filter generation code](64c6b9bad5/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala (L387-L400)).

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

Author: Cheng Lian <lian@databricks.com>

Closes #3318 from liancheng/aggresive-conj-pushdown and squashes the following commits:

78b69d2 [Cheng Lian] Makes conjunction pushdown more aggressive
2014-11-17 15:33:13 -08:00
Michael Armbrust 64c6b9bad5 [SPARK-4410][SQL] Add support for external sort
Adds a new operator that uses Spark's `ExternalSort` class.  It is off by default now, but we might consider making it the default if benchmarks show that it does not regress performance.

Author: Michael Armbrust <michael@databricks.com>

Closes #3268 from marmbrus/externalSort and squashes the following commits:

48b9726 [Michael Armbrust] comments
b98799d [Michael Armbrust] Add test
afd7562 [Michael Armbrust] Add support for external sort.
2014-11-16 21:55:57 -08:00
Michael Armbrust 45ce3273cb Revert "[SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types"
Author: Michael Armbrust <michael@databricks.com>

Closes #3292 from marmbrus/revert4309 and squashes the following commits:

808e96e [Michael Armbrust] Revert "[SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types"
2014-11-16 15:05:08 -08:00
Cheng Lian cb6bd83a91 [SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types
SPARK-4407 was detected while working on SPARK-4309. Merged these two into a single PR since 1.2.0 RC is approaching.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3178 from liancheng/date-for-thriftserver and squashes the following commits:

6f71d0b [Cheng Lian] Makes toHiveString static
26fa955 [Cheng Lian] Fixes complex type support in Hive 0.13.1 shim
a92882a [Cheng Lian] Updates HiveShim for 0.13.1
73f442b [Cheng Lian] Adds Date support for HiveThriftServer2 (Hive 0.12.0)
2014-11-16 14:26:41 -08:00
Kousuke Saruta 84468b2e20 [SPARK-4426][SQL][Minor] The symbol of BitwiseOr is wrong, should not be '&'
The symbol of BitwiseOr is defined as '&' but I think it's wrong. It should be '|'.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #3284 from sarutak/bitwise-or-symbol-fix and squashes the following commits:

aff4be5 [Kousuke Saruta] Fixed symbol of BitwiseOr
2014-11-15 22:23:47 -08:00
kai cbddac2369 Added contains(key) to Metadata
Add contains(key) to org.apache.spark.sql.catalyst.util.Metadata to test the existence of a key. Otherwise, Class Metadata's get methods may throw NoSuchElement exception if the key does not exist.
Testcases are added to MetadataSuite as well.

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

Closes #3273 from kai-zeng/metadata-fix and squashes the following commits:

74b3d03 [kai] Added contains(key) to Metadata
2014-11-14 23:44:23 -08:00
Jim Carroll 37482ce5a7 [SPARK-4412][SQL] Fix Spark's control of Parquet logging.
The Spark ParquetRelation.scala code makes the assumption that the parquet.Log class has already been loaded. If ParquetRelation.enableLogForwarding executes prior to the parquet.Log class being loaded then the code in enableLogForwarding has no affect.

ParquetRelation.scala attempts to override the parquet logger but, at least currently (and if your application simply reads a parquet file before it does anything else with Parquet), the parquet.Log class hasn't been loaded yet. Therefore the code in ParquetRelation.enableLogForwarding has no affect. If you look at the code in parquet.Log there's a static initializer that needs to be called prior to enableLogForwarding or whatever enableLogForwarding does gets undone by this static initializer.

The "fix" would be to force the static initializer to get called in parquet.Log as part of enableForwardLogging.

Author: Jim Carroll <jim@dontcallme.com>

Closes #3271 from jimfcarroll/parquet-logging and squashes the following commits:

37bdff7 [Jim Carroll] Fix Spark's control of Parquet logging.
2014-11-14 15:33:21 -08:00
Yash Datta 63ca3af66f [SPARK-4365][SQL] Remove unnecessary filter call on records returned from parquet library
Since parquet library has been updated , we no longer need to filter the records returned from parquet library for null records , as now the library skips those :

from parquet-hadoop/src/main/java/parquet/hadoop/InternalParquetRecordReader.java

public boolean nextKeyValue() throws IOException, InterruptedException {
boolean recordFound = false;
while (!recordFound) {
// no more records left
if (current >= total)
{ return false; }
try {
checkRead();
currentValue = recordReader.read();
current ++;
if (recordReader.shouldSkipCurrentRecord())
{
 // this record is being filtered via the filter2 package
if (DEBUG) LOG.debug("skipping record");
 continue;
 }
if (currentValue == null)
{
// only happens with FilteredRecordReader at end of block current = totalCountLoadedSoFar;
 if (DEBUG) LOG.debug("filtered record reader reached end of block");
 continue;
}

recordFound = true;
if (DEBUG) LOG.debug("read value: " + currentValue);
} catch (RuntimeException e)
{ throw new ParquetDecodingException(format("Can not read value at %d in block %d in file %s", current, currentBlock, file), e); }

}
return true;
}

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

Closes #3229 from saucam/remove_filter and squashes the following commits:

8909ae9 [Yash Datta] SPARK-4365: Remove unnecessary filter call on records returned from parquet library
2014-11-14 15:16:40 -08:00
Jim Carroll f76b968370 [SPARK-4386] Improve performance when writing Parquet files.
If you profile the writing of a Parquet file, the single worst time consuming call inside of org.apache.spark.sql.parquet.MutableRowWriteSupport.write is actually in the scala.collection.AbstractSequence.size call. This is because the size call actually ends up COUNTING the elements in a scala.collection.LinearSeqOptimized.length ("optimized?").

This doesn't need to be done. "size" is called repeatedly where needed rather than called once at the top of the method and stored in a 'val'.

Author: Jim Carroll <jim@dontcallme.com>

Closes #3254 from jimfcarroll/parquet-perf and squashes the following commits:

30cc0b5 [Jim Carroll] Improve performance when writing Parquet files.
2014-11-14 15:11:53 -08:00
Cheng Lian 0c7b66bd44 [SPARK-4322][SQL] Enables struct fields as sub expressions of grouping fields
While resolving struct fields, the resulted `GetField` expression is wrapped with an `Alias` to make it a named expression. Assume `a` is a struct instance with a field `b`, then `"a.b"` will be resolved as `Alias(GetField(a, "b"), "b")`. Thus, for this following SQL query:

```sql
SELECT a.b + 1 FROM t GROUP BY a.b + 1
```

the grouping expression is

```scala
Add(GetField(a, "b"), Literal(1, IntegerType))
```

while the aggregation expression is

```scala
Add(Alias(GetField(a, "b"), "b"), Literal(1, IntegerType))
```

This mismatch makes the above SQL query fail during the both analysis and execution phases. This PR fixes this issue by removing the alias when substituting aggregation expressions.

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

Author: Cheng Lian <lian@databricks.com>

Closes #3248 from liancheng/spark-4322 and squashes the following commits:

23a46ea [Cheng Lian] Code simplification
dd20a79 [Cheng Lian] Should only trim aliases around `GetField`s
7f46532 [Cheng Lian] Enables struct fields as sub expressions of grouping fields
2014-11-14 15:09:36 -08:00
Michael Armbrust 4b4b50c9e5 [SQL] Don't shuffle code generated rows
When sort based shuffle and code gen are on we were trying to ship the code generated rows during a shuffle.  This doesn't work because the classes don't exist on the other side.  Instead we now copy into a generic row before shipping.

Author: Michael Armbrust <michael@databricks.com>

Closes #3263 from marmbrus/aggCodeGen and squashes the following commits:

f6ba8cf [Michael Armbrust] fix and test
2014-11-14 15:03:23 -08:00
Michael Armbrust f805025e8e [SQL] Minor cleanup of comments, errors and override.
Author: Michael Armbrust <michael@databricks.com>

Closes #3257 from marmbrus/minorCleanup and squashes the following commits:

d8b5abc [Michael Armbrust] Use interpolation.
2fdf903 [Michael Armbrust] Better error message when coalesce can't be resolved.
f9fa6cf [Michael Armbrust] Methods in a final class do not also need to be final, use override.
199fd98 [Michael Armbrust] Fix typo
2014-11-14 15:00:42 -08:00
Michael Armbrust e47c387639 [SPARK-4391][SQL] Configure parquet filters using SQLConf
This is more uniform with the rest of SQL configuration and allows it to be turned on and off without restarting the SparkContext.  In this PR I also turn off filter pushdown by default due to a number of outstanding issues (in particular SPARK-4258).  When those are fixed we should turn it back on by default.

Author: Michael Armbrust <michael@databricks.com>

Closes #3258 from marmbrus/parquetFilters and squashes the following commits:

5655bfe [Michael Armbrust] Remove extra line.
15e9a98 [Michael Armbrust] Enable filters for tests
75afd39 [Michael Armbrust] Fix comments
78fa02d [Michael Armbrust] off by default
e7f9e16 [Michael Armbrust] First draft of correctly configuring parquet filter pushdown
2014-11-14 14:59:35 -08:00
Michael Armbrust a0300ea32a [SPARK-4390][SQL] Handle NaN cast to decimal correctly
Author: Michael Armbrust <michael@databricks.com>

Closes #3256 from marmbrus/NanDecimal and squashes the following commits:

4c3ba46 [Michael Armbrust] fix style
d360f83 [Michael Armbrust] Handle NaN cast to decimal
2014-11-14 14:56:57 -08:00
DoingDone9 0cbdb01e1c [SPARK-4333][SQL] Correctly log number of iterations in RuleExecutor
When iterator of RuleExecutor breaks, the num of iterator should be (iteration - 1) not (iteration ).Because log looks like "Fixed point reached for batch ${batch.name} after 3 iterations.", but it did 2 iterations really!

Author: DoingDone9 <799203320@qq.com>

Closes #3180 from DoingDone9/issue_01 and squashes the following commits:

571e2ed [DoingDone9] Update RuleExecutor.scala
46514b6 [DoingDone9] When iterator of RuleExecutor breaks, the num of iterator should be iteration - 1 not iteration.
2014-11-14 14:28:06 -08:00
Sandy Ryza f5f757e4ed SPARK-4375. no longer require -Pscala-2.10
It seems like the winds might have moved away from this approach, but wanted to post the PR anyway because I got it working and to show what it would look like.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3239 from sryza/sandy-spark-4375 and squashes the following commits:

0ffbe95 [Sandy Ryza] Enable -Dscala-2.11 in sbt
cd42d94 [Sandy Ryza] Update doc
f6644c3 [Sandy Ryza] SPARK-4375 take 2
2014-11-14 14:21:57 -08:00