Commit graph

2526 commits

Author SHA1 Message Date
Michael Armbrust 59a501359a [SPARK-11636][SQL] Support classes defined in the REPL with Encoders
Before this PR there were two things that would blow up if you called `df.as[MyClass]` if `MyClass` was defined in the REPL:
 - [x] Because `classForName` doesn't work on the munged names returned by `tpe.erasure.typeSymbol.asClass.fullName`
 - [x] Because we don't have anything to pass into the constructor for the `$outer` pointer.

Note that this PR is just adding the infrastructure for working with inner classes in encoder and is not yet sufficient to make them work in the REPL.  Currently, the implementation show in 95cec7d413 is causing a bug that breaks code gen due to some interaction between janino and the `ExecutorClassLoader`.  This will be addressed in a follow-up PR.

Author: Michael Armbrust <michael@databricks.com>

Closes #9602 from marmbrus/dataset-replClasses.
2015-11-18 16:48:09 -08:00
Reynold Xin 5df08949f5 [SPARK-11810][SQL] Java-based encoder for opaque types in Datasets.
This patch refactors the existing Kryo encoder expressions and adds support for Java serialization.

Author: Reynold Xin <rxin@databricks.com>

Closes #9802 from rxin/SPARK-11810.
2015-11-18 15:42:07 -08:00
Dilip Biswal 54db797025 [SPARK-11544][SQL] sqlContext doesn't use PathFilter
Apply the user supplied pathfilter while retrieving the files from fs.

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

Closes #9652 from dilipbiswal/spark-11544.
2015-11-18 14:05:18 -08:00
JihongMa 09ad9533d5 [SPARK-11720][SQL][ML] Handle edge cases when count = 0 or 1 for Stats function
return Double.NaN for mean/average when count == 0 for all numeric types that is converted to Double, Decimal type continue to return null.

Author: JihongMa <linlin200605@gmail.com>

Closes #9705 from JihongMA/SPARK-11720.
2015-11-18 13:03:37 -08:00
Davies Liu 94624eacb0 [SPARK-11739][SQL] clear the instantiated SQLContext
Currently, if the first SQLContext is not removed after stopping SparkContext, a SQLContext could set there forever. This patch make this more robust.

Author: Davies Liu <davies@databricks.com>

Closes #9706 from davies/clear_context.
2015-11-18 11:53:28 -08:00
Yin Huai 6f99522d13 [SPARK-11792] [SQL] [FOLLOW-UP] Change SizeEstimation to KnownSizeEstimation and make estimatedSize return Long instead of Option[Long]
https://issues.apache.org/jira/browse/SPARK-11792

The main changes include:
* Renaming `SizeEstimation` to `KnownSizeEstimation`. Hopefully this new name has more information.
* Making `estimatedSize` return `Long` instead of `Option[Long]`.
* In `UnsaveHashedRelation`, `estimatedSize` will delegate the work to `SizeEstimator` if we have not created a `BytesToBytesMap`.

Since we will put `UnsaveHashedRelation` to `BlockManager`, it is generally good to let it provide a more accurate size estimation. Also, if we do not put `BytesToBytesMap` directly into `BlockerManager`, I feel it is not really necessary to make `BytesToBytesMap` extends `KnownSizeEstimation`.

Author: Yin Huai <yhuai@databricks.com>

Closes #9813 from yhuai/SPARK-11792-followup.
2015-11-18 11:49:12 -08:00
Wenchen Fan dbf428c87a [SPARK-11795][SQL] combine grouping attributes into a single NamedExpression
we use `ExpressionEncoder.tuple` to build the result encoder, which assumes the input encoder should point to a struct type field if it’s non-flat.
However, our keyEncoder always point to a flat field/fields: `groupingAttributes`, we should combine them into a single `NamedExpression`.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9792 from cloud-fan/agg.
2015-11-18 10:33:17 -08:00
Wenchen Fan 33b8373334 [SPARK-11725][SQL] correctly handle null inputs for UDF
If user use primitive parameters in UDF, there is no way for him to do the null-check for primitive inputs, so we are assuming the primitive input is null-propagatable for this case and return null if the input is null.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9770 from cloud-fan/udf.
2015-11-18 10:23:12 -08:00
Wenchen Fan cffb899c43 [SPARK-11803][SQL] fix Dataset self-join
When we resolve the join operator, we may change the output of right side if self-join is detected. So in `Dataset.joinWith`, we should resolve the join operator first, and then get the left output and right output from it, instead of using `left.output` and `right.output` directly.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9806 from cloud-fan/self-join.
2015-11-18 10:15:50 -08:00
somideshmukh b8f4379ba1 [SPARK-10946][SQL] JDBC - Use Statement.executeUpdate instead of PreparedStatement.executeUpdate for DDLs
New changes with JDBCRDD

Author: somideshmukh <somilde@us.ibm.com>

Closes #9733 from somideshmukh/SomilBranch-1.1.
2015-11-18 08:51:01 +00:00
Yin Huai 1714350bdd [SPARK-11792][SQL] SizeEstimator cannot provide a good size estimation of UnsafeHashedRelations
https://issues.apache.org/jira/browse/SPARK-11792

Right now, SizeEstimator will "think" a small UnsafeHashedRelation is several GBs.

Author: Yin Huai <yhuai@databricks.com>

Closes #9788 from yhuai/SPARK-11792.
2015-11-18 00:42:52 -08:00
Reynold Xin 5e2b44474c [SPARK-11802][SQL] Kryo-based encoder for opaque types in Datasets
I also found a bug with self-joins returning incorrect results in the Dataset API. Two test cases attached and filed SPARK-11803.

Author: Reynold Xin <rxin@databricks.com>

Closes #9789 from rxin/SPARK-11802.
2015-11-18 00:09:29 -08:00
Davies Liu 2f191c66b6 [SPARK-11643] [SQL] parse year with leading zero
Support the years between 0 <= year < 1000

Author: Davies Liu <davies@databricks.com>

Closes #9701 from davies/leading_zero.
2015-11-17 23:14:05 -08:00
Reynold Xin 91f4b6f2db [SPARK-11797][SQL] collect, first, and take should use encoders for serialization
They were previously using Spark's default serializer for serialization.

Author: Reynold Xin <rxin@databricks.com>

Closes #9787 from rxin/SPARK-11797.
2015-11-17 21:40:58 -08:00
Reynold Xin ed8d1531f9 [SPARK-11793][SQL] Dataset should set the resolved encoders internally for maps.
I also wrote a test case -- but unfortunately the test case is not working due to SPARK-11795.

Author: Reynold Xin <rxin@databricks.com>

Closes #9784 from rxin/SPARK-11503.
2015-11-17 19:02:44 -08:00
Davies Liu 5aca6ad00c [SPARK-11767] [SQL] limit the size of caced batch
Currently the size of cached batch in only controlled by `batchSize` (default value is 10000), which does not work well with the size of serialized columns (for example, complex types). The memory used to build the batch is not accounted, it's easy to OOM (especially after unified memory management).

This PR introduce a hard limit as 4M for total columns (up to 50 columns of uncompressed primitive columns).

This also change the way to grow buffer, double it each time, then trim it once finished.

cc liancheng

Author: Davies Liu <davies@databricks.com>

Closes #9760 from davies/cache_limit.
2015-11-17 12:50:01 -08:00
Wenchen Fan d925149664 [SPARK-10186][SQL] support postgre array type in JDBCRDD
Add ARRAY support to `PostgresDialect`.

Nested ARRAY is not allowed for now because it's hard to get the array dimension info. See http://stackoverflow.com/questions/16619113/how-to-get-array-base-type-in-postgres-via-jdbc

Thanks for the initial work from mariusvniekerk !

Close https://github.com/apache/spark/pull/9137

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9662 from cloud-fan/postgre.
2015-11-17 11:29:02 -08:00
gatorsmile 0158ff7737 [SPARK-8658][SQL][FOLLOW-UP] AttributeReference's equals method compares all the members
Based on the comment of cloud-fan in https://github.com/apache/spark/pull/9216, update the AttributeReference's hashCode function by including the hashCode of the other attributes including name, nullable and qualifiers.

Here, I am not 100% sure if we should include name in the hashCode calculation, since the original hashCode calculation does not include it.

marmbrus cloud-fan Please review if the changes are good.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9761 from gatorsmile/hashCodeNamedExpression.
2015-11-17 11:23:54 -08:00
Cheng Lian 7b1407c7b9 [SPARK-11089][SQL] Adds option for disabling multi-session in Thrift server
This PR adds a new option `spark.sql.hive.thriftServer.singleSession` for disabling multi-session support in the Thrift server.

Note that this option is added as a Spark configuration (retrieved from `SparkConf`) rather than Spark SQL configuration (retrieved from `SQLConf`). This is because all SQL configurations are session-ized. Since multi-session support is by default on, no JDBC connection can modify global configurations like the newly added one.

Author: Cheng Lian <lian@databricks.com>

Closes #9740 from liancheng/spark-11089.single-session-option.
2015-11-17 11:17:52 -08:00
mayuanwen e8833dd12c [SPARK-11679][SQL] Invoking method " apply(fields: java.util.List[StructField])" in "StructType" gets ClassCastException
In the previous method, fields.toArray will cast java.util.List[StructField] into Array[Object] which can not cast into Array[StructField], thus when invoking this method will throw "java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to [Lorg.apache.spark.sql.types.StructField;"
I directly cast java.util.List[StructField] into Array[StructField]  in this patch.

Author: mayuanwen <mayuanwen@qiyi.com>

Closes #9649 from jackieMaKing/Spark-11679.
2015-11-17 11:15:46 -08:00
Cheng Lian fa13301ae4 [SPARK-11191][SQL][FOLLOW-UP] Cleans up unnecessary anonymous HiveFunctionRegistry
According to discussion in PR #9664, the anonymous `HiveFunctionRegistry` in `HiveContext` can be removed now.

Author: Cheng Lian <lian@databricks.com>

Closes #9737 from liancheng/spark-11191.follow-up.
2015-11-17 18:11:08 +08:00
Liang-Chi Hsieh d79d8b08ff [MINOR] [SQL] Fix randomly generated ArrayData in RowEncoderSuite
The randomly generated ArrayData used for the UDT `ExamplePoint` in `RowEncoderSuite` sometimes doesn't have enough elements. In this case, this test will fail. This patch is to fix it.

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

Closes #9757 from viirya/fix-randomgenerated-udt.
2015-11-16 23:16:17 -08:00
Kevin Yu e01865af0d [SPARK-11447][SQL] change NullType to StringType during binaryComparison between NullType and StringType
During executing PromoteStrings rule, if one side of binaryComparison is StringType and the other side is not StringType, the current code will promote(cast) the StringType to DoubleType, and if the StringType doesn't contain the numbers, it will get null value. So if it is doing <=> (NULL-safe equal) with Null, it will not filter anything, caused the problem reported by this jira.

I proposal to the changes through this PR, can you review my code changes ?

This problem only happen for <=>, other operators works fine.

scala> val filteredDF = df.filter(df("column") > (new Column(Literal(null))))
filteredDF: org.apache.spark.sql.DataFrame = [column: string]

scala> filteredDF.show
+------+
|column|
+------+
+------+

scala> val filteredDF = df.filter(df("column") === (new Column(Literal(null))))
filteredDF: org.apache.spark.sql.DataFrame = [column: string]

scala> filteredDF.show
+------+
|column|
+------+
+------+

scala> df.registerTempTable("DF")

scala> sqlContext.sql("select * from DF where 'column' = NULL")
res27: org.apache.spark.sql.DataFrame = [column: string]

scala> res27.show
+------+
|column|
+------+
+------+

Author: Kevin Yu <qyu@us.ibm.com>

Closes #9720 from kevinyu98/working_on_spark-11447.
2015-11-16 22:54:29 -08:00
hyukjinkwon 75d2020731 [SPARK-11694][FOLLOW-UP] Clean up imports, use a common function for metadata and add a test for FIXED_LEN_BYTE_ARRAY
As discussed https://github.com/apache/spark/pull/9660 https://github.com/apache/spark/pull/9060, I cleaned up unused imports, added a test for fixed-length byte array and used a common function for writing metadata for Parquet.

