Commit graph

783 commits

Author SHA1 Message Date
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
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
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
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
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
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
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
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 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
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
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
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
Jakob Odersky 60851bc7bf [SPARK-11110][BUILD] Remove transient annotation for parameters.
`transient` annotations on class parameters (not case class parameters or vals) causes compilation errors during compilation with Scala 2.11.
I understand that transient *parameters* make no sense, however I don't quite understand why the 2.10 compiler accepted them.

Note: in case it is preferred to keep the annotations in case someone would in the future want to redefine them as vals, it would also be possible to just add `val` after the annotation, e.g. `class Foo(transient x: Int)` becomes `class Foo(transient private val x: Int)`.

I chose to remove the annotation as it also reduces needles clutter, however please feel free to tell me if you prefer the second option and I'll update the PR

Author: Jakob Odersky <jodersky@gmail.com>

Closes #9126 from jodersky/sbt-scala-2.11.
2015-10-20 08:54:34 +01:00
Josh Rosen 4ace4f8a9c [SPARK-11017] [SQL] Support ImperativeAggregates in TungstenAggregate
This patch extends TungstenAggregate to support ImperativeAggregate functions. The existing TungstenAggregate operator only supported DeclarativeAggregate functions, which are defined in terms of Catalyst expressions and can be evaluated via generated projections. ImperativeAggregate functions, on the other hand, are evaluated by calling their `initialize`, `update`, `merge`, and `eval` methods.

The basic strategy here is similar to how SortBasedAggregate evaluates both types of aggregate functions: use a generated projection to evaluate the expression-based declarative aggregates with dummy placeholder expressions inserted in place of the imperative aggregate function output, then invoke the imperative aggregate functions and target them against the aggregation buffer. The bulk of the diff here consists of code that was copied and adapted from SortBasedAggregate, with some key changes to handle TungstenAggregate's sort fallback path.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9038 from JoshRosen/support-interpreted-in-tungsten-agg-final.
2015-10-14 17:27:50 -07:00
Wenchen Fan 56d7da14ab [SPARK-10104] [SQL] Consolidate different forms of table identifiers
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.

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

Closes #8453 from cloud-fan/table-name.
2015-10-14 16:05:37 -07:00
Yin Huai ce3f9a8065 [SPARK-11091] [SQL] Change spark.sql.canonicalizeView to spark.sql.nativeView.
https://issues.apache.org/jira/browse/SPARK-11091

Author: Yin Huai <yhuai@databricks.com>

Closes #9103 from yhuai/SPARK-11091.
2015-10-13 18:21:24 -07:00
Davies Liu d0cc79ccd0 [SPARK-11030] [SQL] share the SQLTab across sessions
The SQLTab will be shared by multiple sessions.

If we create multiple independent SQLContexts (not using newSession()), will still see multiple SQLTabs in the Spark UI.

Author: Davies Liu <davies@databricks.com>

Closes #9048 from davies/sqlui.
2015-10-13 09:57:53 -07:00
Davies Liu 6987c06793 [SPARK-11009] [SQL] fix wrong result of Window function in cluster mode
Currently, All windows function could generate wrong result in cluster sometimes.

The root cause is that AttributeReference is called in executor, then id of it may not be unique than others created in driver.

Here is the script that could reproduce the problem (run in local cluster):
```
from pyspark import SparkContext, HiveContext
from pyspark.sql.window import Window
from pyspark.sql.functions import rowNumber

sqlContext = HiveContext(SparkContext())
sqlContext.setConf("spark.sql.shuffle.partitions", "3")
df =  sqlContext.range(1<<20)
df2 = df.select((df.id % 1000).alias("A"), (df.id / 1000).alias('B'))
ws = Window.partitionBy(df2.A).orderBy(df2.B)
df3 = df2.select("client", "date", rowNumber().over(ws).alias("rn")).filter("rn < 0")
assert df3.count() == 0
```

Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #9050 from davies/wrong_window.
2015-10-13 09:43:33 -07:00
Yin Huai 8a354bef55 [SPARK-11042] [SQL] Add a mechanism to ban creating multiple root SQLContexts/HiveContexts in a JVM
https://issues.apache.org/jira/browse/SPARK-11042

Author: Yin Huai <yhuai@databricks.com>

Closes #9058 from yhuai/SPARK-11042.
2015-10-12 13:50:34 -07:00
Liang-Chi Hsieh fcb37a0417 [SPARK-10960] [SQL] SQL with windowing function should be able to refer column in inner select
JIRA: https://issues.apache.org/jira/browse/SPARK-10960

When accessing a column in inner select from a select with window function, `AnalysisException` will be thrown. For example, an query like this:

     select area, rank() over (partition by area order by tmp.month) + tmp.tmp1 as c1 from (select month, area, product, 1 as tmp1 from windowData) tmp

Currently, the rule `ExtractWindowExpressions` in `Analyzer` only extracts regular expressions from `WindowFunction`, `WindowSpecDefinition` and `AggregateExpression`. We need to also extract other attributes as the one in `Alias` as shown in the above query.

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

Closes #9011 from viirya/fix-window-inner-column.
2015-10-12 09:16:14 -07:00
Davies Liu 3390b400d0 [SPARK-10810] [SPARK-10902] [SQL] Improve session management in SQL
This PR improve the sessions management by replacing the thread-local based to one SQLContext per session approach, introduce separated temporary tables and UDFs/UDAFs for each session.

A new session of SQLContext could be created by:

1) create an new SQLContext
2) call newSession() on existing SQLContext

For HiveContext, in order to reduce the cost for each session, the classloader and Hive client are shared across multiple sessions (created by newSession).

CacheManager is also shared by multiple sessions, so cache a table multiple times in different sessions will not cause multiple copies of in-memory cache.

Added jars are still shared by all the sessions, because SparkContext does not support sessions.

cc marmbrus yhuai rxin

Author: Davies Liu <davies@databricks.com>

Closes #8909 from davies/sessions.
2015-10-08 17:34:24 -07:00
Cheng Lian 02149ff08e [SPARK-8848] [SQL] Refactors Parquet write path to follow parquet-format
This PR refactors Parquet write path to follow parquet-format spec.  It's a successor of PR #7679, but with less non-essential changes.

Major changes include:

1.  Replaces `RowWriteSupport` and `MutableRowWriteSupport` with `CatalystWriteSupport`

    - Writes Parquet data using standard layout defined in parquet-format

      Specifically, we are now writing ...

      - ... arrays and maps in standard 3-level structure with proper annotations and field names
      - ... decimals as `INT32` and `INT64` whenever possible, and taking `FIXED_LEN_BYTE_ARRAY` as the final fallback

    - Supports legacy mode which is compatible with Spark 1.4 and prior versions

      The legacy mode is by default off, and can be turned on by flipping SQL option `spark.sql.parquet.writeLegacyFormat` to `true`.

    - Eliminates per value data type dispatching costs via prebuilt composed writer functions

1.  Cleans up the last pieces of old Parquet support code

As pointed out by rxin previously, we probably want to rename all those `Catalyst*` Parquet classes to `Parquet*` for clarity.  But I'd like to do this in a follow-up PR to minimize code review noises in this one.

Author: Cheng Lian <lian@databricks.com>

Closes #8988 from liancheng/spark-8848/standard-parquet-write-path.
2015-10-08 16:18:35 -07:00
Wenchen Fan af2a554487 [SPARK-10337] [SQL] fix hive views on non-hive-compatible tables.
add a new config to deal with this special case.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8990 from cloud-fan/view-master.
2015-10-08 12:42:10 -07:00
Cheng Lian 2df882ef14 [SPARK-5775] [SPARK-5508] [SQL] Re-enable Hive Parquet array reading tests
Since SPARK-5508 has already been fixed.

Author: Cheng Lian <lian@databricks.com>