For the test for fixed-length byte array, I have tested and checked the encoding types with [parquet-tools](https://github.com/Parquet/parquet-mr/tree/master/parquet-tools).

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9754 from HyukjinKwon/SPARK-11694-followup.
2015-11-17 14:35:00 +08:00
Reynold Xin fbad920dbf [SPARK-11768][SPARK-9196][SQL] Support now function in SQL (alias for current_timestamp).
This patch adds an alias for current_timestamp (now function).

Also fixes SPARK-9196 to re-enable the test case for current_timestamp.

Author: Reynold Xin <rxin@databricks.com>

Closes #9753 from rxin/SPARK-11768.
2015-11-16 20:47:46 -08:00
Wenchen Fan fd14936be7 [SPARK-11625][SQL] add java test for typed aggregate
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9591 from cloud-fan/agg-test.
2015-11-16 15:32:49 -08:00
gatorsmile 75ee12f09c [SPARK-8658][SQL] AttributeReference's equals method compares all the members
This fix is to change the equals method to check all of the specified fields for equality of AttributeReference.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9216 from gatorsmile/namedExpressEqual.
2015-11-16 15:22:12 -08:00
Bartlomiej Alberski 31296628ac [SPARK-11553][SQL] Primitive Row accessors should not convert null to default value
Invocation of getters for type extending AnyVal returns default value (if field value is null) instead of throwing NPE. Please check comments for SPARK-11553 issue for more details.

Author: Bartlomiej Alberski <bartlomiej.alberski@allegrogroup.com>

Closes #9642 from alberskib/bugfix/SPARK-11553.
2015-11-16 15:14:38 -08:00
Zee Chen 985b38dd2f [SPARK-11390][SQL] Query plan with/without filterPushdown indistinguishable
…ishable

Propagate pushed filters to PhyicalRDD in DataSourceStrategy.apply

Author: Zee Chen <zeechen@us.ibm.com>

Closes #9679 from zeocio/spark-11390.
2015-11-16 14:21:28 -08:00
Wenchen Fan b1a9662623 [SPARK-11754][SQL] consolidate ExpressionEncoder.tuple and Encoders.tuple
These 2 are very similar, we can consolidate them into one.

Also add tests for it and fix a bug.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9729 from cloud-fan/tuple.
2015-11-16 12:45:34 -08:00
Liang-Chi Hsieh b0c3fd34e4 [SPARK-11743] [SQL] Add UserDefinedType support to RowEncoder
JIRA: https://issues.apache.org/jira/browse/SPARK-11743

RowEncoder doesn't support UserDefinedType now. We should add the support for it.

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

Closes #9712 from viirya/rowencoder-udt.
2015-11-16 09:03:42 -08:00
Wenchen Fan 06f1fdba6d [SPARK-11752] [SQL] fix timezone problem for DateTimeUtils.getSeconds
code snippet to reproduce it:
```
TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
val t = Timestamp.valueOf("1900-06-11 12:14:50.789")
val us = fromJavaTimestamp(t)
assert(getSeconds(us) === t.getSeconds)
```

it will be good to add a regression test for it, but the reproducing code need to change the default timezone, and even we change it back, the `lazy val defaultTimeZone` in `DataTimeUtils` is fixed.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9728 from cloud-fan/seconds.
2015-11-16 08:58:40 -08:00
xin Wu 0e79604aed [SPARK-11522][SQL] input_file_name() returns "" for external tables
When computing partition for non-parquet relation, `HadoopRDD.compute` is used. but it does not set the thread local variable `inputFileName` in `NewSqlHadoopRDD`, like `NewSqlHadoopRDD.compute` does.. Yet, when getting the `inputFileName`, `NewSqlHadoopRDD.inputFileName` is exptected, which is empty now.
Adding the setting inputFileName in HadoopRDD.compute resolves this issue.

Author: xin Wu <xinwu@us.ibm.com>

Closes #9542 from xwu0226/SPARK-11522.
2015-11-16 08:10:48 -08:00
hyukjinkwon e388b39d10 [SPARK-11692][SQL] Support for Parquet logical types, JSON and BSON (embedded types)
Parquet supports some JSON and BSON datatypes. They are represented as binary for BSON and string (UTF-8) for JSON internally.

I searched a bit and found Apache drill also supports both in this way, [link](https://drill.apache.org/docs/parquet-format/).

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #9658 from HyukjinKwon/SPARK-11692.
2015-11-16 21:59:33 +08:00
hyukjinkwon 7f8eb3bf6e [SPARK-11044][SQL] Parquet writer version fixed as version1
https://issues.apache.org/jira/browse/SPARK-11044

Spark writes a parquet file only with writer version1 ignoring the writer version given by user.

So, in this PR, it keeps the writer version if given or sets version1 as default.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: HyukjinKwon <gurwls223@gmail.com>

Closes #9060 from HyukjinKwon/SPARK-11044.
2015-11-16 21:30:10 +08:00
Reynold Xin 42de5253f3 [SPARK-11745][SQL] Enable more JSON parsing options
This patch adds the following options to the JSON data source, for dealing with non-standard JSON files:
* `allowComments` (default `false`): ignores Java/C++ style comment in JSON records
* `allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names
* `allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes
* `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers (e.g. 00012)

To avoid passing a lot of options throughout the json package, I introduced a new JSONOptions case class to define all JSON config options.

Also updated documentation to explain these options.

Scala

![screen shot 2015-11-15 at 6 12 12 pm](https://cloud.githubusercontent.com/assets/323388/11172965/e3ace6ec-8bc4-11e5-805e-2d78f80d0ed6.png)

Python

![screen shot 2015-11-15 at 6 11 28 pm](https://cloud.githubusercontent.com/assets/323388/11172964/e23ed6ee-8bc4-11e5-8216-312f5983acd5.png)

Author: Reynold Xin <rxin@databricks.com>

Closes #9724 from rxin/SPARK-11745.
2015-11-16 00:06:14 -08:00
gatorsmile b58765caa6 [SPARK-9928][SQL] Removal of LogicalLocalTable
LogicalLocalTable in ExistingRDD.scala is replaced by localRelation in LocalRelation.scala?

Do you know any reason why we still keep this class?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9717 from gatorsmile/LogicalLocalTable.
2015-11-15 21:10:46 -08:00
Yu Gao 72c1d68b4a [SPARK-10181][SQL] Do kerberos login for credentials during hive client initialization
On driver process start up, UserGroupInformation.loginUserFromKeytab is called with the principal and keytab passed in, and therefore static var UserGroupInfomation,loginUser is set to that principal with kerberos credentials saved in its private credential set, and all threads within the driver process are supposed to see and use this login credentials to authenticate with Hive and Hadoop. However, because of IsolatedClientLoader, UserGroupInformation class is not shared for hive metastore clients, and instead it is loaded separately and of course not able to see the prepared kerberos login credentials in the main thread.

The first proposed fix would cause other classloader conflict errors, and is not an appropriate solution. This new change does kerberos login during hive client initialization, which will make credentials ready for the particular hive client instance.

 yhuai Please take a look and let me know. If you are not the right person to talk to, could you point me to someone responsible for this?

Author: Yu Gao <ygao@us.ibm.com>
Author: gaoyu <gaoyu@gaoyu-macbookpro.roam.corp.google.com>
Author: Yu Gao <crystalgaoyu@gmail.com>

Closes #9272 from yolandagao/master.
2015-11-15 14:53:59 -08:00
Yin Huai 3e2e1873b2 [SPARK-11738] [SQL] Making ArrayType orderable
https://issues.apache.org/jira/browse/SPARK-11738

Author: Yin Huai <yhuai@databricks.com>

Closes #9718 from yhuai/makingArrayOrderable.
2015-11-15 13:59:59 -08:00
Reynold Xin d22fc10887 [SPARK-11734][SQL] Rename TungstenProject -> Project, TungstenSort -> Sort
I didn't remove the old Sort operator, since we still use it in randomized tests. I moved it into test module and renamed it ReferenceSort.

Author: Reynold Xin <rxin@databricks.com>

Closes #9700 from rxin/SPARK-11734.
2015-11-15 10:33:53 -08:00
Yin Huai d83c2f9f0b [SPARK-11736][SQL] Add monotonically_increasing_id to function registry.
https://issues.apache.org/jira/browse/SPARK-11736

Author: Yin Huai <yhuai@databricks.com>

Closes #9703 from yhuai/MonotonicallyIncreasingID.
2015-11-14 21:04:18 -08:00
hyukjinkwon 139c15b624 [SPARK-11694][SQL] Parquet logical types are not being tested properly
All the physical types are properly tested at `ParquetIOSuite` but logical type mapping is not being tested.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #9660 from HyukjinKwon/SPARK-11694.
2015-11-14 18:36:01 +08:00
nitin goyal c939c70ac1 [SPARK-7970] Skip closure cleaning for SQL operations
Also introduces new spark private API in RDD.scala with name 'mapPartitionsInternal' which doesn't closure cleans the RDD elements.

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

Closes #9253 from nitin2goyal/master.
2015-11-13 18:09:08 -08:00
Wenchen Fan d7b2b97ad6 [SPARK-11727][SQL] Split ExpressionEncoder into FlatEncoder and ProductEncoder
also add more tests for encoders, and fix bugs that I found:

* when convert array to catalyst array, we can only skip element conversion for native types(e.g. int, long, boolean), not `AtomicType`(String is AtomicType but we need to convert it)
* we should also handle scala `BigDecimal` when convert from catalyst `Decimal`.
* complex map type should be supported

other issues that still in investigation:

* encode java `BigDecimal` and decode it back, seems we will loss precision info.
* when encode case class that defined inside a object, `ClassNotFound` exception will be thrown.

I'll remove unused code in a follow-up PR.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9693 from cloud-fan/split.
2015-11-13 11:25:33 -08:00
Wenchen Fan 23b8188f75 [SPARK-11654][SQL][FOLLOW-UP] fix some mistakes and clean up
* rename `AppendColumn` to `AppendColumns` to be consistent with the physical plan name.
* clean up stale comments.
* always pass in resolved encoder to `TypedColumn.withInputType`(test added)
* enable a mistakenly disabled java test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9688 from cloud-fan/follow.
2015-11-13 11:13:09 -08:00
Yin Huai 7b5d9051cf [SPARK-11678][SQL] Partition discovery should stop at the root path of the table.
https://issues.apache.org/jira/browse/SPARK-11678

The change of this PR is to pass root paths of table to the partition discovery logic. So, the process of partition discovery stops at those root paths instead of going all the way to the root path of the file system.

Author: Yin Huai <yhuai@databricks.com>

Closes #9651 from yhuai/SPARK-11678.
2015-11-13 18:36:56 +08:00
Michael Armbrust 41bbd23004 [SPARK-11654][SQL] add reduce to GroupedDataset
This PR adds a new method, `reduce`, to `GroupedDataset`, which allows similar operations to `reduceByKey` on a traditional `PairRDD`.

```scala
val ds = Seq("abc", "xyz", "hello").toDS()
ds.groupBy(_.length).reduce(_ + _).collect()  // not actually commutative :P

res0: Array(3 -> "abcxyz", 5 -> "hello")
```

While implementing this method and its test cases several more deficiencies were found in our encoder handling.  Specifically, in order to support positional resolution, named resolution and tuple composition, it is important to keep the unresolved encoder around and to use it when constructing new `Datasets` with the same object type but different output attributes.  We now divide the encoder lifecycle into three phases (that mirror the lifecycle of standard expressions) and have checks at various boundaries:

 - Unresoved Encoders: all users facing encoders (those constructed by implicits, static methods, or tuple composition) are unresolved, meaning they have only `UnresolvedAttributes` for named fields and `BoundReferences` for fields accessed by ordinal.
 - Resolved Encoders: internal to a `[Grouped]Dataset` the encoder is resolved, meaning all input has been resolved to a specific `AttributeReference`.  Any encoders that are placed into a logical plan for use in object construction should be resolved.
 - BoundEncoder: Are constructed by physical plans, right before actual conversion from row -> object is performed.

It is left to future work to add explicit checks for resolution and provide good error messages when it fails.  We might also consider enforcing the above constraints in the type system (i.e. `fromRow` only exists on a `ResolvedEncoder`), but we should probably wait before spending too much time on this.

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

Closes #9673 from marmbrus/pr/9628.
2015-11-12 17:20:30 -08:00
JihongMa d292f74831 [SPARK-11420] Updating Stddev support via Imperative Aggregate
switched stddev support from DeclarativeAggregate to ImperativeAggregate.

Author: JihongMa <linlin200605@gmail.com>

Closes #9380 from JihongMA/SPARK-11420.
2015-11-12 13:47:34 -08:00
hyukjinkwon f5a9526fec [SPARK-10113][SQL] Explicit error message for unsigned Parquet logical types
Parquet supports some unsigned datatypes. However, Since Spark does not support unsigned datatypes, it needs to emit an exception with a clear message rather then with the one saying illegal datatype.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9646 from HyukjinKwon/SPARK-10113.
2015-11-12 12:29:50 -08:00
Cheng Lian 4fe99c72c6 [SPARK-11191][SQL] Looks up temporary function using execution Hive client
When looking up Hive temporary functions, we should always use the `SessionState` within the execution Hive client, since temporary functions are registered there.

Author: Cheng Lian <lian@databricks.com>

Closes #9664 from liancheng/spark-11191.fix-temp-function.
2015-11-12 12:17:51 -08:00
Reynold Xin 30e7433643 [SPARK-11673][SQL] Remove the normal Project physical operator (and keep TungstenProject)
Also make full outer join being able to produce UnsafeRows.

Author: Reynold Xin <rxin@databricks.com>

Closes #9643 from rxin/SPARK-11673.
2015-11-12 08:14:08 -08:00
Yin Huai 14cf753704 [SPARK-11661][SQL] Still pushdown filters returned by unhandledFilters.
https://issues.apache.org/jira/browse/SPARK-11661

Author: Yin Huai <yhuai@databricks.com>

Closes #9634 from yhuai/unhandledFilters.
2015-11-12 16:47:00 +08:00
Daoyuan Wang 39b1e36fbc [SPARK-11396] [SQL] add native implementation of datetime function to_unix_timestamp
`to_unix_timestamp` is the deterministic version of `unix_timestamp`, as it accepts at least one parameters.

Since the behavior here is quite similar to `unix_timestamp`, I think the dataframe API is not necessary here.

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

Closes #9347 from adrian-wang/to_unix_timestamp.
2015-11-11 20:36:21 -08:00
Reynold Xin e49e723392 [SPARK-11675][SQL] Remove shuffle hash joins.
Author: Reynold Xin <rxin@databricks.com>

Closes #9645 from rxin/SPARK-11675.
2015-11-11 19:32:52 -08:00
Andrew Ray b8ff6888e7 [SPARK-8992][SQL] Add pivot to dataframe api
This adds a pivot method to the dataframe api.

Following the lead of cube and rollup this adds a Pivot operator that is translated into an Aggregate by the analyzer.

Currently the syntax is like:
~~courseSales.pivot(Seq($"year"), $"course", Seq("dotNET", "Java"), sum($"earnings"))~~

~~Would we be interested in the following syntax also/alternatively? and~~

    courseSales.groupBy($"year").pivot($"course", "dotNET", "Java").agg(sum($"earnings"))
    //or
    courseSales.groupBy($"year").pivot($"course").agg(sum($"earnings"))

Later we can add it to `SQLParser`, but as Hive doesn't support it we cant add it there, right?

~~Also what would be the suggested Java friendly method signature for this?~~

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #7841 from aray/sql-pivot.
2015-11-11 16:23:24 -08:00
Josh Rosen 2d76e44b1a [SPARK-11647] Attempt to reduce time/flakiness of Thriftserver CLI and SparkSubmit tests
This patch aims to reduce the test time and flakiness of HiveSparkSubmitSuite, SparkSubmitSuite, and CliSuite.

Key changes:

- Disable IO synchronization calls for Derby writes, since durability doesn't matter for tests. This was done for HiveCompatibilitySuite in #6651 and resulted in huge test speedups.
- Add a few missing `--conf`s to disable various Spark UIs. The CliSuite, in particular, never disabled these UIs, leaving it prone to port-contention-related flakiness.
- Fix two instances where tests defined `beforeAll()` methods which were never called because the appropriate traits were not mixed in. I updated these tests suites to extend `BeforeAndAfterEach` so that they play nicely with our `ResetSystemProperties` trait.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9623 from JoshRosen/SPARK-11647.
2015-11-11 14:30:38 -08:00
Reynold Xin a9a6b80c71 [SPARK-11645][SQL] Remove OpenHashSet for the old aggregate.
Author: Reynold Xin <rxin@databricks.com>

Closes #9621 from rxin/SPARK-11645.
2015-11-11 12:48:51 -08:00
Reynold Xin df97df2b39 [SPARK-11644][SQL] Remove the option to turn off unsafe and codegen.
Author: Reynold Xin <rxin@databricks.com>

Closes #9618 from rxin/SPARK-11644.
2015-11-11 12:47:02 -08:00
Josh Rosen 529a1d3380 [SPARK-6152] Use shaded ASM5 to support closure cleaning of Java 8 compiled classes
This patch modifies Spark's closure cleaner (and a few other places) to use ASM 5, which is necessary in order to support cleaning of closures that were compiled by Java 8.

In order to avoid ASM dependency conflicts, Spark excludes ASM from all of its dependencies and uses a shaded version of ASM 4 that comes from `reflectasm` (see [SPARK-782](https://issues.apache.org/jira/browse/SPARK-782) and #232). This patch updates Spark to use a shaded version of ASM 5.0.4 that was published by the Apache XBean project; the POM used to create the shaded artifact can be found at https://github.com/apache/geronimo-xbean/blob/xbean-4.4/xbean-asm5-shaded/pom.xml.

http://movingfulcrum.tumblr.com/post/80826553604/asm-framework-50-the-missing-migration-guide was a useful resource while upgrading the code to use the new ASM5 opcodes.

I also added a new regression tests in the `java8-tests` subproject; the existing tests were insufficient to catch this bug, which only affected Scala 2.11 user code which was compiled targeting Java 8.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9512 from JoshRosen/SPARK-6152.
2015-11-11 11:16:39 -08:00
Wenchen Fan e71ba56586 [SQL][MINOR] remove newLongEncoder in functions
it may shadows the one from implicits in some case.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9629 from cloud-fan/minor.
2015-11-11 11:04:04 -08:00
Wenchen Fan ec2b807212 [SPARK-11564][SQL][FOLLOW-UP] clean up java tuple encoder
We need to support custom classes like java beans and combine them into tuple, and it's very hard to do it with the  TypeTag-based approach.
We should keep only the compose-based way to create tuple encoder.

This PR also move `Encoder` to `org.apache.spark.sql`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9567 from cloud-fan/java.
2015-11-11 10:52:23 -08:00
Wenchen Fan 9c57bc0efc [SPARK-11656][SQL] support typed aggregate in project list
insert `aEncoder` like we do in `agg`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9630 from cloud-fan/select.
2015-11-11 10:21:53 -08:00
Wenchen Fan c964fc1015 [SQL][MINOR] rename present to finish in Aggregator
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9617 from cloud-fan/tmp.
2015-11-11 10:19:09 -08:00
Wenchen Fan 1510c527b4 [SPARK-10371][SQL][FOLLOW-UP] fix code style
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9627 from cloud-fan/follow.
2015-11-11 09:33:41 -08:00
hyukjinkwon 1bc41125ee [SPARK-11500][SQL] Not deterministic order of columns when using merging schemas.
https://issues.apache.org/jira/browse/SPARK-11500

As filed in SPARK-11500, if merging schemas is enabled, the order of files to touch is a matter which might affect the ordering of the output columns.

This was mostly because of the use of `Set` and `Map` so I replaced them to `LinkedHashSet` and `LinkedHashMap` to keep the insertion order.

Also, I changed `reduceOption` to `reduceLeftOption`, and replaced the order of `filesToTouch` from `metadataStatuses ++ commonMetadataStatuses ++ needMerged` to  `needMerged ++ metadataStatuses ++ commonMetadataStatuses` in order to touch the part-files first which always have the schema in footers whereas the others might not exist.

One nit is, If merging schemas is not enabled, but when multiple files are given, there is no guarantee of the output order, since there might not be a summary file for the first file, which ends up putting ahead the columns of the other files.

However, I thought this should be okay since disabling merging schemas means (assumes) all the files have the same schemas.

In addition, in the test code for this, I only checked the names of fields.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9517 from HyukjinKwon/SPARK-11500.
2015-11-11 16:46:04 +08:00
Marc Prud'hommeaux 745e45d5ff [MINOR] License header formatting fix
The header wasn't indented properly.

Author: Marc Prud'hommeaux <mwp1@cornell.edu>

Closes #9312 from mprudhom/patch-1.
2015-11-10 16:57:12 -08:00
Forest Fang 12c7635dc0 [MINOR] Fix typo in AggregationQuerySuite.scala
Author: Forest Fang <saurfang@users.noreply.github.com>

Closes #9357 from saurfang/patch-1.
2015-11-10 16:56:06 -08:00
tedyu 9009175416 [SPARK-11615] Drop @VisibleForTesting annotation
See http://search-hadoop.com/m/q3RTtjpe8r1iRbTj2 for discussion.

Summary: addition of VisibleForTesting annotation resulted in spark-shell malfunctioning.

Author: tedyu <yuzhihong@gmail.com>

Closes #9585 from tedyu/master.
2015-11-10 16:52:59 -08:00
Herman van Hovell 21c562fa03 [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)
This PR is a 2nd follow-up for [SPARK-9241](https://issues.apache.org/jira/browse/SPARK-9241). It contains the following improvements:
* Fix for a potential bug in distinct child expression and attribute alignment.
* Improved handling of duplicate distinct child expressions.
* Added test for distinct UDAF with multiple children.

cc yhuai

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

Closes #9566 from hvanhovell/SPARK-9241-followup-2.
2015-11-10 16:28:21 -08:00
Yin Huai 3121e78168 [SPARK-9830][SPARK-11641][SQL][FOLLOW-UP] Remove AggregateExpression1 and update toString of Exchange
https://issues.apache.org/jira/browse/SPARK-9830

This is the follow-up pr for https://github.com/apache/spark/pull/9556 to address davies' comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #9607 from yhuai/removeAgg1-followup.
2015-11-10 16:25:22 -08:00
Michael Armbrust 724cf7a38c [SPARK-11616][SQL] Improve toString for Dataset
Author: Michael Armbrust <michael@databricks.com>

Closes #9586 from marmbrus/dataset-toString.
2015-11-10 14:30:19 -08:00
Nong Li 87aedc48c0 [SPARK-10371][SQL] Implement subexpr elimination for UnsafeProjections
This patch adds the building blocks for codegening subexpr elimination and implements
it end to end for UnsafeProjection. The building blocks can be used to do the same thing
for other operators.

It introduces some utilities to compute common sub expressions. Expressions can be added to
this data structure. The expr and its children will be recursively matched against existing
expressions (ones previously added) and grouped into common groups. This is built using
the existing `semanticEquals`. It does not understand things like commutative or associative
expressions. This can be done as future work.

After building this data structure, the codegen process takes advantage of it by:
  1. Generating a helper function in the generated class that computes the common
     subexpression. This is done for all common subexpressions that have at least
     two occurrences and the expression tree is sufficiently complex.
  2. When generating the apply() function, if the helper function exists, call that
     instead of regenerating the expression tree. Repeated calls to the helper function
     shortcircuit the evaluation logic.

Author: Nong Li <nong@databricks.com>
Author: Nong Li <nongli@gmail.com>

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

Closes #9480 from nongli/spark-10371.
2015-11-10 11:28:53 -08:00
Wenchen Fan 53600854c2 [SPARK-11590][SQL] use native json_tuple in lateral view
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9562 from cloud-fan/json-tuple.
2015-11-10 11:21:31 -08:00
Wenchen Fan dfcfcbcc04 [SPARK-11578][SQL][FOLLOW-UP] complete the user facing api for typed aggregation
Currently the user facing api for typed aggregation has some limitations:

* the customized typed aggregation must be the first of aggregation list
* the customized typed aggregation can only use long as buffer type
* the customized typed aggregation can only use flat type as result type

This PR tries to remove these limitations.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9599 from cloud-fan/agg.
2015-11-10 11:14:25 -08:00
Yin Huai e0701c7560 [SPARK-9830][SQL] Remove AggregateExpression1 and Aggregate Operator used to evaluate AggregateExpression1s
https://issues.apache.org/jira/browse/SPARK-9830

This PR contains the following main changes.
* Removing `AggregateExpression1`.
* Removing `Aggregate` operator, which is used to evaluate `AggregateExpression1`.
* Removing planner rule used to plan `Aggregate`.
* Linking `MultipleDistinctRewriter` to analyzer.
* Renaming `AggregateExpression2` to `AggregateExpression` and `AggregateFunction2` to `AggregateFunction`.
* Updating places where we create aggregate expression. The way to create aggregate expressions is `AggregateExpression(aggregateFunction, mode, isDistinct)`.
* Changing `val`s in `DeclarativeAggregate`s that touch children of this function to `lazy val`s (when we create aggregate expression in DataFrame API, children of an aggregate function can be unresolved).

Author: Yin Huai <yhuai@databricks.com>

Closes #9556 from yhuai/removeAgg1.
2015-11-10 11:06:29 -08:00
Davies Liu 521b3cae11 [SPARK-11598] [SQL] enable tests for ShuffledHashOuterJoin
Author: Davies Liu <davies@databricks.com>

Closes #9573 from davies/join_condition.
2015-11-09 23:28:32 -08:00
Davies Liu d6cd3a18e7 [SPARK-11599] [SQL] fix NPE when resolve Hive UDF in SQLParser
The DataFrame APIs that takes a SQL expression always use SQLParser, then the HiveFunctionRegistry will called outside of Hive state, cause NPE if there is not a active Session State for current thread (in PySpark).

cc rxin yhuai

Author: Davies Liu <davies@databricks.com>

Closes #9576 from davies/hive_udf.
2015-11-09 23:27:36 -08:00
Reynold Xin 675c7e723c [SPARK-11564][SQL] Fix documentation for DataFrame.take/collect
Author: Reynold Xin <rxin@databricks.com>

Closes #9557 from rxin/SPARK-11564-1.
2015-11-09 16:22:15 -08:00
Michael Armbrust 9c740a9ddf [SPARK-11578][SQL] User API for Typed Aggregation
This PR adds a new interface for user-defined aggregations, that can be used in `DataFrame` and `Dataset` operations to take all of the elements of a group and reduce them to a single value.

For example, the following aggregator extracts an `int` from a specific class and adds them up:

```scala
  case class Data(i: Int)

  val customSummer =  new Aggregator[Data, Int, Int] {
    def prepare(d: Data) = d.i
    def reduce(l: Int, r: Int) = l + r
    def present(r: Int) = r
  }.toColumn()

  val ds: Dataset[Data] = ...
  val aggregated = ds.select(customSummer)
```

By using helper functions, users can make a generic `Aggregator` that works on any input type:

```scala
/** An `Aggregator` that adds up any numeric type returned by the given function. */
class SumOf[I, N : Numeric](f: I => N) extends Aggregator[I, N, N] with Serializable {
  val numeric = implicitly[Numeric[N]]
  override def zero: N = numeric.zero
  override def reduce(b: N, a: I): N = numeric.plus(b, f(a))
  override def present(reduction: N): N = reduction
}

def sum[I, N : Numeric : Encoder](f: I => N): TypedColumn[I, N] = new SumOf(f).toColumn
```

These aggregators can then be used alongside other built-in SQL aggregations.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
ds
  .groupBy(_._1)
  .agg(
    sum(_._2),                // The aggregator defined above.
    expr("sum(_2)").as[Int],  // A built-in dynatically typed aggregation.
    count("*"))               // A built-in statically typed aggregation.
  .collect()

res0: ("a", 30, 30, 2L), ("b", 3, 3, 2L), ("c", 1, 1, 1L)
```

The current implementation focuses on integrating this into the typed API, but currently only supports running aggregations that return a single long value as explained in `TypedAggregateExpression`.  This will be improved in a followup PR.

Author: Michael Armbrust <michael@databricks.com>

Closes #9555 from marmbrus/dataset-useragg.
2015-11-09 16:11:00 -08:00
hyukjinkwon 9565c246ea [SPARK-9557][SQL] Refactor ParquetFilterSuite and remove old ParquetFilters code
Actually this was resolved by https://github.com/apache/spark/pull/8275.

But I found the JIRA issue for this is not marked as resolved since the PR above was made for another issue but the PR above resolved both.

I commented that this is resolved by the PR above; however, I opened this PR as I would like to just add
a little bit of corrections.

In the previous PR, I refactored the test by not reducing just collecting filters; however, this would not test  properly `And` filter (which is not given to the tests). I unintentionally changed this from the original way (before being refactored).

In this PR, I just followed the original way to collect filters by reducing.

I would like to close this if this PR is inappropriate and somebody would like this deal with it in the separate PR related with this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9554 from HyukjinKwon/SPARK-9557.
2015-11-09 15:20:50 -08:00
Wenchen Fan fcb57e9c73 [SPARK-11564][SQL][FOLLOW-UP] improve java api for GroupedDataset
created `MapGroupFunction`, `FlatMapGroupFunction`, `CoGroupFunction`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9564 from cloud-fan/map.
2015-11-09 15:16:47 -08:00
Cheng Lian 150f6a89b7 [SPARK-11595] [SQL] Fixes ADD JAR when the input path contains URL scheme
Author: Cheng Lian <lian@databricks.com>

Closes #9569 from liancheng/spark-11595.fix-add-jar.
2015-11-09 14:32:52 -08:00
Nick Buroojy f138cb8733 [SPARK-9301][SQL] Add collect_set and collect_list aggregate functions
For now they are thin wrappers around the corresponding Hive UDAFs.

One limitation with these in Hive 0.13.0 is they only support aggregating primitive types.

I chose snake_case here instead of camelCase because it seems to be used in the majority of the multi-word fns.

Do we also want to add these to `functions.py`?

This approach was recommended here: https://github.com/apache/spark/pull/8592#issuecomment-154247089

marmbrus rxin

Author: Nick Buroojy <nick.buroojy@civitaslearning.com>

Closes #9526 from nburoojy/nick/udaf-alias.

(cherry picked from commit a6ee4f989d)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-11-09 14:30:52 -08:00
Wenchen Fan d8b50f7029 [SPARK-11453][SQL] append data to partitioned table will messes up the result
The reason is that:

1. For partitioned hive table, we will move the partitioned columns after data columns. (e.g. `<a: Int, b: Int>` partition by `a` will become `<b: Int, a: Int>`)
2. When append data to table, we use position to figure out how to match input columns to table's columns.

So when we append data to partitioned table, we will match wrong columns between input and table. A solution is reordering the input columns before match by position, like what we did for [`InsertIntoHadoopFsRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala#L101-L105)

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9408 from cloud-fan/append.
2015-11-08 21:01:53 -08:00
Reynold Xin 97b7080cf2 [SPARK-11564][SQL] Dataset Java API audit
A few changes:

1. Removed fold, since it can be confusing for distributed collections.
2. Created specific interfaces for each Dataset function (e.g. MapFunction, ReduceFunction, MapPartitionsFunction)
3. Added more documentation and test cases.

The other thing I'm considering doing is to have a "collector" interface for FlatMapFunction and MapPartitionsFunction, similar to MapReduce's map function.

Author: Reynold Xin <rxin@databricks.com>

Closes #9531 from rxin/SPARK-11564.
2015-11-08 20:57:09 -08:00
Wenchen Fan b2d195e137 [SPARK-11554][SQL] add map/flatMap to GroupedDataset
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9521 from cloud-fan/map.
2015-11-08 12:59:35 -08:00
Herman van Hovell 30c8ba71a7 [SPARK-11451][SQL] Support single distinct count on multiple columns.
This PR adds support for multiple column in a single count distinct aggregate to the new aggregation path.

cc yhuai

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

Closes #9409 from hvanhovell/SPARK-11451.
2015-11-08 11:06:10 -08:00
Liang-Chi Hsieh 4b69a42eda [SPARK-11362] [SQL] Use Spark BitSet in BroadcastNestedLoopJoin
JIRA: https://issues.apache.org/jira/browse/SPARK-11362

We use scala.collection.mutable.BitSet in BroadcastNestedLoopJoin now. We should use Spark's BitSet.

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

Closes #9316 from viirya/use-spark-bitset.
2015-11-07 19:44:45 -08:00
Herman van Hovell ef362846eb [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up
This PR is a follow up for PR https://github.com/apache/spark/pull/9406. It adds more documentation to the rewriting rule, removes a redundant if expression in the non-distinct aggregation path and adds a multiple distinct test to the AggregationQuerySuite.

cc yhuai marmbrus

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

Closes #9541 from hvanhovell/SPARK-9241-followup.
2015-11-07 13:37:37 -08:00
navis.ryu 1c80d66e52 [SPARK-11546] Thrift server makes too many logs about result schema
SparkExecuteStatementOperation logs result schema for each getNextRowSet() calls which is by default every 1000 rows, overwhelming whole log file.

Author: navis.ryu <navis@apache.org>

Closes #9514 from navis/SPARK-11546.
2015-11-06 17:13:46 -08:00
Herman van Hovell 6d0ead322e [SPARK-9241][SQL] Supporting multiple DISTINCT columns (2) - Rewriting Rule
The second PR for SPARK-9241, this adds support for multiple distinct columns to the new aggregation code path.

This PR solves the multiple DISTINCT column problem by rewriting these Aggregates into an Expand-Aggregate-Aggregate combination. See the [JIRA ticket](https://issues.apache.org/jira/browse/SPARK-9241) for some information on this. The advantages over the - competing - [first PR](https://github.com/apache/spark/pull/9280) are:
- This can use the faster TungstenAggregate code path.
- It is impossible to OOM due to an ```OpenHashSet``` allocating to much memory. However, this will multiply the number of input rows by the number of distinct clauses (plus one), and puts a lot more memory pressure on the aggregation code path itself.

The location of this Rule is a bit funny, and should probably change when the old aggregation path is changed.

cc yhuai - Could you also tell me where to add tests for this?

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

Closes #9406 from hvanhovell/SPARK-9241-rewriter.
2015-11-06 16:04:20 -08:00
Wenchen Fan 7e9a9e603a [SPARK-11269][SQL] Java API support & test cases for Dataset
This simply brings https://github.com/apache/spark/pull/9358 up-to-date.

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

Closes #9528 from rxin/dataset-java.
2015-11-06 15:37:07 -08:00
Reynold Xin 3a652f691b [SPARK-11561][SQL] Rename text data source's column name to value.
Author: Reynold Xin <rxin@databricks.com>

Closes #9527 from rxin/SPARK-11561.
2015-11-06 14:47:41 -08:00
Herman van Hovell f328fedafd [SPARK-11450] [SQL] Add Unsafe Row processing to Expand
This PR enables the Expand operator to process and produce Unsafe Rows.

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

Closes #9414 from hvanhovell/SPARK-11450.
2015-11-06 12:21:53 -08:00
Imran Rashid 49f1a82037 [SPARK-10116][CORE] XORShiftRandom.hashSeed is random in high bits
https://issues.apache.org/jira/browse/SPARK-10116

This is really trivial, just happened to notice it -- if `XORShiftRandom.hashSeed` is really supposed to have random bits throughout (as the comment implies), it needs to do something for the conversion to `long`.

mengxr mkolod

Author: Imran Rashid <irashid@cloudera.com>

Closes #8314 from squito/SPARK-10116.
2015-11-06 20:06:24 +00:00
Yin Huai 8211aab079 [SPARK-9858][SQL] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins (follow-up)
https://issues.apache.org/jira/browse/SPARK-9858

This PR is the follow-up work of https://github.com/apache/spark/pull/9276. It addresses JoshRosen's comments.

Author: Yin Huai <yhuai@databricks.com>

Closes #9453 from yhuai/numReducer-followUp.
2015-11-06 11:13:51 -08:00
Cheng Lian c048929c6a [SPARK-10978][SQL][FOLLOW-UP] More comprehensive tests for PR #9399
This PR adds test cases that test various column pruning and filter push-down cases.

Author: Cheng Lian <lian@databricks.com>

Closes #9468 from liancheng/spark-10978.follow-up.
2015-11-06 11:11:36 -08:00
Liang-Chi Hsieh 574141a298 [SPARK-9162] [SQL] Implement code generation for ScalaUDF
JIRA: https://issues.apache.org/jira/browse/SPARK-9162

Currently ScalaUDF extends CodegenFallback and doesn't provide code generation implementation. This path implements code generation for ScalaUDF.

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

Closes #9270 from viirya/scalaudf-codegen.
2015-11-06 10:52:04 -08:00
Wenchen Fan 253e87e8ab [SPARK-11453][SQL][FOLLOW-UP] remove DecimalLit
A cleanup for https://github.com/apache/spark/pull/9085.

The `DecimalLit` is very similar to `FloatLit`, we can just keep one of them.
Also added low level unit test at `SqlParserSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9482 from cloud-fan/parser.
2015-11-06 06:38:49 -08:00
Reynold Xin bc5d6c0389 [SPARK-11541][SQL] Break JdbcDialects.scala into multiple files and mark various dialects as private.
Author: Reynold Xin <rxin@databricks.com>

Closes #9511 from rxin/SPARK-11541.
2015-11-05 22:03:26 -08:00
Michael Armbrust 363a476c3f [SPARK-11528] [SQL] Typed aggregations for Datasets
This PR adds the ability to do typed SQL aggregations.  We will likely also want to provide an interface to allow users to do aggregations on objects, but this is deferred to another PR.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
ds.groupBy(_._1).agg(sum("_2").as[Int]).collect()

res0: Array(("a", 30), ("b", 3), ("c", 1))
```

Author: Michael Armbrust <michael@databricks.com>

Closes #9499 from marmbrus/dataset-agg.
2015-11-05 21:42:32 -08:00
Davies Liu eec74ba8bd [SPARK-7542][SQL] Support off-heap index/sort buffer
This brings the support of off-heap memory for array inside BytesToBytesMap and InMemorySorter, then we could allocate all the memory from off-heap for execution.

Closes #8068

Author: Davies Liu <davies@databricks.com>

Closes #9477 from davies/unsafe_timsort.
2015-11-05 19:02:18 -08:00
Reynold Xin 3cc2c053b5 [SPARK-11540][SQL] API audit for QueryExecutionListener.
Author: Reynold Xin <rxin@databricks.com>

Closes #9509 from rxin/SPARK-11540.
2015-11-05 18:12:54 -08:00
Reynold Xin 6091e91fca Revert "[SPARK-11469][SQL] Allow users to define nondeterministic udfs."
This reverts commit 9cf56c96b7.
2015-11-05 17:10:35 -08:00
Davies Liu 07414afac9 [SPARK-11537] [SQL] fix negative hours/minutes/seconds
Currently, if the Timestamp is before epoch (1970/01/01), the hours, minutes and seconds will be negative (also rounding up).

Author: Davies Liu <davies@databricks.com>

Closes #9502 from davies/neg_hour.
2015-11-05 17:02:22 -08:00
Reynold Xin b6974f8fed [SPARK-11536][SQL] Remove the internal implicit conversion from Expression to Column in functions.scala
Author: Reynold Xin <rxin@databricks.com>

Closes #9505 from rxin/SPARK-11536.
2015-11-05 15:34:05 -08:00
Wenchen Fan d9e30c59ce [SPARK-10656][SQL] completely support special chars in DataFrame
the main problem is: we interpret column name with special handling of `.` for DataFrame. This enables us to write something like `df("a.b")` to get the field `b` of `a`. However, we don't need this feature in `DataFrame.apply("*")` or `DataFrame.withColumnRenamed`. In these 2 cases, the column name is the final name already, we don't need extra process to interpret it.

The solution is simple, use `queryExecution.analyzed.output` to get resolved column directly, instead of using `DataFrame.resolve`.

close https://github.com/apache/spark/pull/8811

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9462 from cloud-fan/special-chars.
2015-11-05 14:53:16 -08:00
Reynold Xin 8a5314efd1 [SPARK-11532][SQL] Remove implicit conversion from Expression to Column
Author: Reynold Xin <rxin@databricks.com>

Closes #9500 from rxin/SPARK-11532.
2015-11-05 13:34:36 -08:00
Travis Hegner 14ee0f5726 [SPARK-10648] Oracle dialect to handle nonspecific numeric types
This is the alternative/agreed upon solution to PR #8780.

Creating an OracleDialect to handle the nonspecific numeric types that can be defined in oracle.

Author: Travis Hegner <thegner@trilliumit.com>

Closes #9495 from travishegner/OracleDialect.
2015-11-05 12:36:57 -08:00
Reynold Xin 6b87acd664 [SPARK-11513][SQL] Remove implicit conversion from LogicalPlan to DataFrame
This internal implicit conversion has been a source of confusion for a lot of new developers.

Author: Reynold Xin <rxin@databricks.com>

Closes #9479 from rxin/SPARK-11513.
2015-11-05 11:58:13 -08:00
Huaxin Gao b072ff4d1d [SPARK-11474][SQL] change fetchSize to fetchsize
In DefaultDataSource.scala, it has
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String]): BaseRelation
The parameters is CaseInsensitiveMap.
After this line
parameters.foreach(kv => properties.setProperty(kv._1, kv._2))
properties is set to all lower case key/value pairs and fetchSize becomes fetchsize.
However, in compute method in JDBCRDD, it has
val fetchSize = properties.getProperty("fetchSize", "0").toInt
so fetchSize value is always 0 and never gets set correctly.

Author: Huaxin Gao <huaxing@oc0558782468.ibm.com>

Closes #9473 from huaxingao/spark-11474.
2015-11-05 09:41:14 -08:00
Cheng Lian 77488fb8e5 [MINOR][SQL] A minor log line fix
`jars` in the log line is an array, so `$jars` doesn't print its content.

Author: Cheng Lian <lian@databricks.com>

Closes #9494 from liancheng/minor.log-fix.
2015-11-05 23:49:44 +08:00
Sean Owen 6f81eae24f [SPARK-11440][CORE][STREAMING][BUILD] Declare rest of @Experimental items non-experimental if they've existed since 1.2.0
Remove `Experimental` annotations in core, streaming for items that existed in 1.2.0 or before. The changes are:

* SparkContext
  * binary{Files,Records} : 1.2.0
  * submitJob : 1.0.0
* JavaSparkContext
  * binary{Files,Records} : 1.2.0
* DoubleRDDFunctions, JavaDoubleRDD
  * {mean,sum}Approx : 1.0.0
* PairRDDFunctions, JavaPairRDD
  * sampleByKeyExact : 1.2.0
  * countByKeyApprox : 1.0.0
* PairRDDFunctions
  * countApproxDistinctByKey : 1.1.0
* RDD
  * countApprox, countByValueApprox, countApproxDistinct : 1.0.0
* JavaRDDLike
  * countApprox : 1.0.0
* PythonHadoopUtil.Converter : 1.1.0
* PortableDataStream : 1.2.0 (related to binaryFiles)
* BoundedDouble : 1.0.0
* PartialResult : 1.0.0
* StreamingContext, JavaStreamingContext
  * binaryRecordsStream : 1.2.0
* HiveContext
  * analyze : 1.2.0

Author: Sean Owen <sowen@cloudera.com>

Closes #9396 from srowen/SPARK-11440.
2015-11-05 09:08:53 +00:00
Davies Liu 81498dd5c8 [SPARK-11425] [SPARK-11486] Improve hybrid aggregation
After aggregation, the dataset could be smaller than inputs, so it's better to do hash based aggregation for all inputs, then using sort based aggregation to merge them.

Author: Davies Liu <davies@databricks.com>

Closes #9383 from davies/fix_switch.
2015-11-04 21:30:21 -08:00
Zhenhua Wang a752ddad7f [SPARK-11398] [SQL] unnecessary def dialectClassName in HiveContext, and misleading dialect conf at the start of spark-sql
1. def dialectClassName in HiveContext is unnecessary.
In HiveContext, if conf.dialect == "hiveql", getSQLDialect() will return new HiveQLDialect(this);
else it will use super.getSQLDialect(). Then in super.getSQLDialect(), it calls dialectClassName, which is overriden in HiveContext and still return super.dialectClassName.
So we'll never reach the code "classOf[HiveQLDialect].getCanonicalName" of def dialectClassName in HiveContext.

2. When we start bin/spark-sql, the default context is HiveContext, and the corresponding dialect is hiveql.
However, if we type "set spark.sql.dialect;", the result is "sql", which is inconsistent with the actual dialect and is misleading. For example, we can use sql like "create table" which is only allowed in hiveql, but this dialect conf shows it's "sql".
Although this problem will not cause any execution error, it's misleading to spark sql users. Therefore I think we should fix it.
In this pr, while procesing “set spark.sql.dialect” in SetCommand, I use "conf.dialect" instead of "getConf()" for the case of key == SQLConf.DIALECT.key, so that it will return the right dialect conf.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #9349 from wzhfy/dialect.
2015-11-04 17:16:00 -08:00
Reynold Xin b6e0a5ae6f [SPARK-11510][SQL] Remove SQL aggregation tests for higher order statistics
We have some aggregate function tests in both DataFrameAggregateSuite and SQLQuerySuite. The two have almost the same coverage and we should just remove the SQL one.

Author: Reynold Xin <rxin@databricks.com>

Closes #9475 from rxin/SPARK-11510.
2015-11-04 16:49:25 -08:00
Reynold Xin d19f4fda63 [SPARK-11505][SQL] Break aggregate functions into multiple files
functions.scala was getting pretty long. I broke it into multiple files.

I also added explicit data types for some public vals, and renamed aggregate function pretty names to lower case, which is more consistent with rest of the functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #9471 from rxin/SPARK-11505.
2015-11-04 13:44:07 -08:00
Reynold Xin abf5e4285d [SPARK-11504][SQL] API audit for distributeBy and localSort
1. Renamed localSort -> sortWithinPartitions to avoid ambiguity in "local"
2. distributeBy -> repartition to match the existing repartition.

Author: Reynold Xin <rxin@databricks.com>

Closes #9470 from rxin/SPARK-11504.
2015-11-04 12:33:47 -08:00
Liang-Chi Hsieh de289bf279 [SPARK-10304][SQL] Following up checking valid dir structure for partition discovery
This patch follows up #8840.

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

Closes #9459 from viirya/detect_invalid_part_dir_following.
2015-11-04 10:56:32 -08:00
Reynold Xin 3bd6f5d2ae [SPARK-11490][SQL] variance should alias var_samp instead of var_pop.
stddev is an alias for stddev_samp. variance should be consistent with stddev.

Also took the chance to remove internal Stddev and Variance, and only kept StddevSamp/StddevPop and VarianceSamp/VariancePop.

Author: Reynold Xin <rxin@databricks.com>

Closes #9449 from rxin/SPARK-11490.
2015-11-04 09:34:52 -08:00
Reynold Xin cd1df66238 [SPARK-11485][SQL] Make DataFrameHolder and DatasetHolder public.
These two classes should be public, since they are used in public code.

Author: Reynold Xin <rxin@databricks.com>

Closes #9445 from rxin/SPARK-11485.
2015-11-04 09:32:30 -08:00
Wenchen Fan 2692bdb7db [SPARK-11455][SQL] fix case sensitivity of partition by
depend on `caseSensitive` to do column name equality check, instead of just `==`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9410 from cloud-fan/partition.
2015-11-03 20:25:58 -08:00
Nong e352de0db2 [SPARK-11329] [SQL] Cleanup from spark-11329 fix.
Author: Nong <nong@cloudera.com>

Closes #9442 from nongli/spark-11483.
2015-11-03 16:44:37 -08:00
Reynold Xin 5051262d4c [SPARK-11489][SQL] Only include common first order statistics in GroupedData
We added a bunch of higher order statistics such as skewness and kurtosis to GroupedData. I don't think they are common enough to justify being listed, since users can always use the normal statistics aggregate functions.

That is to say, after this change, we won't support
```scala
df.groupBy("key").kurtosis("colA", "colB")
```

However, we will still support
```scala
df.groupBy("key").agg(kurtosis(col("colA")), kurtosis(col("colB")))
```

Author: Reynold Xin <rxin@databricks.com>

Closes #9446 from rxin/SPARK-11489.
2015-11-03 16:27:56 -08:00
Wenchen Fan f6fcb4874c [SPARK-11477] [SQL] support create Dataset from RDD
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9434 from cloud-fan/rdd2ds and squashes the following commits:

0892d72 [Wenchen Fan] support create Dataset from RDD
2015-11-04 00:15:50 +01:00
Davies Liu 1d04dc95c0 [SPARK-11467][SQL] add Python API for stddev/variance
Add Python API for stddev/stddev_pop/stddev_samp/variance/var_pop/var_samp/skewness/kurtosis

Author: Davies Liu <davies@databricks.com>

Closes #9424 from davies/py_var.
2015-11-03 13:33:46 -08:00
Cheng Lian ebf8b0b48d [SPARK-10978][SQL] Allow data sources to eliminate filters
This PR adds a new method `unhandledFilters` to `BaseRelation`. Data sources which implement this method properly may avoid the overhead of defensive filtering done by Spark SQL.

Author: Cheng Lian <lian@databricks.com>

Closes #9399 from liancheng/spark-10978.unhandled-filters.
2015-11-03 10:07:45 -08:00
Liang-Chi Hsieh d6035d97c9 [SPARK-10304] [SQL] Partition discovery should throw an exception if the dir structure is invalid
JIRA: https://issues.apache.org/jira/browse/SPARK-10304

This patch detects if the structure of partition directories is not valid.

The test cases are from #8547. Thanks zhzhan.

cc liancheng

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

Closes #8840 from viirya/detect_invalid_part_dir.
2015-11-03 07:41:50 -08:00
Daoyuan Wang d188a67762 [SPARK-10533][SQL] handle scientific notation in sqlParser
https://issues.apache.org/jira/browse/SPARK-10533

val df = sqlContext.createDataFrame(Seq(("a",1.0),("b",2.0),("c",3.0)))
df.filter("_2 < 2.0e1").show

Scientific notation didn't work.

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

Closes #9085 from adrian-wang/scinotation.
2015-11-03 22:30:23 +08:00
Michael Armbrust b86f2cab67 [SPARK-11404] [SQL] Support for groupBy using column expressions
This PR adds a new method `groupBy(cols: Column*)` to `Dataset` that allows users to group using column expressions instead of a lambda function.  Since the return type of these expressions is not known at compile time, we just set the key type as a generic `Row`.  If the user would like to work the key in a type-safe way, they can call `grouped.asKey[Type]`, which is also added in this PR.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupBy($"_1").asKey[String]
val agged = grouped.mapGroups { case (g, iter) =>
  Iterator((g, iter.map(_._2).sum))
}

agged.collect()

res0: Array(("a", 30), ("b", 3), ("c", 1))
```

Author: Michael Armbrust <michael@databricks.com>

Closes #9359 from marmbrus/columnGroupBy and squashes the following commits:

bbcb03b [Michael Armbrust] Update DatasetSuite.scala
8fd2908 [Michael Armbrust] Update DatasetSuite.scala
0b0e2f8 [Michael Armbrust] [SPARK-11404] [SQL] Support for groupBy using column expressions
2015-11-03 13:02:17 +01:00
Wenchen Fan 425ff03f5a [SPARK-11436] [SQL] rebind right encoder when join 2 datasets
When we join 2 datasets, we will combine 2 encoders into a tupled one, and use it as the encoder for the jioned dataset. Assume both of the 2 encoders are flat, their `constructExpression`s both reference to the first element of input row. However, when we combine 2 encoders, the schema of input row changed,  now the right encoder should reference to second element of input row. So we should rebind right encoder to let it know the new schema of input row before combine it.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9391 from cloud-fan/join and squashes the following commits:

846d3ab [Wenchen Fan] rebind right encoder when join 2 datasets
2015-11-03 12:47:39 +01:00
Davies Liu 67e23b39ac [SPARK-10429] [SQL] make mutableProjection atomic
Right now, SQL's mutable projection updates every value of the mutable project after it evaluates the corresponding expression. This makes the behavior of MutableProjection confusing and complicate the implementation of common aggregate functions like stddev because developers need to be aware that when evaluating {{i+1}}th expression of a mutable projection, {{i}}th slot of the mutable row has already been updated.

This PR make the MutableProjection atomic, by generating all the results of expressions first, then copy them into mutableRow.

Had run a mircro-benchmark, there is no notable performance difference between using class members and local variables.

cc yhuai

Author: Davies Liu <davies@databricks.com>

Closes #9422 from davies/atomic_mutable and squashes the following commits:

bbc1758 [Davies Liu] support wide table
8a0ae14 [Davies Liu] fix bug
bec07da [Davies Liu] refactor
2891628 [Davies Liu] make mutableProjection atomic
2015-11-03 11:42:08 +01:00
Yin Huai d728d5c986 [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins
https://issues.apache.org/jira/browse/SPARK-9858
https://issues.apache.org/jira/browse/SPARK-9859
https://issues.apache.org/jira/browse/SPARK-9861

Author: Yin Huai <yhuai@databricks.com>

Closes #9276 from yhuai/numReducer.
2015-11-03 00:12:49 -08:00
navis.ryu c34c27fe92 [SPARK-9034][SQL] Reflect field names defined in GenericUDTF
Hive GenericUDTF#initialize() defines field names in a returned schema though,
the current HiveGenericUDTF drops these names.
We might need to reflect these in a logical plan tree.

Author: navis.ryu <navis@apache.org>

Closes #8456 from navis/SPARK-9034.
2015-11-02 23:52:36 -08:00
Yin Huai 9cf56c96b7 [SPARK-11469][SQL] Allow users to define nondeterministic udfs.
This is the first task (https://issues.apache.org/jira/browse/SPARK-11469) of https://issues.apache.org/jira/browse/SPARK-11438

Author: Yin Huai <yhuai@databricks.com>

Closes #9393 from yhuai/udfNondeterministic.
2015-11-02 21:18:38 -08:00
Nong Li 9cb5c731da [SPARK-11329][SQL] Support star expansion for structs.
1. Supporting expanding structs in Projections. i.e.
  "SELECT s.*" where s is a struct type.
  This is fixed by allowing the expand function to handle structs in addition to tables.

2. Supporting expanding * inside aggregate functions of structs.
   "SELECT max(struct(col1, structCol.*))"
   This requires recursively expanding the expressions. In this case, it it the aggregate
   expression "max(...)" and we need to recursively expand its children inputs.

Author: Nong Li <nongli@gmail.com>

Closes #9343 from nongli/spark-11329.
2015-11-02 20:32:08 -08:00
Nong Li 2cef1bb0b5 [SPARK-5354][SQL] Cached tables should preserve partitioning and ord…
…ering.

For cached tables, we can just maintain the partitioning and ordering from the
source relation.

Author: Nong Li <nongli@gmail.com>

Closes #9404 from nongli/spark-5354.
2015-11-02 19:18:45 -08:00
tedyu db11ee5e56 [SPARK-11371] Make "mean" an alias for "avg" operator
From Reynold in the thread 'Exception when using some aggregate operators' (http://search-hadoop.com/m/q3RTt0xFr22nXB4/):

I don't think these are bugs. The SQL standard for average is "avg", not "mean". Similarly, a distinct count is supposed to be written as "count(distinct col)", not "countDistinct(col)".
We can, however, make "mean" an alias for "avg" to improve compatibility between DataFrame and SQL.

Author: tedyu <yuzhihong@gmail.com>

Closes #9332 from ted-yu/master.
2015-11-02 13:51:53 -08:00
Daoyuan Wang 74ba95228d [SPARK-11311][SQL] spark cannot describe temporary functions
When describe temporary function, spark would return 'Unable to find function', this is not right.

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

Closes #9277 from adrian-wang/functionreg.
2015-11-02 23:07:30 +08:00
huangzhaowei 767522dc4e [SPARK-10786][SQL] Take the whole statement to generate the CommandProcessor
In the now implementation of `SparkSQLCLIDriver.scala`:
`val proc: CommandProcessor = CommandProcessorFactory.get(Array(tokens(0)), hconf)`
`CommandProcessorFactory` only take the first token of the statement, and this will be hard to diff the statement `delete jar xxx` and `delete from xxx`.
So maybe it's better to take the whole statement into the `CommandProcessorFactory`.

And in [HiveCommand](https://github.com/SaintBacchus/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java#L76), it already special handing these two statement.
```java
if(command.length > 1 && "from".equalsIgnoreCase(command[1])) {
  //special handling for SQL "delete from <table> where..."
  return null;
}
```

Author: huangzhaowei <carlmartinmax@gmail.com>

Closes #8895 from SaintBacchus/SPARK-10786.
2015-11-02 21:31:10 +08:00
Liang-Chi Hsieh 3e770a64a4 [SPARK-9298][SQL] Add pearson correlation aggregation function
JIRA: https://issues.apache.org/jira/browse/SPARK-9298

This patch adds pearson correlation aggregation function based on `AggregateExpression2`.

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

Closes #8587 from viirya/corr_aggregation.
2015-11-01 18:37:27 -08:00
Nong Li 046e32ed84 [SPARK-11410][SQL] Add APIs to provide functionality similar to Hive's DISTRIBUTE BY and SORT BY.
DISTRIBUTE BY allows the user to hash partition the data by specified exprs. It also allows for
optioning sorting within each resulting partition. There is no required relationship between the
exprs for partitioning and sorting (i.e. one does not need to be a prefix of the other).

This patch adds to APIs to DataFrames which can be used together to provide this functionality:
  1. distributeBy() which partitions the data frame into a specified number of partitions using the
     partitioning exprs.
  2. localSort() which sorts each partition using the provided sorting exprs.

To get the DISTRIBUTE BY functionality, the user simply does: df.distributeBy(...).localSort(...)

Author: Nong Li <nongli@gmail.com>

Closes #9364 from nongli/spark-11410.
2015-11-01 14:34:06 -08:00
Cheng Lian aa494a9c2e [SPARK-11117] [SPARK-11345] [SQL] Makes all HadoopFsRelation data sources produce UnsafeRow
This PR fixes two issues:

1.  `PhysicalRDD.outputsUnsafeRows` is always `false`

    Thus a `ConvertToUnsafe` operator is often required even if the underlying data source relation does output `UnsafeRow`.

1.  Internal/external row conversion for `HadoopFsRelation` is kinda messy

    Currently we're using `HadoopFsRelation.needConversion` and [dirty type erasure hacks][1] to indicate whether the relation outputs external row or internal row and apply external-to-internal conversion when necessary.  Basically, all builtin `HadoopFsRelation` data sources, i.e. Parquet, JSON, ORC, and Text output `InternalRow`, while typical external `HadoopFsRelation` data sources, e.g. spark-avro and spark-csv, output `Row`.

This PR adds a `private[sql]` interface method `HadoopFsRelation.buildInternalScan`, which by default invokes `HadoopFsRelation.buildScan` and converts `Row`s to `UnsafeRow`s (which are also `InternalRow`s).  All builtin `HadoopFsRelation` data sources override this method and directly output `UnsafeRow`s.  In this way, now `HadoopFsRelation` always produces `UnsafeRow`s. Thus `PhysicalRDD.outputsUnsafeRows` can be properly set by checking whether the underlying data source is a `HadoopFsRelation`.

A remaining question is that, can we assume that all non-builtin `HadoopFsRelation` data sources output external rows?  At least all well known ones do so.  However it's possible that some users implemented their own `HadoopFsRelation` data sources that leverages `InternalRow` and thus all those unstable internal data representations.  If this assumption is safe, we can deprecate `HadoopFsRelation.needConversion` and cleanup some more conversion code (like [here][2] and [here][3]).

This PR supersedes #9125.

Follow-ups:

1.  Makes JSON and ORC data sources output `UnsafeRow` directly

1.  Makes `HiveTableScan` output `UnsafeRow` directly

    This is related to 1 since ORC data source shares the same `Writable` unwrapping code with `HiveTableScan`.

[1]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala#L353
[2]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala#L331-L335
[3]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala#L630-L669

Author: Cheng Lian <lian@databricks.com>

Closes #9305 from liancheng/spark-11345.unsafe-hadoop-fs-relation.
2015-10-31 21:16:09 -07:00
Dilip Biswal fc27dfbf0f [SPARK-11024][SQL] Optimize NULL in <inlist-expressions> by folding it to Literal(null)
Add a rule in optimizer to convert NULL [NOT] IN (expr1,...,expr2) to
Literal(null).

This is a follow up defect to SPARK-8654

cloud-fan Can you please take a look ?

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

Closes #9348 from dilipbiswal/spark_11024.
2015-10-31 12:55:33 -07:00
Jeff Zhang 97b3c8fb47 [SPARK-11226][SQL] Empty line in json file should be skipped
Currently the empty line in json file will be parsed into Row with all null field values. But in json, "{}" represents a json object, empty line is supposed to be skipped.

Make a trivial change for this.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #9211 from zjffdu/SPARK-11226.
2015-10-31 11:10:37 +00:00
Yin Huai 3c471885dc [SPARK-11434][SPARK-11103][SQL] Fix test ": Filter applied on merged Parquet schema with new column fails"
https://issues.apache.org/jira/browse/SPARK-11434

Author: Yin Huai <yhuai@databricks.com>

Closes #9387 from yhuai/SPARK-11434.
2015-10-30 20:05:07 -07:00
Davies Liu 45029bfdea [SPARK-11423] remove MapPartitionsWithPreparationRDD
Since we do not need to preserve a page before calling compute(), MapPartitionsWithPreparationRDD is not needed anymore.

This PR basically revert #8543, #8511, #8038, #8011

Author: Davies Liu <davies@databricks.com>

Closes #9381 from davies/remove_prepare2.
2015-10-30 15:47:40 -07:00
Wenchen Fan 14d08b9908 [SPARK-11393] [SQL] CoGroupedIterator should respect the fact that GroupedIterator.hasNext is not idempotent
When we cogroup 2 `GroupedIterator`s in `CoGroupedIterator`, if the right side is smaller, we will consume right data and keep the left data unchanged. Then we call `hasNext` which will call `left.hasNext`. This will make `GroupedIterator` generate an extra group as the previous one has not been comsumed yet.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9346 from cloud-fan/cogroup and squashes the following commits:

9be67c8 [Wenchen Fan] SPARK-11393
2015-10-30 12:17:51 +01:00
hyukjinkwon 59db9e9c38 [SPARK-11103][SQL] Filter applied on Merged Parquet shema with new column fail
When enabling mergedSchema and predicate filter, this fails since Parquet does not accept filters pushed down when the columns of the filters do not exist in the schema.
This is related with Parquet issue (https://issues.apache.org/jira/browse/PARQUET-389).

For now, it just simply disables predicate push down when using merged schema in this PR.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9327 from HyukjinKwon/SPARK-11103.
2015-10-30 18:17:35 +08:00
Davies Liu eb59b94c45 [SPARK-11417] [SQL] no @Override in codegen
Older version of Janino (>2.7) does not support Override, we should not use that in codegen.

Author: Davies Liu <davies@databricks.com>

Closes #9372 from davies/no_override.
2015-10-30 00:36:20 -07:00
Davies Liu 56419cf11f [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPARK-10929] [SQL] Cooperative memory management
This PR introduce a mechanism to call spill() on those SQL operators that support spilling (for example, BytesToBytesMap, UnsafeExternalSorter and ShuffleExternalSorter) if there is not enough memory for execution. The preserved first page is needed anymore, so removed.

Other Spillable objects in Spark core (ExternalSorter and AppendOnlyMap) are not included in this PR, but those could benefit from this (trigger others' spilling).

The PrepareRDD may be not needed anymore, could be removed in follow up PR.

The following script will fail with OOM before this PR, finished in 150 seconds with 2G heap (also works in 1.5 branch, with similar duration).

```python
sqlContext.setConf("spark.sql.shuffle.partitions", "1")
df = sqlContext.range(1<<25).selectExpr("id", "repeat(id, 2) as s")
df2 = df.select(df.id.alias('id2'), df.s.alias('s2'))
j = df.join(df2, df.id==df2.id2).groupBy(df.id).max("id", "id2")
j.explain()
print j.count()
```

For thread-safety, here what I'm got:

1) Without calling spill(), the operators should only be used by single thread, no safety problems.

2) spill() could be triggered in two cases, triggered by itself, or by other operators. we can check trigger == this in spill(), so it's still in the same thread, so safety problems.

3) if it's triggered by other operators (right now cache will not trigger spill()), we only spill the data into disk when it's in scanning stage (building is finished), so the in-memory sorter or memory pages are read-only, we only need to synchronize the iterator and change it.

4) During scanning, the iterator will only use one record in one page, we can't free this page, because the downstream is currently using it (used by UnsafeRow or other objects). In BytesToBytesMap, we just skip the current page, and dump all others into disk. In UnsafeExternalSorter, we keep the page that is used by current record (having the same baseObject), free it when loading the next record. In ShuffleExternalSorter, the spill() will not trigger during scanning.

5) In order to avoid deadlock, we didn't call acquireMemory during spill (so we reused the pointer array in InMemorySorter).

Author: Davies Liu <davies@databricks.com>

Closes #9241 from davies/force_spill.
2015-10-29 23:38:06 -07:00
Wenchen Fan 96cf87f66d [SPARK-11301] [SQL] fix case sensitivity for filter on partitioned columns
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9271 from cloud-fan/filter.
2015-10-29 16:36:52 -07:00
sethah a01cbf5daa [SPARK-10641][SQL] Add Skewness and Kurtosis Support
Implementing skewness and kurtosis support based on following algorithm:
https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics

Author: sethah <seth.hendrickson16@gmail.com>

Closes #9003 from sethah/SPARK-10641.
2015-10-29 11:58:39 -07:00
Dilip Biswal 8185f038c1 [SPARK-11188][SQL] Elide stacktraces in bin/spark-sql for AnalysisExceptions
Only print the error message to the console for Analysis Exceptions in sql-shell.

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

Closes #9194 from dilipbiswal/spark-11188.
2015-10-29 18:29:50 +01:00
xin Wu f7a51deeba [SPARK-11246] [SQL] Table cache for Parquet broken in 1.5
The root cause is that when spark.sql.hive.convertMetastoreParquet=true by default, the cached InMemoryRelation of the ParquetRelation can not be looked up from the cachedData of CacheManager because the key comparison fails even though it is the same LogicalPlan representing the Subquery that wraps the ParquetRelation.
The solution in this PR is overriding the LogicalPlan.sameResult function in Subquery case class to eliminate subquery node first before directly comparing the child (ParquetRelation), which will find the key  to the cached InMemoryRelation.

Author: xin Wu <xinwu@us.ibm.com>

Closes #9326 from xwu0226/spark-11246-commit.
2015-10-29 07:42:46 -07:00
Wenchen Fan f79ebf2a9e [SPARK-11370] [SQL] fix a bug in GroupedIterator and create unit test for it
Before this PR, user has to consume the iterator of one group before process next group, or we will get into infinite loops.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9330 from cloud-fan/group.
2015-10-29 11:49:45 +01:00
Wenchen Fan 87f28fc240 [SPARK-11379][SQL] ExpressionEncoder can't handle top level primitive type correctly
For inner primitive type(e.g. inside `Product`), we use `schemaFor` to get the catalyst type for it, https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala#L403.

However, for top level primitive type, we use `dataTypeFor`, which is wrong.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9337 from cloud-fan/encoder.
2015-10-29 11:17:03 +01:00
Wenchen Fan 0cb7662d86 [SPARK-11351] [SQL] support hive interval literal
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9304 from cloud-fan/interval.
2015-10-28 21:35:57 -07:00
Cheng Lian e5b89978ed [SPARK-11376][SQL] Removes duplicated mutableRow field
This PR fixes a mistake in the code generated by `GenerateColumnAccessor`. Interestingly, although the code is illegal in Java (the class has two fields with the same name), Janino accepts it happily and accidentally works properly.

Author: Cheng Lian <lian@databricks.com>

Closes #9335 from liancheng/spark-11376.fix-generated-code.
2015-10-29 11:34:54 +08:00
Liang-Chi Hsieh 20dfd46743 [SPARK-11363] [SQL] LeftSemiJoin should be LeftSemi in SparkStrategies
JIRA: https://issues.apache.org/jira/browse/SPARK-11363

In SparkStrategies some places use LeftSemiJoin. It should be LeftSemi.

cc chenghao-intel liancheng

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

Closes #9318 from viirya/no-left-semi-join.
2015-10-28 15:57:01 -07:00
Michael Armbrust 032748bb9a [SPARK-11377] [SQL] withNewChildren should not convert StructType to Seq
This is minor, but I ran into while writing Datasets and while it wasn't needed for the final solution, it was super confusing so we should fix it.

Basically we recurse into `Seq` to see if they have children.  This breaks because we don't preserve the original subclass of `Seq` (and `StructType <:< Seq[StructField]`).  Since a struct can never contain children, lets just not recurse into it.

Author: Michael Armbrust <michael@databricks.com>

Closes #9334 from marmbrus/structMakeCopy.
2015-10-28 09:40:05 -07:00
Wenchen Fan 075ce4914f [SPARK-11313][SQL] implement cogroup on DataSets (support 2 datasets)
A simpler version of https://github.com/apache/spark/pull/9279, only support 2 datasets.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9324 from cloud-fan/cogroup2.
2015-10-28 13:58:52 +01:00
Cheng Hao d9c6039897 [SPARK-10484] [SQL] Optimize the cartesian join with broadcast join for some cases
In some cases, we can broadcast the smaller relation in cartesian join, which improve the performance significantly.

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

Closes #8652 from chenghao-intel/cartesian.
2015-10-27 20:26:38 -07:00
Michael Armbrust 5a5f65905a [SPARK-11347] [SQL] Support for joinWith in Datasets
This PR adds a new operation `joinWith` to a `Dataset`, which returns a `Tuple` for each pair where a given `condition` evaluates to true.

```scala
case class ClassData(a: String, b: Int)

val ds1 = Seq(ClassData("a", 1), ClassData("b", 2)).toDS()
val ds2 = Seq(("a", 1), ("b", 2)).toDS()

> ds1.joinWith(ds2, $"_1" === $"a").collect()
res0: Array((ClassData("a", 1), ("a", 1)), (ClassData("b", 2), ("b", 2)))
```

This operation is similar to the relation `join` function with one important difference in the result schema. Since `joinWith` preserves objects present on either side of the join, the result schema is similarly nested into a tuple under the column names `_1` and `_2`.

This type of join can be useful both for preserving type-safety with the original object types as well as working with relational data where either side of the join has column names in common.

## Required Changes to Encoders
In the process of working on this patch, several deficiencies to the way that we were handling encoders were discovered.  Specifically, it turned out to be very difficult to `rebind` the non-expression based encoders to extract the nested objects from the results of joins (and also typed selects that return tuples).

As a result the following changes were made.
 - `ClassEncoder` has been renamed to `ExpressionEncoder` and has been improved to also handle primitive types.  Additionally, it is now possible to take arbitrary expression encoders and rewrite them into a single encoder that returns a tuple.
 - All internal operations on `Dataset`s now require an `ExpressionEncoder`.  If the users tries to pass a non-`ExpressionEncoder` in, an error will be thrown.  We can relax this requirement in the future by constructing a wrapper class that uses expressions to project the row to the expected schema, shielding the users code from the required remapping.  This will give us a nice balance where we don't force user encoders to understand attribute references and binding, but still allow our native encoder to leverage runtime code generation to construct specific encoders for a given schema that avoid an extra remapping step.
 - Additionally, the semantics for different types of objects are now better defined.  As stated in the `ExpressionEncoder` scaladoc:
  - Classes will have their sub fields extracted by name using `UnresolvedAttribute` expressions
  and `UnresolvedExtractValue` expressions.
  - Tuples will have their subfields extracted by position using `BoundReference` expressions.
  - Primitives will have their values extracted from the first ordinal with a schema that defaults
  to the name `value`.
 - Finally, the binding lifecycle for `Encoders` has now been unified across the codebase.  Encoders are now `resolved` to the appropriate schema in the constructor of `Dataset`.  This process replaces an unresolved expressions with concrete `AttributeReference` expressions.  Binding then happens on demand, when an encoder is going to be used to construct an object.  This closely mirrors the lifecycle for standard expressions when executing normal SQL or `DataFrame` queries.

Author: Michael Armbrust <michael@databricks.com>

Closes #9300 from marmbrus/datasets-tuples.
2015-10-27 13:28:52 -07:00
Yanbo Liang 360ed832f5 [SPARK-11303][SQL] filter should not be pushed down into sample
When sampling and then filtering DataFrame, the SQL Optimizer will push down filter into sample and produce wrong result. This is due to the sampler is calculated based on the original scope rather than the scope after filtering.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #9294 from yanboliang/spark-11303.
2015-10-27 11:28:59 +01:00
Jia Li 958a0ec8fa [SPARK-11277][SQL] sort_array throws exception scala.MatchError
I'm new to spark. I was trying out the sort_array function then hit this exception. I looked into the spark source code. I found the root cause is that sort_array does not check for an array of NULLs. It's not meaningful to sort an array of entirely NULLs anyway.

I'm adding a check on the input array type to SortArray. If the array consists of NULLs entirely, there is no need to sort such array. I have also added a test case for this.

Please help to review my fix. Thanks!

Author: Jia Li <jiali@us.ibm.com>

Closes #9247 from jliwork/SPARK-11277.
2015-10-27 10:57:08 +01:00
Wenchen Fan a150e6c1b0 [SPARK-10562] [SQL] support mixed case partitionBy column names for tables stored in metastore
https://issues.apache.org/jira/browse/SPARK-10562

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9226 from cloud-fan/par.
2015-10-26 21:14:26 -07:00
Stephen De Gennaro 82464fb2e0 [SPARK-10947] [SQL] With schema inference from JSON into a Dataframe, add option to infer all primitive object types as strings
Currently, when a schema is inferred from a JSON file using sqlContext.read.json, the primitive object types are inferred as string, long, boolean, etc.

However, if the inferred type is too specific (JSON obviously does not enforce types itself), this can cause issues with merging dataframe schemas.

This pull request adds the option "primitivesAsString" to the JSON DataFrameReader which when true (defaults to false if not set) will infer all primitives as strings.

Below is an example usage of this new functionality.
```
val jsonDf = sqlContext.read.option("primitivesAsString", "true").json(sampleJsonFile)

scala> jsonDf.printSchema()
root
|-- bigInteger: string (nullable = true)
|-- boolean: string (nullable = true)
|-- double: string (nullable = true)
|-- integer: string (nullable = true)
|-- long: string (nullable = true)
|-- null: string (nullable = true)
|-- string: string (nullable = true)
```

Author: Stephen De Gennaro <stepheng@realitymine.com>

Closes #9249 from stephend-realitymine/stephend-primitives.
2015-10-26 19:55:10 -07:00
Nong Li d4c397a64a [SPARK-11325] [SQL] Alias 'alias' in Scala's DataFrame API
Author: Nong Li <nongli@gmail.com>

Closes #9286 from nongli/spark-11325.
2015-10-26 18:27:02 -07:00
Alexander Slesarenko 4bb2b3698f [SQL][DOC] Minor document fixes in interfaces.scala
rxin just noticed this while reading the code.

Author: Alexander Slesarenko <avslesarenko@gmail.com>

Closes #9284 from aslesarenko/doc-typos.
2015-10-26 23:49:14 +01:00
Frank Rosner b60aab8a95 [SPARK-11258] Converting a Spark DataFrame into an R data.frame is slow / requires a lot of memory
https://issues.apache.org/jira/browse/SPARK-11258

I was not able to locate an existing unit test for this function so I wrote one.

Author: Frank Rosner <frank@fam-rosner.de>

Closes #9222 from FRosner/master.
2015-10-26 15:46:59 -07:00
Wenchen Fan 07ced43424 [SPARK-11253] [SQL] reset all accumulators in physical operators before execute an action
With this change, our query execution listener can get the metrics correctly.

The UI still looks good after this change.
<img width="257" alt="screen shot 2015-10-23 at 11 25 14 am" src="https://cloud.githubusercontent.com/assets/3182036/10683834/d516f37e-7978-11e5-8118-343ed40eb824.png">
<img width="494" alt="screen shot 2015-10-23 at 11 25 01 am" src="https://cloud.githubusercontent.com/assets/3182036/10683837/e1fa60da-7978-11e5-8ec8-178b88f27764.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9215 from cloud-fan/metric.
2015-10-25 22:47:39 -07:00
Josh Rosen 85e654c5ec [SPARK-10984] Simplify *MemoryManager class structure
This patch refactors the MemoryManager class structure. After #9000, Spark had the following classes:

- MemoryManager
- StaticMemoryManager
- ExecutorMemoryManager
- TaskMemoryManager
- ShuffleMemoryManager

This is fairly confusing. To simplify things, this patch consolidates several of these classes:

- ShuffleMemoryManager and ExecutorMemoryManager were merged into MemoryManager.
- TaskMemoryManager is moved into Spark Core.

**Key changes and tasks**:

- [x] Merge ExecutorMemoryManager into MemoryManager.
  - [x] Move pooling logic into Allocator.
- [x] Move TaskMemoryManager from `spark-unsafe` to `spark-core`.
- [x] Refactor the existing Tungsten TaskMemoryManager interactions so Tungsten code use only this and not both this and ShuffleMemoryManager.
- [x] Refactor non-Tungsten code to use the TaskMemoryManager instead of ShuffleMemoryManager.
- [x] Merge ShuffleMemoryManager into MemoryManager.
  - [x] Move code
  - [x] ~~Simplify 1/n calculation.~~ **Will defer to followup, since this needs more work.**
- [x] Port ShuffleMemoryManagerSuite tests.
- [x] Move classes from `unsafe` package to `memory` package.
- [ ] Figure out how to handle the hacky use of the memory managers in HashedRelation's broadcast variable construction.
- [x] Test porting and cleanup: several tests relied on mock functionality (such as `TestShuffleMemoryManager.markAsOutOfMemory`) which has been changed or broken during the memory manager consolidation
  - [x] AbstractBytesToBytesMapSuite
  - [x] UnsafeExternalSorterSuite
  - [x] UnsafeFixedWidthAggregationMapSuite
  - [x] UnsafeKVExternalSorterSuite

**Compatiblity notes**:

- This patch introduces breaking changes in `ExternalAppendOnlyMap`, which is marked as `DevloperAPI` (likely for legacy reasons): this class now cannot be used outside of a task.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9127 from JoshRosen/SPARK-10984.
2015-10-25 21:19:52 -07:00
Alexander Slesarenko 92b9c5edd9 [SPARK-6428][SQL] Removed unnecessary typecasts in MutableInt, MutableDouble etc.
marmbrus rxin I believe these typecasts are not required in the presence of explicit return types.

Author: Alexander Slesarenko <avslesarenko@gmail.com>

Closes #9262 from aslesarenko/remove-typecasts.
2015-10-25 10:37:10 +01:00
Yin Huai 4725cb988b [SPARK-11194] [SQL] Use MutableURLClassLoader for the classLoader in IsolatedClientLoader.
https://issues.apache.org/jira/browse/SPARK-11194

Author: Yin Huai <yhuai@databricks.com>

Closes #9170 from yhuai/SPARK-11194.
2015-10-23 17:15:13 -07:00
Reynold Xin e1a897b657 [SPARK-11274] [SQL] Text data source support for Spark SQL.
This adds API for reading and writing text files, similar to SparkContext.textFile and RDD.saveAsTextFile.
```
SQLContext.read.text("/path/to/something.txt")
DataFrame.write.text("/path/to/write.txt")
```

Using the new Dataset API, this also supports
```
val ds: Dataset[String] = SQLContext.read.text("/path/to/something.txt").as[String]
```

Author: Reynold Xin <rxin@databricks.com>

Closes #9240 from rxin/SPARK-11274.
2015-10-23 13:04:06 -07:00
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
Cheng Hao d4950e6be4 [SPARK-9735][SQL] Respect the user specified schema than the infer partition schema for HadoopFsRelation
To enable the unit test of `hadoopFsRelationSuite.Partition column type casting`. It previously threw exception like below, as we treat the auto infer partition schema with higher priority than the user specified one.

```
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getUTF8String(rows.scala:220)
	at org.apache.spark.sql.catalyst.expressions.JoinedRow.getUTF8String(JoinedRow.scala:102)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(generated.java:62)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
	at scala.collection.AbstractIterator.to(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
07:44:01.344 ERROR org.apache.spark.executor.Executor: Exception in task 14.0 in stage 3.0 (TID 206)
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getUTF8String(rows.scala:220)
	at org.apache.spark.sql.catalyst.expressions.JoinedRow.getUTF8String(JoinedRow.scala:102)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(generated.java:62)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
	at scala.collection.AbstractIterator.to(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
```

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

Closes #8026 from chenghao-intel/partition_discovery.
2015-10-22 13:11:37 -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
Josh Rosen f6d06adf05 [SPARK-10708] Consolidate sort shuffle implementations
There's a lot of duplication between SortShuffleManager and UnsafeShuffleManager. Given that these now provide the same set of functionality, now that UnsafeShuffleManager supports large records, I think that we should replace SortShuffleManager's serialized shuffle implementation with UnsafeShuffleManager's and should merge the two managers together.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8829 from JoshRosen/consolidate-sort-shuffle-implementations.
2015-10-22 09:46:30 -07:00
Davies Liu 1d97332715 [SPARK-11243][SQL] output UnsafeRow from columnar cache
This PR change InMemoryTableScan to output UnsafeRow, and optimize the unrolling and scanning by coping the bytes for var-length types between UnsafeRow and ByteBuffer directly without creating the wrapper objects. When scanning the decimals in TPC-DS store_sales table, it's 80% faster (copy it as long without create Decimal objects).

Author: Davies Liu <davies@databricks.com>

Closes #9203 from davies/unsafe_cache.
2015-10-21 19:20:31 -07:00
Yanbo Liang 40a10d7675 [SPARK-9392][SQL] Dataframe drop should work on unresolved columns
Dataframe drop should work on unresolved columns

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #8821 from yanboliang/spark-9392.
2015-10-21 17:50:33 -07:00
navis.ryu f481090a71 [SPARK-10151][SQL] Support invocation of hive macro
Macro in hive (which is GenericUDFMacro) contains real function inside of it but it's not conveyed to tasks, resulting null-pointer exception.

Author: navis.ryu <navis@apache.org>

Closes #8354 from navis/SPARK-10151.
2015-10-21 15:07:08 -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
Shagun Sodhani 19ad18638e [SPARK-11233][SQL] register cosh in function registry
Author: Shagun Sodhani <sshagunsodhani@gmail.com>

Closes #9199 from shagunsodhani/proposed-fix-#11233.
2015-10-21 14:18:06 -07:00
Artem Aliev a37cd87048 [SPARK-11208][SQL] Filter out 'hive.metastore.rawstore.impl' from executionHive temporary config
The executionHive assumed to be a standard meta store located in temporary directory as a derby db. But hive.metastore.rawstore.impl was not filtered out so any custom implementation of the metastore with other storage properties (not JDO) will persist that temporary functions. CassandraHiveMetaStore from DataStax Enterprise is one of examples.

Author: Artem Aliev <artem.aliev@datastax.com>

Closes #9178 from artem-aliev/SPARK-11208.
2015-10-21 14:12:07 -07:00
Yin Huai 3afe448d39 [SPARK-9740][SPARK-9592][SPARK-9210][SQL] Change the default behavior of First/Last to RESPECT NULLS.
I am changing the default behavior of `First`/`Last` to respect null values (the SQL standard default behavior).

https://issues.apache.org/jira/browse/SPARK-9740

Author: Yin Huai <yhuai@databricks.com>

Closes #8113 from yhuai/firstLast.
2015-10-21 13:43:17 -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
Wenchen Fan 7c74ebca05 [SPARK-10743][SQL] keep the name of expression if possible when do cast
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8859 from cloud-fan/cast.
2015-10-21 13:22:35 -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
Pravin Gadakh 8e82e59834 [SPARK-11037][SQL] using Option instead of Some in JdbcDialects
Using Option instead of Some in getCatalystType method.

Author: Pravin Gadakh <prgadakh@in.ibm.com>

Closes #9195 from pravingadakh/master.
2015-10-21 10:35:09 -07:00
Cheng Lian 89e6db6150 [SPARK-11153][SQL] Disables Parquet filter push-down for string and binary columns
Due to PARQUET-251, `BINARY` columns in existing Parquet files may be written with corrupted statistics information. This information is used by filter push-down optimization. Since Spark 1.5 turns on Parquet filter push-down by default, we may end up with wrong query results. PARQUET-251 has been fixed in parquet-mr 1.8.1, but Spark 1.5 is still using 1.7.0.

This affects all Spark SQL data types that can be mapped to Parquet {{BINARY}}, namely:

- `StringType`

- `BinaryType`

- `DecimalType`

  (But Spark SQL doesn't support pushing down filters involving `DecimalType` columns for now.)

To avoid wrong query results, we should disable filter push-down for columns of `StringType` and `BinaryType` until we upgrade to parquet-mr 1.8.

Author: Cheng Lian <lian@databricks.com>

Closes #9152 from liancheng/spark-11153.workaround-parquet-251.

(cherry picked from commit 0887e5e878)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-10-21 09:02:59 +08: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
Daoyuan Wang 94139557c5 [SPARK-10463] [SQL] remove PromotePrecision during optimization
PromotePrecision is not necessary after HiveTypeCoercion done.

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

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

Closes #8621 from adrian-wang/promoterm.
2015-10-20 09:20:35 -07:00