Closes #8999 from liancheng/spark-5775.enable-array-tests.
2015-10-08 09:22:42 -07:00
navis.ryu 713e4f44e9 [SPARK-10679] [CORE] javax.jdo.JDOFatalUserException in executor
HadoopRDD throws exception in executor, something like below.
{noformat}
5/09/17 18:51:21 INFO metastore.HiveMetaStore: 0: Opening raw store with implemenation class:org.apache.hadoop.hive.metastore.ObjectStore
15/09/17 18:51:21 INFO metastore.ObjectStore: ObjectStore, initialize called
15/09/17 18:51:21 WARN metastore.HiveMetaStore: Retrying creating default database after error: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
javax.jdo.JDOFatalUserException: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
	at javax.jdo.JDOHelper.invokeGetPersistenceManagerFactoryOnImplementation(JDOHelper.java:1175)
	at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:808)
	at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:701)
	at org.apache.hadoop.hive.metastore.ObjectStore.getPMF(ObjectStore.java:365)
	at org.apache.hadoop.hive.metastore.ObjectStore.getPersistenceManager(ObjectStore.java:394)
	at org.apache.hadoop.hive.metastore.ObjectStore.initialize(ObjectStore.java:291)
	at org.apache.hadoop.hive.metastore.ObjectStore.setConf(ObjectStore.java:258)
	at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:73)
	at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133)
	at org.apache.hadoop.hive.metastore.RawStoreProxy.<init>(RawStoreProxy.java:57)
	at org.apache.hadoop.hive.metastore.RawStoreProxy.getProxy(RawStoreProxy.java:66)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.newRawStore(HiveMetaStore.java:593)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.getMS(HiveMetaStore.java:571)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:620)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:461)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.<init>(RetryingHMSHandler.java:66)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.getProxy(RetryingHMSHandler.java:72)
	at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762)
	at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.<init>(HiveMetaStoreClient.java:199)
	at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.<init>(SessionHiveMetaStoreClient.java:74)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1521)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:86)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:132)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:104)
	at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3005)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3024)
	at org.apache.hadoop.hive.ql.metadata.Hive.getAllDatabases(Hive.java:1234)
	at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:174)
	at org.apache.hadoop.hive.ql.metadata.Hive.<clinit>(Hive.java:166)
	at org.apache.hadoop.hive.ql.plan.PlanUtils.configureJobPropertiesForStorageHandler(PlanUtils.java:803)
	at org.apache.hadoop.hive.ql.plan.PlanUtils.configureInputJobPropertiesForStorageHandler(PlanUtils.java:782)
	at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:298)
	at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
	at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
	at scala.Option.map(Option.scala:145)
	at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:176)
	at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:220)
	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:216)
	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:87)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	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)
{noformat}

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

Closes #8804 from navis/SPARK-10679.
2015-10-07 14:56:02 -07:00
Marcelo Vanzin 94fc57afdf [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8775 from vanzin/SPARK-10300.
2015-10-07 14:11:21 -07:00
Josh Rosen a9ecd06149 [SPARK-10941] [SQL] Refactor AggregateFunction2 and AlgebraicAggregate interfaces to improve code clarity
This patch refactors several of the Aggregate2 interfaces in order to improve code clarity.

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

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

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8973 from JoshRosen/tungsten-agg-comments.
2015-10-07 13:19:49 -07:00
Cheng Lian 01cd688f52 [SPARK-10400] [SQL] Renames SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC
We introduced SQL option `spark.sql.parquet.followParquetFormatSpec` while working on implementing Parquet backwards-compatibility rules in SPARK-6777. It indicates whether we should use legacy Parquet format adopted by Spark 1.4 and prior versions or the standard format defined in parquet-format spec to write Parquet files.

This option defaults to `false` and is marked as a non-public option (`isPublic = false`) because we haven't finished refactored Parquet write path. The problem is, the name of this option is somewhat confusing, because it's not super intuitive why we shouldn't follow the spec. Would be nice to rename it to `spark.sql.parquet.writeLegacyFormat`, and invert its default value (the two option names have opposite meanings).

Although this option is private in 1.5, we'll make it public in 1.6 after refactoring Parquet write path. So that users can decide whether to write Parquet files in standard format or legacy format.

Author: Cheng Lian <lian@databricks.com>

Closes #8566 from liancheng/spark-10400/deprecate-follow-parquet-format-spec.
2015-10-01 17:23:27 -07:00
Wenchen Fan 02026a8132 [SPARK-10671] [SQL] Throws an analysis exception if we cannot find Hive UDFs
Takes over https://github.com/apache/spark/pull/8800

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8941 from cloud-fan/hive-udf.
2015-10-01 13:23:59 -07:00
Reynold Xin 03cca5dce2 [SPARK-10770] [SQL] SparkPlan.executeCollect/executeTake should return InternalRow rather than external Row.
Author: Reynold Xin <rxin@databricks.com>

Closes #8900 from rxin/SPARK-10770-1.
2015-09-30 14:36:54 -04:00
Wenchen Fan 418e5e4cbd [SPARK-10741] [SQL] Hive Query Having/OrderBy against Parquet table is not working
https://issues.apache.org/jira/browse/SPARK-10741
I choose the second approach: do not change output exprIds when convert MetastoreRelation to LogicalRelation

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8889 from cloud-fan/hot-bug.
2015-09-27 09:08:38 -07:00
Cheng Lian 6f94d56a95 [SPARK-10845] [SQL] Makes spark.sql.hive.version a SQLConfEntry
When refactoring SQL options from plain strings to the strongly typed `SQLConfEntry`, `spark.sql.hive.version` wasn't migrated, and doesn't show up in the result of `SET -v`, as `SET -v` only shows public `SQLConfEntry` instances. This affects compatibility with Simba ODBC driver.

This PR migrates this SQL option as a `SQLConfEntry` to fix this issue.

Author: Cheng Lian <lian@databricks.com>

Closes #8925 from liancheng/spark-10845/hive-version-conf.
2015-09-26 19:08:55 -07:00
Wenchen Fan 341b13f8f5 [SPARK-10765] [SQL] use new aggregate interface for hive UDAF
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8874 from cloud-fan/hive-agg.
2015-09-24 09:54:07 -07:00
Zhichao Li 84f81e035e [SPARK-10310] [SQL] Fixes script transformation field/line delimiters
**Please attribute this PR to `Zhichao Li <zhichao.liintel.com>`.**

This PR is based on PR #8476 authored by zhichao-li. It fixes SPARK-10310 by adding field delimiter SerDe property to the default `LazySimpleSerDe`, and enabling default record reader/writer classes.

Currently, we only support `LazySimpleSerDe`, used together with `TextRecordReader` and `TextRecordWriter`, and don't support customizing record reader/writer using `RECORDREADER`/`RECORDWRITER` clauses. This should be addressed in separate PR(s).

Author: Cheng Lian <lian@databricks.com>

Closes #8860 from liancheng/spark-10310/fix-script-trans-delimiters.
2015-09-22 19:41:57 -07:00
Yin Huai 2204cdb284 [SPARK-10672] [SQL] Do not fail when we cannot save the metadata of a data source table in a hive compatible way
https://issues.apache.org/jira/browse/SPARK-10672

With changes in this PR, we will fallback to same the metadata of a table in Spark SQL specific way if we fail to save it in a hive compatible way (Hive throws an exception because of its internal restrictions, e.g. binary and decimal types cannot be saved to parquet if the metastore is running Hive 0.13). I manually tested the fix with the following test in `DataSourceWithHiveMetastoreCatalogSuite` (`spark.sql.hive.metastore.version=0.13` and `spark.sql.hive.metastore.jars`=`maven`).

```
    test(s"fail to save metadata of a parquet table in hive 0.13") {
      withTempPath { dir =>
        withTable("t") {
          val path = dir.getCanonicalPath

          sql(
            s"""CREATE TABLE t USING $provider
               |OPTIONS (path '$path')
               |AS SELECT 1 AS d1, cast("val_1" as binary) AS d2
             """.stripMargin)

          sql(
            s"""describe formatted t
             """.stripMargin).collect.foreach(println)

          sqlContext.table("t").show
        }
      }
    }
  }
```

Without this fix, we will fail with the following error.
```
org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.UnsupportedOperationException: Unknown field type: binary
	at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:619)
	at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:576)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply$mcV$sp(ClientWrapper.scala:359)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply(ClientWrapper.scala:357)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply(ClientWrapper.scala:357)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:256)
	at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:211)
	at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:248)
	at org.apache.spark.sql.hive.client.ClientWrapper.createTable(ClientWrapper.scala:357)
	at org.apache.spark.sql.hive.HiveMetastoreCatalog.createDataSourceTable(HiveMetastoreCatalog.scala:358)
	at org.apache.spark.sql.hive.execution.CreateMetastoreDataSourceAsSelect.run(commands.scala:285)
	at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57)
	at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57)
	at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:69)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:140)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:138)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:138)
	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:58)
	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:58)
	at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:144)
	at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:129)
	at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51)
	at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:725)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$sql$1.apply(SQLTestUtils.scala:56)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$sql$1.apply(SQLTestUtils.scala:56)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2$$anonfun$apply$2.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:165)
	at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:150)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.withTable(HiveMetastoreCatalogSuite.scala:52)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2.apply(HiveMetastoreCatalogSuite.scala:162)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2.apply(HiveMetastoreCatalogSuite.scala:161)
	at org.apache.spark.sql.test.SQLTestUtils$class.withTempPath(SQLTestUtils.scala:125)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.withTempPath(HiveMetastoreCatalogSuite.scala:52)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:161)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:161)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:161)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
	at org.scalatest.FunSuite.runTest(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
	at scala.collection.immutable.List.foreach(List.scala:318)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
	at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
	at org.scalatest.Suite$class.run(Suite.scala:1424)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
	at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.org$scalatest$BeforeAndAfterAll$$super$run(HiveMetastoreCatalogSuite.scala:52)
	at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
	at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.run(HiveMetastoreCatalogSuite.scala:52)
	at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
	at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
	at sbt.ForkMain$Run$2.call(ForkMain.java:294)
	at sbt.ForkMain$Run$2.call(ForkMain.java:284)
	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
	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)
Caused by: java.lang.UnsupportedOperationException: Unknown field type: binary
	at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.getObjectInspector(ArrayWritableObjectInspector.java:108)
	at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.<init>(ArrayWritableObjectInspector.java:60)
	at org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe.initialize(ParquetHiveSerDe.java:113)
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:339)
	at org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:288)
	at org.apache.hadoop.hive.ql.metadata.Table.checkValidity(Table.java:194)
	at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:597)
	... 76 more
```

Author: Yin Huai <yhuai@databricks.com>

Closes #8824 from yhuai/datasourceMetadata.
2015-09-22 13:29:39 -07:00
Davies Liu 22d40159e6 [SPARK-10593] [SQL] fix resolve output of Generate
The output of Generate should not be resolved as Reference.

Author: Davies Liu <davies@databricks.com>

Closes #8755 from davies/view.
2015-09-22 11:07:10 -07:00
Yin Huai 4da32bc0e7 [SPARK-8567] [SQL] Increase the timeout of o.a.s.sql.hive.HiveSparkSubmitSuite to 5 minutes.
https://issues.apache.org/jira/browse/SPARK-8567

Looks like "SPARK-8368: includes jars passed in through --jars" is pretty flaky now. Based on some history runs, the time spent on a successful run may be from 1.5 minutes to almost 3 minutes. Let's try to increase the timeout and see if we can fix this test.

https://amplab.cs.berkeley.edu/jenkins/job/Spark-1.5-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.0,label=spark-test/385/testReport/junit/org.apache.spark.sql.hive/HiveSparkSubmitSuite/SPARK_8368__includes_jars_passed_in_through___jars/history/?start=25

Author: Yin Huai <yhuai@databricks.com>

Closes #8850 from yhuai/SPARK-8567-anotherTry.
2015-09-22 00:07:30 -07:00
zsxwing e789000b88 [SPARK-10155] [SQL] Change SqlParser to object to avoid memory leak
Since `scala.util.parsing.combinator.Parsers` is thread-safe since Scala 2.10 (See [SI-4929](https://issues.scala-lang.org/browse/SI-4929)), we can change SqlParser to object to avoid memory leak.

I didn't change other subclasses of `scala.util.parsing.combinator.Parsers` because there is only one instance in one SQLContext, which should not be an issue.

Author: zsxwing <zsxwing@gmail.com>

Closes #8357 from zsxwing/sql-memory-leak.
2015-09-19 18:22:43 -07:00
Cheng Lian 22be2ae147 [SPARK-10623] [SQL] Fixes ORC predicate push-down
When pushing down a leaf predicate, ORC `SearchArgument` builder requires an extra "parent" predicate (any one among `AND`/`OR`/`NOT`) to wrap the leaf predicate. E.g., to push down `a < 1`, we must build `AND(a < 1)` instead. Fortunately, when actually constructing the `SearchArgument`, the builder will eliminate all those unnecessary wrappers.

This PR is based on #8783 authored by zhzhan. I also took the chance to simply `OrcFilters` a little bit to improve readability.

Author: Cheng Lian <lian@databricks.com>

Closes #8799 from liancheng/spark-10623/fix-orc-ppd.
2015-09-18 18:42:20 -07:00
Cheng Lian 00a2911c5b [SPARK-10540] Fixes flaky all-data-type test
This PR breaks the original test case into multiple ones (one test case for each data type). In this way, test failure output can be much more readable.

Within each test case, we build a table with two columns, one of them is for the data type to test, the other is an "index" column, which is used to sort the DataFrame and workaround [SPARK-10591] [1]

[1]: https://issues.apache.org/jira/browse/SPARK-10591

Author: Cheng Lian <lian@databricks.com>

Closes #8768 from liancheng/spark-10540/test-all-data-types.
2015-09-18 12:19:08 -07:00
Yin Huai aad644fbe2 [SPARK-10639] [SQL] Need to convert UDAF's result from scala to sql type
https://issues.apache.org/jira/browse/SPARK-10639

Author: Yin Huai <yhuai@databricks.com>

Closes #8788 from yhuai/udafConversion.
2015-09-17 11:14:52 -07:00
Josh Rosen 38700ea40c [SPARK-10381] Fix mixup of taskAttemptNumber & attemptId in OutputCommitCoordinator
When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop.

This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish).

This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8544 from JoshRosen/SPARK-10381.
2015-09-15 17:11:21 -07:00
Marcelo Vanzin b42059d2ef Revert "[SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py."
This reverts commit 8abef21dac.
2015-09-15 13:03:38 -07:00
Marcelo Vanzin 8abef21dac [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py.
This change does two things:

- tag a few tests and adds the mechanism in the build to be able to disable those tags,
  both in maven and sbt, for both junit and scalatest suites.
- add some logic to run-tests.py to disable some tags depending on what files have
  changed; that's used to disable expensive tests when a module hasn't explicitly
  been changed, to speed up testing for changes that don't directly affect those
  modules.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8437 from vanzin/test-tags.
2015-09-15 10:45:02 -07:00
Reynold Xin 09b7e7c198 Update version to 1.6.0-SNAPSHOT.
Author: Reynold Xin <rxin@databricks.com>

Closes #8350 from rxin/1.6.
2015-09-15 00:54:20 -07:00
Kousuke Saruta cf2821ef5f [SPARK-10584] [DOC] [SQL] Documentation about spark.sql.hive.metastore.version is wrong.
The default value of hive metastore version is 1.2.1 but the documentation says the value of `spark.sql.hive.metastore.version` is 0.13.1.
Also, we cannot get the default value by `sqlContext.getConf("spark.sql.hive.metastore.version")`.

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

Closes #8739 from sarutak/SPARK-10584.
2015-09-14 12:06:23 -07:00
Wenchen Fan 32407bfd2b [SPARK-9899] [SQL] log warning for direct output committer with speculation enabled
This is a follow-up of https://github.com/apache/spark/pull/8317.

When speculation is enabled, there may be multiply tasks writing to the same path. Generally it's OK as we will write to a temporary directory first and only one task can commit the temporary directory to target path.

However, when we use direct output committer, tasks will write data to target path directly without temporary directory. This causes problems like corrupted data. Please see [PR comment](https://github.com/apache/spark/pull/8191#issuecomment-131598385) for more details.

Unfortunately, we don't have a simple flag to tell if a output committer will write to temporary directory or not, so for safety, we have to disable any customized output committer when `speculation` is true.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8687 from cloud-fan/direct-committer.
2015-09-14 11:51:39 -07:00