Commit graph

2006 commits

Author SHA1 Message Date
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
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
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
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
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 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
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
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
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
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
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 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
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
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
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
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
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
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
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
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
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
Cheng Lian 8b877cc4ee [SPARK-11088][SQL] Merges partition values using UnsafeProjection
`DataSourceStrategy.mergeWithPartitionValues` is essentially a projection implemented in a quite inefficient way. This PR optimizes this method with `UnsafeProjection` to avoid unnecessary boxing costs.

Author: Cheng Lian <lian@databricks.com>

Closes #9104 from liancheng/spark-11088.faster-partition-values-merging.
2015-10-19 16:57:20 -07:00
Rishabh Bhardwaj 5966817941 [SPARK-11180][SQL] Support BooleanType in DataFrame.na.fill
Added support for boolean types in fill and replace methods

Author: Rishabh Bhardwaj <rbnext29@gmail.com>

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

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9131 from cloud-fan/unsafe.
2015-10-19 11:02:26 -07:00
zsxwing 94c8fef296 [SPARK-11126][SQL] Fix a memory leak in SQLListener._stageIdToStageMetrics
SQLListener adds all stage infos to `_stageIdToStageMetrics`, but only removes stage infos belonging to SQL executions. This PR fixed it by ignoring stages that don't belong to SQL executions.

Reported by Terry Hoo in https://www.mail-archive.com/userspark.apache.org/msg38810.html

Author: zsxwing <zsxwing@gmail.com>

Closes #9132 from zsxwing/SPARK-11126.
2015-10-18 13:51:45 -07:00
Koert Kuipers 57f83e36d6 [SPARK-10185] [SQL] Feat sql comma separated paths
Make sure comma-separated paths get processed correcly in ResolvedDataSource for a HadoopFsRelationProvider

Author: Koert Kuipers <koert@tresata.com>

Closes #8416 from koertkuipers/feat-sql-comma-separated-paths.
2015-10-17 14:56:24 -07:00
Pravin Gadakh 3d683a139b [SPARK-10581] [DOCS] Groups are not resolved in scaladoc in sql classes
Groups are not resolved properly in scaladoc in following classes:

sql/core/src/main/scala/org/apache/spark/sql/Column.scala
sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
sql/core/src/main/scala/org/apache/spark/sql/functions.scala

Author: Pravin Gadakh <pravingadakh177@gmail.com>

Closes #9148 from pravingadakh/master.
2015-10-16 13:38:50 -07:00
navis.ryu b9c5e5d4ac [SPARK-11124] JsonParser/Generator should be closed for resource recycle
Some json parsers are not closed. parser in JacksonParser#parseJson, for example.

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

Closes #9130 from navis/SPARK-11124.
2015-10-16 11:19:37 -07:00
Josh Rosen eb0b4d6e2d [SPARK-11135] [SQL] Exchange incorrectly skips sorts when existing ordering is non-empty subset of required ordering
In Spark SQL, the Exchange planner tries to avoid unnecessary sorts in cases where the data has already been sorted by a superset of the requested sorting columns. For instance, let's say that a query calls for an operator's input to be sorted by `a.asc` and the input happens to already be sorted by `[a.asc, b.asc]`. In this case, we do not need to re-sort the input. The converse, however, is not true: if the query calls for `[a.asc, b.asc]`, then `a.asc` alone will not satisfy the ordering requirements, requiring an additional sort to be planned by Exchange.

However, the current Exchange code gets this wrong and incorrectly skips sorting when the existing output ordering is a subset of the required ordering. This is simple to fix, however.

This bug was introduced in https://github.com/apache/spark/pull/7458, so it affects 1.5.0+.

This patch fixes the bug and significantly improves the unit test coverage of Exchange's sort-planning logic.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9140 from JoshRosen/SPARK-11135.
2015-10-15 17:36:55 -07:00
Wenchen Fan 6a2359ff1f [SPARK-10412] [SQL] report memory usage for tungsten sql physical operator
https://issues.apache.org/jira/browse/SPARK-10412

some screenshots:
### aggregate:
![screen shot 2015-10-12 at 2 23 11 pm](https://cloud.githubusercontent.com/assets/3182036/10439534/618320a4-70ef-11e5-94d8-62ea7f2d1531.png)

### join
![screen shot 2015-10-12 at 2 23 29 pm](https://cloud.githubusercontent.com/assets/3182036/10439537/6724797c-70ef-11e5-8f75-0cf5cbd42048.png)

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

Closes #8931 from cloud-fan/viz.
2015-10-15 14:50:58 -07: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
Cheng Hao 1baaf2b9bd [SPARK-10829] [SQL] Filter combine partition key and attribute doesn't work in DataSource scan
```scala
withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") {
      withTempPath { dir =>
        val path = s"${dir.getCanonicalPath}/part=1"
        (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(path)

        // If the "part = 1" filter gets pushed down, this query will throw an exception since
        // "part" is not a valid column in the actual Parquet file
        checkAnswer(
          sqlContext.read.parquet(path).filter("a > 0 and (part = 0 or a > 1)"),
          (2 to 3).map(i => Row(i, i.toString, 1)))
      }
    }
```

We expect the result to be:
```
2,1
3,1
```
But got
```
1,1
2,1
3,1
```

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

Closes #8916 from chenghao-intel/partition_filter.
2015-10-14 16:29:32 -07:00
Reynold Xin 2b5e31c7e9 [SPARK-11113] [SQL] Remove DeveloperApi annotation from private classes.
o.a.s.sql.catalyst and o.a.s.sql.execution are supposed to be private.

Author: Reynold Xin <rxin@databricks.com>

Closes #9121 from rxin/SPARK-11113.
2015-10-14 16:27:43 -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
Wenchen Fan 9a430a027f [SPARK-11068] [SQL] [FOLLOW-UP] move execution listener to util
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9119 from cloud-fan/callback.
2015-10-14 15:08:13 -07:00
Huaxin Gao 7e1308d37f [SPARK-8386] [SQL] add write.mode for insertIntoJDBC when the parm overwrite is false
the fix is for jira https://issues.apache.org/jira/browse/SPARK-8386

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

Closes #9042 from huaxingao/spark8386.
2015-10-14 12:31:29 -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
Wenchen Fan 15ff85b316 [SPARK-11068] [SQL] add callback to query execution
With this feature, we can track the query plan, time cost, exception during query execution for spark users.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #9078 from cloud-fan/callback.
2015-10-13 17:59:32 -07:00
Sun Rui 5e3868ba13 [SPARK-10051] [SPARKR] Support collecting data of StructType in DataFrame
Two points in this PR:

1.    Originally thought was that a named R list is assumed to be a struct in SerDe. But this is problematic because some R functions will implicitly generate named lists that are not intended to be a struct when transferred by SerDe. So SerDe clients have to explicitly mark a names list as struct by changing its class from "list" to "struct".

2.    SerDe is in the Spark Core module, and data of StructType is represented as GenricRow which is defined in Spark SQL module. SerDe can't import GenricRow as in maven build  Spark SQL module depends on Spark Core module. So this PR adds a registration hook in SerDe to allow SQLUtils in Spark SQL module to register its functions for serialization and deserialization of StructType.

Author: Sun Rui <rui.sun@intel.com>

Closes #8794 from sun-rui/SPARK-10051.
2015-10-13 10:02:21 -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
Davies Liu c4da5345a0 [SPARK-10990] [SPARK-11018] [SQL] improve unrolling of complex types
This PR improve the unrolling and read of complex types in columnar cache:
1) Using UnsafeProjection to do serialization of complex types, so they will not be serialized three times (two for actualSize)
2) Copy the bytes from UnsafeRow/UnsafeArrayData to ByteBuffer directly, avoiding the immediate byte[]
3) Using the underlying array in ByteBuffer to create UTF8String/UnsafeRow/UnsafeArrayData without copy.

Combine these optimizations,  we can reduce the unrolling time from 25s to 21s (20% less), reduce the scanning time from 3.5s to 2.5s (28% less).

```
df = sqlContext.read.parquet(path)
t = time.time()
df.cache()
df.count()
print 'unrolling', time.time() - t

for i in range(10):
    t = time.time()
    print df.select("*")._jdf.queryExecution().toRdd().count()
    print time.time() - t
```

The schema is
```
root
 |-- a: struct (nullable = true)
 |    |-- b: long (nullable = true)
 |    |-- c: string (nullable = true)
 |-- d: array (nullable = true)
 |    |-- element: long (containsNull = true)
 |-- e: map (nullable = true)
 |    |-- key: long
 |    |-- value: string (valueContainsNull = true)
```

Now the columnar cache depends on that UnsafeProjection support all the data types (including UDT), this PR also fix that.

Author: Davies Liu <davies@databricks.com>

Closes #9016 from davies/complex2.
2015-10-12 21:12:59 -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
Cheng Lian 64b1d00e1a [SPARK-11007] [SQL] Adds dictionary aware Parquet decimal converters
For Parquet decimal columns that are encoded using plain-dictionary encoding, we can make the upper level converter aware of the dictionary, so that we can pre-instantiate all the decimals to avoid duplicated instantiation.

Note that plain-dictionary encoding isn't available for `FIXED_LEN_BYTE_ARRAY` for Parquet writer version `PARQUET_1_0`. So currently only decimals written as `INT32` and `INT64` can benefit from this optimization.

Author: Cheng Lian <lian@databricks.com>

Closes #9040 from liancheng/spark-11007.decimal-converter-dict-support.
2015-10-12 10:17:19 -07:00
Josh Rosen 595012ea8b [SPARK-11053] Remove use of KVIterator in SortBasedAggregationIterator
SortBasedAggregationIterator uses a KVIterator interface in order to process input rows as key-value pairs, but this use of KVIterator is unnecessary, slightly complicates the code, and might hurt performance. This patch refactors this code to remove the use of this extra layer of iterator wrapping and simplifies other parts of the code in the process.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9066 from JoshRosen/sort-iterator-cleanup.
2015-10-11 18:11:08 -07:00
Rick Hillegas 12b7191d20 [SPARK-10855] [SQL] Add a JDBC dialect for Apache Derby
marmbrus
rxin

This patch adds a JdbcDialect class, which customizes the datatype mappings for Derby backends. The patch also adds unit tests for the new dialect, corresponding to the existing tests for other JDBC dialects.

JDBCSuite runs cleanly for me with this patch. So does JDBCWriteSuite, although it produces noise as described here: https://issues.apache.org/jira/browse/SPARK-10890

This patch is my original work, which I license to the ASF. I am a Derby contributor, so my ICLA is on file under SVN id "rhillegas": http://people.apache.org/committer-index.html

Touches the following files:

---------------------------------

org.apache.spark.sql.jdbc.JdbcDialects

Adds a DerbyDialect.

---------------------------------

org.apache.spark.sql.jdbc.JDBCSuite

Adds unit tests for the new DerbyDialect.

Author: Rick Hillegas <rhilleg@us.ibm.com>

Closes #8982 from rick-ibm/b_10855.
2015-10-09 13:36:51 -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
Josh Rosen 2816c89b6a [SPARK-10988] [SQL] Reduce duplication in Aggregate2's expression rewriting logic
In `aggregate/utils.scala`, there is a substantial amount of duplication in the expression-rewriting logic. As a prerequisite to supporting imperative aggregate functions in `TungstenAggregate`, this patch refactors this file so that the same expression-rewriting logic is used for both `SortAggregate` and `TungstenAggregate`.

In order to allow both operators to use the same rewriting logic, `TungstenAggregationIterator. generateResultProjection()` has been updated so that it first evaluates all declarative aggregate functions' `evaluateExpression`s and writes the results into a temporary buffer, and then uses this temporary buffer and the grouping expressions to evaluate the final resultExpressions. This matches the logic in SortAggregateIterator, where this two-pass approach is necessary in order to support imperative aggregates. If this change turns out to cause performance regressions, then we can look into re-implementing the single-pass evaluation in a cleaner way as part of a followup patch.

Since the rewriting logic is now shared across both operators, this patch also extracts that logic and places it in `SparkStrategies`. This makes the rewriting logic a bit easier to follow, I think.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9015 from JoshRosen/SPARK-10988.
2015-10-08 14:56:27 -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
Yin Huai 82d275f27c [SPARK-10887] [SQL] Build HashedRelation outside of HashJoinNode.
This PR refactors `HashJoinNode` to take a existing `HashedRelation`. So, we can reuse this node for both `ShuffledHashJoin` and `BroadcastHashJoin`.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #8953 from yhuai/SPARK-10887.
2015-10-08 11:56:44 -07:00
tedyu 2a6f614cd6 [SPARK-11006] Rename NullColumnAccess as NullColumnAccessor
davies
I think NullColumnAccessor follows same convention for other accessors

Author: tedyu <yuzhihong@gmail.com>

Closes #9028 from tedyu/master.
2015-10-08 11:51:58 -07:00
Cheng Lian 59b0606f33 [SPARK-10999] [SQL] Coalesce should be able to handle UnsafeRow
Author: Cheng Lian <lian@databricks.com>

Closes #9024 from liancheng/spark-10999.coalesce-unsafe-row-handling.
2015-10-08 09:20:36 -07:00
0x0FFF b8f849b546 [SPARK-7869][SQL] Adding Postgres JSON and JSONb data types support
This PR addresses [SPARK-7869](https://issues.apache.org/jira/browse/SPARK-7869)

Before the patch, attempt to load the table from Postgres with JSON/JSONb datatype caused error `java.sql.SQLException: Unsupported type 1111`
Postgres data types JSON and JSONb are now mapped to String on Spark side thus they can be loaded into DF and processed on Spark side

Example

Postgres:
```
create table test_json  (id int, value json);
create table test_jsonb (id int, value jsonb);

insert into test_json (id, value) values
(1, '{"field1":"value1","field2":"value2","field3":[1,2,3]}'::json),
(2, '{"field1":"value3","field2":"value4","field3":[4,5,6]}'::json),
(3, '{"field3":"value5","field4":"value6","field3":[7,8,9]}'::json);

insert into test_jsonb (id, value) values
(4, '{"field1":"value1","field2":"value2","field3":[1,2,3]}'::jsonb),
(5, '{"field1":"value3","field2":"value4","field3":[4,5,6]}'::jsonb),
(6, '{"field3":"value5","field4":"value6","field3":[7,8,9]}'::jsonb);
```

PySpark:
```
>>> import json
>>> df1 = sqlContext.read.jdbc("jdbc:postgresql://127.0.0.1:5432/test?user=testuser", "test_json")
>>> df1.map(lambda x: (x.id, json.loads(x.value))).map(lambda (id, value): (id, value.get('field3'))).collect()
[(1, [1, 2, 3]), (2, [4, 5, 6]), (3, [7, 8, 9])]
>>> df2 = sqlContext.read.jdbc("jdbc:postgresql://127.0.0.1:5432/test?user=testuser", "test_jsonb")
>>> df2.map(lambda x: (x.id, json.loads(x.value))).map(lambda (id, value): (id, value.get('field1'))).collect()
[(4, u'value1'), (5, u'value3'), (6, None)]
```

Author: 0x0FFF <programmerag@gmail.com>

Closes #8948 from 0x0FFF/SPARK-7869.
2015-10-07 23:12:35 -07:00
Davies Liu 075a0b6582 [SPARK-10917] [SQL] improve performance of complex type in columnar cache
This PR improve the performance of complex types in columnar cache by using UnsafeProjection instead of KryoSerializer.

A simple benchmark show that this PR could improve the performance of scanning a cached table with complex columns by 15x (comparing to Spark 1.5).

Here is the code used to benchmark:

```
df = sc.range(1<<23).map(lambda i: Row(a=Row(b=i, c=str(i)), d=range(10), e=dict(zip(range(10), [str(i) for i in range(10)])))).toDF()
df.write.parquet("table")
```
```
df = sqlContext.read.parquet("table")
df.cache()
df.count()
t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```

Author: Davies Liu <davies@databricks.com>

Closes #8971 from davies/complex.
2015-10-07 15:58:07 -07:00
Josh Rosen 7e2e268289 [SPARK-9702] [SQL] Use Exchange to implement logical Repartition operator
This patch allows `Repartition` to support UnsafeRows. This is accomplished by implementing the logical `Repartition` operator in terms of `Exchange` and a new `RoundRobinPartitioning`.

Author: Josh Rosen <joshrosen@databricks.com>
Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #8083 from JoshRosen/SPARK-9702.
2015-10-07 15:53:37 -07:00
Reynold Xin 6dbfd7ecf4 [SPARK-10982] [SQL] Rename ExpressionAggregate -> DeclarativeAggregate.
DeclarativeAggregate matches more closely with ImperativeAggregate we already have.

Author: Reynold Xin <rxin@databricks.com>

Closes #9013 from rxin/SPARK-10982.
2015-10-07 15:38:46 -07:00
Liang-Chi Hsieh c14aee4da9 [SPARK-10856][SQL] Mapping TimestampType to DATETIME for SQL Server jdbc dialect
JIRA: https://issues.apache.org/jira/browse/SPARK-10856

For Microsoft SQL Server, TimestampType should be mapped to DATETIME instead of TIMESTAMP.

Related information for the datatype mapping: https://msdn.microsoft.com/en-us/library/ms378878(v=sql.110).aspx

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

Closes #8978 from viirya/mysql-jdbc-timestamp.
2015-10-07 14:49:08 -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
Davies Liu 27ecfe61f0 [SPARK-10938] [SQL] remove typeId in columnar cache
This PR remove the typeId in columnar cache, it's not needed anymore, it also remove DATE and TIMESTAMP (use INT/LONG instead).

Author: Davies Liu <davies@databricks.com>

Closes #8989 from davies/refactor_cache.
2015-10-06 08:45:31 -07:00
gweidner 314bc68435 [SPARK-7275] [SQL] Make LogicalRelation public
Given LogicalRelation (and other classes) were moved from sources package to execution.sources package, removed private[sql] to make LogicalRelation public to facilitate access for data sources.

Author: gweidner <gweidner@us.ibm.com>

Closes #8965 from gweidner/SPARK-7275.
2015-10-03 01:04:14 -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
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
Cheng Lian 4d5a005b0d [SPARK-10811] [SQL] Eliminates unnecessary byte array copying
When reading Parquet string and binary-backed decimal values, Parquet `Binary.getBytes` always returns a copied byte array, which is unnecessary. Since the underlying implementation of `Binary` values there is guaranteed to be `ByteArraySliceBackedBinary`, and Parquet itself never reuses underlying byte arrays, we can use `Binary.toByteBuffer.array()` to steal the underlying byte arrays without copying them.

This brings performance benefits when scanning Parquet string and binary-backed decimal columns. Note that, this trick doesn't cover binary-backed decimals with precision greater than 18.

My micro-benchmark result is that, this brings a ~15% performance boost for scanning TPC-DS `store_sales` table (scale factor 15).

Another minor optimization done in this PR is that, now we directly construct a Java `BigDecimal` in `Decimal.toJavaBigDecimal` without constructing a Scala `BigDecimal` first. This brings another ~5% performance gain.

Author: Cheng Lian <lian@databricks.com>

Closes #8907 from liancheng/spark-10811/eliminate-array-copying.
2015-09-29 23:30:27 -07:00
Davies Liu ea02e5513a [SPARK-10859] [SQL] fix stats of StringType in columnar cache
The UTF8String may come from UnsafeRow, then underline buffer of it is not copied, so we should clone it in order to hold it in Stats.

cc yhuai

Author: Davies Liu <davies@databricks.com>

Closes #8929 from davies/pushdown_string.
2015-09-28 14:40:40 -07:00
Cheng Lian 14978b785a [SPARK-10395] [SQL] Simplifies CatalystReadSupport
Please refer to [SPARK-10395] [1] for details.

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

Author: Cheng Lian <lian@databricks.com>

Closes #8553 from liancheng/spark-10395/simplify-parquet-read-support.
2015-09-28 13:53:45 -07:00
Holden Karau 8ecba3e86e [SPARK-10720] [SQL] [JAVA] Add a java wrapper to create a dataframe from a local list of java beans
Similar to SPARK-10630 it would be nice if Java users didn't have to parallelize there data explicitly (as Scala users already can skip). Issue came up in http://stackoverflow.com/questions/32613413/apache-spark-machine-learning-cant-get-estimator-example-to-work

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8879 from holdenk/SPARK-10720-add-a-java-wrapper-to-create-a-dataframe-from-a-local-list-of-java-beans.
2015-09-27 21:16:15 +01: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
Matei Zaharia 21fd12cb17 [SPARK-9852] Let reduce tasks fetch multiple map output partitions
This makes two changes:

- Allow reduce tasks to fetch multiple map output partitions -- this is a pretty small change to HashShuffleFetcher
- Move shuffle locality computation out of DAGScheduler and into ShuffledRDD / MapOutputTracker; this was needed because the code in DAGScheduler wouldn't work for RDDs that fetch multiple map output partitions from each reduce task

I also added an AdaptiveSchedulingSuite that creates RDDs depending on multiple map output partitions.

Author: Matei Zaharia <matei@databricks.com>

Closes #8844 from mateiz/spark-9852.
2015-09-24 23:39:04 -04:00
Liang-Chi Hsieh b3862d3c59 [SPARK-10705] [SQL] Avoid using external rows in DataFrame.toJSON
JIRA: https://issues.apache.org/jira/browse/SPARK-10705

As described in the JIRA ticket, `DataFrame.toJSON` uses `DataFrame.mapPartitions`, which converts internal rows to external rows. We should use `queryExecution.toRdd.mapPartitions` that directly uses internal rows for better performance.

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

Closes #8865 from viirya/df-tojson-internalrow.
2015-09-24 12:52:11 -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
Andrew Or 83f6f54d12 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array (round 2)
This patch reverts most of the changes in a previous fix #8827.

The real cause of the issue is that in `TungstenAggregate`'s prepare method we only reserve 1 page, but later when we switch to sort-based aggregation we try to acquire 1 page AND a pointer array. The longer-term fix should be to reserve also the pointer array, but for now ***we will simply not track the pointer array***. (Note that elsewhere we already don't track the pointer array, e.g. [here](a18208047f/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java (L88)))

Note: This patch reuses the unit test added in #8827 so it doesn't show up in the diff.

Author: Andrew Or <andrew@databricks.com>

Closes #8888 from andrewor14/dont-track-pointer-array.
2015-09-23 19:34:31 -07:00
Reynold Xin 9952217749 [SPARK-10731] [SQL] Delegate to Scala's DataFrame.take implementation in Python DataFrame.
Python DataFrame.head/take now requires scanning all the partitions. This pull request changes them to delegate the actual implementation to Scala DataFrame (by calling DataFrame.take).

This is more of a hack for fixing this issue in 1.5.1. A more proper fix is to change executeCollect and executeTake to return InternalRow rather than Row, and thus eliminate the extra round-trip conversion.

Author: Reynold Xin <rxin@databricks.com>

Closes #8876 from rxin/SPARK-10731.
2015-09-23 16:43:21 -07:00
Josh Rosen a18208047f [SPARK-10403] Allow UnsafeRowSerializer to work with tungsten-sort ShuffleManager
This patch attempts to fix an issue where Spark SQL's UnsafeRowSerializer was incompatible with the `tungsten-sort` ShuffleManager.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8873 from JoshRosen/SPARK-10403.
2015-09-23 11:31:01 -07:00
Reynold Xin a96ba40f7e [SPARK-10714] [SPARK-8632] [SPARK-10685] [SQL] Refactor Python UDF handling
This patch refactors Python UDF handling:

1. Extract the per-partition Python UDF calling logic from PythonRDD into a PythonRunner. PythonRunner itself expects iterator as input/output, and thus has no dependency on RDD. This way, we can use PythonRunner directly in a mapPartitions call, or in the future in an environment without RDDs.
2. Use PythonRunner in Spark SQL's BatchPythonEvaluation.
3. Updated BatchPythonEvaluation to only use its input once, rather than twice. This should fix Python UDF performance regression in Spark 1.5.

There are a number of small cleanups I wanted to do when I looked at the code, but I kept most of those out so the diff looks small.

This basically implements the approach in https://github.com/apache/spark/pull/8833, but with some code moving around so the correctness doesn't depend on the inner workings of Spark serialization and task execution.

Author: Reynold Xin <rxin@databricks.com>

Closes #8835 from rxin/python-iter-refactor.
2015-09-22 14:11:46 -07:00
Yin Huai 5aea987c90 [SPARK-10737] [SQL] When using UnsafeRows, SortMergeJoin may return wrong results
https://issues.apache.org/jira/browse/SPARK-10737

Author: Yin Huai <yhuai@databricks.com>

Closes #8854 from yhuai/SMJBug.
2015-09-22 13:31:35 -07:00
Reynold Xin f3b727c801 [SQL] [MINOR] map -> foreach.
DataFrame.explain should use foreach to print the explain content.

Author: Reynold Xin <rxin@databricks.com>

Closes #8862 from rxin/map-foreach.
2015-09-22 00:09:29 -07:00
Liang-Chi Hsieh 1fcefef069 [SPARK-10446][SQL] Support to specify join type when calling join with usingColumns
JIRA: https://issues.apache.org/jira/browse/SPARK-10446

Currently the method `join(right: DataFrame, usingColumns: Seq[String])` only supports inner join. It is more convenient to have it support other join types.

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

Closes #8600 from viirya/usingcolumns_df.
2015-09-21 23:46:00 -07:00
Ewan Leith 781b21ba2a [SPARK-10419] [SQL] Adding SQLServer support for datetimeoffset types to JdbcDialects
Reading from Microsoft SQL Server over jdbc fails when the table contains datetimeoffset types.

This patch registers a SQLServer JDBC Dialect that maps datetimeoffset to a String, as Microsoft suggest.

Author: Ewan Leith <ewan.leith@realitymine.com>

Closes #8575 from realitymine-coordinator/sqlserver.
2015-09-21 23:43:20 -07:00
Yin Huai 0494c80ef5 [SPARK-10495] [SQL] Read date values in JSON data stored by Spark 1.5.0.
https://issues.apache.org/jira/browse/SPARK-10681

Author: Yin Huai <yhuai@databricks.com>

Closes #8806 from yhuai/SPARK-10495.
2015-09-21 18:06:45 -07:00
Holden Karau 362539f8d9 [SPARK-10630] [SQL] Add a createDataFrame API that takes in a java list
It would be nice to support creating a DataFrame directly from a Java List of Row.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8779 from holdenk/SPARK-10630-create-DataFrame-from-Java-List.
2015-09-21 13:33:10 -07:00
Josh Rosen 2117eea71e [SPARK-10710] Remove ability to disable spilling in core and SQL
It does not make much sense to set `spark.shuffle.spill` or `spark.sql.planner.externalSort` to false: I believe that these configurations were initially added as "escape hatches" to guard against bugs in the external operators, but these operators are now mature and well-tested. In addition, these configurations are not handled in a consistent way anymore: SQL's Tungsten codepath ignores these configurations and will continue to use spilling operators. Similarly, Spark Core's `tungsten-sort` shuffle manager does not respect `spark.shuffle.spill=false`.

This pull request removes these configurations, adds warnings at the appropriate places, and deletes a large amount of code which was only used in code paths that did not support spilling.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8831 from JoshRosen/remove-ability-to-disable-spilling.
2015-09-19 21:40:21 -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
Andrew Or 7ff8d68cc1 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array
When `TungstenAggregation` hits memory pressure, it switches from hash-based to sort-based aggregation in-place. However, in the process we try to allocate the pointer array for writing to the new `UnsafeExternalSorter` *before* actually freeing the memory from the hash map. This lead to the following exception:
```
 java.io.IOException: Could not acquire 65536 bytes of memory
        at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.initializeForWriting(UnsafeExternalSorter.java:169)
        at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:220)
        at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:126)
        at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:257)
        at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:435)
```

Author: Andrew Or <andrew@databricks.com>

Closes #8827 from andrewor14/allocate-pointer-array.
2015-09-18 23:58:25 -07:00
Yash Datta 20fd35dfd1 [SPARK-10451] [SQL] Prevent unnecessary serializations in InMemoryColumnarTableScan
Many of the fields in InMemoryColumnar scan and InMemoryRelation can be made transient.

This  reduces my 1000ms job to abt 700 ms . The task size reduces from 2.8 mb to ~1300kb

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

Closes #8604 from saucam/serde.
2015-09-18 08:22:38 -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
Liang-Chi Hsieh 2a508df20d [SPARK-10459] [SQL] Do not need to have ConvertToSafe for PythonUDF
JIRA: https://issues.apache.org/jira/browse/SPARK-10459

As mentioned in the JIRA, `PythonUDF` actually could process `UnsafeRow`.

Specially, the rows in `childResults` in `BatchPythonEvaluation` will be projected to a `MutableRow`. So I think we can enable `canProcessUnsafeRows` for `BatchPythonEvaluation` and get rid of redundant `ConvertToSafe`.

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

Closes #8616 from viirya/pyudf-unsafe.
2015-09-17 09:21:21 -07:00
Sun Rui 896edb51ab [SPARK-10050] [SPARKR] Support collecting data of MapType in DataFrame.
1. Support collecting data of MapType from DataFrame.
2. Support data of MapType in createDataFrame.

Author: Sun Rui <rui.sun@intel.com>

Closes #8711 from sun-rui/SPARK-10050.
2015-09-16 13:20:39 -07:00
sureshthalamati 64c29afcb7 [SPARK-9078] [SQL] Allow jdbc dialects to override the query used to check the table.
Current implementation uses query with a LIMIT clause to find if table already exists. This syntax works only in some database systems. This patch changes the default query to the one that is likely to work on most databases, and adds a new method to the  JdbcDialect abstract class to allow  dialects to override the default query.

I looked at using the JDBC meta data calls, it turns out there is no common way to find the current schema, catalog..etc.  There is a new method Connection.getSchema() , but that is available only starting jdk1.7 , and existing jdbc drivers may not have implemented it.  Other option was to use jdbc escape syntax clause for LIMIT, not sure on how well this supported in all the databases also. After looking at all the jdbc metadata options my conclusion was most common way is to use the simple select query with 'where 1 =0' , and allow dialects to customize as needed

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #8676 from sureshthalamati/table_exists_spark-9078.
2015-09-15 19:41:38 -07:00
Andrew Or 35a19f3357 [SPARK-10613] [SPARK-10624] [SQL] Reduce LocalNode tests dependency on SQLContext
Instead of relying on `DataFrames` to verify our answers, we can just use simple arrays. This significantly simplifies the test logic for `LocalNode`s and reduces a lot of code duplicated from `SparkPlanTest`.

This also fixes an additional issue [SPARK-10624](https://issues.apache.org/jira/browse/SPARK-10624) where the output of `TakeOrderedAndProjectNode` is not actually ordered.

Author: Andrew Or <andrew@databricks.com>

Closes #8764 from andrewor14/sql-local-tests-cleanup.
2015-09-15 17:24:32 -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
Reynold Xin a63cdc769f [SPARK-10612] [SQL] Add prepare to LocalNode.
The idea is that we should separate the function call that does memory reservation (i.e. prepare) from the function call that consumes the input (e.g. open()), so all operators can be a chance to reserve memory before they are all consumed.

Author: Reynold Xin <rxin@databricks.com>

Closes #8761 from rxin/SPARK-10612.
2015-09-15 16:53:27 -07:00
zsxwing 217e496444 [SPARK-9996] [SPARK-9997] [SQL] Add local expand and NestedLoopJoin operators
This PR is in conflict with #8535 and #8573. Will update this one when they are merged.

Author: zsxwing <zsxwing@gmail.com>

Closes #8642 from zsxwing/expand-nest-join.
2015-09-14 15:00:27 -07:00
Edoardo Vacchi 64f04154e3 [SPARK-6981] [SQL] Factor out SparkPlanner and QueryExecution from SQLContext
Alternative to PR #6122; in this case the refactored out classes are replaced by inner classes with the same name for backwards binary compatibility

   * process in a lighter-weight, backwards-compatible way

Author: Edoardo Vacchi <uncommonnonsense@gmail.com>

Closes #6356 from evacchi/sqlctx-refactoring-lite.
2015-09-14 14:56:04 -07:00
Josh Rosen b3a7480ab0 [SPARK-10330] Add Scalastyle rule to require use of SparkHadoopUtil JobContext methods
This is a followup to #8499 which adds a Scalastyle rule to mandate the use of SparkHadoopUtil's JobContext accessor methods and fixes the existing violations.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8521 from JoshRosen/SPARK-10330-part2.
2015-09-12 16:23:55 -07:00
JihongMa f4a22808e0 [SPARK-6548] Adding stddev to DataFrame functions
Adding STDDEV support for DataFrame using 1-pass online /parallel algorithm to compute variance. Please review the code change.

Author: JihongMa <linlin200605@gmail.com>
Author: Jihong MA <linlin200605@gmail.com>
Author: Jihong MA <jihongma@jihongs-mbp.usca.ibm.com>
Author: Jihong MA <jihongma@Jihongs-MacBook-Pro.local>

Closes #6297 from JihongMA/SPARK-SQL.
2015-09-12 10:17:15 -07:00
Andrew Or c2af42b5f3 [SPARK-9990] [SQL] Local hash join follow-ups
1. Hide `LocalNodeIterator` behind the `LocalNode#asIterator` method
2. Add tests for this

Author: Andrew Or <andrew@databricks.com>

Closes #8708 from andrewor14/local-hash-join-follow-up.
2015-09-11 15:01:37 -07:00
zsxwing e626ac5f5c [SPARK-9992] [SPARK-9994] [SPARK-9998] [SQL] Implement the local TopK, sample and intersect operators
This PR is in conflict with #8535. I will update this one when #8535 gets merged.

Author: zsxwing <zsxwing@gmail.com>

Closes #8573 from zsxwing/more-local-operators.
2015-09-11 15:00:13 -07:00
Andrew Or 3db72554be [SPARK-10443] [SQL] Refactor SortMergeOuterJoin to reduce duplication
`LeftOutputIterator` and `RightOutputIterator` are symmetrically identical and can share a lot of code. If someone makes a change in one but forgets to do the same thing in the other we'll end up with inconsistent behavior. This patch also adds inline comments to clarify the intention of the code.

Author: Andrew Or <andrew@databricks.com>

Closes #8596 from andrewor14/smoj-cleanup.
2015-09-10 13:22:35 -07:00
Sun Rui 45e3be5c13 [SPARK-10049] [SPARKR] Support collecting data of ArraryType in DataFrame.
this PR :
1.  Enhance reflection in RBackend. Automatically matching a Java array to Scala Seq when finding methods. Util functions like seq(), listToSeq() in R side can be removed, as they will conflict with the Serde logic that transferrs a Scala seq to R side.

2.  Enhance the SerDe to support transferring  a Scala seq to R side. Data of ArrayType in DataFrame
after collection is observed to be of Scala Seq type.

3.  Support ArrayType in createDataFrame().

Author: Sun Rui <rui.sun@intel.com>

Closes #8458 from sun-rui/SPARK-10049.
2015-09-10 12:21:13 -07:00
zsxwing d88abb7e21 [SPARK-9990] [SQL] Create local hash join operator
This PR includes the following changes:
- Add SQLConf to LocalNode
- Add HashJoinNode
- Add ConvertToUnsafeNode and ConvertToSafeNode.scala to test unsafe hash join.

Author: zsxwing <zsxwing@gmail.com>

Closes #8535 from zsxwing/SPARK-9990.
2015-09-10 12:06:49 -07:00
Cheng Hao e048111376 [SPARK-10466] [SQL] UnsafeRow SerDe exception with data spill
Data Spill with UnsafeRow causes assert failure.

```
java.lang.AssertionError: assertion failed
	at scala.Predef$.assert(Predef.scala:165)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2.writeKey(UnsafeRowSerializer.scala:75)
	at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:180)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:688)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:687)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:687)
	at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:683)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at org.apache.spark.util.collection.ExternalSorter.writePartitionedFile(ExternalSorter.scala:683)
	at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:80)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
```

To reproduce that with code (thanks andrewor14):
```scala
bin/spark-shell --master local
  --conf spark.shuffle.memoryFraction=0.005
  --conf spark.shuffle.sort.bypassMergeThreshold=0

sc.parallelize(1 to 2 * 1000 * 1000, 10)
  .map { i => (i, i) }.toDF("a", "b").groupBy("b").avg().count()
```

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

Closes #8635 from chenghao-intel/unsafe_spill.
2015-09-10 11:48:43 -07:00
Cheng Lian 49da38e5f7 [SPARK-10301] [SPARK-10428] [SQL] Addresses comments of PR #8583 and #8509 for master
Author: Cheng Lian <lian@databricks.com>

Closes #8670 from liancheng/spark-10301/address-pr-comments.
2015-09-10 11:01:08 -07:00
Liang-Chi Hsieh 45de518742 [SPARK-9730] [SQL] Add Full Outer Join support for SortMergeJoin
This PR is based on #8383 , thanks to viirya

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

This patch adds the Full Outer Join support for SortMergeJoin. A new class SortMergeFullJoinScanner is added to scan rows from left and right iterators. FullOuterIterator is simply a wrapper of type RowIterator to consume joined rows from SortMergeFullJoinScanner.

Closes #8383

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Davies Liu <davies@databricks.com>

Closes #8579 from davies/smj_fullouter.
2015-09-09 16:02:27 -07:00
Luc Bourlier c1bc4f439f [SPARK-10227] fatal warnings with sbt on Scala 2.11
The bulk of the changes are on `transient` annotation on class parameter. Often the compiler doesn't generate a field for this parameters, so the the transient annotation would be unnecessary.
But if the class parameter are used in methods, then fields are created. So it is safer to keep the annotations.

The remainder are some potential bugs, and deprecated syntax.

Author: Luc Bourlier <luc.bourlier@typesafe.com>

Closes #8433 from skyluc/issue/sbt-2.11.
2015-09-09 09:57:58 +01:00
Yin Huai 7a9dcbc91d [SPARK-10441] [SQL] Save data correctly to json.
https://issues.apache.org/jira/browse/SPARK-10441

Author: Yin Huai <yhuai@databricks.com>

Closes #8597 from yhuai/timestampJson.
2015-09-08 14:10:12 -07:00
Cheng Lian bca8c072bd [SPARK-10434] [SQL] Fixes Parquet schema of arrays that may contain null
To keep full compatibility of Parquet write path with Spark 1.4, we should rename the innermost field name of arrays that may contain null from "array_element" to "array".

Please refer to [SPARK-10434] [1] for more details.

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

Author: Cheng Lian <lian@databricks.com>

Closes #8586 from liancheng/spark-10434/fix-parquet-array-type.
2015-09-05 17:50:12 +08:00
Andrew Or 3339e6f674 [SPARK-10450] [SQL] Minor improvements to readability / style / typos etc.
Author: Andrew Or <andrew@databricks.com>

Closes #8603 from andrewor14/minor-sql-changes.
2015-09-04 15:20:20 -07:00
zsxwing 0349b5b438 [SPARK-10411] [SQL] Move visualization above explain output and hide explain by default
New screenshots after this fix:

<img width="627" alt="s1" src="https://cloud.githubusercontent.com/assets/1000778/9625782/4b2dba36-518b-11e5-9104-c713ff026e3d.png">

Default:
<img width="462" alt="s2" src="https://cloud.githubusercontent.com/assets/1000778/9625817/92366e50-518b-11e5-9981-cdfb774d66b8.png">

After clicking `+details`:
<img width="377" alt="s3" src="https://cloud.githubusercontent.com/assets/1000778/9625784/4ba24342-518b-11e5-8522-846a16a95d44.png">

Author: zsxwing <zsxwing@gmail.com>

Closes #8570 from zsxwing/SPARK-10411.
2015-09-02 22:17:39 -07:00
Yin Huai 03f3e91ff2 [SPARK-10422] [SQL] String column in InMemoryColumnarCache needs to override clone method
https://issues.apache.org/jira/browse/SPARK-10422

Author: Yin Huai <yhuai@databricks.com>

Closes #8578 from yhuai/SPARK-10422.
2015-09-02 21:00:13 -07:00
Cheng Lian 391e6be0ae [SPARK-10301] [SQL] Fixes schema merging for nested structs
This PR can be quite challenging to review.  I'm trying to give a detailed description of the problem as well as its solution here.

When reading Parquet files, we need to specify a potentially nested Parquet schema (of type `MessageType`) as requested schema for column pruning.  This Parquet schema is translated from a Catalyst schema (of type `StructType`), which is generated by the query planner and represents all requested columns.  However, this translation can be fairly complicated because of several reasons:

1.  Requested schema must conform to the real schema of the physical file to be read.

    This means we have to tailor the actual file schema of every individual physical Parquet file to be read according to the given Catalyst schema.  Fortunately we are already doing this in Spark 1.5 by pushing request schema conversion to executor side in PR #7231.

1.  Support for schema merging.

    A single Parquet dataset may consist of multiple physical Parquet files come with different but compatible schemas.  This means we may request for a column path that doesn't exist in a physical Parquet file.  All requested column paths can be nested.  For example, for a Parquet file schema

    ```
    message root {
      required group f0 {
        required group f00 {
          required int32 f000;
          required binary f001 (UTF8);
        }
      }
    }
    ```

    we may request for column paths defined in the following schema:

    ```
    message root {
      required group f0 {
        required group f00 {
          required binary f001 (UTF8);
          required float f002;
        }
      }

      optional double f1;
    }
    ```

    Notice that we pruned column path `f0.f00.f000`, but added `f0.f00.f002` and `f1`.

    The good news is that Parquet handles non-existing column paths properly and always returns null for them.

1.  The map from `StructType` to `MessageType` is a one-to-many map.

    This is the most unfortunate part.

    Due to historical reasons (dark histories!), schemas of Parquet files generated by different libraries have different "flavors".  For example, to handle a schema with a single non-nullable column, whose type is an array of non-nullable integers, parquet-protobuf generates the following Parquet schema:

    ```
    message m0 {
      repeated int32 f;
    }
    ```

    while parquet-avro generates another version:

    ```
    message m1 {
      required group f (LIST) {
        repeated int32 array;
      }
    }
    ```

    and parquet-thrift spills this:

    ```
    message m1 {
      required group f (LIST) {
        repeated int32 f_tuple;
      }
    }
    ```

    All of them can be mapped to the following _unique_ Catalyst schema:

    ```
    StructType(
      StructField(
        "f",
        ArrayType(IntegerType, containsNull = false),
        nullable = false))
    ```

    This greatly complicates Parquet requested schema construction, since the path of a given column varies in different cases.  To read the array elements from files with the above schemas, we must use `f` for `m0`, `f.array` for `m1`, and `f.f_tuple` for `m2`.

In earlier Spark versions, we didn't try to fix this issue properly.  Spark 1.4 and prior versions simply translate the Catalyst schema in a way more or less compatible with parquet-hive and parquet-avro, but is broken in many other cases.  Earlier revisions of Spark 1.5 only try to tailor the Parquet file schema at the first level, and ignore nested ones.  This caused [SPARK-10301] [spark-10301] as well as [SPARK-10005] [spark-10005].  In PR #8228, I tried to avoid the hard part of the problem and made a minimum change in `CatalystRowConverter` to fix SPARK-10005.  However, when taking SPARK-10301 into consideration, keeping hacking `CatalystRowConverter` doesn't seem to be a good idea.  So this PR is an attempt to fix the problem in a proper way.

For a given physical Parquet file with schema `ps` and a compatible Catalyst requested schema `cs`, we use the following algorithm to tailor `ps` to get the result Parquet requested schema `ps'`:

For a leaf column path `c` in `cs`:

- if `c` exists in `cs` and a corresponding Parquet column path `c'` can be found in `ps`, `c'` should be included in `ps'`;
- otherwise, we convert `c` to a Parquet column path `c"` using `CatalystSchemaConverter`, and include `c"` in `ps'`;
- no other column paths should exist in `ps'`.

Then comes the most tedious part:

> Given `cs`, `ps`, and `c`, how to locate `c'` in `ps`?

Unfortunately, there's no quick answer, and we have to enumerate all possible structures defined in parquet-format spec.  They are:

1.  the standard structure of nested types, and
1.  cases defined in all backwards-compatibility rules for `LIST` and `MAP`.

The core part of this PR is `CatalystReadSupport.clipParquetType()`, which tailors a given Parquet file schema according to a requested schema in its Catalyst form.  Backwards-compatibility rules of `LIST` and `MAP` are covered in `clipParquetListType()` and `clipParquetMapType()` respectively.  The column path selection algorithm is implemented in `clipParquetGroupFields()`.

With this PR, we no longer need to do schema tailoring in `CatalystReadSupport` and `CatalystRowConverter`.  Another benefit is that, now we can also read Parquet datasets consist of files with different physical Parquet schema but share the same logical schema, for example, files generated by different Parquet libraries.  This situation is illustrated by [this test case] [test-case].

[spark-10301]: https://issues.apache.org/jira/browse/SPARK-10301
[spark-10005]: https://issues.apache.org/jira/browse/SPARK-10005
[test-case]: 38644d8a45 (diff-a9b98e28ce3ae30641829dffd1173be2R26)

Author: Cheng Lian <lian@databricks.com>

Closes #8509 from liancheng/spark-10301/fix-parquet-requested-schema.
2015-09-01 16:52:59 +08:00
sureshthalamati a2d5c72091 [SPARK-10170] [SQL] Add DB2 JDBC dialect support.
Data frame write to DB2 database is failing because by default JDBC data source implementation is generating a table schema with DB2 unsupported data types TEXT for String, and BIT1(1) for Boolean.

This patch registers DB2 JDBC Dialect that maps String, Boolean to valid DB2 data types.

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #8393 from sureshthalamati/db2_dialect_spark-10170.
2015-08-31 12:39:58 -07:00
zsxwing 13f5f8ec97 [SPARK-9986] [SPARK-9991] [SPARK-9993] [SQL] Create a simple test framework for local operators
This PR includes the following changes:
- Add `LocalNodeTest` for local operator tests and add unit tests for FilterNode and ProjectNode.
- Add `LimitNode` and `UnionNode` and their unit tests to show how to use `LocalNodeTest`. (SPARK-9991, SPARK-9993)

Author: zsxwing <zsxwing@gmail.com>

Closes #8464 from zsxwing/local-execution.
2015-08-29 18:10:44 -07:00
Yin Huai 097a7e36e0 [SPARK-10339] [SPARK-10334] [SPARK-10301] [SQL] Partitioned table scan can OOM driver and throw a better error message when users need to enable parquet schema merging
This fixes the problem that scanning partitioned table causes driver have a high memory pressure and takes down the cluster. Also, with this fix, we will be able to correctly show the query plan of a query consuming partitioned tables.

https://issues.apache.org/jira/browse/SPARK-10339
https://issues.apache.org/jira/browse/SPARK-10334

Finally, this PR squeeze in a "quick fix" for SPARK-10301. It is not a real fix, but it just throw a better error message to let user know what to do.

Author: Yin Huai <yhuai@databricks.com>

Closes #8515 from yhuai/partitionedTableScan.
2015-08-29 16:39:40 -07:00
Josh Rosen 6a6f3c91ee [SPARK-10330] Use SparkHadoopUtil TaskAttemptContext reflection methods in more places
SparkHadoopUtil contains methods that use reflection to work around TaskAttemptContext binary incompatibilities between Hadoop 1.x and 2.x. We should use these methods in more places.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8499 from JoshRosen/use-hadoop-reflection-in-more-places.
2015-08-29 13:36:25 -07:00
Davies Liu 54cda0deb6 [SPARK-10321] sizeInBytes in HadoopFsRelation
Having sizeInBytes in HadoopFsRelation to enable broadcast join.

cc marmbrus

Author: Davies Liu <davies@databricks.com>

Closes #8490 from davies/sizeInByte.
2015-08-27 16:38:00 -07:00
Yin Huai b3dd569ad4 [SPARK-10287] [SQL] Fixes JSONRelation refreshing on read path
https://issues.apache.org/jira/browse/SPARK-10287

After porting json to HadoopFsRelation, it seems hard to keep the behavior of picking up new files automatically for JSON. This PR removes this behavior, so JSON is consistent with others (ORC and Parquet).

Author: Yin Huai <yhuai@databricks.com>

Closes #8469 from yhuai/jsonRefresh.
2015-08-27 16:11:25 -07:00
Sun Rui 71a138cd0e [SPARK-10048] [SPARKR] Support arbitrary nested Java array in serde.
This PR:
1. supports transferring arbitrary nested array from JVM to R side in SerDe;
2. based on 1, collect() implemenation is improved. Now it can support collecting data of complex types
   from a DataFrame.

Author: Sun Rui <rui.sun@intel.com>

Closes #8276 from sun-rui/SPARK-10048.
2015-08-25 13:14:10 -07:00
Michael Armbrust 5c08c86bfa [SPARK-10198] [SQL] Turn off partition verification by default
Author: Michael Armbrust <michael@databricks.com>

Closes #8404 from marmbrus/turnOffPartitionVerification.
2015-08-25 10:22:54 -07:00
Sean Owen 69c9c17716 [SPARK-9613] [CORE] Ban use of JavaConversions and migrate all existing uses to JavaConverters
Replace `JavaConversions` implicits with `JavaConverters`

Most occurrences I've seen so far are necessary conversions; a few have been avoidable. None are in critical code as far as I see, yet.

Author: Sean Owen <sowen@cloudera.com>

Closes #8033 from srowen/SPARK-9613.
2015-08-25 12:33:13 +01:00
Josh Rosen 7bc9a8c624 [SPARK-10195] [SQL] Data sources Filter should not expose internal types
Spark SQL's data sources API exposes Catalyst's internal types through its Filter interfaces. This is a problem because types like UTF8String are not stable developer APIs and should not be exposed to third-parties.

This issue caused incompatibilities when upgrading our `spark-redshift` library to work against Spark 1.5.0.  To avoid these issues in the future we should only expose public types through these Filter objects. This patch accomplishes this by using CatalystTypeConverters to add the appropriate conversions.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8403 from JoshRosen/datasources-internal-vs-external-types.
2015-08-25 01:06:36 -07:00
Cheng Lian bf03fe68d6 [SPARK-10136] [SQL] A more robust fix for SPARK-10136
PR #8341 is a valid fix for SPARK-10136, but it didn't catch the real root cause.  The real problem can be rather tricky to explain, and requires audiences to be pretty familiar with parquet-format spec, especially details of `LIST` backwards-compatibility rules.  Let me have a try to give an explanation here.

The structure of the problematic Parquet schema generated by parquet-avro is something like this:

```
message m {
  <repetition> group f (LIST) {         // Level 1
    repeated group array (LIST) {       // Level 2
      repeated <primitive-type> array;  // Level 3
    }
  }
}
```

(The schema generated by parquet-thrift is structurally similar, just replace the `array` at level 2 with `f_tuple`, and the other one at level 3 with `f_tuple_tuple`.)

This structure consists of two nested legacy 2-level `LIST`-like structures:

1. The repeated group type at level 2 is the element type of the outer array defined at level 1

   This group should map to an `CatalystArrayConverter.ElementConverter` when building converters.

2. The repeated primitive type at level 3 is the element type of the inner array defined at level 2

   This group should also map to an `CatalystArrayConverter.ElementConverter`.

The root cause of SPARK-10136 is that, the group at level 2 isn't properly recognized as the element type of level 1.  Thus, according to parquet-format spec, the repeated primitive at level 3 is left as a so called "unannotated repeated primitive type", and is recognized as a required list of required primitive type, thus a `RepeatedPrimitiveConverter` instead of a `CatalystArrayConverter.ElementConverter` is created for it.

According to  parquet-format spec, unannotated repeated type shouldn't appear in a `LIST`- or `MAP`-annotated group.  PR #8341 fixed this issue by allowing such unannotated repeated type appear in `LIST`-annotated groups, which is a non-standard, hacky, but valid fix.  (I didn't realize this when authoring #8341 though.)

As for the reason why level 2 isn't recognized as a list element type, it's because of the following `LIST` backwards-compatibility rule defined in the parquet-format spec:

> If the repeated field is a group with one field and is named either `array` or uses the `LIST`-annotated group's name with `_tuple` appended then the repeated type is the element type and elements are required.

(The `array` part is for parquet-avro compatibility, while the `_tuple` part is for parquet-thrift.)

This rule is implemented in [`CatalystSchemaConverter.isElementType`] [1], but neglected in [`CatalystRowConverter.isElementType`] [2].  This PR delivers a more robust fix by adding this rule in the latter method.

Note that parquet-avro 1.7.0 also suffers from this issue. Details can be found at [PARQUET-364] [3].

[1]: 85f9a61357/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala (L259-L305)
[2]: 85f9a61357/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala (L456-L463)
[3]: https://issues.apache.org/jira/browse/PARQUET-364

Author: Cheng Lian <lian@databricks.com>

Closes #8361 from liancheng/spark-10136/proper-version.
2015-08-25 14:58:42 +08:00
Yin Huai df7041d02d [SPARK-10196] [SQL] Correctly saving decimals in internal rows to JSON.
https://issues.apache.org/jira/browse/SPARK-10196

Author: Yin Huai <yhuai@databricks.com>

Closes #8408 from yhuai/DecimalJsonSPARK-10196.
2015-08-24 23:38:32 -07:00
Feynman Liang 642c43c81c [SQL] [MINOR] [DOC] Clarify docs for inferring DataFrame from RDD of Products
* Makes `SQLImplicits.rddToDataFrameHolder` scaladoc consistent with `SQLContext.createDataFrame[A <: Product](rdd: RDD[A])` since the former is essentially a wrapper for the latter
 * Clarifies `createDataFrame[A <: Product]` scaladoc to apply for any `RDD[Product]`, not just case classes

Author: Feynman Liang <fliang@databricks.com>

Closes #8406 from feynmanliang/sql-doc-fixes.
2015-08-24 19:45:41 -07:00
Burak Yavuz 9ce0c7ad33 [SPARK-7710] [SPARK-7998] [DOCS] Docs for DataFrameStatFunctions
This PR contains examples on how to use some of the Stat Functions available for DataFrames under `df.stat`.

rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #8378 from brkyvz/update-sql-docs.
2015-08-24 13:48:01 -07:00
Yin Huai e3355090d4 [SPARK-10143] [SQL] Use parquet's block size (row group size) setting as the min split size if necessary.
https://issues.apache.org/jira/browse/SPARK-10143

With this PR, we will set min split size to parquet's block size (row group size) set in the conf if the min split size is smaller. So, we can avoid have too many tasks and even useless tasks for reading parquet data.

I tested it locally. The table I have has 343MB and it is in my local FS. Because I did not set any min/max split size, the default split size was 32MB and the map stage had 11 tasks. But there were only three tasks that actually read data. With my PR, there were only three tasks in the map stage. Here is the difference.

Without this PR:
![image](https://cloud.githubusercontent.com/assets/2072857/9399179/8587dba6-4765-11e5-9189-7ebba52a2b6d.png)

With this PR:
![image](https://cloud.githubusercontent.com/assets/2072857/9399185/a4735d74-4765-11e5-8848-1f1e361a6b4b.png)

Even if the block size setting does match the actual block size of parquet file, I think it is still generally good to use parquet's block size setting if min split size is smaller than this block size.

Tested it on a cluster using
```
val count = sqlContext.table("""store_sales""").groupBy().count().queryExecution.executedPlan(3).execute().count
```
Basically, it reads 0 column of table `store_sales`. My table has 1824 parquet files with size from 80MB to 280MB (1 to 3 row group sizes). Without this patch, in a 16 worker cluster, the job had 5023 tasks and spent 102s. With this patch, the job had 2893 tasks and spent 64s. It is still not as good as using one mapper per file (1824 tasks and 42s), but it is much better than our master.

Author: Yin Huai <yhuai@databricks.com>

Closes #8346 from yhuai/parquetMinSplit.
2015-08-21 14:30:00 -07:00
Liang-Chi Hsieh bb220f6570 [SPARK-10040] [SQL] Use batch insert for JDBC writing
JIRA: https://issues.apache.org/jira/browse/SPARK-10040

We should use batch insert instead of single row in JDBC.

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

Closes #8273 from viirya/jdbc-insert-batch.
2015-08-21 01:43:49 -07:00
Wenchen Fan 907df2fce0 [SQL] [MINOR] remove unnecessary class
This class is identical to `org.apache.spark.sql.execution.datasources.jdbc. DefaultSource` and is not needed.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8334 from cloud-fan/minor.
2015-08-20 13:51:54 -07:00
Cheng Lian 85f9a61357 [SPARK-10136] [SQL] Fixes Parquet support for Avro array of primitive array
I caught SPARK-10136 while adding more test cases to `ParquetAvroCompatibilitySuite`. Actual bug fix code lies in `CatalystRowConverter.scala`.

Author: Cheng Lian <lian@databricks.com>

Closes #8341 from liancheng/spark-10136/parquet-avro-nested-primitive-array.
2015-08-20 11:00:29 -07:00
Reynold Xin b4f4e91c39 [SPARK-10100] [SQL] Eliminate hash table lookup if there is no grouping key in aggregation.
This improves performance by ~ 20 - 30% in one of my local test and should fix the performance regression from 1.4 to 1.5 on ss_max.

Author: Reynold Xin <rxin@databricks.com>

Closes #8332 from rxin/SPARK-10100.
2015-08-20 07:53:27 -07:00
Yin Huai 43e0135421 [SPARK-10092] [SQL] Multi-DB support follow up.
https://issues.apache.org/jira/browse/SPARK-10092

This pr is a follow-up one for Multi-DB support. It has the following changes:

* `HiveContext.refreshTable` now accepts `dbName.tableName`.
* `HiveContext.analyze` now accepts `dbName.tableName`.
* `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateTempTableUsing`, `CreateTempTableUsingAsSelect`, `CreateMetastoreDataSource`, and `CreateMetastoreDataSourceAsSelect` all take `TableIdentifier` instead of the string representation of table name.
* When you call `saveAsTable` with a specified database, the data will be saved to the correct location.
* Explicitly do not allow users to create a temporary with a specified database name (users cannot do it before).
* When we save table to metastore, we also check if db name and table name can be accepted by hive (using `MetaStoreUtils.validateName`).

Author: Yin Huai <yhuai@databricks.com>

Closes #8324 from yhuai/saveAsTableDB.
2015-08-20 15:30:31 +08:00
Reynold Xin 2f2686a73f [SPARK-9242] [SQL] Audit UDAF interface.
A few minor changes:

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

And unrelated to UDAFs:

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

Author: Reynold Xin <rxin@databricks.com>

Closes #8321 from rxin/SPARK-9242.
2015-08-19 17:35:41 -07:00
hyukjinkwon ba5f7e1842 [SPARK-10035] [SQL] Parquet filters does not process EqualNullSafe filter.
As I talked with Lian,

1. I added EquelNullSafe to ParquetFilters
 - It uses the same equality comparison filter with EqualTo since the Parquet filter performs actually null-safe equality comparison.

2. Updated the test code (ParquetFilterSuite)
 - Convert catalyst.Expression to sources.Filter
 - Removed Cast since only Literal is picked up as a proper Filter in DataSourceStrategy
 - Added EquelNullSafe comparison

3. Removed deprecated createFilter for catalyst.Expression

Author: hyukjinkwon <gurwls223@gmail.com>
Author: 권혁진 <gurwls223@gmail.com>

Closes #8275 from HyukjinKwon/master.
2015-08-20 08:13:25 +08:00
Cheng Lian f3ff4c41d2 [SPARK-9899] [SQL] Disables customized output committer when speculation is on
Speculation hates direct output committer, as there are multiple corner cases that may cause data corruption and/or data loss.

Please see this [PR comment] [1] for more details.

[1]: https://github.com/apache/spark/pull/8191#issuecomment-131598385

Author: Cheng Lian <lian@databricks.com>

Closes #8317 from liancheng/spark-9899/speculation-hates-direct-output-committer.
2015-08-19 14:15:28 -07:00
Cheng Lian 21bdbe9fe6 [SPARK-9627] [SQL] Stops using Scala runtime reflection in DictionaryEncoding
`DictionaryEncoding` uses Scala runtime reflection to avoid boxing costs while building the directory array. However, this code path may hit [SI-6240] [1] and throw exception.

[1]: https://issues.scala-lang.org/browse/SI-6240

Author: Cheng Lian <lian@databricks.com>

Closes #8306 from liancheng/spark-9627/in-memory-cache-scala-reflection.
2015-08-19 13:57:52 -07:00
Davies Liu 08887369c8 [SPARK-10073] [SQL] Python withColumn should replace the old column
DataFrame.withColumn in Python should be consistent with the Scala one (replacing the existing column  that has the same name).

cc marmbrus

Author: Davies Liu <davies@databricks.com>

Closes #8300 from davies/with_column.
2015-08-19 13:56:40 -07:00
Reynold Xin 1ff0580eda [SPARK-10093] [SPARK-10096] [SQL] Avoid transformation on executors & fix UDFs on complex types
This is kind of a weird case, but given a sufficiently complex query plan (in this case a TungstenProject with an Exchange underneath), we could have NPEs on the executors due to the time when we were calling transformAllExpressions

In general we should ensure that all transformations occur on the driver and not on the executors. Some reasons for avoid executor side transformations include:

* (this case) Some operator constructors require state such as access to the Spark/SQL conf so doing a makeCopy on the executor can fail.
* (unrelated reason for avoid executor transformations) ExprIds are calculated using an atomic integer, so you can violate their uniqueness constraint by constructing them anywhere other than the driver.

This subsumes #8285.

Author: Reynold Xin <rxin@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #8295 from rxin/SPARK-10096.
2015-08-18 22:08:15 -07:00
Michael Armbrust 80cb25b228 [SPARK-10080] [SQL] Fix binary incompatibility for $ column interpolation
Turns out that inner classes of inner objects are referenced directly, and thus moving it will break binary compatibility.

Author: Michael Armbrust <michael@databricks.com>

Closes #8281 from marmbrus/binaryCompat.
2015-08-18 13:50:51 -07:00
Cheng Lian 5723d26d7e [SPARK-8118] [SQL] Redirects Parquet JUL logger via SLF4J
Parquet hard coded a JUL logger which always writes to stdout. This PR redirects it via SLF4j JUL bridge handler, so that we can control Parquet logs via `log4j.properties`.

This solution is inspired by https://github.com/Parquet/parquet-mr/issues/390#issuecomment-46064909.

Author: Cheng Lian <lian@databricks.com>

Closes #8196 from liancheng/spark-8118/redirect-parquet-jul.
2015-08-18 20:15:33 +08:00
Yu ISHIKAWA a0910315da [MINOR] Format the comment of translate at functions.scala
Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #8265 from yu-iskw/minor-translate-comment.
2015-08-17 23:27:11 -07:00
zsxwing f10660fe7b [SPARK-10036] [SQL] Load JDBC driver in DataFrameReader.jdbc and DataFrameWriter.jdbc
This PR uses `JDBCRDD.getConnector` to load JDBC driver before creating connection in `DataFrameReader.jdbc` and `DataFrameWriter.jdbc`.

Author: zsxwing <zsxwing@gmail.com>

Closes #8232 from zsxwing/SPARK-10036 and squashes the following commits:

adf75de [zsxwing] Add extraOptions to the connection properties
57f59d4 [zsxwing] Load JDBC driver in DataFrameReader.jdbc and DataFrameWriter.jdbc
2015-08-17 11:53:33 -07:00
Cheng Lian 76c155dd44 [SPARK-7837] [SQL] Avoids double closing output writers when commitTask() fails
When inserting data into a `HadoopFsRelation`, if `commitTask()` of the writer container fails, `abortTask()` will be invoked. However, both `commitTask()` and `abortTask()` try to close the output writer(s). The problem is that, closing underlying writers may not be an idempotent operation. E.g., `ParquetRecordWriter.close()` throws NPE when called twice.

Author: Cheng Lian <lian@databricks.com>

Closes #8236 from liancheng/spark-7837/double-closing.
2015-08-18 00:59:05 +08:00
Cheng Lian ae2370e72f [SPARK-10005] [SQL] Fixes schema merging for nested structs
In case of schema merging, we only handled first level fields when converting Parquet groups to `InternalRow`s. Nested struct fields are not properly handled.

For example, the schema of a Parquet file to be read can be:

```
message individual {
  required group f1 {
    optional binary f11 (utf8);
  }
}
```

while the global schema is:

```
message global {
  required group f1 {
    optional binary f11 (utf8);
    optional int32 f12;
  }
}
```

This PR fixes this issue by padding missing fields when creating actual converters.

Author: Cheng Lian <lian@databricks.com>

Closes #8228 from liancheng/spark-10005/nested-schema-merging.
2015-08-16 10:17:58 -07:00
Kun Xu 182f9b7a6d [SPARK-9973] [SQL] Correct in-memory columnar buffer size
The `initialSize` argument of `ColumnBuilder.initialize()` should be the
number of rows rather than bytes.  However `InMemoryColumnarTableScan`
passes in a byte size, which makes Spark SQL allocate more memory than
necessary when building in-memory columnar buffers.

Author: Kun Xu <viper_kun@163.com>

Closes #8189 from viper-kun/errorSize.
2015-08-16 14:44:45 +08:00
Reynold Xin 609ce3c07d [SPARK-9984] [SQL] Create local physical operator interface.
This pull request creates a new operator interface that is more similar to traditional database query iterators (with open/close/next/get).

These local operators are not currently used anywhere, but will become the basis for SPARK-9983 (local physical operators for query execution).

cc zsxwing

Author: Reynold Xin <rxin@databricks.com>

Closes #8212 from rxin/SPARK-9984.
2015-08-14 21:12:11 -07:00
Yijie Shen 6c4fdbec33 [SPARK-8887] [SQL] Explicit define which data types can be used as dynamic partition columns
This PR enforce dynamic partition column data type requirements by adding analysis rules.

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

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

Closes #8201 from yjshen/dynamic_partition_columns.
2015-08-14 21:03:14 -07:00
Wenchen Fan ec29f2034a [SPARK-9634] [SPARK-9323] [SQL] cleanup unnecessary Aliases in LogicalPlan at the end of analysis
Also alias the ExtractValue instead of wrapping it with UnresolvedAlias when resolve attribute in LogicalPlan, as this alias will be trimmed if it's unnecessary.

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

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

Closes #8215 from marmbrus/pr/7957.
2015-08-14 20:59:54 -07:00
Davies Liu 37586e5449 [HOTFIX] fix duplicated braces
Author: Davies Liu <davies@databricks.com>

Closes #8219 from davies/fix_typo.
2015-08-14 20:56:55 -07:00
Yin Huai 932b24fd14 [SPARK-9949] [SQL] Fix TakeOrderedAndProject's output.
https://issues.apache.org/jira/browse/SPARK-9949

Author: Yin Huai <yhuai@databricks.com>

Closes #8179 from yhuai/SPARK-9949.
2015-08-14 17:35:17 -07:00
Wenchen Fan 1150a19b18 [SPARK-8670] [SQL] Nested columns can't be referenced in pyspark
This bug is caused by a wrong column-exist-check in `__getitem__` of pyspark dataframe. `DataFrame.apply` accepts not only top level column names, but also nested column name like `a.b`, so we should remove that check from `__getitem__`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8202 from cloud-fan/nested.
2015-08-14 14:09:46 -07:00
Wenchen Fan 34d610be85 [SPARK-9929] [SQL] support metadata in withColumn
in MLlib sometimes we need to set metadata for the new column, thus we will alias the new column with metadata before call `withColumn` and in `withColumn` we alias this clolumn again. Here I overloaded `withColumn` to allow user set metadata, just like what we did  for `Column.as`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8159 from cloud-fan/withColumn.
2015-08-14 12:00:01 -07:00
Davies Liu bd35385d53 [SPARK-9945] [SQL] pageSize should be calculated from executor.memory
Currently, pageSize of TungstenSort is calculated from driver.memory, it should use executor.memory instead.

Also, in the worst case, the safeFactor could be 4 (because of rounding), increase it to 16.

cc rxin

Author: Davies Liu <davies@databricks.com>

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

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

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

Author: Andrew Or <andrew@databricks.com>

Closes #8111 from andrewor14/sql-tests-refactor.
2015-08-13 17:42:01 -07:00
Davies Liu c50f97dafd [SPARK-9943] [SQL] deserialized UnsafeHashedRelation should be serializable
When the free memory in executor goes low, the cached broadcast objects need to serialized into disk, but currently the deserialized UnsafeHashedRelation can't be serialized , fail with NPE. This PR fixes that.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #8174 from davies/serialize_hashed.
2015-08-13 17:35:11 -07:00
Yijie Shen d0b18919d1 [SPARK-9927] [SQL] Revert 8049 since it's pushing wrong filter down
I made a mistake in #8049 by casting literal value to attribute's data type, which would cause simply truncate the literal value and push a wrong filter down.

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

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

Closes #8157 from yjshen/rever8049.
2015-08-13 13:33:39 +08:00
Davies Liu a8ab2634c1 [SPARK-9832] [SQL] add a thread-safe lookup for BytesToBytseMap
This patch add a thread-safe lookup for BytesToBytseMap, and use that in broadcasted HashedRelation.

Author: Davies Liu <davies@databricks.com>

Closes #8151 from davies/safeLookup.
2015-08-12 21:26:00 -07:00
Yin Huai 2278219054 [SPARK-9920] [SQL] The simpleString of TungstenAggregate does not show its output
https://issues.apache.org/jira/browse/SPARK-9920

Taking `sqlContext.sql("select i, sum(j1) as sum from testAgg group by i").explain()` as an example, the output of our current master is
```
== Physical Plan ==
TungstenAggregate(key=[i#0], value=[(sum(cast(j1#1 as bigint)),mode=Final,isDistinct=false)]
 TungstenExchange hashpartitioning(i#0)
  TungstenAggregate(key=[i#0], value=[(sum(cast(j1#1 as bigint)),mode=Partial,isDistinct=false)]
   Scan ParquetRelation[file:/user/hive/warehouse/testagg][i#0,j1#1]
```
With this PR, the output will be
```
== Physical Plan ==
TungstenAggregate(key=[i#0], functions=[(sum(cast(j1#1 as bigint)),mode=Final,isDistinct=false)], output=[i#0,sum#18L])
 TungstenExchange hashpartitioning(i#0)
  TungstenAggregate(key=[i#0], functions=[(sum(cast(j1#1 as bigint)),mode=Partial,isDistinct=false)], output=[i#0,currentSum#22L])
   Scan ParquetRelation[file:/user/hive/warehouse/testagg][i#0,j1#1]
```

Author: Yin Huai <yhuai@databricks.com>

Closes #8150 from yhuai/SPARK-9920.
2015-08-12 21:24:15 -07:00
Yin Huai 4413d0855a [SPARK-9908] [SQL] When spark.sql.tungsten.enabled is false, broadcast join does not work
https://issues.apache.org/jira/browse/SPARK-9908

Author: Yin Huai <yhuai@databricks.com>

Closes #8149 from yhuai/SPARK-9908.
2015-08-12 20:03:55 -07:00
Davies Liu 7c35746c91 [SPARK-9827] [SQL] fix fd leak in UnsafeRowSerializer
Currently, UnsafeRowSerializer does not close the InputStream, will cause fd leak if the InputStream has an open fd in it.

TODO: the fd could still be leaked, if any items in the stream is not consumed. Currently it replies on GC to close the fd in this case.

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #8116 from davies/fd_leak.
2015-08-12 20:02:55 -07:00
Michael Armbrust 660e6dcff8 [SPARK-9449] [SQL] Include MetastoreRelation's inputFiles
Author: Michael Armbrust <michael@databricks.com>

Closes #8119 from marmbrus/metastoreInputFiles.
2015-08-12 17:07:29 -07:00
Yin Huai 7035d880a0 [SPARK-9894] [SQL] Json writer should handle MapData.
https://issues.apache.org/jira/browse/SPARK-9894

Author: Yin Huai <yhuai@databricks.com>

Closes #8137 from yhuai/jsonMapData.
2015-08-12 16:45:15 -07:00
Andrew Or e0110792ef [SPARK-9747] [SQL] Avoid starving an unsafe operator in aggregation
This is the sister patch to #8011, but for aggregation.

In a nutshell: create the `TungstenAggregationIterator` before computing the parent partition. Internally this creates a `BytesToBytesMap` which acquires a page in the constructor as of this patch. This ensures that the aggregation operator is not starved since we reserve at least 1 page in advance.

rxin yhuai

Author: Andrew Or <andrew@databricks.com>

Closes #8038 from andrewor14/unsafe-starve-memory-agg.
2015-08-12 10:08:35 -07:00
Cheng Lian 3ecb379430 [SPARK-9407] [SQL] Relaxes Parquet ValidTypeMap to allow ENUM predicates to be pushed down
This PR adds a hacky workaround for PARQUET-201, and should be removed once we upgrade to parquet-mr 1.8.1 or higher versions.

In Parquet, not all types of columns can be used for filter push-down optimization.  The set of valid column types is controlled by `ValidTypeMap`.  Unfortunately, in parquet-mr 1.7.0 and prior versions, this limitation is too strict, and doesn't allow `BINARY (ENUM)` columns to be pushed down.  On the other hand, `BINARY (ENUM)` is commonly seen in Parquet files written by libraries like `parquet-avro`.

This restriction is problematic for Spark SQL, because Spark SQL doesn't have a type that maps to Parquet `BINARY (ENUM)` directly, and always converts `BINARY (ENUM)` to Catalyst `StringType`.  Thus, a predicate involving a `BINARY (ENUM)` is recognized as one involving a string field instead and can be pushed down by the query optimizer.  Such predicates are actually perfectly legal except that it fails the `ValidTypeMap` check.

The workaround added here is relaxing `ValidTypeMap` to include `BINARY (ENUM)`.  I also took the chance to simplify `ParquetCompatibilityTest` a little bit when adding regression test.

Author: Cheng Lian <lian@databricks.com>

Closes #8107 from liancheng/spark-9407/parquet-enum-filter-push-down.
2015-08-12 20:01:34 +08:00
Yijie Shen 9d0822455d [SPARK-9182] [SQL] Filters are not passed through to jdbc source
This PR fixes unable to push filter down to JDBC source caused by `Cast` during pattern matching.

While we are comparing columns of different type, there's a big chance we need a cast on the column, therefore not match the pattern directly on Attribute and would fail to push down.

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

Closes #8049 from yjshen/jdbc_pushdown.
2015-08-12 19:54:00 +08:00
Davies Liu c3e9a120e3 [SPARK-9831] [SQL] fix serialization with empty broadcast
Author: Davies Liu <davies@databricks.com>

Closes #8117 from davies/fix_serialization and squashes the following commits:

d21ac71 [Davies Liu] fix serialization with empty broadcast
2015-08-11 22:45:18 -07:00
Reynold Xin afa757c98c [SPARK-9849] [SQL] DirectParquetOutputCommitter qualified name should be backward compatible
DirectParquetOutputCommitter was moved in SPARK-9763. However, users can explicitly set the class as a config option, so we must be able to resolve the old committer qualified name.

Author: Reynold Xin <rxin@databricks.com>

Closes #8114 from rxin/SPARK-9849.
2015-08-11 18:08:49 -07:00
hyukjinkwon 00c02728a6 [SPARK-9814] [SQL] EqualNotNull not passing to data sources
Author: hyukjinkwon <gurwls223@gmail.com>
Author: 권혁진 <gurwls223@gmail.com>

Closes #8096 from HyukjinKwon/master.
2015-08-11 14:04:09 -07:00
zsxwing 5831294a7a [SPARK-9646] [SQL] Add metrics for all join and aggregate operators
This PR added metrics for all join and aggregate operators. However, I found the metrics may be confusing in the following two case:
1. The iterator is not totally consumed and the metric values will be less.
2. Recreating the iterators will make metric values look bigger than the size of the input source, such as `CartesianProduct`.

Author: zsxwing <zsxwing@gmail.com>

Closes #8060 from zsxwing/sql-metrics and squashes the following commits:

40f3fc1 [zsxwing] Mark LongSQLMetric private[metric] to avoid using incorrectly and leak memory
b1b9071 [zsxwing] Merge branch 'master' into sql-metrics
4bef25a [zsxwing] Add metrics for SortMergeOuterJoin
95ccfc6 [zsxwing] Merge branch 'master' into sql-metrics
67cb4dd [zsxwing] Add metrics for Project and TungstenProject; remove metrics from PhysicalRDD and LocalTableScan
0eb47d4 [zsxwing] Merge branch 'master' into sql-metrics
dd9d932 [zsxwing] Avoid creating new Iterators
589ea26 [zsxwing] Add metrics for all join and aggregate operators
2015-08-11 12:39:13 -07:00
Reynold Xin d378396f86 [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
PlatformDependent.UNSAFE is way too verbose.

Author: Reynold Xin <rxin@databricks.com>

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

229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
2015-08-11 08:41:06 -07:00
Josh Rosen 91e9389f39 [SPARK-9729] [SPARK-9363] [SQL] Use sort merge join for left and right outer join
This patch adds a new `SortMergeOuterJoin` operator that performs left and right outer joins using sort merge join.  It also refactors `SortMergeJoin` in order to improve performance and code clarity.

Along the way, I also performed a couple pieces of minor cleanup and optimization:

- Rename the `HashJoin` physical planner rule to `EquiJoinSelection`, since it's also used for non-hash joins.
- Rewrite the comment at the top of `HashJoin` to better explain the precedence for choosing join operators.
- Update `JoinSuite` to use `SqlTestUtils.withConf` for changing SQLConf settings.

This patch incorporates several ideas from adrian-wang's patch, #5717.

Closes #5717.

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

Author: Josh Rosen <joshrosen@databricks.com>
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #7904 from JoshRosen/outer-join-smj and squashes 1 commits.
2015-08-10 22:04:41 -07:00
Damian Guy 071bbad5db [SPARK-9340] [SQL] Fixes converting unannotated Parquet lists
This PR is inspired by #8063 authored by dguy. Especially, testing Parquet files added here are all taken from that PR.

**Committer who merges this PR should attribute it to "Damian Guy <damian.guygmail.com>".**

----

SPARK-6776 and SPARK-6777 followed `parquet-avro` to implement backwards-compatibility rules defined in `parquet-format` spec. However, both Spark SQL and `parquet-avro` neglected the following statement in `parquet-format`:

> This does not affect repeated fields that are not annotated: A repeated field that is neither contained by a `LIST`- or `MAP`-annotated group nor annotated by `LIST` or `MAP` should be interpreted as a required list of required elements where the element type is the type of the field.

One of the consequences is that, Parquet files generated by `parquet-protobuf` containing unannotated repeated fields are not correctly converted to Catalyst arrays.

This PR fixes this issue by

1. Handling unannotated repeated fields in `CatalystSchemaConverter`.
2. Converting this kind of special repeated fields to Catalyst arrays in `CatalystRowConverter`.

   Two special converters, `RepeatedPrimitiveConverter` and `RepeatedGroupConverter`, are added. They delegate actual conversion work to a child `elementConverter` and accumulates elements in an `ArrayBuffer`.

   Two extra methods, `start()` and `end()`, are added to `ParentContainerUpdater`. So that they can be used to initialize new `ArrayBuffer`s for unannotated repeated fields, and propagate converted array values to upstream.

Author: Cheng Lian <lian@databricks.com>

Closes #8070 from liancheng/spark-9340/unannotated-parquet-list and squashes the following commits:

ace6df7 [Cheng Lian] Moves ParquetProtobufCompatibilitySuite
f1c7bfd [Cheng Lian] Updates .rat-excludes
420ad2b [Cheng Lian] Fixes converting unannotated Parquet lists
2015-08-11 12:46:33 +08:00
Reynold Xin 40ed2af587 [SPARK-9763][SQL] Minimize exposure of internal SQL classes.
There are a few changes in this pull request:

1. Moved all data sources to execution.datasources, except the public JDBC APIs.
2. In order to maintain backward compatibility from 1, added a backward compatibility translation map in data source resolution.
3. Moved ui and metric package into execution.
4. Added more documentation on some internal classes.
5. Renamed DataSourceRegister.format -> shortName.
6. Added "override" modifier on shortName.
7. Removed IntSQLMetric.

Author: Reynold Xin <rxin@databricks.com>

Closes #8056 from rxin/SPARK-9763 and squashes the following commits:

9df4801 [Reynold Xin] Removed hardcoded name in test cases.
d9babc6 [Reynold Xin] Shorten.
e484419 [Reynold Xin] Removed VisibleForTesting.
171b812 [Reynold Xin] MimaExcludes.
2041389 [Reynold Xin] Compile ...
79dda42 [Reynold Xin] Compile.
0818ba3 [Reynold Xin] Removed IntSQLMetric.
c46884f [Reynold Xin] Two more fixes.
f9aa88d [Reynold Xin] [SPARK-9763][SQL] Minimize exposure of internal SQL classes.
2015-08-10 13:49:23 -07:00
Josh Rosen 0fe66744f1 [SPARK-9784] [SQL] Exchange.isUnsafe should check whether codegen and unsafe are enabled
Exchange.isUnsafe should check whether codegen and unsafe are enabled.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8073 from JoshRosen/SPARK-9784 and squashes the following commits:

7a1019f [Josh Rosen] [SPARK-9784] Exchange.isUnsafe should check whether codegen and unsafe are enabled
2015-08-10 13:05:03 -07:00
Cheng Lian e3fef0f9e1 [SPARK-9743] [SQL] Fixes JSONRelation refreshing
PR #7696 added two `HadoopFsRelation.refresh()` calls ([this] [1], and [this] [2]) in `DataSourceStrategy` to make test case `InsertSuite.save directly to the path of a JSON table` pass. However, this forces every `HadoopFsRelation` table scan to do a refresh, which can be super expensive for tables with large number of partitions.

The reason why the original test case fails without the `refresh()` calls is that, the old JSON relation builds the base RDD with the input paths, while `HadoopFsRelation` provides `FileStatus`es of leaf files. With the old JSON relation, we can create a temporary table based on a path, writing data to that, and then read newly written data without refreshing the table. This is no long true for `HadoopFsRelation`.

This PR removes those two expensive refresh calls, and moves the refresh into `JSONRelation` to fix this issue. We might want to update `HadoopFsRelation` interface to provide better support for this use case.

[1]: ebfd91c542/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala (L63)
[2]: ebfd91c542/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala (L91)

Author: Cheng Lian <lian@databricks.com>

Closes #8035 from liancheng/spark-9743/fix-json-relation-refreshing and squashes the following commits:

ec1957d [Cheng Lian] Fixes JSONRelation refreshing
2015-08-10 09:07:08 -07:00
Yin Huai be80def0d0 [SPARK-9777] [SQL] Window operator can accept UnsafeRows
https://issues.apache.org/jira/browse/SPARK-9777

Author: Yin Huai <yhuai@databricks.com>

Closes #8064 from yhuai/windowUnsafe and squashes the following commits:

8fb3537 [Yin Huai] Set canProcessUnsafeRows to true.
2015-08-09 22:33:53 -07:00
Josh Rosen 23cf5af08d [SPARK-9703] [SQL] Refactor EnsureRequirements to avoid certain unnecessary shuffles
This pull request refactors the `EnsureRequirements` planning rule in order to avoid the addition of certain unnecessary shuffles.

As an example of how unnecessary shuffles can occur, consider SortMergeJoin, which requires clustered distribution and sorted ordering of its children's input rows. Say that both of SMJ's children produce unsorted output but are both SinglePartition. In this case, we will need to inject sort operators but should not need to inject Exchanges. Unfortunately, it looks like the EnsureRequirements unnecessarily repartitions using a hash partitioning.

This patch solves this problem by refactoring `EnsureRequirements` to properly implement the `compatibleWith` checks that were broken in earlier implementations. See the significant inline comments for a better description of how this works. The majority of this PR is new comments and test cases, with few actual changes to the code.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7988 from JoshRosen/exchange-fixes and squashes the following commits:

38006e7 [Josh Rosen] Rewrite EnsureRequirements _yet again_ to make things even simpler
0983f75 [Josh Rosen] More guarantees vs. compatibleWith cleanup; delete BroadcastPartitioning.
8784bd9 [Josh Rosen] Giant comment explaining compatibleWith vs. guarantees
1307c50 [Josh Rosen] Update conditions for requiring child compatibility.
18cddeb [Josh Rosen] Rename DummyPlan to DummySparkPlan.
2c7e126 [Josh Rosen] Merge remote-tracking branch 'origin/master' into exchange-fixes
fee65c4 [Josh Rosen] Further refinement to comments / reasoning
642b0bb [Josh Rosen] Further expand comment / reasoning
06aba0c [Josh Rosen] Add more comments
8dbc845 [Josh Rosen] Add even more tests.
4f08278 [Josh Rosen] Fix the test by adding the compatibility check to EnsureRequirements
a1c12b9 [Josh Rosen] Add failing test to demonstrate allCompatible bug
0725a34 [Josh Rosen] Small assertion cleanup.
5172ac5 [Josh Rosen] Add test for requiresChildrenToProduceSameNumberOfPartitions.
2e0f33a [Josh Rosen] Write a more generic test for EnsureRequirements.
752b8de [Josh Rosen] style fix
c628daf [Josh Rosen] Revert accidental ExchangeSuite change.
c9fb231 [Josh Rosen] Rewrite exchange to fix better handle this case.
adcc742 [Josh Rosen] Move test to PlannerSuite.
0675956 [Josh Rosen] Preserving ordering and partitioning in row format converters also does not help.
cc5669c [Josh Rosen] Adding outputPartitioning to Repartition does not fix the test.
2dfc648 [Josh Rosen] Add failing test illustrating bad exchange planning.
2015-08-09 14:26:01 -07:00
Reynold Xin e9c36938ba [SPARK-9752][SQL] Support UnsafeRow in Sample operator.
In order for this to work, I had to disable gap sampling.

Author: Reynold Xin <rxin@databricks.com>

Closes #8040 from rxin/SPARK-9752 and squashes the following commits:

f9e248c [Reynold Xin] Fix the test case for real this time.
adbccb3 [Reynold Xin] Fixed test case.
589fb23 [Reynold Xin] Merge branch 'SPARK-9752' of github.com:rxin/spark into SPARK-9752
55ccddc [Reynold Xin] Fixed core test.
78fa895 [Reynold Xin] [SPARK-9752][SQL] Support UnsafeRow in Sample operator.
c9e7112 [Reynold Xin] [SPARK-9752][SQL] Support UnsafeRow in Sample operator.
2015-08-09 10:58:36 -07:00
Yijie Shen 3ca995b78f [SPARK-6212] [SQL] The EXPLAIN output of CTAS only shows the analyzed plan
JIRA: https://issues.apache.org/jira/browse/SPARK-6212

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

Closes #7986 from yjshen/ctas_explain and squashes the following commits:

bb6fee5 [Yijie Shen] refine test
f731041 [Yijie Shen] address comment
b2cf8ab [Yijie Shen] bug fix
bd7eb20 [Yijie Shen] ctas explain
2015-08-08 21:05:50 -07:00
CodingCat 25c363e93b [MINOR] inaccurate comments for showString()
Author: CodingCat <zhunansjtu@gmail.com>

Closes #8050 from CodingCat/minor and squashes the following commits:

5bc4b89 [CodingCat] inaccurate comments
2015-08-08 18:22:46 -07:00
Joseph Batchik a3aec918be [SPARK-9486][SQL] Add data source aliasing for external packages
Users currently have to provide the full class name for external data sources, like:

`sqlContext.read.format("com.databricks.spark.avro").load(path)`

This allows external data source packages to register themselves using a Service Loader so that they can add custom alias like:

`sqlContext.read.format("avro").load(path)`

This makes it so that using external data source packages uses the same format as the internal data sources like parquet, json, etc.

Author: Joseph Batchik <joseph.batchik@cloudera.com>
Author: Joseph Batchik <josephbatchik@gmail.com>

Closes #7802 from JDrit/service_loader and squashes the following commits:

49a01ec [Joseph Batchik] fixed a couple of format / error bugs
e5e93b2 [Joseph Batchik] modified rat file to only excluded added services
72b349a [Joseph Batchik] fixed error with orc data source actually
9f93ea7 [Joseph Batchik] fixed error with orc data source
87b7f1c [Joseph Batchik] fixed typo
101cd22 [Joseph Batchik] removing unneeded changes
8f3cf43 [Joseph Batchik] merged in changes
b63d337 [Joseph Batchik] merged in master
95ae030 [Joseph Batchik] changed the new trait to be used as a mixin for data source to register themselves
74db85e [Joseph Batchik] reformatted class loader
ac2270d [Joseph Batchik] removing some added test
a6926db [Joseph Batchik] added test cases for data source loader
208a2a8 [Joseph Batchik] changes to do error catching if there are multiple data sources
946186e [Joseph Batchik] started working on service loader
2015-08-08 11:03:01 -07:00
Cheng Lian 11caf1ce29 [SPARK-4176] [SQL] [MINOR] Should use unscaled Long to write decimals for precision <= 18 rather than 8
This PR fixes a minor bug introduced in #7455: when writing decimals, we should use the unscaled Long for better performance when the precision <= 18 rather than 8 (should be a typo). This bug doesn't affect correctness, but hurts Parquet decimal writing performance.

This PR also replaced similar magic numbers with newly defined constants.

Author: Cheng Lian <lian@databricks.com>

Closes #8031 from liancheng/spark-4176/minor-fix-for-writing-decimals and squashes the following commits:

10d4ea3 [Cheng Lian] Should use unscaled Long to write decimals for precision <= 18 rather than 8
2015-08-08 18:09:48 +08:00
Yin Huai c564b27447 [SPARK-9753] [SQL] TungstenAggregate should also accept InternalRow instead of just UnsafeRow
https://issues.apache.org/jira/browse/SPARK-9753

This PR makes TungstenAggregate to accept `InternalRow` instead of just `UnsafeRow`. Also, it adds an `getAggregationBufferFromUnsafeRow` method to `UnsafeFixedWidthAggregationMap`. It is useful when we already have grouping keys stored in `UnsafeRow`s. Finally, it wraps `InputStream` and `OutputStream` in `UnsafeRowSerializer` with `BufferedInputStream` and `BufferedOutputStream`, respectively.

Author: Yin Huai <yhuai@databricks.com>

Closes #8041 from yhuai/joinedRowForProjection and squashes the following commits:

7753e34 [Yin Huai] Use BufferedInputStream and BufferedOutputStream.
d68b74e [Yin Huai] Use joinedRow instead of UnsafeRowJoiner.
e93c009 [Yin Huai] Add getAggregationBufferFromUnsafeRow for cases that the given groupingKeyRow is already an UnsafeRow.
2015-08-07 20:04:17 -07:00
Reynold Xin 998f4ff94d [SPARK-9754][SQL] Remove TypeCheck in debug package.
TypeCheck no longer applies in the new "Tungsten" world.

Author: Reynold Xin <rxin@databricks.com>

Closes #8043 from rxin/SPARK-9754 and squashes the following commits:

4ec471e [Reynold Xin] [SPARK-9754][SQL] Remove TypeCheck in debug package.
2015-08-07 19:09:28 -07:00
Michael Armbrust 49702bd738 [SPARK-8890] [SQL] Fallback on sorting when writing many dynamic partitions
Previously, we would open a new file for each new dynamic written out using `HadoopFsRelation`.  For formats like parquet this is very costly due to the buffers required to get good compression.  In this PR I refactor the code allowing us to fall back on an external sort when many partitions are seen.  As such each task will open no more than `spark.sql.sources.maxFiles` files.  I also did the following cleanup:

 - Instead of keying the file HashMap on an expensive to compute string representation of the partition, we now use a fairly cheap UnsafeProjection that avoids heap allocations.
 - The control flow for instantiating and invoking a writer container has been simplified.  Now instead of switching in two places based on the use of partitioning, the specific writer container must implement a single method `writeRows` that is invoked using `runJob`.
 - `InternalOutputWriter` has been removed.  Instead we have a `private[sql]` method `writeInternal` that converts and calls the public method.  This method can be overridden by internal datasources to avoid the conversion.  This change remove a lot of code duplication and per-row `asInstanceOf` checks.
 - `commands.scala` has been split up.

Author: Michael Armbrust <michael@databricks.com>

Closes #8010 from marmbrus/fsWriting and squashes the following commits:

00804fe [Michael Armbrust] use shuffleMemoryManager.pageSizeBytes
775cc49 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into fsWriting
17b690e [Michael Armbrust] remove comment
40f0372 [Michael Armbrust] address comments
f5675bd [Michael Armbrust] char -> string
7e2d0a4 [Michael Armbrust] make sure we close current writer
8100100 [Michael Armbrust] delete empty commands.scala
71cc717 [Michael Armbrust] update comment
8ec75ac [Michael Armbrust] [SPARK-8890][SQL] Fallback on sorting when writing many dynamic partitions
2015-08-07 16:24:50 -07:00
Andrew Or 881548ab20 [SPARK-9674] Re-enable ignored test in SQLQuerySuite
The original code that this test tests is removed in 9270bd06fd. It was ignored shortly before that so we never caught it. This patch re-enables the test and adds the code necessary to make it pass.

JoshRosen yhuai

Author: Andrew Or <andrew@databricks.com>

Closes #8015 from andrewor14/SPARK-9674 and squashes the following commits:

225eac2 [Andrew Or] Merge branch 'master' of github.com:apache/spark into SPARK-9674
8c24209 [Andrew Or] Fix NPE
e541d64 [Andrew Or] Track aggregation memory for both sort and hash
0be3a42 [Andrew Or] Fix test
2015-08-07 14:20:13 -07:00
Reynold Xin 05d04e10a8 [SPARK-9733][SQL] Improve physical plan explain for data sources
All data sources show up as "PhysicalRDD" in physical plan explain. It'd be better if we can show the name of the data source.

Without this patch:
```
== Physical Plan ==
NewAggregate with UnsafeHybridAggregationIterator ArrayBuffer(date#0, cat#1) ArrayBuffer((sum(CAST((CAST(count#2, IntegerType) + 1), LongType))2,mode=Final,isDistinct=false))
 Exchange hashpartitioning(date#0,cat#1)
  NewAggregate with UnsafeHybridAggregationIterator ArrayBuffer(date#0, cat#1) ArrayBuffer((sum(CAST((CAST(count#2, IntegerType) + 1), LongType))2,mode=Partial,isDistinct=false))
   PhysicalRDD [date#0,cat#1,count#2], MapPartitionsRDD[3] at
```

With this patch:
```
== Physical Plan ==
TungstenAggregate(key=[date#0,cat#1], value=[(sum(CAST((CAST(count#2, IntegerType) + 1), LongType)),mode=Final,isDistinct=false)]
 Exchange hashpartitioning(date#0,cat#1)
  TungstenAggregate(key=[date#0,cat#1], value=[(sum(CAST((CAST(count#2, IntegerType) + 1), LongType)),mode=Partial,isDistinct=false)]
   ConvertToUnsafe
    Scan ParquetRelation[file:/scratch/rxin/spark/sales4][date#0,cat#1,count#2]
```

Author: Reynold Xin <rxin@databricks.com>

Closes #8024 from rxin/SPARK-9733 and squashes the following commits:

811b90e [Reynold Xin] Fixed Python test case.
52cab77 [Reynold Xin] Cast.
eea9ccc [Reynold Xin] Fix test case.
fcecb22 [Reynold Xin] [SPARK-9733][SQL] Improve explain message for data source scan node.
2015-08-07 13:41:45 -07:00
Reynold Xin aeddeafc03 [SPARK-9667][SQL] followup: Use GenerateUnsafeProjection.canSupport to test Exchange supported data types.
This way we recursively test the data types.

cc chenghao-intel

Author: Reynold Xin <rxin@databricks.com>

Closes #8036 from rxin/cansupport and squashes the following commits:

f7302ff [Reynold Xin] Can GenerateUnsafeProjection.canSupport to test Exchange supported data types.
2015-08-07 13:26:03 -07:00
Reynold Xin 76eaa70183 [SPARK-9674][SPARK-9667] Remove SparkSqlSerializer2
It is now subsumed by various Tungsten operators.

Author: Reynold Xin <rxin@databricks.com>

Closes #7981 from rxin/SPARK-9674 and squashes the following commits:

144f96e [Reynold Xin] Re-enable test
58b7332 [Reynold Xin] Disable failing list.
fb797e3 [Reynold Xin] Match all UDTs.
be9f243 [Reynold Xin] Updated if.
71fc99c [Reynold Xin] [SPARK-9674][SPARK-9667] Remove GeneratedAggregate & SparkSqlSerializer2.
2015-08-07 11:02:53 -07:00
zsxwing ebfd91c542 [SPARK-9467][SQL]Add SQLMetric to specialize accumulators to avoid boxing
This PR adds SQLMetric/SQLMetricParam/SQLMetricValue to specialize accumulators to avoid boxing. All SQL metrics should use these classes rather than `Accumulator`.

Author: zsxwing <zsxwing@gmail.com>

Closes #7996 from zsxwing/sql-accu and squashes the following commits:

14a5f0a [zsxwing] Address comments
367ca23 [zsxwing] Use localValue directly to avoid changing Accumulable
42f50c3 [zsxwing] Add SQLMetric to specialize accumulators to avoid boxing
2015-08-07 00:09:58 -07:00
Reynold Xin 4309262ec9 [SPARK-9700] Pick default page size more intelligently.
Previously, we use 64MB as the default page size, which was way too big for a lot of Spark applications (especially for single node).

This patch changes it so that the default page size, if unset by the user, is determined by the number of cores available and the total execution memory available.

Author: Reynold Xin <rxin@databricks.com>

Closes #8012 from rxin/pagesize and squashes the following commits:

16f4756 [Reynold Xin] Fixed failing test.
5afd570 [Reynold Xin] private...
0d5fb98 [Reynold Xin] Update default value.
674a6cd [Reynold Xin] Address review feedback.
dc00e05 [Reynold Xin] Merge with master.
73ebdb6 [Reynold Xin] [SPARK-9700] Pick default page size more intelligently.
2015-08-06 23:18:29 -07:00
zsxwing 7aaed1b114 [SPARK-8862][SQL]Support multiple SQLContexts in Web UI
This is a follow-up PR to solve the UI issue when there are multiple SQLContexts. Each SQLContext has a separate tab and contains queries which are executed by this SQLContext.

<img width="1366" alt="multiple sqlcontexts" src="https://cloud.githubusercontent.com/assets/1000778/9088391/54584434-3bc2-11e5-9caf-94c2b0da528e.png">

Author: zsxwing <zsxwing@gmail.com>

Closes #7962 from zsxwing/multi-sqlcontext-ui and squashes the following commits:

cf661e1 [zsxwing] sql -> SQL
39b0c97 [zsxwing] Support multiple SQLContexts in Web UI
2015-08-06 22:52:23 -07:00
Davies Liu 17284db314 [SPARK-9228] [SQL] use tungsten.enabled in public for both of codegen/unsafe
spark.sql.tungsten.enabled will be the default value for both codegen and unsafe, they are kept internally for debug/testing.

cc marmbrus rxin

Author: Davies Liu <davies@databricks.com>

Closes #7998 from davies/tungsten and squashes the following commits:

c1c16da [Davies Liu] update doc
1a47be1 [Davies Liu] use tungsten.enabled for both of codegen/unsafe

(cherry picked from commit 4e70e8256c)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-08-06 19:42:02 -07:00
Andrew Or 014a9f9d8c [SPARK-9709] [SQL] Avoid starving unsafe operators that use sort
The issue is that a task may run multiple sorts, and the sorts run by the child operator (i.e. parent RDD) may acquire all available memory such that other sorts in the same task do not have enough to proceed. This manifests itself in an `IOException("Unable to acquire X bytes of memory")` thrown by `UnsafeExternalSorter`.

The solution is to reserve a page in each sorter in the chain before computing the child operator's (parent RDD's) partitions. This requires us to use a new special RDD that does some preparation before computing the parent's partitions.

Author: Andrew Or <andrew@databricks.com>

Closes #8011 from andrewor14/unsafe-starve-memory and squashes the following commits:

35b69a4 [Andrew Or] Simplify test
0b07782 [Andrew Or] Minor: update comments
5d5afdf [Andrew Or] Merge branch 'master' of github.com:apache/spark into unsafe-starve-memory
254032e [Andrew Or] Add tests
234acbd [Andrew Or] Reserve a page in sorter when preparing each partition
b889e08 [Andrew Or] MapPartitionsWithPreparationRDD
2015-08-06 19:04:57 -07:00
Reynold Xin b87825310a [SPARK-9692] Remove SqlNewHadoopRDD's generated Tuple2 and InterruptibleIterator.
A small performance optimization – we don't need to generate a Tuple2 and then immediately discard the key. We also don't need an extra wrapper from InterruptibleIterator.

Author: Reynold Xin <rxin@databricks.com>

Closes #8000 from rxin/SPARK-9692 and squashes the following commits:

1d4d0b3 [Reynold Xin] [SPARK-9692] Remove SqlNewHadoopRDD's generated Tuple2 and InterruptibleIterator.
2015-08-06 18:25:38 -07:00
Davies Liu 49b1504fe3 Revert "[SPARK-9228] [SQL] use tungsten.enabled in public for both of codegen/unsafe"
This reverts commit 4e70e8256c.
2015-08-06 17:36:12 -07:00
Michael Armbrust 0867b23c74 [SPARK-9650][SQL] Fix quoting behavior on interpolated column names
Make sure that `$"column"` is consistent with other methods with respect to backticks.  Adds a bunch of tests for various ways of constructing columns.

Author: Michael Armbrust <michael@databricks.com>

Closes #7969 from marmbrus/namesWithDots and squashes the following commits:

53ef3d7 [Michael Armbrust] [SPARK-9650][SQL] Fix quoting behavior on interpolated column names
2bf7a92 [Michael Armbrust] WIP
2015-08-06 17:31:16 -07:00
Davies Liu 4e70e8256c [SPARK-9228] [SQL] use tungsten.enabled in public for both of codegen/unsafe
spark.sql.tungsten.enabled will be the default value for both codegen and unsafe, they are kept internally for debug/testing.

cc marmbrus rxin

Author: Davies Liu <davies@databricks.com>

Closes #7998 from davies/tungsten and squashes the following commits:

c1c16da [Davies Liu] update doc
1a47be1 [Davies Liu] use tungsten.enabled for both of codegen/unsafe
2015-08-06 17:30:31 -07:00
Yin Huai 3504bf3aa9 [SPARK-9630] [SQL] Clean up new aggregate operators (SPARK-9240 follow up)
This is the followup of https://github.com/apache/spark/pull/7813. It renames `HybridUnsafeAggregationIterator` to `TungstenAggregationIterator` and makes it only work with `UnsafeRow`. Also, I add a `TungstenAggregate` that uses `TungstenAggregationIterator` and make `SortBasedAggregate` (renamed from `SortBasedAggregate`) only works with `SafeRow`.

Author: Yin Huai <yhuai@databricks.com>

Closes #7954 from yhuai/agg-followUp and squashes the following commits:

4d2f4fc [Yin Huai] Add comments and free map.
0d7ddb9 [Yin Huai] Add TungstenAggregationQueryWithControlledFallbackSuite to test fall back process.
91d69c2 [Yin Huai] Rename UnsafeHybridAggregationIterator to  TungstenAggregateIteraotr and make it only work with UnsafeRow.
2015-08-06 15:04:44 -07:00
Liang-Chi Hsieh 21fdfd7d6f [SPARK-9548][SQL] Add a destructive iterator for BytesToBytesMap
This pull request adds a destructive iterator to BytesToBytesMap. When used, the iterator frees pages as it traverses them. This is part of the effort to avoid starving when we have more than one operators that can exhaust memory.

This is based on #7924, but fixes a bug there (Don't use destructive iterator in UnsafeKVExternalSorter).

Closes #7924.

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

Closes #8003 from rxin/map-destructive-iterator and squashes the following commits:

6b618c3 [Reynold Xin] Don't use destructive iterator in UnsafeKVExternalSorter.
a7bd8ec [Reynold Xin] Merge remote-tracking branch 'viirya/destructive_iter' into map-destructive-iterator
7652083 [Liang-Chi Hsieh] For comments: add destructiveIterator(), modify unit test, remove code block.
4a3e9de [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into destructive_iter
581e9e3 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into destructive_iter
f0ff783 [Liang-Chi Hsieh] No need to free last page.
9e9d2a3 [Liang-Chi Hsieh] Add a destructive iterator for BytesToBytesMap.
2015-08-06 14:33:29 -07:00
Wenchen Fan 1f62f104c7 [SPARK-9632][SQL] update InternalRow.toSeq to make it accept data type info
This re-applies #7955, which was reverted due to a race condition to fix build breaking.

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

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

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

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

21665e2 [Wenchen Fan] fix hive again...
4addf29 [Wenchen Fan] fix hive
bc16c59 [Wenchen Fan] minor fix
33d802c [Wenchen Fan] pass data type info to InternalRow.toSeq
3dd033e [Wenchen Fan] move the default special getters implementation from InternalRow to BaseGenericInternalRow
2015-08-06 10:40:54 -07:00
Reynold Xin 5e1b0ef079 [SPARK-9659][SQL] Rename inSet to isin to match Pandas function.
Inspiration drawn from this blog post: https://lab.getbase.com/pandarize-spark-dataframes/

Author: Reynold Xin <rxin@databricks.com>

Closes #7977 from rxin/isin and squashes the following commits:

9b1d3d6 [Reynold Xin] Added return.
2197d37 [Reynold Xin] Fixed test case.
7c1b6cf [Reynold Xin] Import warnings.
4f4a35d [Reynold Xin] [SPARK-9659][SQL] Rename inSet to isin to match Pandas function.
2015-08-06 10:39:16 -07:00
Burak Yavuz 98e69467d4 [SPARK-9615] [SPARK-9616] [SQL] [MLLIB] Bugs related to FrequentItems when merging and with Tungsten
In short:
1- FrequentItems should not use the InternalRow representation, because the keys in the map get messed up. For example, every key in the Map correspond to the very last element observed in the partition, when the elements are strings.

2- Merging two partitions had a bug:

**Existing behavior with size 3**
Partition A -> Map(1 -> 3, 2 -> 3, 3 -> 4)
Partition B -> Map(4 -> 25)
Result -> Map()

**Correct Behavior:**
Partition A -> Map(1 -> 3, 2 -> 3, 3 -> 4)
Partition B -> Map(4 -> 25)
Result -> Map(3 -> 1, 4 -> 22)

cc mengxr rxin JoshRosen

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #7945 from brkyvz/freq-fix and squashes the following commits:

07fa001 [Burak Yavuz] address 2
1dc61a8 [Burak Yavuz] address 1
506753e [Burak Yavuz] fixed and added reg test
47bfd50 [Burak Yavuz] pushing
2015-08-06 10:29:40 -07:00
Davies Liu 93085c992e [SPARK-9482] [SQL] Fix thread-safey issue of using UnsafeProjection in join
This PR also change to use `def` instead of `lazy val` for UnsafeProjection, because it's not thread safe.

TODO: cleanup the debug code once the flaky test passed 100 times.

Author: Davies Liu <davies@databricks.com>

Closes #7940 from davies/semijoin and squashes the following commits:

93baac7 [Davies Liu] fix outerjoin
5c40ded [Davies Liu] address comments
aa3de46 [Davies Liu] Merge branch 'master' of github.com:apache/spark into semijoin
7590a25 [Davies Liu] Merge branch 'master' of github.com:apache/spark into semijoin
2d4085b [Davies Liu] use def for resultProjection
0833407 [Davies Liu] Merge branch 'semijoin' of github.com:davies/spark into semijoin
e0d8c71 [Davies Liu] use lazy val
6a59e8f [Davies Liu] Update HashedRelation.scala
0fdacaf [Davies Liu] fix broadcast and thread-safety of UnsafeProjection
2fc3ef6 [Davies Liu] reproduce failure in semijoin
2015-08-06 09:12:41 -07:00
Davies Liu 5b965d64ee [SPARK-9644] [SQL] Support update DecimalType with precision > 18 in UnsafeRow
In order to support update a varlength (actually fixed length) object, the space should be preserved even  it's null. And, we can't call setNullAt(i) for it anymore, we because setNullAt(i) will remove the offset of the preserved space, should call setDecimal(i, null, precision) instead.

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

cc rxin

Author: Davies Liu <davies@databricks.com>

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

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

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

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

9418088 [zhichao.li] refine checking condition
f2ab77a [zhichao.li] clone string
9d88f2d [zhichao.li] fix indent
6aa2962 [zhichao.li] style
e575ead [zhichao.li] add python api
9d4bab0 [zhichao.li] add special case for fodable and refactor unittest
eda7ad6 [zhichao.li] update to use TernaryExpression
cdfd4be [zhichao.li] add function translate
2015-08-06 09:02:30 -07:00
Yin Huai d5a9af3230 [SPARK-9664] [SQL] Remove UDAFRegistration and add apply to UserDefinedAggregateFunction.
https://issues.apache.org/jira/browse/SPARK-9664

Author: Yin Huai <yhuai@databricks.com>

Closes #7982 from yhuai/udafRegister and squashes the following commits:

0cc2287 [Yin Huai] Remove UDAFRegistration and add apply to UserDefinedAggregateFunction.
2015-08-05 21:50:35 -07:00
Reynold Xin 9270bd06fd [SPARK-9674][SQL] Remove GeneratedAggregate.
The new aggregate replaces the old GeneratedAggregate.

Author: Reynold Xin <rxin@databricks.com>

Closes #7983 from rxin/remove-generated-agg and squashes the following commits:

8334aae [Reynold Xin] [SPARK-9674][SQL] Remove GeneratedAggregate.
2015-08-05 21:50:14 -07:00
Cheng Hao 119b590538 [SPARK-6923] [SPARK-7550] [SQL] Persists data source relations in Hive compatible format when possible
This PR is a fork of PR #5733 authored by chenghao-intel.  For committers who's going to merge this PR, please set the author to "Cheng Hao <hao.chengintel.com>".

----

When a data source relation meets the following requirements, we persist it in Hive compatible format, so that other systems like Hive can access it:

1. It's a `HadoopFsRelation`
2. It has only one input path
3. It's non-partitioned
4. It's data source provider can be naturally mapped to a Hive builtin SerDe (e.g. ORC and Parquet)

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

Closes #7967 from liancheng/spark-6923/refactoring-pr-5733 and squashes the following commits:

5175ee6 [Cheng Lian] Fixes an oudated comment
3870166 [Cheng Lian] Fixes build error and comments
864acee [Cheng Lian] Refactors PR #5733
3490cdc [Cheng Hao] update the scaladoc
6f57669 [Cheng Hao] write schema info to hivemetastore for data source
2015-08-06 11:13:44 +08:00
Yin Huai 4581badbc8 [SPARK-9611] [SQL] Fixes a few corner cases when we spill a UnsafeFixedWidthAggregationMap
This PR has the following three small fixes.

1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an UnsafeInMemorySorter if its BytesToBytesMap is empty.
2. We will not not spill a InMemorySorter if it is empty.
3. We will not add a SpillReader to a SpillMerger if this SpillReader is empty.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #7948 from yhuai/unsafeEmptyMap and squashes the following commits:

9727abe [Yin Huai] Address Josh's comments.
34b6f76 [Yin Huai] 1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an UnsafeInMemorySorter if its BytesToBytesMap is empty. 2. Do not spill a InMemorySorter if it is empty. 3. Do not add spill to SpillMerger if this spill is empty.
2015-08-05 19:19:09 -07:00
Josh Rosen 9c878923db [SPARK-9054] [SQL] Rename RowOrdering to InterpretedOrdering; use newOrdering in SMJ
This patches renames `RowOrdering` to `InterpretedOrdering` and updates SortMergeJoin to use the `SparkPlan` methods for constructing its ordering so that it may benefit from codegen.

This is an updated version of #7408.

Author: Josh Rosen <joshrosen@databricks.com>

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

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

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

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

81ff97b [Liang-Chi Hsieh] For comments.
47761c6 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
cf4bf41 [Liang-Chi Hsieh] For comments.
f532b3c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
446bbcd [Liang-Chi Hsieh] Fix bug.
b3d0ab4 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into codegen_in
4610eff [Liang-Chi Hsieh] Relax the types of references and update optimizer test.
224f18e [Liang-Chi Hsieh] Beef up the test cases for In and InSet to include all primitive data types.
86dc8aa [Liang-Chi Hsieh] Only convert In to InSet when the number of items in set is more than the threshold.
b7ded7e [Tarek Auel] [SPARK-9403][SQL] codeGen in / inSet
2015-08-05 11:38:56 -07:00
Michael Armbrust 23d982204b [SPARK-9141] [SQL] Remove project collapsing from DataFrame API
Currently we collapse successive projections that are added by `withColumn`.  However, this optimization violates the constraint that adding nodes to a plan will never change its analyzed form and thus breaks caching.  Instead of doing early optimization, in this PR I just fix some low-hanging slowness in the analyzer.  In particular, I add a mechanism for skipping already analyzed subplans, `resolveOperators` and `resolveExpression`.  Since trees are generally immutable after construction, it's safe to annotate a plan as already analyzed as any transformation will create a new tree with this bit no longer set.

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

2145031 [Michael Armbrust] fix hive udfs tests
5a5a525 [Michael Armbrust] remove wrong comment
7a507d5 [Michael Armbrust] style
b59d710 [Michael Armbrust] revert small change
1fa5949 [Michael Armbrust] move logic into LogicalPlan, add tests
0e2cb43 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into withColumnCache
c926e24 [Michael Armbrust] naming
e593a2d [Michael Armbrust] style
f5a929e [Michael Armbrust] [SPARK-9141][SQL] Remove project collapsing from DataFrame API
38b1c83 [Michael Armbrust] WIP
2015-08-05 09:01:45 -07:00
Cheng Hao 519cf6d3f7 [SPARK-9381] [SQL] Migrate JSON data source to the new partitioning data source
Support partitioning for the JSON data source.

Still 2 open issues for the `HadoopFsRelation`
- `refresh()` will invoke the `discoveryPartition()`, which will auto infer the data type for the partition columns, and maybe conflict with the given partition columns. (TODO enable `HadoopFsRelationSuite.Partition column type casting"
- When insert data into a cached HadoopFsRelation based table, we need to invalidate the cache after the insertion (TODO enable `InsertSuite.Caching`)

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

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

d90b104 [Cheng Hao] revert the change for JacksonGenerator.apply
307111d [Cheng Hao] fix bug in the unit test
8738c8a [Cheng Hao] fix bug in unit testing
35f2cde [Cheng Hao] support partition for json format
2015-08-05 22:35:55 +08:00
Nathan Howell eb8bfa3eaa [SPARK-9618] [SQL] Use the specified schema when reading Parquet files
The user specified schema is currently ignored when loading Parquet files.

One workaround is to use the `format` and `load` methods instead of `parquet`, e.g.:

```
val schema = ???

// schema is ignored
sqlContext.read.schema(schema).parquet("hdfs:///test")

// schema is retained
sqlContext.read.schema(schema).format("parquet").load("hdfs:///test")
```

The fix is simple, but I wonder if the `parquet` method should instead be written in a similar fashion to `orc`:

```
def parquet(path: String): DataFrame = format("parquet").load(path)
```

Author: Nathan Howell <nhowell@godaddy.com>

Closes #7947 from NathanHowell/SPARK-9618 and squashes the following commits:

d1ea62c [Nathan Howell] [SPARK-9618] [SQL] Use the specified schema when reading Parquet files
2015-08-05 22:16:56 +08:00
zsxwing 1b0317f64c [SPARK-8861][SPARK-8862][SQL] Add basic instrumentation to each SparkPlan operator and add a new SQL tab
This PR includes the following changes:

### SPARK-8862: Add basic instrumentation to each SparkPlan operator

A SparkPlan can override `def accumulators: Map[String, Accumulator[_]]` to expose its metrics that can be displayed in UI. The UI will use them to track the updates and show them in the web page in real-time.

### SparkSQLExecution and SQLSparkListener

`SparkSQLExecution.withNewExecutionId` will set `spark.sql.execution.id` to the local properties so that we can use it to track all jobs that belong to the same query.

SQLSparkListener is a listener to track all accumulator updates of all tasks for a query. It receives them from heartbeats can the UI can query them in real-time.

When running a query, `SQLSparkListener.onExecutionStart` will be called. When a query is finished,  `SQLSparkListener.onExecutionEnd` will be called. And the Spark jobs with the same execution id will be tracked and stored with this query.

`SQLSparkListener` has to store all accumulator updates for tasks separately. When a task fails and starts to retry, we need to drop the old accumulator updates. Because we can not revert our changes to an accumulator, we have to maintain these accumulator updates by ourselves so as to drop accumulator updates for a failed task.

### SPARK-8862: A new SQL tab
Includes two pages:
#### A page for all DataFrame/SQL queries
It will show the running, completed and failed queries in 3 tables. It also displays the jobs and their links for a query in each row.
#### A detail page for a DataFrame/SQL query
In this page, it also shows the SparkPlan metrics in real-time. Run a long-running query, such as
```
val testData = sc.parallelize((1 to 1000000).map(i => (i, i.toString))).toDF()
testData.select($"_1").filter($"_1" < 1000).foreach(_ => Thread.sleep(60))
```
and you will see the metrics keep updating in real-time.

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

Author: zsxwing <zsxwing@gmail.com>

Closes #7774 from zsxwing/sql-ui and squashes the following commits:

5a2bc99 [zsxwing] Remove UISeleniumSuite and its dependency
57d4cd2 [zsxwing] Use VisibleForTesting annotation
cc1c736 [zsxwing] Add SparkPlan.trackNumOfRowsEnabled to make subclasses easy to track the number of rows; fix the issue that the "save" action cannot collect metrics
3771ab0 [zsxwing] Register SQL metrics accmulators
3a101c0 [zsxwing] Change prepareCalled's type to AtomicBoolean for thread-safety
b8d5605 [zsxwing] Make prepare idempotent; call children's prepare in SparkPlan.prepare; change doPrepare to def
4ed11a1 [zsxwing] var -> val
332639c [zsxwing] Ignore UISeleniumSuite and SQLListenerSuite."no memory leak" because of SPARK-9580
bb52359 [zsxwing] Address other commens in SQLListener
c4d0f5d [zsxwing] Move newPredicate out of the iterator loop
957473c [zsxwing] Move STATIC_RESOURCE_DIR to object SQLTab
7ab4816 [zsxwing] Make SparkPlan accumulator API private[sql]
dae195e [zsxwing] Fix the code style and comments
3a66207 [zsxwing] Ignore irrelevant accumulators
b8484a1 [zsxwing] Merge branch 'master' into sql-ui
9406592 [zsxwing] Implement the SparkPlan viz
4ebce68 [zsxwing] Add SparkPlan.prepare to support BroadcastHashJoin to run background work in parallel
ca1811f [zsxwing] Merge branch 'master' into sql-ui
fef6fc6 [zsxwing] Fix a corner case
25f335c [zsxwing] Fix the code style
6eae828 [zsxwing] SQLSparkListener -> SQLListener; SparkSQLExecutionUIData -> SQLExecutionUIData; SparkSQLExecution -> SQLExecution
822af75 [zsxwing] Add SQLSparkListenerSuite and fix the issue about onExecutionEnd and onJobEnd
6be626f [zsxwing] Add UISeleniumSuite to test UI
d02a24d [zsxwing] Make ExecutionPage private
23abf73 [zsxwing] [SPARK-8862][SPARK-8862][SQL] Add basic instrumentation to each SparkPlan operator and add a new SQL tab
2015-08-05 01:51:22 -07:00
Takeshi YAMAMURO 6d8a6e4161 [SPARK-9360] [SQL] Support BinaryType in PrefixComparators for UnsafeExternalSort
The current implementation of UnsafeExternalSort uses NoOpPrefixComparator for binary-typed data.
So, we need to add BinaryPrefixComparator in PrefixComparators.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #7676 from maropu/BinaryTypePrefixComparator and squashes the following commits:

fe6f31b [Takeshi YAMAMURO] Apply comments
d943c04 [Takeshi YAMAMURO] Add a codegen'd entry for BinaryType in SortPrefix
ecf3ac5 [Takeshi YAMAMURO] Support BinaryType in PrefixComparator
2015-08-05 00:56:35 -07:00
Emiliano Leporati 1d1a76c8c5 [SPARK-9581][SQL] Add unit test for JSON UDT
This brings #7416 up-to-date by drubbo.

Author: Emiliano Leporati <emiliano.leporati@gmail.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #7917 from rxin/udt-json-test and squashes the following commits:

93e3954 [Reynold Xin] Fix test.
7035308 [Reynold Xin] Merge pull request #7416 from drubbo/master
b5bcd94 [Emiliano Leporati] removed unneded case in MyDenseVector::equals
508a399 [Emiliano Leporati] Merge remote branch 'upstream/master'
7569e42 [Emiliano Leporati] using checkAnswer
62daccd [Emiliano Leporati] added coverage for UDTs in JSON RDDs
2015-08-05 00:42:08 -07:00
Davies Liu 781c8d71a0 [SPARK-9119] [SPARK-8359] [SQL] match Decimal.precision/scale with DecimalType
Let Decimal carry the correct precision and scale with DecimalType.

cc rxin yhuai

Author: Davies Liu <davies@databricks.com>

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

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

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

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

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

Closes #7580

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

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

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

There is issue with serialize byte_array in Python 3, so some of functions (for BinaryType) does not have tests.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7922 from davies/python_functions and squashes the following commits:

8ad942f [Davies Liu] fix test
5fb6ec3 [Davies Liu] fix bugs
3495ed3 [Davies Liu] fix issues
ea5f7bb [Davies Liu] Add python API for DataFrame functions
2015-08-04 19:25:24 -07:00
Wenchen Fan 7c8fc1f7cb [SPARK-9598][SQL] do not expose generic getter in internal row
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7932 from cloud-fan/generic-getter and squashes the following commits:

c60de4c [Wenchen Fan] do not expose generic getter in internal row
2015-08-04 17:05:19 -07:00
Josh Rosen ab8ee1a3b9 [SPARK-9452] [SQL] Support records larger than page size in UnsafeExternalSorter
This patch extends UnsafeExternalSorter to support records larger than the page size. The basic strategy is the same as in #7762: store large records in their own overflow pages.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7891 from JoshRosen/large-records-in-sql-sorter and squashes the following commits:

967580b [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
948c344 [Josh Rosen] Add large records tests for KV sorter.
3c17288 [Josh Rosen] Combine memory and disk cleanup into general cleanupResources() method
380f217 [Josh Rosen] Merge remote-tracking branch 'origin/master' into large-records-in-sql-sorter
27eafa0 [Josh Rosen] Fix page size in PackedRecordPointerSuite
a49baef [Josh Rosen] Address initial round of review comments
3edb931 [Josh Rosen] Remove accidentally-committed debug statements.
2b164e2 [Josh Rosen] Support large records in UnsafeExternalSorter.
2015-08-04 14:42:11 -07:00
Reynold Xin 5eb89f67e3 [SPARK-9577][SQL] Surface concrete iterator types in various sort classes.
We often return abstract iterator types in various sort-related classes (e.g. UnsafeKVExternalSorter). It is actually better to return a more concrete type, so the callsite uses that type and JIT can inline the iterator calls.

Author: Reynold Xin <rxin@databricks.com>

Closes #7911 from rxin/surface-concrete-type and squashes the following commits:

0422add [Reynold Xin] [SPARK-9577][SQL] Surface concrete iterator types in various sort classes.
2015-08-03 18:47:02 -07:00
Reynold Xin b2e4b85d2d Revert "[SPARK-9372] [SQL] Filter nulls in join keys"
This reverts commit 687c8c3715.
2015-08-03 14:51:15 -07:00
Andrew Or 702aa9d7fb [SPARK-8735] [SQL] Expose memory usage for shuffles, joins and aggregations
This patch exposes the memory used by internal data structures on the SparkUI. This tracks memory used by all spilling operations and SQL operators backed by Tungsten, e.g. `BroadcastHashJoin`, `ExternalSort`, `GeneratedAggregate` etc. The metric exposed is "peak execution memory", which broadly refers to the peak in-memory sizes of each of these data structure.

A separate patch will extend this by linking the new information to the SQL operators themselves.

<img width="950" alt="screen shot 2015-07-29 at 7 43 17 pm" src="https://cloud.githubusercontent.com/assets/2133137/8974776/b90fc980-362a-11e5-9e2b-842da75b1641.png">
<img width="802" alt="screen shot 2015-07-29 at 7 43 05 pm" src="https://cloud.githubusercontent.com/assets/2133137/8974777/baa76492-362a-11e5-9b77-e364a6a6b64e.png">

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

Author: Andrew Or <andrew@databricks.com>

Closes #7770 from andrewor14/expose-memory-metrics and squashes the following commits:

9abecb9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
f5b0d68 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
d7df332 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
8eefbc5 [Andrew Or] Fix non-failing tests
9de2a12 [Andrew Or] Fix tests due to another logical merge conflict
876bfa4 [Andrew Or] Fix failing test after logical merge conflict
361a359 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
40b4802 [Andrew Or] Fix style?
d0fef87 [Andrew Or] Fix tests?
b3b92f6 [Andrew Or] Address comments
0625d73 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
c00a197 [Andrew Or] Fix potential NPEs
10da1cd [Andrew Or] Fix compile
17f4c2d [Andrew Or] Fix compile?
a87b4d0 [Andrew Or] Fix compile?
d70874d [Andrew Or] Fix test compile + address comments
2840b7d [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
6aa2f7a [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
b889a68 [Andrew Or] Minor changes: comments, spacing, style
663a303 [Andrew Or] UnsafeShuffleWriter: update peak memory before close
d090a94 [Andrew Or] Fix style
2480d84 [Andrew Or] Expand test coverage
5f1235b [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
1ecf678 [Andrew Or] Minor changes: comments, style, unused imports
0b6926c [Andrew Or] Oops
111a05e [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
a7a39a5 [Andrew Or] Strengthen presence check for accumulator
a919eb7 [Andrew Or] Add tests for unsafe shuffle writer
23c845d [Andrew Or] Add tests for SQL operators
a757550 [Andrew Or] Address comments
b5c51c1 [Andrew Or] Re-enable test in JavaAPISuite
5107691 [Andrew Or] Add tests for internal accumulators
59231e4 [Andrew Or] Fix tests
9528d09 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
5b5e6f3 [Andrew Or] Add peak execution memory to summary table + tooltip
92b4b6b [Andrew Or] Display peak execution memory on the UI
eee5437 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
d9b9015 [Andrew Or] Track execution memory in unsafe shuffles
770ee54 [Andrew Or] Track execution memory in broadcast joins
9c605a4 [Andrew Or] Track execution memory in GeneratedAggregate
9e824f2 [Andrew Or] Add back execution memory tracking for *ExternalSort
4ef4cb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into expose-memory-metrics
e6c3e2f [Andrew Or] Move internal accumulators creation to Stage
a417592 [Andrew Or] Expose memory metrics in UnsafeExternalSorter
3c4f042 [Andrew Or] Track memory usage in ExternalAppendOnlyMap / ExternalSorter
bd7ab3f [Andrew Or] Add internal accumulators to TaskContext
2015-08-03 14:22:07 -07:00
Cheng Lian 703e44bff1 [SPARK-9554] [SQL] Enables in-memory partition pruning by default
Author: Cheng Lian <lian@databricks.com>

Closes #7895 from liancheng/spark-9554/enable-in-memory-partition-pruning and squashes the following commits:

67c403e [Cheng Lian] Enables in-memory partition pruning by default
2015-08-03 12:06:58 -07:00
Yin Huai 1ebd41b141 [SPARK-9240] [SQL] Hybrid aggregate operator using unsafe row
This PR adds a base aggregation iterator `AggregationIterator`, which is used to create `SortBasedAggregationIterator` (for sort-based aggregation) and `UnsafeHybridAggregationIterator` (first it tries hash-based aggregation and falls back to the sort-based aggregation (using external sorter) if we cannot allocate memory for the map). With these two iterators, we will not need existing iterators and I am removing those. Also, we can use a single physical `Aggregate` operator and it internally determines what iterators to used.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #7813 from yhuai/AggregateOperator and squashes the following commits:

e317e2b [Yin Huai] Remove unnecessary change.
74d93c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into AggregateOperator
ba6afbc [Yin Huai] Add a little bit more comments.
c9cf3b6 [Yin Huai] update
0f1b06f [Yin Huai] Remove unnecessary code.
21fd15f [Yin Huai] Remove unnecessary change.
964f88b [Yin Huai] Implement fallback strategy.
b1ea5cf [Yin Huai] wip
7fcbd87 [Yin Huai] Add a flag to control what iterator to use.
533d5b2 [Yin Huai] Prepare for fallback!
33b7022 [Yin Huai] wip
bd9282b [Yin Huai] UDAFs now supports UnsafeRow.
f52ee53 [Yin Huai] wip
3171f44 [Yin Huai] wip
d2c45a0 [Yin Huai] wip
f60cc83 [Yin Huai] Also check input schema.
af32210 [Yin Huai] Check iter.hasNext before we create an iterator because the constructor of the iterato will read at least one row from a non-empty input iter.
299008c [Yin Huai] First round cleanup.
3915bac [Yin Huai] Create a base iterator class for aggregation iterators and add the initial version of the hybrid iterator.
2015-08-03 00:23:08 -07:00
Yin Huai 687c8c3715 [SPARK-9372] [SQL] Filter nulls in join keys
This PR adds an optimization rule, `FilterNullsInJoinKey`, to add `Filter` before join operators to filter out rows having null values for join keys.

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

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

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

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

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

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

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

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

f9e322d [Reynold Xin] Fixed tests.
0439b43 [Reynold Xin] [SPARK-9546][SQL] Centralize orderable data type checking.
2015-08-02 20:12:03 -07:00
Reynold Xin 2e981b7bfa [SPARK-9531] [SQL] UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter
This pull request adds a destructAndCreateExternalSorter method to UnsafeFixedWidthAggregationMap. The new method does the following:

1. Creates a new external sorter UnsafeKVExternalSorter
2. Adds all the data into an in-memory sorter, sorts them
3. Spills the sorted in-memory data to disk

This method can be used to fallback to sort-based aggregation when under memory pressure.

The pull request also includes accounting fixes from JoshRosen.

TODOs (that can be done in follow-up PRs)
- [x] Address Josh's feedbacks from #7849
- [x] More documentation and test cases
- [x] Make sure we are doing memory accounting correctly with test cases (e.g. did we release the memory in BytesToBytesMap twice?)
- [ ] Look harder at possible memory leaks and exception handling
- [ ] Randomized tester for the KV sorter as well as the aggregation map

Author: Reynold Xin <rxin@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #7860 from rxin/kvsorter and squashes the following commits:

986a58c [Reynold Xin] Bug fix.
599317c [Reynold Xin] Style fix and slightly more compact code.
fe7bd4e [Reynold Xin] Bug fixes.
fd71bef [Reynold Xin] Merge remote-tracking branch 'josh/large-records-in-sql-sorter' into kvsorter-with-josh-fix
3efae38 [Reynold Xin] More fixes and documentation.
45f1b09 [Josh Rosen] Ensure that spill files are cleaned up
f6a9bd3 [Reynold Xin] Josh feedback.
9be8139 [Reynold Xin] Remove testSpillFrequency.
7cbe759 [Reynold Xin] [SPARK-9531][SQL] UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter.
ae4a8af [Josh Rosen] Detect leaked unsafe memory in UnsafeExternalSorterSuite.
52f9b06 [Josh Rosen] Detect ShuffleMemoryManager leaks in UnsafeExternalSorter.
2015-08-02 12:32:14 -07:00
Reynold Xin 8eafa2aeb6 [SPARK-9208][SQL] Sort DataFrame functions alphabetically.
Author: Reynold Xin <rxin@databricks.com>

Closes #7861 from rxin/api-audit and squashes the following commits:

7200256 [Reynold Xin] [SPARK-9208][SQL] Sort DataFrame functions alphabetically.
2015-08-02 11:36:11 -07:00
Davies Liu 16b928c543 [SPARK-9529] [SQL] improve TungstenSort on DecimalType
Generate prefix for DecimalType, fix the random generator of decimal

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

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

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

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

Author: Davies Liu <davies@databricks.com>

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

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

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

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

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

Closes #7208

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

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

54472e9 [Davies Liu] fix python test
17ffe51 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
ca46390 [Davies Liu] Merge branch 'master' of github.com:apache/spark into initcap
3a906e4 [Davies Liu] implement title case in UTF8String
8b2506a [HuJiayin] Update functions.py
2cd43e5 [HuJiayin] fix python style check
b616c0e [HuJiayin] add python api
1f5a0ef [HuJiayin] add codegen
7e0c604 [HuJiayin] Merge branch 'master' of https://github.com/apache/spark into initcap
6a0b958 [HuJiayin] add column
c79482d [HuJiayin] support soundex
7ce416b [HuJiayin] support initcap rebase code
2015-08-01 21:44:57 -07:00
Davies Liu 5d9e33d9a2 [SPARK-9495] prefix of DateType/TimestampType
cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #7856 from davies/sort_improve and squashes the following commits:

5fc81bd [Davies Liu] support DateType/TimestampType
2015-08-01 18:22:46 -07:00
Reynold Xin 3d1535d488 [SPARK-9520] [SQL] Support in-place sort in UnsafeFixedWidthAggregationMap
This pull request adds a sortedIterator method to UnsafeFixedWidthAggregationMap that sorts its data in-place by the grouping key.

This is needed so we can fallback to external sorting for aggregation.

Author: Reynold Xin <rxin@databricks.com>

Closes #7849 from rxin/bytes2bytes-sorting and squashes the following commits:

75018c6 [Reynold Xin] Updated documentation.
81a8694 [Reynold Xin] [SPARK-9520][SQL] Support in-place sort in UnsafeFixedWidthAggregationMap.
2015-08-01 13:20:26 -07:00
zhichao.li c5166f7a69 [SPARK-8263] [SQL] substr/substring should also support binary type
This is based on #7641, thanks to zhichao-li

Closes #7641

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

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

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

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

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

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

77d482f [Wenchen Fan] fix python
e8f6682 [Wenchen Fan] skip MapData equality check in HiveInspectorSuite
40cc9db [Wenchen Fan] add toString
6e06ec9 [Wenchen Fan] some more cleanup
a90aca1 [Wenchen Fan] add MapData
2015-08-01 00:17:15 -07:00
Reynold Xin d90f2cf7a2 [SPARK-9517][SQL] BytesToBytesMap should encode data the same way as UnsafeExternalSorter
BytesToBytesMap current encodes key/value data in the following format:
```
8B key length, key data, 8B value length, value data
```

UnsafeExternalSorter, on the other hand, encodes data this way:
```
4B record length, data
```

As a result, we cannot pass records encoded by BytesToBytesMap directly into UnsafeExternalSorter for sorting. However, if we rearrange data slightly, we can then pass the key/value records directly into UnsafeExternalSorter:
```
4B key+value length, 4B key length, key data, value data
```

Author: Reynold Xin <rxin@databricks.com>

Closes #7845 from rxin/kvsort-rebase and squashes the following commits:

5716b59 [Reynold Xin] Fixed test.
2e62ccb [Reynold Xin] Updated BytesToBytesMap's data encoding to put the key first.
a51b641 [Reynold Xin] Added a KV sorter interface.
2015-07-31 23:55:16 -07:00
Cheng Hao 67ad4e21fc [SPARK-8232] [SQL] Add sort_array support
Add expression `sort_array` support.

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

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

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

664c960 [Cheng Hao] update the sort_array by using the ArrayData
276d2d5 [Cheng Hao] add empty line
0edab9c [Cheng Hao] Add asending/descending support for sort_array
80fc0f8 [Cheng Hao] Add type checking
a42b678 [Cheng Hao] Add sort_array support
2015-07-31 23:11:22 -07:00
zhichao.li 6996bd2e81 [SPARK-8264][SQL]add substring_index function
This PR is based on #7533 , thanks to zhichao-li

Closes #7533

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

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

In addition, this patch integrates BytesToBytesMap with the ShuffleMemoryManager:

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

Author: Josh Rosen <joshrosen@databricks.com>

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

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

It's based on #7115 , thanks to HuJiayin

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

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

fa75941 [Davies Liu] Merge branch 'master' of github.com:apache/spark into soundex
a4bd6d8 [Davies Liu] fix soundex
2538908 [HuJiayin] add codegen soundex
d15d329 [HuJiayin] add back ut
ded1a14 [HuJiayin] Merge branch 'master' of https://github.com/apache/spark
e2dec2c [HuJiayin] support soundex rebase code
2015-07-31 16:05:26 -07:00
Herman van Hovell 39ab199a3f [SPARK-8640] [SQL] Enable Processing of Multiple Window Frames in a Single Window Operator
This PR enables the processing of multiple window frames in a single window operator. This should improve the performance of processing multiple window expressions wich share partition by/order by clauses, because it will be more efficient with respect to memory use and group processing.

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

Closes #7515 from hvanhovell/SPARK-8640 and squashes the following commits:

f0e1c21 [Herman van Hovell] Changed Window Logical/Physical plans to use partition by/order by specs directly instead of using WindowSpec.
e1711c2 [Herman van Hovell] Enabled the processing of multiple window frames in a single Window operator.
2015-07-31 12:08:25 -07:00
Davies Liu 0024da9157 [SQL] address comments for to_date/trunc
This PR address the comments in #7805

cc rxin

Author: Davies Liu <davies@databricks.com>

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

f729d5f [Davies Liu] rollback
cb7f7832 [Davies Liu] genCode() is protected
31e52ef [Davies Liu] fix style
ed1edc7 [Davies Liu] address comments for #7805
2015-07-31 11:07:34 -07:00
Daoyuan Wang 83670fc9e6 [SPARK-8176] [SPARK-8197] [SQL] function to_date/ trunc
This PR is based on #6988 , thanks to adrian-wang .

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

Closes #6988

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

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

2c7beba [Davies Liu] Merge branch 'master' of github.com:apache/spark into to_date
310dd55 [Daoyuan Wang] remove dup test in rebase
980b092 [Daoyuan Wang] resolve rebase conflict
a476c5a [Daoyuan Wang] address comments from davies
d44ea5f [Daoyuan Wang] function to_date, trunc
2015-07-30 19:22:38 -07:00
Josh Rosen 3c66ff727d [SPARK-9489] Remove unnecessary compatibility and requirements checks from Exchange
While reviewing yhuai's patch for SPARK-2205 (#7773), I noticed that Exchange's `compatible` check may be incorrectly returning `false` in many cases.  As far as I know, this is not actually a problem because the `compatible`, `meetsRequirements`, and `needsAnySort` checks are serving only as short-circuit performance optimizations that are not necessary for correctness.

In order to reduce code complexity, I think that we should remove these checks and unconditionally rewrite the operator's children.  This should be safe because we rewrite the tree in a single bottom-up pass.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7807 from JoshRosen/SPARK-9489 and squashes the following commits:

9d76ce9 [Josh Rosen] [SPARK-9489] Remove compatibleWith, meetsRequirements, and needsAnySort checks from Exchange
2015-07-30 17:38:48 -07:00
Davies Liu 0b1a464b6e [SPARK-9425] [SQL] support DecimalType in UnsafeRow
This PR brings the support of DecimalType in UnsafeRow, for precision <= 18, it's settable, otherwise it's not settable.

Author: Davies Liu <davies@databricks.com>

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

478b1ba [Davies Liu] address comments
536314c [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_decimal
7c2e77a [Davies Liu] fix JoinedRow
76d6fa4 [Davies Liu] fix tests
99d3151 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_decimal
d49c6ae [Davies Liu] support DecimalType in UnsafeRow
2015-07-30 17:18:32 -07:00
Reynold Xin e7a0976e99 [SPARK-9458][SPARK-9469][SQL] Code generate prefix computation in sorting & moves unsafe conversion out of TungstenSort.
Author: Reynold Xin <rxin@databricks.com>

Closes #7803 from rxin/SPARK-9458 and squashes the following commits:

5b032dc [Reynold Xin] Fix string.
b670dbb [Reynold Xin] [SPARK-9458][SPARK-9469][SQL] Code generate prefix computation in sorting & moves unsafe conversion out of TungstenSort.
2015-07-30 17:17:27 -07:00
Xiangrui Meng df32669514 [SPARK-7157][SQL] add sampleBy to DataFrame
This was previously committed but then reverted due to test failures (see #6769).

Author: Xiangrui Meng <meng@databricks.com>

Closes #7755 from rxin/SPARK-7157 and squashes the following commits:

fbf9044 [Xiangrui Meng] fix python test
542bd37 [Xiangrui Meng] update test
604fe6d [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-7157
f051afd [Xiangrui Meng] use udf instead of building expression
f4e9425 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-7157
8fb990b [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-7157
103beb3 [Xiangrui Meng] add Java-friendly sampleBy
991f26f [Xiangrui Meng] fix seed
4a14834 [Xiangrui Meng] move sampleBy to stat
832f7cc [Xiangrui Meng] add sampleBy to DataFrame
2015-07-30 17:16:03 -07:00
Daoyuan Wang 1abf7dc16c [SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang

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

Closes #7589

cc rxin

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

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

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

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

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

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

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

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

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

2fe20c4 [Daoyuan Wang] util.Date
ea2ec16 [Daoyuan Wang] use util.Date for better performance
a2cf929 [Daoyuan Wang] doc return null instead of 0
f6f070a [Daoyuan Wang] address comments from davies
6a4cbb3 [Daoyuan Wang] temp
56ded53 [Daoyuan Wang] rebase and address comments
14a8b37 [Daoyuan Wang] function unix_timestamp, from_unixtime
2015-07-30 11:14:09 -07:00
Josh Rosen 520ec0ff9d [SPARK-8850] [SQL] Enable Unsafe mode by default
This pull request enables Unsafe mode by default in Spark SQL. In order to do this, we had to fix a number of small issues:

**List of fixed blockers**:

- [x] Make some default buffer sizes configurable so that HiveCompatibilitySuite can run properly (#7741).
- [x] Memory leak on grouped aggregation of empty input (fixed by #7560 to fix this)
- [x] Update planner to also check whether codegen is enabled before planning unsafe operators.
- [x] Investigate failing HiveThriftBinaryServerSuite test.  This turns out to be caused by a ClassCastException that occurs when Exchange tries to apply an interpreted RowOrdering to an UnsafeRow when range partitioning an RDD.  This could be fixed by #7408, but a shorter-term fix is to just skip the Unsafe exchange path when RangePartitioner is used.
- [x] Memory leak exceptions masking exceptions that actually caused tasks to fail (will be fixed by #7603).
- [x]  ~~https://issues.apache.org/jira/browse/SPARK-9162, to implement code generation for ScalaUDF.  This is necessary for `UDFSuite` to pass.  For now, I've just ignored this test in order to try to find other problems while we wait for a fix.~~ This is no longer necessary as of #7682.
- [x] Memory leaks from Limit after UnsafeExternalSort cause the memory leak detector to fail tests. This is a huge problem in the HiveCompatibilitySuite (fixed by f4ac642a4e5b2a7931c5e04e086bb10e263b1db6).
- [x] Tests in `AggregationQuerySuite` are failing due to NaN-handling issues in UnsafeRow, which were fixed in #7736.
- [x] `org.apache.spark.sql.ColumnExpressionSuite.rand` needs to be updated so that the planner check also matches `TungstenProject`.
- [x] After having lowered the buffer sizes to 4MB so that most of HiveCompatibilitySuite runs:
  - [x] Wrong answer in `join_1to1` (fixed by #7680)
  - [x] Wrong answer in `join_nulls` (fixed by #7680)
  - [x] Managed memory OOM / leak in `lateral_view`
  - [x] Seems to hang indefinitely in `partcols1`.  This might be a deadlock in script transformation or a bug in error-handling code? The hang was fixed by #7710.
  - [x] Error while freeing memory in `partcols1`: will be fixed by #7734.
- [x] After fixing the `partcols1` hang, it appears that a number of later tests have issues as well.
- [x] Fix thread-safety bug in codegen fallback expression evaluation (#7759).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7564 from JoshRosen/unsafe-by-default and squashes the following commits:

83c0c56 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-by-default
f4cc859 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-by-default
963f567 [Josh Rosen] Reduce buffer size for R tests
d6986de [Josh Rosen] Lower page size in PySpark tests
013b9da [Josh Rosen] Also match TungstenProject in checkNumProjects
5d0b2d3 [Josh Rosen] Add task completion callback to avoid leak in limit after sort
ea250da [Josh Rosen] Disable unsafe Exchange path when RangePartitioning is used
715517b [Josh Rosen] Enable Unsafe by default
2015-07-30 10:45:32 -07:00
Liang-Chi Hsieh 5363ed7156 [SPARK-9361] [SQL] Refactor new aggregation code to reduce the times of checking compatibility
JIRA: https://issues.apache.org/jira/browse/SPARK-9361

Currently, we call `aggregate.Utils.tryConvert` in many places to check it the logical.Aggregate can be run with new aggregation. But looks like `aggregate.Utils.tryConvert` will cost considerable time to run. We should only call `tryConvert` once and keep it value in `logical.Aggregate` and reuse it.

In `org.apache.spark.sql.execution.aggregate.Utils`, the codes involving with `tryConvert` should be moved to catalyst because it actually doesn't deal with execution details.

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

Closes #7677 from viirya/refactor_aggregate and squashes the following commits:

babea30 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into refactor_aggregate
9a589d7 [Liang-Chi Hsieh] Fix scala style.
0a91329 [Liang-Chi Hsieh] Refactor new aggregation code to reduce the times to call tryConvert.
2015-07-30 10:32:12 -07:00
Wenchen Fan c0cc0eaec6 [SPARK-9390][SQL] create a wrapper for array type
Author: Wenchen Fan <cloud0fan@outlook.com>

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

d0408a1 [Wenchen Fan] fix python
661e608 [Wenchen Fan] rebase
f39256c [Wenchen Fan] fix hive...
6dbfa6f [Wenchen Fan] fix hive again...
8cb8842 [Wenchen Fan] remove element type parameter from getArray
43e9816 [Wenchen Fan] fix mllib
e719afc [Wenchen Fan] fix hive
4346290 [Wenchen Fan] address comment
d4a38da [Wenchen Fan] remove sizeInBytes and add license
7e283e2 [Wenchen Fan] create a wrapper for array type
2015-07-30 10:04:30 -07:00
Liang-Chi Hsieh 6175d6cfe7 [SPARK-8838] [SQL] Add config to enable/disable merging part-files when merging parquet schema
JIRA: https://issues.apache.org/jira/browse/SPARK-8838

Currently all part-files are merged when merging parquet schema. However, in case there are many part-files and we can make sure that all the part-files have the same schema as their summary file. If so, we provide a configuration to disable merging part-files when merging parquet schema.

In short, we need to merge parquet schema because different summary files may contain different schema. But the part-files are confirmed to have the same schema with summary files.

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

Closes #7238 from viirya/option_partfile_merge and squashes the following commits:

71d5b5f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into option_partfile_merge
8816f44 [Liang-Chi Hsieh] For comments.
dbc8e6b [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into option_partfile_merge
afc2fa1 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into option_partfile_merge
d4ed7e6 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into option_partfile_merge
df43027 [Liang-Chi Hsieh] Get dataStatuses' partitions based on all paths.
4eb2f00 [Liang-Chi Hsieh] Use given parameter.
ea8f6e5 [Liang-Chi Hsieh] Correct the code comments.
a57be0e [Liang-Chi Hsieh] Merge part-files if there are no summary files.
47df981 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into option_partfile_merge
4caf293 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into option_partfile_merge
0e734e0 [Liang-Chi Hsieh] Use correct API.
3b6be5b [Liang-Chi Hsieh] Fix key not found.
4bdd7e0 [Liang-Chi Hsieh] Don't read footer files if we can skip them.
8bbebcb [Liang-Chi Hsieh] Figure out how to test the config.
bbd4ce7 [Liang-Chi Hsieh] Add config to enable/disable merging part-files when merging parquet schema.
2015-07-30 17:45:30 +08:00
Reynold Xin 5ba2d44068 Fix flaky HashedRelationSuite
SparkEnv might not have been set in local unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #7784 from rxin/HashedRelationSuite and squashes the following commits:

435d64b [Reynold Xin] Fix flaky HashedRelationSuite
2015-07-30 01:21:39 -07:00
Reynold Xin 4a8bb9d00d Revert "[SPARK-9458] Avoid object allocation in prefix generation."
This reverts commit 9514d874f0.
2015-07-30 01:04:24 -07:00
Joseph Batchik 1221849f91 [SPARK-8005][SQL] Input file name
Users can now get the file name of the partition being read in. A thread local variable is in `SQLNewHadoopRDD` and is set when the partition is computed. `SQLNewHadoopRDD` is moved to core so that the catalyst package can reach it.

This supports:

`df.select(inputFileName())`

and

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

Author: Joseph Batchik <josephbatchik@gmail.com>

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

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

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

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

e0c2b3d [Yijie Shen] update codegen in RegExpExtract and RegExpReplace
26614d2 [Yijie Shen] MathFunctionSuite
a402859 [Yijie Shen] complex_create, conditional and cast
6e4e608 [Yijie Shen] arithmetic and cast
52593c1 [Yijie Shen] null input test cases for StringExpressionSuite
2015-07-29 23:24:20 -07:00
Davies Liu e044705b44 [SPARK-9116] [SQL] [PYSPARK] support Python only UDT in __main__
Also we could create a Python UDT without having a Scala one, it's important for Python users.

cc mengxr JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #7453 from davies/class_in_main and squashes the following commits:

4dfd5e1 [Davies Liu] add tests for Python and Scala UDT
793d9b2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into class_in_main
dc65f19 [Davies Liu] address comment
a9a3c40 [Davies Liu] Merge branch 'master' of github.com:apache/spark into class_in_main
a86e1fc [Davies Liu] fix serialization
ad528ba [Davies Liu] Merge branch 'master' of github.com:apache/spark into class_in_main
63f52ef [Davies Liu] fix pylint check
655b8a9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into class_in_main
316a394 [Davies Liu] support Python UDT with UTF
0bcb3ef [Davies Liu] fix bug in mllib
de986d6 [Davies Liu] fix test
83d65ac [Davies Liu] fix bug in StructType
55bb86e [Davies Liu] support Python UDT in __main__ (without Scala one)
2015-07-29 22:30:49 -07:00
Reynold Xin 9514d874f0 [SPARK-9458] Avoid object allocation in prefix generation.
In our existing sort prefix generation code, we use expression's eval method to generate the prefix, which results in object allocation for every prefix. We can use the specialized getters available on InternalRow directly to avoid the object allocation.

I also removed the FLOAT prefix, opting for converting float directly to double.

Author: Reynold Xin <rxin@databricks.com>

Closes #7763 from rxin/sort-prefix and squashes the following commits:

5dc2f06 [Reynold Xin] [SPARK-9458] Avoid object allocation in prefix generation.
2015-07-29 20:46:03 -07:00
Josh Rosen 1b0099fc62 [SPARK-9411] [SQL] Make Tungsten page sizes configurable
We need to make page sizes configurable so we can reduce them in unit tests and increase them in real production workloads.  These sizes are now controlled by a new configuration, `spark.buffer.pageSize`.  The new default is 64 megabytes.

Author: Josh Rosen <joshrosen@databricks.com>

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

99f64e8 [Reynold Xin] One more line ...
13466c8 [Reynold Xin] Fixed tests.
e20f24e [Reynold Xin] [SPARK-9430][SQL] Rename IntervalType to CalendarIntervalType.
2015-07-29 13:49:22 -07:00
Yin Huai 3744b7fd42 [SPARK-9422] [SQL] Remove the placeholder attributes used in the aggregation buffers
https://issues.apache.org/jira/browse/SPARK-9422

Author: Yin Huai <yhuai@databricks.com>

Closes #7737 from yhuai/removePlaceHolder and squashes the following commits:

ec29b44 [Yin Huai]  Remove placeholder attributes.
2015-07-28 19:01:25 -07:00
Reynold Xin 6662ee2124 [SPARK-9418][SQL] Use sort-merge join as the default shuffle join.
Sort-merge join is more robust in Spark since sorting can be made using the Tungsten sort operator.

Author: Reynold Xin <rxin@databricks.com>

Closes #7733 from rxin/smj and squashes the following commits:

61e4d34 [Reynold Xin] Fixed test case.
5ffd731 [Reynold Xin] Fixed JoinSuite.
a137dc0 [Reynold Xin] [SPARK-9418][SQL] Use sort-merge join as the default shuffle join.
2015-07-28 17:42:35 -07:00
Reynold Xin b7f54119f8 [SPARK-9420][SQL] Move expressions in sql/core package to catalyst.
Since catalyst package already depends on Spark core, we can move those expressions
into catalyst, and simplify function registry.

This is a followup of #7478.

Author: Reynold Xin <rxin@databricks.com>

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

2ffbdc3 [Reynold Xin] [SPARK-8003][SQL] Move expressions in sql/core package to catalyst.
2015-07-28 17:03:59 -07:00
Davies Liu 21825529ea [SPARK-9247] [SQL] Use BytesToBytesMap for broadcast join
This PR introduce BytesToBytesMap to UnsafeHashedRelation, use it in executor for better performance.

It serialize all the key and values from java HashMap, put them into a BytesToBytesMap while deserializing. All the values for a same key are stored continuous to have better memory locality.

This PR also address the comments for #7480 , do some clean up.

Author: Davies Liu <davies@databricks.com>

Closes #7592 from davies/unsafe_map2 and squashes the following commits:

42c578a [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_map2
fd09528 [Davies Liu] remove thread local cache and update docs
1c5ad8d [Davies Liu] fix test
5eb1b5a [Davies Liu] address comments in #7480
46f1f22 [Davies Liu] fix style
fc221e0 [Davies Liu] use BytesToBytesMap for broadcast join
2015-07-28 15:56:19 -07:00
Joseph Batchik b88b868eb3 [SPARK-8003][SQL] Added virtual column support to Spark
Added virtual column support by adding a new resolution role to the query analyzer. Additional virtual columns can be added by adding case expressions to [the new rule](https://github.com/JDrit/spark/blob/virt_columns/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala#L1026) and my modifying the [logical plan](https://github.com/JDrit/spark/blob/virt_columns/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala#L216) to resolve them.

This also solves [SPARK-8003](https://issues.apache.org/jira/browse/SPARK-8003)

This allows you to perform queries such as:
```sql
select spark__partition__id, count(*) as c from table group by spark__partition__id;
```

Author: Joseph Batchik <josephbatchik@gmail.com>
Author: JD <jd@csh.rit.edu>

Closes #7478 from JDrit/virt_columns and squashes the following commits:

7932bf0 [Joseph Batchik] adding spark__partition__id to hive as well
f8a9c6c [Joseph Batchik] merging in master
e49da48 [JD] fixes for @rxin's suggestions
60e120b [JD] fixing test in merge
4bf8554 [JD] merging in master
c68bc0f [Joseph Batchik] Adding function register ability to SQLContext and adding a function for spark__partition__id()
2015-07-28 14:39:25 -07:00
Aaron Davidson 35ef853b3f [SPARK-9397] DataFrame should provide an API to find source data files if applicable
Certain applications would benefit from being able to inspect DataFrames that are straightforwardly produced by data sources that stem from files, and find out their source data. For example, one might want to display to a user the size of the data underlying a table, or to copy or mutate it.

This PR exposes an `inputFiles` method on DataFrame which attempts to discover the source data in a best-effort manner, by inspecting HadoopFsRelations and JSONRelations.

Author: Aaron Davidson <aaron@databricks.com>

Closes #7717 from aarondav/paths and squashes the following commits:

ff67430 [Aaron Davidson] inputFiles
0acd3ad [Aaron Davidson] [SPARK-9397] DataFrame should provide an API to find source data files if applicable
2015-07-28 10:12:09 -07:00
Reynold Xin 9bbe0171cb [SPARK-8196][SQL] Fix null handling & documentation for next_day.
The original patch didn't handle nulls correctly for next_day.

Author: Reynold Xin <rxin@databricks.com>

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

616a425 [Reynold Xin] Merged DatetimeExpressionsSuite into DateFunctionsSuite.
faa78cf [Reynold Xin] Merged DatetimeFunctionsSuite into DateExpressionsSuite.
6c4fb6a [Reynold Xin] [SPARK-8196][SQL] Fix null handling & documentation for next_day.
2015-07-28 09:43:39 -07:00
Reynold Xin c740bed172 [SPARK-9373][SQL] follow up for StructType support in Tungsten projection.
Author: Reynold Xin <rxin@databricks.com>

Closes #7720 from rxin/struct-followup and squashes the following commits:

d9757f5 [Reynold Xin] [SPARK-9373][SQL] follow up for StructType support in Tungsten projection.
2015-07-28 09:43:12 -07:00
Reynold Xin 60f08c7c87 [SPARK-9373][SQL] Support StructType in Tungsten projection
This pull request updates GenerateUnsafeProjection to support StructType. If an input struct type is backed already by an UnsafeRow, GenerateUnsafeProjection copies the bytes directly into its buffer space without any conversion. However, if the input is not an UnsafeRow, GenerateUnsafeProjection runs the code generated recursively to convert the input into an UnsafeRow and then copies it into the buffer space.

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

Author: Reynold Xin <rxin@databricks.com>

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

9162f42 [Reynold Xin] Support IntervalType in UnsafeRow's getter.
be9f377 [Reynold Xin] Fixed tests.
10c4b7c [Reynold Xin] Format generated code.
77e8d0e [Reynold Xin] Fixed NondeterministicSuite.
ac4951d [Reynold Xin] Yay.
ac203bf [Reynold Xin] More comments.
9f36216 [Reynold Xin] Updated comment.
6b781fe [Reynold Xin] Reset the change in DataFrameSuite.
525b95b [Reynold Xin] Merged with master, more documentation & test cases.
321859a [Reynold Xin] [SPARK-9373][SQL] Support StructType in Tungsten projection [WIP]
2015-07-27 22:51:15 -07:00
Yijie Shen 63a492b931 [SPARK-8828] [SQL] Revert SPARK-5680
JIRA: https://issues.apache.org/jira/browse/SPARK-8828

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

Closes #7667 from yjshen/revert_combinesum_2 and squashes the following commits:

c37ccb1 [Yijie Shen] add test case
8377214 [Yijie Shen] revert spark.sql.useAggregate2 to its default value
e2305ac [Yijie Shen] fix bug - avg on decimal column
7cb0e95 [Yijie Shen] [wip] resolving bugs
1fadb5a [Yijie Shen] remove occurance
17c6248 [Yijie Shen] revert SPARK-5680
2015-07-27 22:47:33 -07:00
Daoyuan Wang 2e7f99a004 [SPARK-8195] [SPARK-8196] [SQL] udf next_day last_day
next_day, returns next certain dayofweek.
last_day, returns the last day of the month which given date belongs to.

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

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

ef7e3da [Daoyuan Wang] fix
02b3426 [Daoyuan Wang] address 2 comments
dc69630 [Daoyuan Wang] address comments from rxin
8846086 [Daoyuan Wang] address comments from rxin
d09bcce [Daoyuan Wang] multi fix
1a9de3d [Daoyuan Wang] function next_day and last_day
2015-07-27 21:08:56 -07:00
Michael Armbrust ce89ff477a [SPARK-9386] [SQL] Feature flag for metastore partition pruning
Since we have been seeing a lot of failures related to this new feature, lets put it behind a flag and turn it off by default.

Author: Michael Armbrust <michael@databricks.com>

Closes #7703 from marmbrus/optionalMetastorePruning and squashes the following commits:

6ad128c [Michael Armbrust] style
8447835 [Michael Armbrust] [SPARK-9386][SQL] Feature flag for metastore partition pruning
fd37b87 [Michael Armbrust] add config flag
2015-07-27 17:32:34 -07:00
Wenchen Fan 3ab7525dce [SPARK-9355][SQL] Remove InternalRow.get generic getter call in columnar cache code
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7673 from cloud-fan/row-generic-getter-columnar and squashes the following commits:

88b1170 [Wenchen Fan] fix style
eeae712 [Wenchen Fan] Remove Internal.get generic getter call in columnar cache code
2015-07-27 13:40:50 -07:00
Yin Huai 55946e76fd [SPARK-9349] [SQL] UDAF cleanup
https://issues.apache.org/jira/browse/SPARK-9349

With this PR, we only expose `UserDefinedAggregateFunction` (an abstract class) and `MutableAggregationBuffer` (an interface). Other internal wrappers and helper classes are moved to `org.apache.spark.sql.execution.aggregate` and marked as `private[sql]`.

Author: Yin Huai <yhuai@databricks.com>

Closes #7687 from yhuai/UDAF-cleanup and squashes the following commits:

db36542 [Yin Huai] Add comments to UDAF examples.
ae17f66 [Yin Huai] Address comments.
9c9fa5f [Yin Huai] UDAF cleanup.
2015-07-27 13:26:57 -07:00
Rene Treffer aa19c696e2 [SPARK-4176] [SQL] Supports decimal types with precision > 18 in Parquet
This PR is based on #6796 authored by rtreffer.

To support large decimal precisions (> 18), we do the following things in this PR:

1. Making `CatalystSchemaConverter` support large decimal precision

   Decimal types with large precision are always converted to fixed-length byte array.

2. Making `CatalystRowConverter` support reading decimal values with large precision

   When the precision is > 18, constructs `Decimal` values with an unscaled `BigInteger` rather than an unscaled `Long`.

3. Making `RowWriteSupport` support writing decimal values with large precision

   In this PR we always write decimals as fixed-length byte array, because Parquet write path hasn't been refactored to conform Parquet format spec (see SPARK-6774 & SPARK-8848).

Two follow-up tasks should be done in future PRs:

- [ ] Writing decimals as `INT32`, `INT64` when possible while fixing SPARK-8848
- [ ] Adding compatibility tests as part of SPARK-5463

Author: Cheng Lian <lian@databricks.com>

Closes #7455 from liancheng/spark-4176 and squashes the following commits:

a543d10 [Cheng Lian] Fixes errors introduced while rebasing
9e31cdf [Cheng Lian] Supports decimals with precision > 18 for Parquet
2015-07-27 23:29:40 +08:00
Cheng Lian 72981bc8f0 [SPARK-7943] [SPARK-8105] [SPARK-8435] [SPARK-8714] [SPARK-8561] Fixes multi-database support
This PR fixes a set of issues related to multi-database. A new data structure `TableIdentifier` is introduced to identify a table among multiple databases. We should stop using a single `String` (table name without database name), or `Seq[String]` (optional database name plus table name) to identify tables internally.

Author: Cheng Lian <lian@databricks.com>

Closes #7623 from liancheng/spark-8131-multi-db and squashes the following commits:

f3bcd4b [Cheng Lian] Addresses PR comments
e0eb76a [Cheng Lian] Fixes styling issues
41e2207 [Cheng Lian] Fixes multi-database support
d4d1ec2 [Cheng Lian] Adds multi-database test cases
2015-07-27 17:15:35 +08:00
Liang-Chi Hsieh 945d8bcbf6 [SPARK-9306] [SQL] Don't use SortMergeJoin when joining on unsortable columns
JIRA: https://issues.apache.org/jira/browse/SPARK-9306

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

Closes #7645 from viirya/smj_unsortable and squashes the following commits:

a240707 [Liang-Chi Hsieh] Use forall instead of exists for readability.
55221fa [Liang-Chi Hsieh] Shouldn't use SortMergeJoin when joining on unsortable columns.
2015-07-26 22:13:37 -07:00
Cheng Hao 1efe97dc9e [SPARK-8867][SQL] Support list / describe function usage
As Hive does, we need to list all of the registered UDF and its usage for user.

We add the annotation to describe a UDF, so we can get the literal description info while registering the UDF.
e.g.
```scala
ExpressionDescription(
    usage = "_FUNC_(expr) - Returns the absolute value of the numeric value",
    extended = """> SELECT _FUNC_('-1')
                  1""")
 case class Abs(child: Expression) extends UnaryArithmetic {
...
```

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

Closes #7259 from chenghao-intel/desc_function and squashes the following commits:

cf29bba [Cheng Hao] fixing the code style issue
5193855 [Cheng Hao] Add more powerful parser for show functions
c645a6b [Cheng Hao] fix bug in unit test
78d40f1 [Cheng Hao] update the padding issue for usage
48ee4b3 [Cheng Hao] update as feedback
70eb4e9 [Cheng Hao] add show/describe function support
2015-07-26 18:34:19 -07:00
Cheng Lian c025c3d0a1 [SPARK-9095] [SQL] Removes the old Parquet support
This PR removes the old Parquet support:

- Removes the old `ParquetRelation` together with related SQL configuration, plan nodes, strategies, utility classes, and test suites.

- Renames `ParquetRelation2` to `ParquetRelation`

- Renames `RowReadSupport` and `RowRecordMaterializer` to `CatalystReadSupport` and `CatalystRecordMaterializer` respectively, and moved them to separate files.

  This follows naming convention used in other Parquet data models implemented in parquet-mr. It should be easier for developers who are familiar with Parquet to follow.

There's still some other code that can be cleaned up. Especially `RowWriteSupport`. But I'd like to leave this part to SPARK-8848.

Author: Cheng Lian <lian@databricks.com>

Closes #7441 from liancheng/spark-9095 and squashes the following commits:

c7b6e38 [Cheng Lian] Removes WriteToFile
2d688d6 [Cheng Lian] Renames ParquetRelation2 to ParquetRelation
ca9e1b7 [Cheng Lian] Removes old Parquet support
2015-07-26 16:49:19 -07:00
Yijie Shen fb5d43fb25 [SPARK-9356][SQL]Remove the internal use of DecimalType.Unlimited
JIRA: https://issues.apache.org/jira/browse/SPARK-9356

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

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

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

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

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

6c6f332 [Wenchen Fan] add test
ef68ff4 [Wenchen Fan] fix comments
9eac85e [Wenchen Fan] move init code to interpreted class
bb7d838 [Wenchen Fan] pulls out nondeterministic expressions into a project
b4a4fc7 [Wenchen Fan] revert a refactor
86fee36 [Wenchen Fan] add initialization phase for nondeterministic expression
2015-07-25 12:10:02 -07:00
Cheng Lian e2ec018e37 [SPARK-9285] [SQL] Fixes Row/InternalRow conversion for HadoopFsRelation
This is a follow-up of #7626. It fixes `Row`/`InternalRow` conversion for data sources extending `HadoopFsRelation` with `needConversion` being `true`.

Author: Cheng Lian <lian@databricks.com>

Closes #7649 from liancheng/spark-9285-conversion-fix and squashes the following commits:

036a50c [Cheng Lian] Addresses PR comment
f6d7c6a [Cheng Lian] Fixes Row/InternalRow conversion for HadoopFsRelation
2015-07-25 11:42:49 -07:00
Reynold Xin f0ebab3f6d [SPARK-9336][SQL] Remove extra JoinedRows
They were added to improve performance (so JIT can inline the JoinedRow calls). However, we can also just improve it by projecting output out to UnsafeRow in Tungsten variant of the operators.

Author: Reynold Xin <rxin@databricks.com>

Closes #7659 from rxin/remove-joinedrows and squashes the following commits:

7510447 [Reynold Xin] [SPARK-9336][SQL] Remove extra JoinedRows
2015-07-25 01:28:46 -07:00
JD 723db13e06 [Spark-8668][SQL] Adding expr to functions
Author: JD <jd@csh.rit.edu>
Author: Joseph Batchik <josephbatchik@gmail.com>

Closes #7606 from JDrit/expr and squashes the following commits:

ad7f607 [Joseph Batchik] fixing python linter error
9d6daea [Joseph Batchik] removed order by per @rxin's comment
707d5c6 [Joseph Batchik] Added expr to fuctions.py
79df83c [JD] added example to the docs
b89eec8 [JD] moved function up as per @rxin's comment
4960909 [JD] updated per @JoshRosen's comment
2cb329c [JD] updated per @rxin's comment
9a9ad0c [JD] removing unused import
6dc26d0 [JD] removed split
7f2222c [JD] Adding expr function as per SPARK-8668
2015-07-25 00:34:59 -07:00
Liang-Chi Hsieh 64135cbb33 [SPARK-9067] [SQL] Close reader in NewHadoopRDD early if there is no more data
JIRA: https://issues.apache.org/jira/browse/SPARK-9067

According to the description of the JIRA ticket, calling `reader.close()` only after the task is finished will cause memory and file open limit problem since these resources are occupied even we don't need that anymore.

This PR simply closes the reader early when we know there is no more data to read.

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

Closes #7424 from viirya/close_reader and squashes the following commits:

3ff64e5 [Liang-Chi Hsieh] For comments.
3d20267 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into close_reader
e152182 [Liang-Chi Hsieh] For comments.
5116cbe [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into close_reader
3ceb755 [Liang-Chi Hsieh] For comments.
e34d98e [Liang-Chi Hsieh] For comments.
50ed729 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into close_reader
216912f [Liang-Chi Hsieh] Fix it.
f429016 [Liang-Chi Hsieh] Release reader if we don't need it.
a305621 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into close_reader
67569da [Liang-Chi Hsieh] Close reader early if there is no more data.
2015-07-24 12:36:44 -07:00
Reynold Xin 431ca39be5 [SPARK-9285][SQL] Remove InternalRow's inheritance from Row.
I also changed InternalRow's size/length function to numFields, to make it more obvious that it is not about bytes, but the number of fields.

Author: Reynold Xin <rxin@databricks.com>

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

e124daf [Reynold Xin] Fixed test case.
805ceb7 [Reynold Xin] Commented out the failed test suite.
f8a9ca5 [Reynold Xin] Fixed more bugs. Still at least one more remaining.
76d9081 [Reynold Xin] Fixed data sources.
7807f70 [Reynold Xin] Fixed DataFrameSuite.
cb60cd2 [Reynold Xin] Code review & small bug fixes.
0a2948b [Reynold Xin] Fixed style.
3280d03 [Reynold Xin] [SPARK-9285][SQL] Remove InternalRow's inheritance from Row.
2015-07-24 09:37:36 -07:00
Liang-Chi Hsieh 6a7e537f3a [SPARK-8756] [SQL] Keep cached information and avoid re-calculating footers in ParquetRelation2
JIRA: https://issues.apache.org/jira/browse/SPARK-8756

Currently, in ParquetRelation2, footers are re-read every time refresh() is called. But we can check if it is possibly changed before we do the reading because reading all footers will be expensive when there are too many partitions. This pr fixes this by keeping some cached information to check it.

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

Closes #7154 from viirya/cached_footer_parquet_relation and squashes the following commits:

92e9347 [Liang-Chi Hsieh] Fix indentation.
ae0ec64 [Liang-Chi Hsieh] Fix wrong assignment.
c8fdfb7 [Liang-Chi Hsieh] Fix it.
a52b6d1 [Liang-Chi Hsieh] For comments.
c2a2420 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cached_footer_parquet_relation
fa5458f [Liang-Chi Hsieh] Use Map to cache FileStatus and do merging previously loaded schema and newly loaded one.
6ae0911 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cached_footer_parquet_relation
21bbdec [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cached_footer_parquet_relation
12a0ed9 [Liang-Chi Hsieh] Add check of FileStatus's modification time.
186429d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cached_footer_parquet_relation
0ef8caf [Liang-Chi Hsieh] Keep cached information and avoid re-calculating footers.
2015-07-24 17:39:57 +08:00
Wenchen Fan 408e64b284 [SPARK-9294][SQL] cleanup comments, code style, naming typo for the new aggregation
fix some comments and code style for https://github.com/apache/spark/pull/7458

Author: Wenchen Fan <cloud0fan@outlook.com>

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

3925457 [Wenchen Fan] one more...
cc78357 [Wenchen Fan] one more cleanup
26f6a93 [Wenchen Fan] some minor cleanup for the new aggregation
2015-07-23 23:40:01 -07:00
Davies Liu 8a94eb23d5 [SPARK-9069] [SPARK-9264] [SQL] remove unlimited precision support for DecimalType
Romove Decimal.Unlimited (change to support precision up to 38, to match with Hive and other databases).

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

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

Author: Davies Liu <davies@databricks.com>

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

aa3f115 [Davies Liu] fix tests and style
fb0d20d [Davies Liu] address comments
bfaae35 [Davies Liu] fix style
df93657 [Davies Liu] address comments and clean up
06727fd [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_unlimited
4c28969 [Davies Liu] fix tests
8d783cc [Davies Liu] fix tests
788631c [Davies Liu] fix double with decimal in Union/except
1779bde [Davies Liu] fix scala style
c9c7c78 [Davies Liu] remove Decimal.Unlimited
2015-07-23 18:31:13 -07:00
Cheng Lian bebe3f7b45 [SPARK-9207] [SQL] Enables Parquet filter push-down by default
PARQUET-136 and PARQUET-173 have been fixed in parquet-mr 1.7.0. It's time to enable filter push-down by default now.

Author: Cheng Lian <lian@databricks.com>

Closes #7612 from liancheng/spark-9207 and squashes the following commits:

77e6b5e [Cheng Lian] Enables Parquet filter push-down by default
2015-07-23 17:49:33 -07:00
David Arroyo Cazorla 662d60db3f [SPARK-5447][SQL] Replace reference 'schema rdd' with DataFrame @rxin.
Author: David Arroyo Cazorla <darroyo@stratio.com>

Closes #7618 from darroyocazorla/master and squashes the following commits:

5f91379 [David Arroyo Cazorla] [SPARK-5447][SQL] Replace reference 'schema rdd' with DataFrame
2015-07-23 10:34:32 -07:00
Xiangrui Meng ecfb312767 [SPARK-9243] [Documentation] null -> zero in crosstab doc
We forgot to update doc. brkyvz

Author: Xiangrui Meng <meng@databricks.com>

Closes #7608 from mengxr/SPARK-9243 and squashes the following commits:

0ea3236 [Xiangrui Meng] null -> zero in crosstab doc
2015-07-23 10:32:11 -07:00
Reynold Xin fb36397b3c Revert "[SPARK-8579] [SQL] support arbitrary object in UnsafeRow"
Reverts ObjectPool. As it stands, it has a few problems:

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

01db0bc [Reynold Xin] Scala style.
eda89fc [Reynold Xin] Fixed describe.
2967118 [Reynold Xin] Fixed accessor for JoinedRow.
e3294eb [Reynold Xin] Merge branch 'master' into revert-object-pool
657855f [Reynold Xin] Temp commit.
c20f2c8 [Reynold Xin] Style fix.
fe37079 [Reynold Xin] Revert "[SPARK-8579] [SQL] support arbitrary object in UnsafeRow"
2015-07-23 01:51:34 -07:00
Josh Rosen b217230f2a [SPARK-9144] Remove DAGScheduler.runLocallyWithinThread and spark.localExecution.enabled
Spark has an option called spark.localExecution.enabled; according to the docs:

> Enables Spark to run certain jobs, such as first() or take() on the driver, without sending tasks to the cluster. This can make certain jobs execute very quickly, but may require shipping a whole partition of data to the driver.

This feature ends up adding quite a bit of complexity to DAGScheduler, especially in the runLocallyWithinThread method, but as far as I know nobody uses this feature (I searched the mailing list and haven't seen any recent mentions of the configuration nor stacktraces including the runLocally method). As a step towards scheduler complexity reduction, I propose that we remove this feature and all code related to it for Spark 1.5.

This pull request simply brings #7484 up to date.

Author: Josh Rosen <joshrosen@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #7585 from rxin/remove-local-exec and squashes the following commits:

84bd10e [Reynold Xin] Python fix.
1d9739a [Reynold Xin] Merge pull request #7484 from JoshRosen/remove-localexecution
eec39fa [Josh Rosen] Remove allowLocal(); deprecate user-facing uses of it.
b0835dc [Josh Rosen] Remove local execution code in DAGScheduler
8975d96 [Josh Rosen] Remove local execution tests.
ffa8c9b [Josh Rosen] Remove documentation for configuration
2015-07-22 21:04:04 -07:00
Reynold Xin d71a13f475 [SPARK-9262][build] Treat Scala compiler warnings as errors
I've seen a few cases in the past few weeks that the compiler is throwing warnings that are caused by legitimate bugs. This patch upgrades warnings to errors, except deprecation warnings.

Note that ideally we should be able to mark deprecation warnings as errors as well. However, due to the lack of ability to suppress individual warning messages in the Scala compiler, we cannot do that (since we do need to access deprecated APIs in Hadoop).

Most of the work are done by ericl.

Author: Reynold Xin <rxin@databricks.com>
Author: Eric Liang <ekl@databricks.com>

Closes #7598 from rxin/warnings and squashes the following commits:

beb311b [Reynold Xin] Fixed tests.
542c031 [Reynold Xin] Fixed one more warning.
87c354a [Reynold Xin] Fixed all non-deprecation warnings.
78660ac [Eric Liang] first effort to fix warnings
2015-07-22 21:02:19 -07:00
Davies Liu e0b7ba59a1 [SPARK-9024] Unsafe HashJoin/HashOuterJoin/HashSemiJoin
This PR introduce unsafe version (using UnsafeRow) of HashJoin, HashOuterJoin and HashSemiJoin, including the broadcast one and shuffle one (except FullOuterJoin, which is better to be implemented using SortMergeJoin).

It use HashMap to store UnsafeRow right now, will change to use BytesToBytesMap for better performance (in another PR).

Author: Davies Liu <davies@databricks.com>

Closes #7480 from davies/unsafe_join and squashes the following commits:

6294b1e [Davies Liu] fix projection
10583f1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_join
dede020 [Davies Liu] fix test
84c9807 [Davies Liu] address comments
a05b4f6 [Davies Liu] support UnsafeRow in LeftSemiJoinBNL and BroadcastNestedLoopJoin
611d2ed [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_join
9481ae8 [Davies Liu] return UnsafeRow after join()
ca2b40f [Davies Liu] revert unrelated change
68f5cd9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_join
0f4380d [Davies Liu] ada a comment
69e38f5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_join
1a40f02 [Davies Liu] refactor
ab1690f [Davies Liu] address comments
60371f2 [Davies Liu] use UnsafeRow in SemiJoin
a6c0b7d [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_join
184b852 [Davies Liu] fix style
6acbb11 [Davies Liu] fix tests
95d0762 [Davies Liu] remove println
bea4a50 [Davies Liu] Unsafe HashJoin
2015-07-22 13:02:43 -07:00
Yin Huai c03299a18b [SPARK-4233] [SPARK-4367] [SPARK-3947] [SPARK-3056] [SQL] Aggregation Improvement
This is the first PR for the aggregation improvement, which is tracked by https://issues.apache.org/jira/browse/SPARK-4366 (umbrella JIRA). This PR contains work for its subtasks, SPARK-3056, SPARK-3947, SPARK-4233, and SPARK-4367.

This PR introduces a new code path for evaluating aggregate functions. This code path is guarded by `spark.sql.useAggregate2` and by default the value of this flag is true.

This new code path contains:
* A new aggregate function interface (`AggregateFunction2`) and 7 built-int aggregate functions based on this new interface (`AVG`, `COUNT`, `FIRST`, `LAST`, `MAX`, `MIN`, `SUM`)
* A UDAF interface (`UserDefinedAggregateFunction`) based on the new code path and two example UDAFs (`MyDoubleAvg` and `MyDoubleSum`).
* A sort-based aggregate operator (`Aggregate2Sort`) for the new aggregate function interface .
* A sort-based aggregate operator (`FinalAndCompleteAggregate2Sort`) for distinct aggregations (for distinct aggregations the query plan will use `Aggregate2Sort` and `FinalAndCompleteAggregate2Sort` together).

With this change, `spark.sql.useAggregate2` is `true`, the flow of compiling an aggregation query is:
1. Our analyzer looks up functions and returns aggregate functions built based on the old aggregate function interface.
2. When our planner is compiling the physical plan, it tries try to convert all aggregate functions to the ones built based on the new interface. The planner will fallback to the old code path if any of the following two conditions is true:
* code-gen is disabled.
* there is any function that cannot be converted (right now, Hive UDAFs).
* the schema of grouping expressions contain any complex data type.
* There are multiple distinct columns.

Right now, the new code path handles a single distinct column in the query (you can have multiple aggregate functions using that distinct column). For a query having a aggregate function with DISTINCT and regular aggregate functions, the generated plan will do partial aggregations for those regular aggregate function.

Thanks chenghao-intel for his initial work on it.

Author: Yin Huai <yhuai@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #7458 from yhuai/UDAF and squashes the following commits:

7865f5e [Yin Huai] Put the catalyst expression in the comment of the generated code for it.
b04d6c8 [Yin Huai] Remove unnecessary change.
f1d5901 [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
35b0520 [Yin Huai] Use semanticEquals to replace grouping expressions in the output of the aggregate operator.
3b43b24 [Yin Huai] bug fix.
00eb298 [Yin Huai] Make it compile.
a3ca551 [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
e0afca3 [Yin Huai] Gracefully fallback to old aggregation code path.
8a8ac4a [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
88c7d4d [Yin Huai] Enable spark.sql.useAggregate2 by default for testing purpose.
dc96fd1 [Yin Huai] Many updates:
85c9c4b [Yin Huai] newline.
43de3de [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
c3614d7 [Yin Huai] Handle single distinct column.
68b8ee9 [Yin Huai] Support single distinct column set. WIP
3013579 [Yin Huai] Format.
d678aee [Yin Huai] Remove AggregateExpressionSuite.scala since our built-in aggregate functions will be based on AlgebraicAggregate and we need to have another way to test it.
e243ca6 [Yin Huai] Add aggregation iterators.
a101960 [Yin Huai] Change MyJavaUDAF to MyDoubleSum.
594cdf5 [Yin Huai] Change existing AggregateExpression to AggregateExpression1 and add an AggregateExpression as the common interface for both AggregateExpression1 and AggregateExpression2.
380880f [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
0a827b3 [Yin Huai] Add comments and doc. Move some classes to the right places.
a19fea6 [Yin Huai] Add UDAF interface.
262d4c4 [Yin Huai] Make it compile.
b2e358e [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
6edb5ac [Yin Huai] Format update.
70b169c [Yin Huai] Remove groupOrdering.
4721936 [Yin Huai] Add CheckAggregateFunction to extendedCheckRules.
d821a34 [Yin Huai] Cleanup.
32aea9c [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
5b46d41 [Yin Huai] Bug fix.
aff9534 [Yin Huai] Make Aggregate2Sort work with both algebraic AggregateFunctions and non-algebraic AggregateFunctions.
2857b55 [Yin Huai] Merge remote-tracking branch 'upstream/master' into UDAF
4435f20 [Yin Huai] Add ConvertAggregateFunction to HiveContext's analyzer.
1b490ed [Michael Armbrust] make hive test
8cfa6a9 [Michael Armbrust] add test
1b0bb3f [Yin Huai] Do not bind references in AlgebraicAggregate and use code gen for all places.
072209f [Yin Huai] Bug fix: Handle expressions in grouping columns that are not attribute references.
f7d9e54 [Michael Armbrust] Merge remote-tracking branch 'apache/master' into UDAF
39ee975 [Yin Huai] Code cleanup: Remove unnecesary AttributeReferences.
b7720ba [Yin Huai] Add an analysis rule to convert aggregate function to the new version.
5c00f3f [Michael Armbrust] First draft of codegen
6bbc6ba [Michael Armbrust] now with correct answers\!
f7996d0 [Michael Armbrust] Add AlgebraicAggregate
dded1c5 [Yin Huai] wip
2015-07-21 23:26:11 -07:00
Andrew Or f4785f5b82 [SPARK-9232] [SQL] Duplicate code in JSONRelation
Author: Andrew Or <andrew@databricks.com>

Closes #7576 from andrewor14/clean-up-json-relation and squashes the following commits:

ea80803 [Andrew Or] Clean up duplicate code
2015-07-21 23:00:13 -07:00
Reynold Xin a4c83cb1e4 [SPARK-9154][SQL] Rename formatString to format_string.
Also make format_string the canonical form, rather than printf.

Author: Reynold Xin <rxin@databricks.com>

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

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

fixes bug of #7546

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

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

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

dcae272 [Tarek Auel] [SPARK-9154][SQL] build fix
1487602 [Tarek Auel] Merge remote-tracking branch 'upstream/master' into SPARK-9154
f512c5f [Tarek Auel] [SPARK-9154][SQL] build fix
a943d3e [Tarek Auel] [SPARK-9154] implicit input cast, added tests for null, support for null primitives
10b4de8 [Tarek Auel] [SPARK-9154][SQL] codegen removed fallback trait
cd8322b [Tarek Auel] [SPARK-9154][SQL] codegen string format
086caba [Tarek Auel] [SPARK-9154][SQL] codegen string format
2015-07-21 15:47:40 -07:00
Reynold Xin 60c0ce134d [SPARK-8906][SQL] Move all internal data source classes into execution.datasources.
This way, the sources package contains only public facing interfaces.

Author: Reynold Xin <rxin@databricks.com>

Closes #7565 from rxin/move-ds and squashes the following commits:

7661aff [Reynold Xin] Mima
9d5196a [Reynold Xin] Rearranged imports.
3dd7174 [Reynold Xin] [SPARK-8906][SQL] Move all internal data source classes into execution.datasources.
2015-07-21 11:56:38 -07:00
navis.ryu 9ba7c64dec [SPARK-8357] Fix unsafe memory leak on empty inputs in GeneratedAggregate
This patch fixes a managed memory leak in GeneratedAggregate.  The leak occurs when the unsafe aggregation path is used to perform grouped aggregation on an empty input; in this case, GeneratedAggregate allocates an UnsafeFixedWidthAggregationMap that is never cleaned up because `next()` is never called on the aggregate result iterator.

This patch fixes this by short-circuiting on empty inputs.

This patch is an updated version of #6810.

Closes #6810.

Author: navis.ryu <navis@apache.org>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #7560 from JoshRosen/SPARK-8357 and squashes the following commits:

3486ce4 [Josh Rosen] Some minor cleanup
c649310 [Josh Rosen] Revert SparkPlan change:
3c7db0f [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-8357
adc8239 [Josh Rosen] Back out Projection changes.
c5419b3 [navis.ryu] addressed comments
143e1ef [navis.ryu] fixed format & added test for CCE case
735972f [navis.ryu] used new conf apis
1a02a55 [navis.ryu] Rolled-back test-conf cleanup & fixed possible CCE & added more tests
51178e8 [navis.ryu] addressed comments
4d326b9 [navis.ryu] fixed test fails
15c5afc [navis.ryu] added a test as suggested by JoshRosen
d396589 [navis.ryu] added comments
1b07556 [navis.ryu] [SPARK-8357] [SQL] Memory leakage on unsafe aggregation path with empty input
2015-07-21 11:52:52 -07:00
Yijie Shen be5c5d3741 [SPARK-9081] [SPARK-9168] [SQL] nanvl & dropna/fillna supporting nan as well
JIRA:
https://issues.apache.org/jira/browse/SPARK-9081
https://issues.apache.org/jira/browse/SPARK-9168

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

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

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

f0a51db [Yijie Shen] make coalesce untouched and implement nanvl
1d3e35f [Yijie Shen] make Coalesce aware of NaN in order to support fillna
2760cbc [Yijie Shen] change isNaN(null) to false as well as implement dropna
2015-07-21 08:25:50 -07:00
Pedro Rodriguez 560c658a74 [SPARK-8230][SQL] Add array/map size method
Pull Request for: https://issues.apache.org/jira/browse/SPARK-8230

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

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

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

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

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

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

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

e5ea476 [Cheng Hao] minor update for documentation
ef96fd6 [Cheng Hao] update the code gen
72cf28f [Cheng Hao] Add more log for compilation error
4e11381 [Cheng Hao] Add regexp_replace / regexp_extract support
2015-07-21 00:48:07 -07:00
Cheng Lian d38c5029a2 [SPARK-9100] [SQL] Adds DataFrame reader/writer shortcut methods for ORC
This PR adds DataFrame reader/writer shortcut methods for ORC in both Scala and Python.

Author: Cheng Lian <lian@databricks.com>

Closes #7444 from liancheng/spark-9100 and squashes the following commits:

284d043 [Cheng Lian] Fixes PySpark test cases and addresses PR comments
e0b09fb [Cheng Lian] Adds DataFrame reader/writer shortcut methods for ORC
2015-07-21 15:08:44 +08:00
Josh Rosen 48f8fd46b3 [SPARK-9023] [SQL] Followup for #7456 (Efficiency improvements for UnsafeRows in Exchange)
This patch addresses code review feedback from #7456.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7551 from JoshRosen/unsafe-exchange-followup and squashes the following commits:

76dbdf8 [Josh Rosen] Add comments + more methods to UnsafeRowSerializer
3d7a1f2 [Josh Rosen] Add writeToStream() method to UnsafeRow
2015-07-20 23:28:35 -07:00
Reynold Xin 67570beed5 [SPARK-9208][SQL] Remove variant of DataFrame string functions that accept column names.
It can be ambiguous whether that is a string literal or a column name.

cc marmbrus

Author: Reynold Xin <rxin@databricks.com>

Closes #7556 from rxin/str-exprs and squashes the following commits:

92afa83 [Reynold Xin] [SPARK-9208][SQL] Remove variant of DataFrame string functions that accept column names.
2015-07-20 22:48:13 -07:00
Cheng Lian a1064df0ee [SPARK-8125] [SQL] Accelerates Parquet schema merging and partition discovery
This PR tries to accelerate Parquet schema discovery and `HadoopFsRelation` partition discovery.  The acceleration is done by the following means:

- Turning off schema merging by default

  Schema merging is not the most common case, but requires reading footers of all Parquet part-files and can be very slow.

- Avoiding `FileSystem.globStatus()` call when possible

  `FileSystem.globStatus()` may issue multiple synchronous RPC calls, and can be very slow (esp. on S3).  This PR adds `SparkHadoopUtil.globPathIfNecessary()`, which only issues RPC calls when the path contain glob-pattern specific character(s) (`{}[]*?\`).

  This is especially useful when converting a metastore Parquet table with lots of partitions, since Spark SQL adds all partition directories as the input paths, and currently we do a `globStatus` call on each input path sequentially.

- Listing leaf files in parallel when the number of input paths exceeds a threshold

  Listing leaf files is required by partition discovery.  Currently it is done on driver side, and can be slow when there are lots of (nested) directories, since each `FileSystem.listStatus()` call issues an RPC.  In this PR, we list leaf files in a BFS style, and resort to a Spark job once we found that the number of directories need to be listed exceed a threshold.

  The threshold is controlled by `SQLConf` option `spark.sql.sources.parallelPartitionDiscovery.threshold`, which defaults to 32.

- Discovering Parquet schema in parallel

  Currently, schema merging is also done on driver side, and needs to read footers of all part-files.  This PR uses a Spark job to do schema merging.  Together with task side metadata reading in Parquet 1.7.0, we never read any footers on driver side now.

Author: Cheng Lian <lian@databricks.com>

Closes #7396 from liancheng/accel-parquet and squashes the following commits:

5598efc [Cheng Lian] Uses ParquetInputFormat[InternalRow] instead of ParquetInputFormat[Row]
ff32cd0 [Cheng Lian] Excludes directories while listing leaf files
3c580f1 [Cheng Lian] Fixes test failure caused by making "mergeSchema" default to "false"
b1646aa [Cheng Lian] Should allow empty input paths
32e5f0d [Cheng Lian] Moves schema merging to executor side
2015-07-20 16:42:43 -07:00
Davies Liu 9f913c4fd6 [SPARK-9114] [SQL] [PySpark] convert returned object from UDF into internal type
This PR also remove the duplicated code between registerFunction and UserDefinedFunction.

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #7450 from davies/fix_return_type and squashes the following commits:

e80bf9f [Davies Liu] remove debugging code
f94b1f6 [Davies Liu] fix mima
8f9c58b [Davies Liu] convert returned object from UDF into internal type
2015-07-20 12:14:47 -07:00
Reynold Xin c6fe9b4a17 [SQL] Remove space from DataFrame Scala/Java API.
I don't think this function is useful at all in Scala/Java, since users can easily compute n * space easily.

Author: Reynold Xin <rxin@databricks.com>

Closes #7530 from rxin/remove-space and squashes the following commits:

c147873 [Reynold Xin] [SQL] Remove space from DataFrame Scala/Java API.
2015-07-20 09:43:25 -07:00
Wenchen Fan 04db58ae30 [SPARK-9186][SQL] make deterministic describing the tree rather than the expression
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7525 from cloud-fan/deterministic and squashes the following commits:

4189bfa [Wenchen Fan] make deterministic describing the tree rather than the expression
2015-07-20 09:42:18 -07:00
Josh Rosen 79ec07290d [SPARK-9023] [SQL] Efficiency improvements for UnsafeRows in Exchange
This pull request aims to improve the performance of SQL's Exchange operator when shuffling UnsafeRows.  It also makes several general efficiency improvements to Exchange.

Key changes:

- When performing hash partitioning, the old Exchange projected the partitioning columns into a new row then passed a `(partitioningColumRow: InternalRow, row: InternalRow)` pair into the shuffle. This is very inefficient because it ends up redundantly serializing the partitioning columns only to immediately discard them after the shuffle.  After this patch's changes, Exchange now shuffles `(partitionId: Int, row: InternalRow)` pairs.  This still isn't optimal, since we're still shuffling extra data that we don't need, but it's significantly more efficient than the old implementation; in the future, we may be able to further optimize this once we implement a new shuffle write interface that accepts non-key-value-pair inputs.
- Exchange's `compute()` method has been significantly simplified; the new code has less duplication and thus is easier to understand.
- When the Exchange's input operator produces UnsafeRows, Exchange will use a specialized `UnsafeRowSerializer` to serialize these rows.  This serializer is significantly more efficient since it simply copies the UnsafeRow's underlying bytes.  Note that this approach does not work for UnsafeRows that use the ObjectPool mechanism; I did not add support for this because we are planning to remove ObjectPool in the next few weeks.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7456 from JoshRosen/unsafe-exchange and squashes the following commits:

7e75259 [Josh Rosen] Fix cast in SparkSqlSerializer2Suite
0082515 [Josh Rosen] Some additional comments + small cleanup to remove an unused parameter
a27cfc1 [Josh Rosen] Add missing newline
741973c [Josh Rosen] Add simple test of UnsafeRow shuffling in Exchange.
359c6a4 [Josh Rosen] Remove println() and add comments
93904e7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-exchange
8dd3ff2 [Josh Rosen] Exchange outputs UnsafeRows when its child outputs them
dd9c66d [Josh Rosen] Fix for copying logic
035af21 [Josh Rosen] Add logic for choosing when to use UnsafeRowSerializer
7876f31 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-shuffle
cbea80b [Josh Rosen] Add UnsafeRowSerializer
0f2ac86 [Josh Rosen] Import ordering
3ca8515 [Josh Rosen] Big code simplification in Exchange
3526868 [Josh Rosen] Iniitial cut at removing shuffle on KV pairs
2015-07-19 23:41:28 -07:00
Jacky Li 972d8900a1 [SQL][DOC] Minor document fix in HadoopFsRelationProvider
Catch this while reading the code

Author: Jacky Li <lee.unreal@gmail.com>
Author: Jacky Li <jackylk@users.noreply.github.com>

Closes #7524 from jackylk/patch-11 and squashes the following commits:

b679011 [Jacky Li] fix doc
e10e211 [Jacky Li] [SQL] Minor document fix in HadoopFsRelationProvider
2015-07-19 23:19:17 -07:00
Wenchen Fan 930253e076 [SPARK-9185][SQL] improve code gen for mutable states to support complex initialization
Sometimes we need more than one step to initialize the mutable states in code gen like https://github.com/apache/spark/pull/7516

Author: Wenchen Fan <cloud0fan@outlook.com>

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

2106445 [Wenchen Fan] improve code gen for mutable states
2015-07-19 22:42:44 -07:00
Reynold Xin 163e3f1df9 [SPARK-8241][SQL] string function: concat_ws.
I also changed the semantics of concat w.r.t. null back to the same behavior as Hive.
That is to say, concat now returns null if any input is null.

Author: Reynold Xin <rxin@databricks.com>

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

83fd950 [Reynold Xin] Fixed type casting.
3ae85f7 [Reynold Xin] Write null better.
cdc7be6 [Reynold Xin] Added code generation for pure string mode.
a61c4e4 [Reynold Xin] Updated comments.
2d51406 [Reynold Xin] [SPARK-8241][SQL] string function: concat_ws.
2015-07-19 16:48:47 -07:00
Herman van Hovell 7a81245345 [SPARK-8638] [SQL] Window Function Performance Improvements - Cleanup
This PR contains a few clean-ups that are a part of SPARK-8638: a few style issues got fixed, and a few tests were moved.

Git commit message is wrong BTW :(...

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

Closes #7513 from hvanhovell/SPARK-8638-cleanup and squashes the following commits:

4e69d08 [Herman van Hovell] Fixed Perfomance Regression for Shrinking Window Frames (+Rebase)
2015-07-19 16:29:50 -07:00
Reynold Xin 3427937ea2 [SQL] Make date/time functions more consistent with other database systems.
This pull request fixes some of the problems in #6981.

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

Author: Reynold Xin <rxin@databricks.com>

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

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

0cb24d9 [Reynold Xin] Export all functions in Python.
e44a4a0 [Reynold Xin] Removed day function from Scala and Python.
9c08fdc [Reynold Xin] [SQL] Make date/time functions more consistent with other database systems.
2015-07-19 01:17:22 -07:00
Herman van Hovell a9a0d0cebf [SPARK-8638] [SQL] Window Function Performance Improvements
## Description
Performance improvements for Spark Window functions. This PR will also serve as the basis for moving away from Hive UDAFs to Spark UDAFs. See JIRA tickets SPARK-8638 and SPARK-7712 for more information.

## Improvements
* Much better performance (10x) in running cases (e.g. BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) and UNBOUDED FOLLOWING cases. The current implementation in spark uses a sliding window approach in these cases. This means that an aggregate is maintained for every row, so space usage is N (N being the number of rows). This also means that all these aggregates all need to be updated separately, this takes N*(N-1)/2 updates. The running case differs from the Sliding case because we are only adding data to an aggregate function (no reset is required), we only need to maintain one aggregate (like in the UNBOUNDED PRECEDING AND UNBOUNDED case), update the aggregate for each row, and get the aggregate value after each update. This is what the new implementation does. This approach only uses 1 buffer, and only requires N updates; I am currently working on data with window sizes of 500-1000 doing running sums and this saves a lot of time. The CURRENT ROW AND UNBOUNDED FOLLOWING case also uses this approach and the fact that aggregate operations are communitative, there is one twist though it will process the input buffer in reverse.
* Fewer comparisons in the sliding case. The current implementation determines frame boundaries for every input row. The new implementation makes more use of the fact that the window is sorted, maintains the boundaries, and only moves them when the current row order changes. This is a minor improvement.
* A single Window node is able to process all types of Frames for the same Partitioning/Ordering. This saves a little time/memory spent buffering and managing partitions. This will be enabled in a follow-up PR.
* A lot of the staging code is moved from the execution phase to the initialization phase. Minor performance improvement, and improves readability of the execution code.

## Benchmarking
I have done a small benchmark using [on time performance](http://www.transtats.bts.gov) data of the month april. I have used the origin as a partioning key, as a result there is quite some variation in window sizes. The code for the benchmark can be found in the JIRA ticket. These are the results per Frame type:

Frame | Master | SPARK-8638
----- | ------ | ----------
Entire Frame | 2 s | 1 s
Sliding | 18 s | 1 s
Growing | 14 s | 0.9 s
Shrinking | 13 s | 1 s

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

Closes #7057 from hvanhovell/SPARK-8638 and squashes the following commits:

3bfdc49 [Herman van Hovell] Fixed Perfomance Regression for Shrinking Window Frames (+Rebase)
2eb3b33 [Herman van Hovell] Corrected reverse range frame processing.
2cd2d5b [Herman van Hovell] Corrected reverse range frame processing.
b0654d7 [Herman van Hovell] Tests for exotic frame specifications.
e75b76e [Herman van Hovell] More docs, added support for reverse sliding range frames, and some reorganization of code.
1fdb558 [Herman van Hovell] Changed Data In HiveDataFrameWindowSuite.
ac2f682 [Herman van Hovell] Added a few more comments.
1938312 [Herman van Hovell] Added Documentation to the createBoundOrdering methods.
bb020e6 [Herman van Hovell] Major overhaul of Window operator.
2015-07-18 23:44:38 -07:00
Tarek Auel 83b682beec [SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions
Jira:
https://issues.apache.org/jira/browse/SPARK-8199
https://issues.apache.org/jira/browse/SPARK-8184
https://issues.apache.org/jira/browse/SPARK-8183
https://issues.apache.org/jira/browse/SPARK-8182
https://issues.apache.org/jira/browse/SPARK-8181
https://issues.apache.org/jira/browse/SPARK-8180
https://issues.apache.org/jira/browse/SPARK-8179
https://issues.apache.org/jira/browse/SPARK-8177
https://issues.apache.org/jira/browse/SPARK-8179
https://issues.apache.org/jira/browse/SPARK-9115

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

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

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

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

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

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

Closes #7501 from rxin/jsonrdd and squashes the following commits:

767ec55 [Reynold Xin] More Mima
51f456e [Reynold Xin] Mima exclude.
789cb80 [Reynold Xin] Fixed compilation error.
b4cf50d [Reynold Xin] [SPARK-8278] Remove non-streaming JSON reader.
2015-07-18 20:27:55 -07:00
Reynold Xin 9914b1b2c5 [SPARK-9150][SQL] Create CodegenFallback and Unevaluable trait
It is very hard to track which expressions have code gen implemented or not. This patch removes the default fallback gencode implementation from Expression, and moves that into a new trait called CodegenFallback. Each concrete expression needs to either implement code generation, or mix in CodegenFallback. This makes it very easy to track which expressions have code generation implemented already.

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

Author: Reynold Xin <rxin@databricks.com>

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

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

Closes #7500 from rxin/sqlconf and squashes the following commits:

a5726c8 [Reynold Xin] [SPARK-9174][SQL] Add documentation for all public SQLConfs.
2015-07-18 15:29:38 -07:00
Reynold Xin 6e1e2eba69 [SPARK-8240][SQL] string function: concat
Author: Reynold Xin <rxin@databricks.com>

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

5217d6e [Reynold Xin] Removed Hive's concat test.
f5cb7a3 [Reynold Xin] Concat is never nullable.
ae4e61f [Reynold Xin] Removed extra import.
fddcbbd [Reynold Xin] Fixed NPE.
22e831c [Reynold Xin] Added missing file.
57a2352 [Reynold Xin] [SPARK-8240][SQL] string function: concat
2015-07-18 14:07:56 -07:00
Wenchen Fan 1b4ff05538 [SPARK-9142][SQL] remove more self type in catalyst
a follow up of https://github.com/apache/spark/pull/7479.
The `TreeNode` is the root case of the requirement of `self: Product =>` stuff, so why not make `TreeNode` extend `Product`?

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7495 from cloud-fan/self-type and squashes the following commits:

8676af7 [Wenchen Fan] remove more self type
2015-07-18 11:13:49 -07:00
Josh Rosen b8aec6cd23 [SPARK-9143] [SQL] Add planner rule for automatically inserting Unsafe <-> Safe row format converters
Now that we have two different internal row formats, UnsafeRow and the old Java-object-based row format, we end up having to perform conversions between these two formats. These conversions should not be performed by the operators themselves; instead, the planner should be responsible for inserting appropriate format conversions when they are needed.

This patch makes the following changes:

- Add two new physical operators for performing row format conversions, `ConvertToUnsafe` and `ConvertFromUnsafe`.
- Add new methods to `SparkPlan` to allow operators to express whether they output UnsafeRows and whether they can handle safe or unsafe rows as inputs.
- Implement an `EnsureRowFormats` rule to automatically insert converter operators where necessary.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7482 from JoshRosen/unsafe-converter-planning and squashes the following commits:

7450fa5 [Josh Rosen] Resolve conflicts in favor of choosing UnsafeRow
5220cce [Josh Rosen] Add roundtrip converter test
2bb8da8 [Josh Rosen] Add Union unsafe support + tests to bump up test coverage
6f79449 [Josh Rosen] Add even more assertions to execute()
08ce199 [Josh Rosen] Rename ConvertFromUnsafe -> ConvertToSafe
0e2d548 [Josh Rosen] Add assertion if operators' input rows are in different formats
cabb703 [Josh Rosen] Add tests for Filter
3b11ce3 [Josh Rosen] Add missing test file.
ae2195a [Josh Rosen] Fixes
0fef0f8 [Josh Rosen] Rename file.
d5f9005 [Josh Rosen] Finish writing EnsureRowFormats planner rule
b5df19b [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-converter-planning
9ba3038 [Josh Rosen] WIP
2015-07-18 11:08:18 -07:00
Daoyuan Wang 1707238601 [SPARK-7026] [SQL] fix left semi join with equi key and non-equi condition
When the `condition` extracted by `ExtractEquiJoinKeys` contain join Predicate for left semi join, we can not plan it as semiJoin. Such as

    SELECT * FROM testData2 x
    LEFT SEMI JOIN testData2 y
    ON x.b = y.b
    AND x.a >= y.a + 2
Condition `x.a >= y.a + 2` can not evaluate on table `x`, so it throw errors

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

Closes #5643 from adrian-wang/spark7026 and squashes the following commits:

cc09809 [Daoyuan Wang] refactor semijoin and add plan test
575a7c8 [Daoyuan Wang] fix notserializable
27841de [Daoyuan Wang] fix rebase
10bf124 [Daoyuan Wang] fix style
72baa02 [Daoyuan Wang] fix style
8e0afca [Daoyuan Wang] merge commits for rebase
2015-07-17 16:45:46 -07:00
Yijie Shen 15fc2ffe55 [SPARK-9080][SQL] add isNaN predicate expression
JIRA: https://issues.apache.org/jira/browse/SPARK-9080

cc rxin

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

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

11ae039 [Yijie Shen] add isNaN in functions
666718e [Yijie Shen] add isNaN predicate expression
2015-07-17 15:49:31 -07:00
Reynold Xin b2aa490bb6 [SPARK-9142] [SQL] Removing unnecessary self types in Catalyst.
Just a small change to add Product type to the base expression/plan abstract classes, based on suggestions on #7434 and offline discussions.

Author: Reynold Xin <rxin@databricks.com>

Closes #7479 from rxin/remove-self-types and squashes the following commits:

e407ffd [Reynold Xin] [SPARK-9142][SQL] Removing unnecessary self types in Catalyst.
2015-07-17 15:02:13 -07:00
zhichao.li 305e77cd83 [SPARK-8209[SQL]Add function conv
cc chenghao-intel  adrian-wang

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

Closes #6872 from zhichao-li/conv and squashes the following commits:

6ef3b37 [zhichao.li] add unittest and comments
78d9836 [zhichao.li] polish dataframe api and add unittest
e2bace3 [zhichao.li] update to use ImplicitCastInputTypes
cbcad3f [zhichao.li] add function conv
2015-07-17 09:32:27 -07:00
Reynold Xin fec10f0c63 [SPARK-9085][SQL] Remove LeafNode, UnaryNode, BinaryNode from TreeNode.
This builds on #7433 but also removes LeafNode/UnaryNode. These are slightly more complicated to remove. I had to change some abstract classes to traits in order for it to work.

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

Author: Reynold Xin <rxin@databricks.com>

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

9e8a4de [Reynold Xin] Generator should not be foldable.
3135a8b [Reynold Xin] SortOrder should not be foldable.
9c589cf [Reynold Xin] Fixed one more test case...
2225331 [Reynold Xin] Aggregate expressions should not be foldable.
16b5c90 [Reynold Xin] [SPARK-9085][SQL] Remove LeafNode, UnaryNode, BinaryNode from TreeNode.
2015-07-16 13:58:39 -07:00
Yijie Shen 43dac2c880 [SPARK-6941] [SQL] Provide a better error message to when inserting into RDD based table
JIRA: https://issues.apache.org/jira/browse/SPARK-6941

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

Closes #7342 from yijieshen/SPARK-6941 and squashes the following commits:

f82cbe7 [Yijie Shen] reorder import
dd67e40 [Yijie Shen] resolve comments
09518af [Yijie Shen] fix import order in DataframeSuite
0c635d4 [Yijie Shen] make match more specific
9df388d [Yijie Shen] move check into PreWriteCheck
847ab20 [Yijie Shen] Detect insertion error in DataSourceStrategy
2015-07-16 10:52:09 -07:00
Cheng Hao 42dea3acf9 [SPARK-8245][SQL] FormatNumber/Length Support for Expression
- `BinaryType` for `Length`
- `FormatNumber`

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

Closes #7034 from chenghao-intel/expression and squashes the following commits:

e534b87 [Cheng Hao] python api style issue
601bbf5 [Cheng Hao] add python API support
3ebe288 [Cheng Hao] update as feedback
52274f7 [Cheng Hao] add support for udf_format_number and length for binary
2015-07-15 21:47:21 -07:00
Reynold Xin b0645195d0 [SPARK-9086][SQL] Remove BinaryNode from TreeNode.
These traits are not super useful, and yet cause problems with toString in expressions due to the orders they are mixed in.

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Reynold Xin <rxin@databricks.com>

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

a760827 [Reynold Xin] [SPARK-9071][SQL] MonotonicallyIncreasingID and SparkPartitionID should be marked as nondeterministic.
2015-07-15 14:52:02 -07:00
zhichao.li a9385271a9 [SPARK-8221][SQL]Add pmod function
https://issues.apache.org/jira/browse/SPARK-8221

One concern is the result would be negative if the divisor is not positive( i.e pmod(7, -3) ), but the behavior is the same as hive.

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

Closes #6783 from zhichao-li/pmod2 and squashes the following commits:

7083eb9 [zhichao.li] update to the latest type checking
d26dba7 [zhichao.li] add pmod
2015-07-15 10:43:38 -07:00
Wenchen Fan fa4ec3606a [SPARK-9020][SQL] Support mutable state in code gen expressions
We can keep expressions' mutable states in generated class(like `SpecificProjection`) as member variables, so that we can read and modify them inside codegened expressions.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

eb3a221 [Wenchen Fan] fix order
73144d8 [Wenchen Fan] naming improvement
318f41d [Wenchen Fan] address more comments
d43b65d [Wenchen Fan] address comments
fd45c7a [Wenchen Fan] Support mutable state in code gen expressions
2015-07-15 10:31:39 -07:00
Liang-Chi Hsieh 6f6902597d [SPARK-8840] [SPARKR] Add float coercion on SparkR
JIRA: https://issues.apache.org/jira/browse/SPARK-8840

Currently the type coercion rules don't include float type. This PR simply adds it.

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

Closes #7280 from viirya/add_r_float_coercion and squashes the following commits:

c86dc0e [Liang-Chi Hsieh] For comments.
dbf0c1b [Liang-Chi Hsieh] Implicitly convert Double to Float based on provided schema.
733015a [Liang-Chi Hsieh] Add test case for DataFrame with float type.
30c2a40 [Liang-Chi Hsieh] Update test case.
52b5294 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_r_float_coercion
6f9159d [Liang-Chi Hsieh] Add another test case.
8db3244 [Liang-Chi Hsieh] schema also needs to support float. add test case.
0dcc992 [Liang-Chi Hsieh] Add float coercion on SparkR.
2015-07-15 09:48:33 -07:00
Yijie Shen f0e129740d [SPARK-8279][SQL]Add math function round
JIRA: https://issues.apache.org/jira/browse/SPARK-8279

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

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

07a124c [Yijie Shen] remove useless def children
392b65b [Yijie Shen] add negative scale test in DecimalSuite
61760ee [Yijie Shen] address reviews
302a78a [Yijie Shen] Add dataframe function test
31dfe7c [Yijie Shen] refactor round to make it readable
8c7a949 [Yijie Shen] rebase & inputTypes update
9555e35 [Yijie Shen] tiny style fix
d10be4a [Yijie Shen] use TypeCollection to specify wanted input and implicit cast
c3b9839 [Yijie Shen] rely on implict cast to handle string input
b0bff79 [Yijie Shen] make round's inner method's name more meaningful
9bd6930 [Yijie Shen] revert accidental change
e6f44c4 [Yijie Shen] refactor eval and genCode
1b87540 [Yijie Shen] modify checkInputDataTypes using foldable
5486b2d [Yijie Shen] DataFrame API modification
2077888 [Yijie Shen] codegen versioned eval
6cd9a64 [Yijie Shen] refactor Round's constructor
9be894e [Yijie Shen] add round functions in o.a.s.sql.functions
7c83e13 [Yijie Shen] more tests on round
56db4bb [Yijie Shen] Add decimal support to Round
7e163ae [Yijie Shen] style fix
653d047 [Yijie Shen] Add math function round
2015-07-14 23:30:41 -07:00
Josh Rosen cc57d705e7 [SPARK-9050] [SQL] Remove unused newOrdering argument from Exchange (cleanup after SPARK-8317)
SPARK-8317 changed the SQL Exchange operator so that it no longer pushed sorting into Spark's shuffle layer, a change which allowed more efficient SQL-specific sorters to be used.

This patch performs some leftover cleanup based on those changes:

- Exchange's constructor should no longer accept a `newOrdering` since it's no longer used and no longer works as expected.
- `addOperatorsIfNecessary` looked at shuffle input's output ordering to decide whether to sort, but this is the wrong node to be examining: it needs to look at whether the post-shuffle node has the right ordering, since shuffling will not preserve row orderings.  Thanks to davies for spotting this.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7407 from JoshRosen/SPARK-9050 and squashes the following commits:

e70be50 [Josh Rosen] No need to wrap line
e866494 [Josh Rosen] Refactor addOperatorsIfNecessary to make code clearer
2e467da [Josh Rosen] Remove `newOrdering` from Exchange.
2015-07-14 18:55:34 -07:00
Josh Rosen 11e5c37286 [SPARK-8962] Add Scalastyle rule to ban direct use of Class.forName; fix existing uses
This pull request adds a Scalastyle regex rule which fails the style check if `Class.forName` is used directly.  `Class.forName` always loads classes from the default / system classloader, but in a majority of cases, we should be using Spark's own `Utils.classForName` instead, which tries to load classes from the current thread's context classloader and falls back to the classloader which loaded Spark when the context classloader is not defined.

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

Author: Josh Rosen <joshrosen@databricks.com>

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

e3e96f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
c0b7885 [Josh Rosen] Hopefully fix the last two cases
d707ba7 [Josh Rosen] Fix uses of Class.forName that I missed in my first cleanup pass
046470d [Josh Rosen] Merge remote-tracking branch 'origin/master' into ban-Class.forName
62882ee [Josh Rosen] Fix uses of Class.forName or add exclusion.
d9abade [Josh Rosen] Add stylechecker rule to ban uses of Class.forName
2015-07-14 16:08:17 -07:00
Daoyuan Wang 257236c3e1 [SPARK-6851] [SQL] function least/greatest follow up
This is a follow up of remaining comments from #6851

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

Closes #7387 from adrian-wang/udflgfollow and squashes the following commits:

6163e62 [Daoyuan Wang] add skipping null values
e8c2e09 [Daoyuan Wang] use seq
8362966 [Daoyuan Wang] pr6851 follow up
2015-07-14 01:09:33 -07:00
Daoyuan Wang 92540d22e4 [SPARK-8203] [SPARK-8204] [SQL] conditional function: least/greatest
chenghao-intel zhichao-li qiansl127

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

Closes #6851 from adrian-wang/udflg and squashes the following commits:

0f1bff2 [Daoyuan Wang] address comments from davis
7a6bdbb [Daoyuan Wang] add '.' for hex()
c1f6824 [Daoyuan Wang] add codegen, test for all types
ec625b0 [Daoyuan Wang] conditional function: least/greatest
2015-07-13 00:14:32 -07:00
Cheng Lian 3363088368 [SPARK-8961] [SQL] Makes BaseWriterContainer.outputWriterForRow accepts InternalRow instead of Row
This is a follow-up of [SPARK-8888] [1], which also aims to optimize writing dynamic partitions.

Three more changes can be made here:

1. Using `InternalRow` instead of `Row` in `BaseWriterContainer.outputWriterForRow`
2. Using `Cast` expressions to convert partition columns to strings, so that we can leverage code generation.
3. Replacing the FP-style `zip` and `map` calls with a faster imperative `while` loop.

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

Author: Cheng Lian <lian@databricks.com>

Closes #7331 from liancheng/spark-8961 and squashes the following commits:

b5ab9ae [Cheng Lian] Casts Java iterator to Scala iterator explicitly
719e63b [Cheng Lian] Makes BaseWriterContainer.outputWriterForRow accepts InternalRow instead of Row
2015-07-10 18:15:36 -07:00
Cheng Lian 857e325f30 [SPARK-8990] [SQL] SPARK-8990 DataFrameReader.parquet() should respect user specified options
Author: Cheng Lian <lian@databricks.com>

Closes #7347 from liancheng/spark-8990 and squashes the following commits:

045698c [Cheng Lian] SPARK-8990 DataFrameReader.parquet() should respect user specified options
2015-07-10 16:49:45 -07:00
Josh Rosen fb8807c9b0 [SPARK-7078] [SPARK-7079] Binary processing sort for Spark SQL
This patch adds a cache-friendly external sorter which operates on serialized bytes and uses this sorter to implement a new sort operator for Spark SQL and DataFrames.

### Overview of the new sorter

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

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

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

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

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

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

### Sorting rows in Spark SQL

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

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

### Misc. additional improvements made in this patch

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

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

### Tasks deferred to followup patches

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

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

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

Author: Josh Rosen <joshrosen@databricks.com>

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

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

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

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

There is no generated `Row` anymore.

Author: Davies Liu <davies@databricks.com>

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

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

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

b09a909 [Cheng Hao] update the code as feedback
7ebbf4c [Cheng Hao] Add more string expressions
2015-07-09 11:11:34 -07:00
Davies Liu 23448a9e98 [SPARK-8931] [SQL] Fallback to interpreted evaluation if failed to compile in codegen
Exception will not be catched during tests.

cc marmbrus rxin

Author: Davies Liu <davies@databricks.com>

Closes #7309 from davies/fallback and squashes the following commits:

969a612 [Davies Liu] throw exception during tests
f844f77 [Davies Liu] fallback
a3091bc [Davies Liu] Merge branch 'master' of github.com:apache/spark into fallback
364a0d6 [Davies Liu] fallback to interpret mode if failed to compile
2015-07-09 09:20:16 -07:00
Weizhong Lin 851e247caa [SPARK-8928] [SQL] Makes CatalystSchemaConverter sticking to 1.4.x- when handling Parquet LISTs in compatible mode
This PR is based on #7209 authored by Sephiroth-Lin.

Author: Weizhong Lin <linweizhong@huawei.com>

Closes #7314 from liancheng/spark-8928 and squashes the following commits:

75267fe [Cheng Lian] Makes CatalystSchemaConverter sticking to 1.4.x- when handling LISTs in compatible mode
2015-07-08 22:19:19 -07:00
Cheng Lian c056484c07 Revert "[SPARK-8928] [SQL] Makes CatalystSchemaConverter sticking to 1.4.x- when handling Parquet LISTs in compatible mode"
This reverts commit 3dab0da429.
2015-07-08 22:14:38 -07:00
Cheng Lian 3dab0da429 [SPARK-8928] [SQL] Makes CatalystSchemaConverter sticking to 1.4.x- when handling Parquet LISTs in compatible mode
This PR is based on #7209 authored by Sephiroth-Lin.

Author: Weizhong Lin <linweizhong@huawei.com>

Closes #7304 from liancheng/spark-8928 and squashes the following commits:

75267fe [Cheng Lian] Makes CatalystSchemaConverter sticking to 1.4.x- when handling LISTs in compatible mode
2015-07-08 22:09:14 -07:00
Andrew Or 47ef423f86 [SPARK-8910] Fix MiMa flaky due to port contention issue
Due to the way MiMa works, we currently start a `SQLContext` pretty early on. This causes us to start a `SparkUI` that attempts to bind to port 4040. Because many tests run in parallel on the Jenkins machines, this  causes port contention sometimes and fails the MiMa tests.

Note that we already disabled the SparkUI for scalatests. However, the MiMa test is run before we even have a chance to load the default scalatest settings, so we need to explicitly disable the UI ourselves.

Author: Andrew Or <andrew@databricks.com>

Closes #7300 from andrewor14/mima-flaky and squashes the following commits:

b55a547 [Andrew Or] Do not enable SparkUI during tests
2015-07-08 20:29:08 -07:00
Yijie Shen a290814877 [SPARK-8866][SQL] use 1us precision for timestamp type
JIRA: https://issues.apache.org/jira/browse/SPARK-8866

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

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

dc735df [Yijie Shen] update CastSuite to avoid round error
714eaea [Yijie Shen] add timestamp_udf into blacklist due to precision lose
c3ca2f4 [Yijie Shen] fix unhandled case in CurrentTimestamp
8d4aa6b [Yijie Shen] use 1us precision for timestamp type
2015-07-08 20:20:17 -07:00
Davies Liu 74d8d3d928 [SPARK-8450] [SQL] [PYSARK] cleanup type converter for Python DataFrame
This PR fixes the converter for Python DataFrame, especially for DecimalType

Closes #7106

Author: Davies Liu <davies@databricks.com>

Closes #7131 from davies/decimal_python and squashes the following commits:

4d3c234 [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_python
20531d6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_python
7d73168 [Davies Liu] fix conflit
6cdd86a [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_python
7104e97 [Davies Liu] improve type infer
9cd5a21 [Davies Liu] run python tests with SPARK_PREPEND_CLASSES
829a05b [Davies Liu] fix UDT in python
c99e8c5 [Davies Liu] fix mima
c46814a [Davies Liu] convert decimal for Python DataFrames
2015-07-08 18:22:53 -07:00
Kousuke Saruta 2a4f88b6c1 [SPARK-8914][SQL] Remove RDDApi
As rxin suggested in #7298 , we should consider to remove `RDDApi`.

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

Closes #7302 from sarutak/remove-rddapi and squashes the following commits:

e495d35 [Kousuke Saruta] Fixed mima
cb7ebb9 [Kousuke Saruta] Removed overriding RDDApi
2015-07-08 18:09:39 -07:00
Cheng Lian 4ffc27caaf [SPARK-6123] [SPARK-6775] [SPARK-6776] [SQL] Refactors Parquet read path for interoperability and backwards-compatibility
This PR is a follow-up of #6617 and is part of [SPARK-6774] [2], which aims to ensure interoperability and backwards-compatibility for Spark SQL Parquet support.  And this one fixes the read path.  Now Spark SQL is expected to be able to read legacy Parquet data files generated by most (if not all) common libraries/tools like parquet-thrift, parquet-avro, and parquet-hive. However, we still need to refactor the write path to write standard Parquet LISTs and MAPs ([SPARK-8848] [4]).

### Major changes

1. `CatalystConverter` class hierarchy refactoring

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

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

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

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

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

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

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

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

   - Implements backwards-compatibility rules in `CatalystArrayConverter`

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

2. Requested columns handling

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

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

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

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

### Testing

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

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

Author: Cheng Lian <lian@databricks.com>

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

360fe18 [Cheng Lian] Adds ParquetHiveCompatibilitySuite
c6fbc06 [Cheng Lian] Removes WIP file committed by mistake
b8c1295 [Cheng Lian] Excludes the whole parquet package from MiMa
598c3e8 [Cheng Lian] Adds extra Maven repo for hadoop-lzo, which is a transitive dependency of parquet-thrift
926af87 [Cheng Lian] Simplifies Parquet compatibility test suites
7946ee1 [Cheng Lian] Fixes Scala styling issues
3d7ab36 [Cheng Lian] Fixes .rat-excludes
a8f13bb [Cheng Lian] Using Parquet writer API to do compatibility tests
f2208cd [Cheng Lian] Adds README.md for Thrift/Avro code generation
1d390aa [Cheng Lian] Adds parquet-thrift compatibility test
440f7b3 [Cheng Lian] Adds generated files to .rat-excludes
13b9121 [Cheng Lian] Adds ParquetAvroCompatibilitySuite
06cfe9d [Cheng Lian] Adds comments about TimestampType handling
a099d3e [Cheng Lian] More comments
0cc1b37 [Cheng Lian] Fixes MiMa checks
884d3e6 [Cheng Lian] Fixes styling issue and reverts unnecessary changes
802cbd7 [Cheng Lian] Fixes bugs related to schema merging and empty requested columns
38fe1e7 [Cheng Lian] Adds explicit return type
7fb21f1 [Cheng Lian] Reverts an unnecessary debugging change
1781dff [Cheng Lian] Adds test case for SPARK-8811
6437d4b [Cheng Lian] Assembles requested schema from Parquet file schema
bcac49f [Cheng Lian] Removes the 16-byte restriction of decimals
a74fb2c [Cheng Lian] More comments
0525346 [Cheng Lian] Removes old Parquet record converters
03c3bd9 [Cheng Lian] Refactors Parquet read path to implement backwards-compatibility rules
2015-07-08 15:51:01 -07:00
Cheolsoo Park 00b265f12c [SPARK-8908] [SQL] Add () to distinct definition in dataframe
Adding `()` to the definition of `distinct` in DataFrame allows distinct to be called with parentheses, which is consistent with `dropDuplicates`.

Author: Cheolsoo Park <cheolsoop@netflix.com>

Closes #7298 from piaozhexiu/SPARK-8908 and squashes the following commits:

7f0d923 [Cheolsoo Park] Add () to distinct definition in dataframe
2015-07-08 15:18:24 -07:00
Reynold Xin f61c989b40 [SPARK-8888][SQL] Use java.util.HashMap in DynamicPartitionWriterContainer.
Just a baby step towards making it more efficient.

Author: Reynold Xin <rxin@databricks.com>

Closes #7282 from rxin/SPARK-8888 and squashes the following commits:

3da51ae [Reynold Xin] [SPARK-8888][SQL] Use java.util.HashMap in DynamicPartitionWriterContainer.
2015-07-08 10:56:31 -07:00
Wenchen Fan 0ba98c04c7 [SPARK-8753][SQL] Create an IntervalType data type
We need a new data type to represent time intervals. Because we can't determine how many days in a month, so we need 2 values for interval: a int `months`, a long `microseconds`.

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

632062d [Wenchen Fan] address comments
ac348c3 [Wenchen Fan] use case class
0342d2e [Wenchen Fan] use array byte
df9256c [Wenchen Fan] fix style
fd6f18a [Wenchen Fan] address comments
1856af3 [Wenchen Fan] support interval type
2015-07-08 10:51:32 -07:00
Davies Liu 74335b3107 [SPARK-5707] [SQL] fix serialization of generated projection
Author: Davies Liu <davies@databricks.com>

Closes #7272 from davies/fix_projection and squashes the following commits:

075ef76 [Davies Liu] fix codegen with BroadcastHashJion
2015-07-08 10:43:00 -07:00
Liang-Chi Hsieh 6722aca809 [SPARK-8785] [SQL] Improve Parquet schema merging
JIRA: https://issues.apache.org/jira/browse/SPARK-8785

Currently, the parquet schema merging (`ParquetRelation2.readSchema`) may spend much time to merge duplicate schema. We can select only non duplicate schema and merge them later.

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

Closes #7182 from viirya/improve_parquet_merging and squashes the following commits:

5cf934f [Liang-Chi Hsieh] Refactor it to make it faster.
f3411ea [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into improve_parquet_merging
a63c3ff [Liang-Chi Hsieh] Improve Parquet schema merging.
2015-07-08 10:09:50 -07:00
Cheng Hao 351a36d0c5 [SPARK-8883][SQL]Remove the OverrideFunctionRegistry
Remove the `OverrideFunctionRegistry` from the Spark SQL, as the subclasses of `FunctionRegistry` have their own way to the delegate to the right underlying `FunctionRegistry`.

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

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

164d093 [Cheng Hao] enable the function registry
2ca8459 [Cheng Hao] remove the OverrideFunctionRegistry
2015-07-08 00:10:24 -07:00
Yin Huai 68a4a16971 [SPARK-8868] SqlSerializer2 can go into infinite loop when row consists only of NullType columns
https://issues.apache.org/jira/browse/SPARK-8868

Author: Yin Huai <yhuai@databricks.com>

Closes #7262 from yhuai/SPARK-8868 and squashes the following commits:

cb58780 [Yin Huai] Andrew's comment.
e456857 [Yin Huai] Josh's comments.
5122e65 [Yin Huai] If types of all columns are NullTypes, do not use serializer2.
2015-07-07 18:36:35 -07:00
Reynold Xin 770ff1025e [SPARK-8876][SQL] Remove InternalRow type alias in expressions package.
The type alias was there because initially when I moved Row around, I didn't want to do massive changes to the expression code. But now it should be pretty easy to just remove it. One less concept to worry about.

Author: Reynold Xin <rxin@databricks.com>

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

72fc842 [Reynold Xin] [SPARK-8876][SQL] Remove InternalRow type alias in expressions package.
2015-07-07 17:40:14 -07:00
Liang-Chi Hsieh d4d6d31db5 [SPARK-8463][SQL] Use DriverRegistry to load jdbc driver at writing path
JIRA: https://issues.apache.org/jira/browse/SPARK-8463

Currently, at the reading path, `DriverRegistry` is used to load needed jdbc driver at executors. However, at the writing path, we also need `DriverRegistry` to load jdbc driver.

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

Closes #6900 from viirya/jdbc_write_driver and squashes the following commits:

16cd04b [Liang-Chi Hsieh] Use DriverRegistry to load jdbc driver at writing path.
2015-07-06 17:16:44 -07:00
animesh 09a06418de [SPARK-8072] [SQL] Better AnalysisException for writing DataFrame with identically named columns
Adding a function checkConstraints which will check for the constraints to be applied on the dataframe / dataframe schema. Function called before storing the dataframe to an external storage. Function added in the corresponding datasource API.
cc rxin marmbrus

Author: animesh <animesh@apache.spark>

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

Closes #7013 from animeshbaranawal/8072 and squashes the following commits:

f70dd0e [animesh] Change IO exception to Analysis Exception
fd45e1b [animesh] 8072: Fix Style Issues
a8a964f [animesh] 8072: Improving on previous commits
3cc4d2c [animesh] Fix Style Issues
1a89115 [animesh] Fix Style Issues
98b4399 [animesh] 8072 : Moved the exception handling to ResolvedDataSource specific to parquet format
7c3d928 [animesh] 8072: Adding check to DataFrameWriter.scala
2015-07-06 16:39:49 -07:00
Daoyuan Wang 132e7fca12 [MINOR] [SQL] remove unused code in Exchange
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #7234 from adrian-wang/exchangeclean and squashes the following commits:

b093ec9 [Daoyuan Wang] remove unused code
2015-07-06 15:54:43 -07:00
kai 2471c0bf7f [SPARK-4485] [SQL] 1) Add broadcast hash outer join, (2) Fix SparkPlanTest
This pull request
(1) extracts common functions used by hash outer joins and put it in interface HashOuterJoin
(2) adds ShuffledHashOuterJoin and BroadcastHashOuterJoin
(3) adds test cases for shuffled and broadcast hash outer join
(3) makes SparkPlanTest to support binary or more complex operators, and fixes bugs in plan composition in SparkPlanTest

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

Closes #7162 from kai-zeng/outer and squashes the following commits:

3742359 [kai] Fix not-serializable exception for code-generated keys in broadcasted relations
14e4bf8 [kai] Use CanBroadcast in broadcast outer join planning
dc5127e [kai] code style fixes
b5a4efa [kai] (1) Add broadcast hash outer join, (2) Fix SparkPlanTest
2015-07-06 14:33:30 -07:00
Davies Liu 37e4d92142 [SPARK-8784] [SQL] Add Python API for hex and unhex
Add Python API for hex/unhex,  also cleanup Hex/Unhex

Author: Davies Liu <davies@databricks.com>

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

6f1249d [Davies Liu] no explicit rule to cast string into binary
711a6ed [Davies Liu] fix test
f9fe5a3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex
f032fbb [Davies Liu] Merge branch 'hex' of github.com:davies/spark into hex
49e325f [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex
b31fc9a [Davies Liu] Update math.scala
25156b7 [Davies Liu] address comments and fix test
c3af78c [Davies Liu] address commments
1a24082 [Davies Liu] Add Python API for hex and unhex
2015-07-06 13:31:31 -07:00
Steve Lindemann 39e4e7e4d8 [SPARK-8841] [SQL] Fix partition pruning percentage log message
When pruning partitions for a query plan, a message is logged indicating what how many partitions were selected based on predicate criteria, and what percent were pruned.

The current release erroneously uses `1 - total/selected` to compute this quantity, leading to nonsense messages like "pruned -1000% partitions". The fix is simple and obvious.

Author: Steve Lindemann <steve.lindemann@engineersgatelp.com>

Closes #7227 from srlindemann/master and squashes the following commits:

c788061 [Steve Lindemann] fix percentPruned log message
2015-07-06 10:17:05 -07:00
Cheng Hao 6d0411b4f3 [SQL][Minor] Update the DataFrame API for encode/decode
This is a the follow up of #6843.

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

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

52cc553 [Cheng Hao] update the code as comment
2015-07-05 21:50:52 -07:00
Liang-Chi Hsieh 2b820f2a4b [MINOR] [SQL] Minor fix for CatalystSchemaConverter
ping liancheng

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

Closes #7224 from viirya/few_fix_catalystschema and squashes the following commits:

d994330 [Liang-Chi Hsieh] Minor fix for CatalystSchemaConverter.
2015-07-04 22:52:50 -07:00
Reynold Xin 48f7aed686 Fixed minor style issue with the previous merge. 2015-07-04 01:11:35 -07:00
Tarek Auel 6b3574e687 [SPARK-8270][SQL] levenshtein distance
Jira: https://issues.apache.org/jira/browse/SPARK-8270

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

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

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

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

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

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

78dee7d [Cheng Hao] base 64 -> base64
9d6f9f4 [Cheng Hao] remove the toString method for expressions
ed5c19c [Cheng Hao] update code as comments
96170fc [Cheng Hao] scalastyle issues
e2df768 [Cheng Hao] remove the unused import
491ce7b [Cheng Hao] add ascii/base64/unbase64/encode/decode functions
2015-07-03 23:45:21 -07:00
Daoyuan Wang 9fb6b832bc [SPARK-8192] [SPARK-8193] [SQL] udf current_date, current_timestamp
Author: Daoyuan Wang <daoyuan.wang@intel.com>

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

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

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

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

Author: Cheolsoo Park <cheolsoop@netflix.com>

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

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

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

6bcca5a [zhichao.li] change coding style
3e9f5ae [zhichao.li] python style
d85ae0b [zhichao.li] add shiftrightunsigned
2015-07-03 15:39:16 -07:00
Burak Yavuz 9b23e92c72 [SPARK-8803] handle special characters in elements in crosstab
cc rxin

Having back ticks or null as elements causes problems.
Since elements become column names, we have to drop them from the element as back ticks are special characters.
Having null throws exceptions, we could replace them with empty strings.

Handling back ticks should be improved for 1.5

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #7201 from brkyvz/weird-ct-elements and squashes the following commits:

e06b840 [Burak Yavuz] fix scalastyle
93a0d3f [Burak Yavuz] added tests for NaN and Infinity
9dba6ce [Burak Yavuz] address cr1
db71dbd [Burak Yavuz] handle special characters in elements in crosstab
2015-07-02 22:10:24 -07:00
zhichao.li 1a7a7d7d57 [SPARK-8213][SQL]Add function factorial
Author: zhichao.li <zhichao.li@intel.com>

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

26edf4f [zhichao.li] add factorial
2015-07-02 20:37:31 -07:00
Reynold Xin e589e71a29 Revert "[SPARK-8784] [SQL] Add Python API for hex and unhex"
This reverts commit fc7aebd94a.
2015-07-02 16:25:10 -07:00
Davies Liu fc7aebd94a [SPARK-8784] [SQL] Add Python API for hex and unhex
Also improve the performance of hex/unhex

Author: Davies Liu <davies@databricks.com>

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

f032fbb [Davies Liu] Merge branch 'hex' of github.com:davies/spark into hex
49e325f [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex
b31fc9a [Davies Liu] Update math.scala
25156b7 [Davies Liu] address comments and fix test
c3af78c [Davies Liu] address commments
1a24082 [Davies Liu] Add Python API for hex and unhex
2015-07-02 15:43:02 -07:00
Yijie Shen 52302a8039 [SPARK-8407] [SQL] complex type constructors: struct and named_struct
This is a follow up of [SPARK-8283](https://issues.apache.org/jira/browse/SPARK-8283) ([PR-6828](https://github.com/apache/spark/pull/6828)), to support both `struct` and `named_struct` in Spark SQL.

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

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

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

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

4cd3375ac [Yijie Shen] change struct documentation
d599d0b [Yijie Shen] rebase code
9a7039e [Yijie Shen] fix reviews and regenerate golden answers
b487354 [Yijie Shen] replace assert using checkAnswer
f07e114 [Yijie Shen] tiny fix
9613be9 [Yijie Shen] review fix
7fef712 [Yijie Shen] Fix checkInputTypes' implementation using foldable and nullable
60812a7 [Yijie Shen] Fix type check
828d694 [Yijie Shen] remove unnecessary resolved assertion inside dataType method
fd3cd8e [Yijie Shen] remove type check from eval
7a71255 [Yijie Shen] tiny fix
ccbbd86 [Yijie Shen] Fix reviews
47da332 [Yijie Shen] remove nameStruct API from DataFrame
917e680 [Yijie Shen] Fix reviews
4bd75ad [Yijie Shen] loosen struct method in functions.scala to take Expression children
0acb7be [Yijie Shen] Add CreateNamedStruct in both DataFrame function API and FunctionRegistery
2015-07-02 10:12:25 -07:00
Tarek Auel 5b3338130d [SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224

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

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

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

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

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

8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 10:02:19 -07:00
Wisely Chen 246265f2bb [SPARK-8690] [SQL] Add a setting to disable SparkSQL parquet schema merge by using datasource API
The detail problem story is in https://issues.apache.org/jira/browse/SPARK-8690

General speaking, I add a config spark.sql.parquet.mergeSchema to achieve the  sqlContext.load("parquet" , Map( "path" -> "..." , "mergeSchema" -> "false" ))

It will become a simple flag and without any side affect.

Author: Wisely Chen <wiselychen@appier.com>

Closes #7070 from thegiive/SPARK8690 and squashes the following commits:

c6f3e86 [Wisely Chen] Refactor some code style and merge the test case to ParquetSchemaMergeConfigSuite
94c9307 [Wisely Chen] Remove some style problem
db8ef1b [Wisely Chen] Change config to SQLConf and add test case
b6806fb [Wisely Chen] remove text
c0edb8c [Wisely Chen] [SPARK-8690] add a config spark.sql.parquet.mergeSchema to disable datasource API schema merge feature.
2015-07-02 09:58:12 -07:00
Vinod K C c572e25617 [SPARK-8787] [SQL] Changed parameter order of @deprecated in package object sql
Parameter order of deprecated annotation in package object sql is wrong
>>deprecated("1.3.0", "use DataFrame") .

This has to be changed to deprecated("use DataFrame", "1.3.0")

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

Closes #7183 from vinodkc/fix_deprecated_param_order and squashes the following commits:

1cbdbe8 [Vinod K C] Modified the message
700911c [Vinod K C] Changed order of parameters
2015-07-02 13:42:48 +01:00
Kousuke Saruta 41588365ad [DOCS] Fix minor wrong lambda expression example.
It's a really minor issue but there is an example with wrong lambda-expression usage in `SQLContext.scala` like as follows.

```
sqlContext.udf().register("myUDF",
       (Integer arg1, String arg2) -> arg2 + arg1),  <- We have an extra `)` here.
       DataTypes.StringType);
```

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

Closes #7187 from sarutak/fix-minor-wrong-lambda-expression and squashes the following commits:

a13196d [Kousuke Saruta] Fixed minor wrong lambda expression example.
2015-07-02 21:16:35 +09:00
zhichao.li b285ac5ba8 [SPARK-8227] [SQL] Add function unhex
cc chenghao-intel  adrian-wang

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

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

379356e [zhichao.li] remove exception checking
a4ae6dc [zhichao.li] add udf_unhex to whitelist
fe5c14a [zhichao.li] add todigit
607d7a3 [zhichao.li] use checkInputTypes
bffd37f [zhichao.li] change to use Hex in apache common package
cde73f5 [zhichao.li] update to use AutoCastInputTypes
11945c7 [zhichao.li] style
c852d46 [zhichao.li] Add function unhex
2015-07-01 22:19:51 -07:00
Reynold Xin 97652416e2 [SPARK-8750][SQL] Remove the closure in functions.callUdf.
Author: Reynold Xin <rxin@databricks.com>

Closes #7148 from rxin/calludf-closure and squashes the following commits:

00df372 [Reynold Xin] Fixed index out of bound exception.
4beba76 [Reynold Xin] [SPARK-8750][SQL] Remove the closure in functions.callUdf.
2015-07-01 01:08:20 -07:00
Wenchen Fan 0eee061589 [SQL] [MINOR] remove internalRowRDD in DataFrame
Developers have already familiar with `queryExecution.toRDD` as internal row RDD, and we should not add new concept.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #7116 from cloud-fan/internal-rdd and squashes the following commits:

24756ca [Wenchen Fan] remove internalRowRDD
2015-07-01 01:02:33 -07:00
Reynold Xin 8133125ca0 [SPARK-8741] [SQL] Remove e and pi from DataFrame functions.
Author: Reynold Xin <rxin@databricks.com>

Closes #7137 from rxin/SPARK-8741 and squashes the following commits:

32c7e75 [Reynold Xin] [SPARK-8741][SQL] Remove e and pi from DataFrame functions.
2015-06-30 16:54:51 -07:00
Davies Liu fbb267ed6f [SPARK-8713] Make codegen thread safe
Codegen takes three steps:

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

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

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

cc marmbrus rxin JoshRosen

Author: Davies Liu <davies@databricks.com>

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

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

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

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

5477352 [Shilei] Change to AutoCastInputTypes
5f16e5d [Shilei] Add misc function crc32
2015-06-30 09:49:58 -07:00
zsxwing 12671dd5e4 [SPARK-8434][SQL]Add a "pretty" parameter to the "show" method to display long strings
Sometimes the user may want to show the complete content of cells. Now `sql("set -v").show()` displays:

![screen shot 2015-06-18 at 4 34 51 pm](https://cloud.githubusercontent.com/assets/1000778/8227339/14d3c5ea-15d9-11e5-99b9-f00b7e93beef.png)

The user needs to use something like `sql("set -v").collect().foreach(r => r.toSeq.mkString("\t"))` to show the complete content.

This PR adds a `pretty` parameter to show. If `pretty` is false, `show` won't truncate strings or align cells right.

![screen shot 2015-06-18 at 4 21 44 pm](https://cloud.githubusercontent.com/assets/1000778/8227407/b6f8dcac-15d9-11e5-8219-8079280d76fc.png)

Author: zsxwing <zsxwing@gmail.com>

Closes #6877 from zsxwing/show and squashes the following commits:

22e28e9 [zsxwing] pretty -> truncate
e582628 [zsxwing] Add pretty parameter to the show method in R
a3cd55b [zsxwing] Fix calling showString in R
923cee4 [zsxwing] Add a "pretty" parameter to show to display long strings
2015-06-29 23:44:11 -07:00
Steven She 4915e9e3bf [SPARK-8669] [SQL] Fix crash with BINARY (ENUM) fields with Parquet 1.7
Patch to fix crash with BINARY fields with ENUM original types.

Author: Steven She <steven@canopylabs.com>

Closes #7048 from stevencanopy/SPARK-8669 and squashes the following commits:

2e72979 [Steven She] [SPARK-8669] [SQL] Fix crash with BINARY (ENUM) fields with Parquet 1.7
2015-06-29 18:50:09 -07:00
Davies Liu ed359de595 [SPARK-8579] [SQL] support arbitrary object in UnsafeRow
This PR brings arbitrary object support in UnsafeRow (both in grouping key and aggregation buffer).

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

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

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

cc JoshRosen rxin

Author: Davies Liu <davies@databricks.com>

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

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

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

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

c500f29 [BenFradet] renamed a few variables in functions to use UDF
8ab0f2d [BenFradet] renamed idUdf to idUDF in SQLQuerySuite
98696c2 [BenFradet] renamed originalUdfs in TestHive to originalUDFs
7738f74 [BenFradet] modified HiveUDFSuite to use only UDF
c52608d [BenFradet] renamed HiveUdfSuite to HiveUDFSuite
e51b9ac [BenFradet] renamed ExtractPythonUdfs to ExtractPythonUDFs
8c756f1 [BenFradet] renamed Hive UDF related code
2a1ca76 [BenFradet] renamed pythonUdfs to pythonUDFs
261e6fb [BenFradet] renamed ScalaUdf to ScalaUDF
2015-06-29 15:27:13 -07:00
Burak Yavuz be7ef06762 [SPARK-8681] fixed wrong ordering of columns in crosstab
I specifically randomized the test. What crosstab does is equivalent to a countByKey, therefore if this test fails again for any reason, we will know that we hit a corner case or something.

cc rxin marmbrus

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #7060 from brkyvz/crosstab-fixes and squashes the following commits:

0a65234 [Burak Yavuz] addressed comments v1
d96da7e [Burak Yavuz] fixed wrong ordering of columns in crosstab
2015-06-29 13:15:04 -07:00
zhichao.li 637b4eedad [SPARK-8214] [SQL] Add function hex
cc chenghao-intel  adrian-wang

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

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

e218d1b [zhichao.li] turn off scalastyle for non-ascii
de3f5ea [zhichao.li] non-ascii char
cf9c936 [zhichao.li] give separated buffer for each hex method
967ec90 [zhichao.li] Make 'value' as a feild of Hex
3b2fa13 [zhichao.li] tiny fix
a647641 [zhichao.li] remove duplicate null check
7cab020 [zhichao.li] tiny refactoring
35ecfe5 [zhichao.li] add function hex
2015-06-29 12:25:16 -07:00
Kousuke Saruta 94e040d059 [SQL][DOCS] Remove wrong example from DataFrame.scala
In DataFrame.scala, there are examples like as follows.

```
 * // The following are equivalent:
 * peopleDf.filter($"age" > 15)
 * peopleDf.where($"age" > 15)
 * peopleDf($"age" > 15)
```

But, I think the last example doesn't work.

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

Closes #6977 from sarutak/fix-dataframe-example and squashes the following commits:

46efbd7 [Kousuke Saruta] Removed wrong example
2015-06-29 12:16:12 -07:00
Tarek Auel a5c2961caa [SPARK-8235] [SQL] misc function sha / sha1
Jira: https://issues.apache.org/jira/browse/SPARK-8235

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

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

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

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

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

463044d [Wenchen Fan] fix style
51cd347 [Wenchen Fan] refactor handling of date and timestmap
2015-06-29 11:41:26 -07:00
Kousuke Saruta ec78438196 [SPARK-8686] [SQL] DataFrame should support where with expression represented by String
DataFrame supports `filter` function with two types of argument, `Column` and `String`. But `where` doesn't.

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

Closes #7063 from sarutak/SPARK-8686 and squashes the following commits:

180f9a4 [Kousuke Saruta] Added test
d61aec4 [Kousuke Saruta] Add "where" method with String argument to DataFrame
2015-06-28 08:29:07 -07:00
Davies Liu 77da5be6f1 [SPARK-8610] [SQL] Separate Row and InternalRow (part 2)
Currently, we use GenericRow both for Row and InternalRow, which is confusing because it could contain Scala type also Catalyst types.

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

Also fixes some incorrect use of InternalRow or Row.

Author: Davies Liu <davies@databricks.com>

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

8b16630 [Wenchen Fan] another fix
326c82c [Wenchen Fan] optimize type converter
2015-06-25 22:44:26 -07:00
Liang-Chi Hsieh 47c874babe [SPARK-8237] [SQL] Add misc function sha2
JIRA: https://issues.apache.org/jira/browse/SPARK-8237

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

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

35e0bb3 [Liang-Chi Hsieh] For comments.
68b5284 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_sha2
8573aff [Liang-Chi Hsieh] Remove unnecessary Product.
ee61e06 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_sha2
59e41aa [Liang-Chi Hsieh] Add misc function: sha2.
2015-06-25 22:07:37 -07:00
Cheng Lian c337844ed7 [SPARK-8604] [SQL] HadoopFsRelation subclasses should set their output format class
`HadoopFsRelation` subclasses, especially `ParquetRelation2` should set its own output format class, so that the default output committer can be setup correctly when doing appending (where we ignore user defined output committers).

Author: Cheng Lian <lian@databricks.com>

Closes #6998 from liancheng/spark-8604 and squashes the following commits:

9be51d1 [Cheng Lian] Adds more comments
6db1368 [Cheng Lian] HadoopFsRelation subclasses should set their output format class
2015-06-25 00:06:23 -07:00
Wenchen Fan b71d3254e5 [SPARK-8075] [SQL] apply type check interface to more expressions
a follow up of https://github.com/apache/spark/pull/6405.
Note: It's not a big change, a lot of changing is due to I swap some code in `aggregates.scala` to make aggregate functions right below its corresponding aggregate expressions.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

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

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

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

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

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

TODO

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

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

Author: Cheng Lian <lian@databricks.com>

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

34aa07b [Wenchen Fan] revert
07d5456 [Wenchen Fan] clean closure
20821ec [Wenchen Fan] fix
3676a82 [Wenchen Fan] address comments
b558549 [Wenchen Fan] address comments
214842b [Wenchen Fan] fix style
2d8be83 [Wenchen Fan] add LimitPushDown
948f740 [Wenchen Fan] fix existing
2015-06-24 13:28:50 -07:00
Yin Huai bba6699d0e [SPARK-8578] [SQL] Should ignore user defined output committer when appending data
https://issues.apache.org/jira/browse/SPARK-8578

It is not very safe to use a custom output committer when append data to an existing dir. This changes adds the logic to check if we are appending data, and if so, we use the output committer associated with the file output format.

Author: Yin Huai <yhuai@databricks.com>

Closes #6964 from yhuai/SPARK-8578 and squashes the following commits:

43544c4 [Yin Huai] Do not use a custom output commiter when appendiing data.
2015-06-24 09:50:03 -07:00
Cheng Lian cc465fd924 [SPARK-8138] [SQL] Improves error message when conflicting partition columns are found
This PR improves the error message shown when conflicting partition column names are detected.  This can be particularly annoying and confusing when there are a large number of partitions while a handful of them happened to contain unexpected temporary file(s).  Now all suspicious directories are listed as below:

```
java.lang.AssertionError: assertion failed: Conflicting partition column names detected:

        Partition column name list #0: b, c, d
        Partition column name list #1: b, c
        Partition column name list #2: b

For partitioned table directories, data files should only live in leaf directories. Please check the following directories for unexpected files:

        file:/tmp/foo/b=0
        file:/tmp/foo/b=1
        file:/tmp/foo/b=1/c=1
        file:/tmp/foo/b=0/c=0
```

Author: Cheng Lian <lian@databricks.com>

Closes #6610 from liancheng/part-errmsg and squashes the following commits:

7d05f2c [Cheng Lian] Fixes Scala style issue
a149250 [Cheng Lian] Adds test case for the error message
6b74dd8 [Cheng Lian] Also lists suspicious non-leaf partition directories
a935eb8 [Cheng Lian] Improves error message when conflicting partition columns are found
2015-06-24 02:17:12 -07:00
Reynold Xin a458efc66c Revert "[SPARK-7157][SQL] add sampleBy to DataFrame"
This reverts commit 0401cbaa8e.

The new test case on Jenkins is failing.
2015-06-23 19:30:25 -07:00
Xiangrui Meng 0401cbaa8e [SPARK-7157][SQL] add sampleBy to DataFrame
Add `sampleBy` to DataFrame. rxin

Author: Xiangrui Meng <meng@databricks.com>

Closes #6769 from mengxr/SPARK-7157 and squashes the following commits:

991f26f [Xiangrui Meng] fix seed
4a14834 [Xiangrui Meng] move sampleBy to stat
832f7cc [Xiangrui Meng] add sampleBy to DataFrame
2015-06-23 17:46:29 -07:00
Cheng Lian 111d6b9b8a [SPARK-8139] [SQL] Updates docs and comments of data sources and Parquet output committer options
This PR only applies to master branch (1.5.0-SNAPSHOT) since it references `org.apache.parquet` classes which only appear in Parquet 1.7.0.

Author: Cheng Lian <lian@databricks.com>

Closes #6683 from liancheng/output-committer-docs and squashes the following commits:

b4648b8 [Cheng Lian] Removes spark.sql.sources.outputCommitterClass as it's not a public option
ee63923 [Cheng Lian] Updates docs and comments of data sources and Parquet output committer options
2015-06-23 17:24:26 -07:00
Cheng Hao 7b1450b666 [SPARK-7235] [SQL] Refactor the grouping sets
The logical plan `Expand` takes the `output` as constructor argument, which break the references chain. We need to refactor the code, as well as the column pruning.

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

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

76e4aa4 [Cheng Hao] revert the change for case insenstive
7c10a83 [Cheng Hao] refactor the grouping sets
2015-06-23 10:52:17 -07:00
lockwobr 4f7fbefb8d [SQL] [DOCS] updated the documentation for explode
the syntax was incorrect in the example in explode

Author: lockwobr <lockwobr@gmail.com>

Closes #6943 from lockwobr/master and squashes the following commits:

3d864d1 [lockwobr] updated the documentation for explode
2015-06-24 02:48:56 +09:00
Reynold Xin 6ceb169608 [SPARK-8300] DataFrame hint for broadcast join.
Users can now do
```scala
left.join(broadcast(right), "joinKey")
```
to give the query planner a hint that "right" DataFrame is small and should be broadcasted.

Author: Reynold Xin <rxin@databricks.com>

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

953eec2 [Reynold Xin] Code review feedback.
88752d8 [Reynold Xin] Fixed import.
8187b88 [Reynold Xin] [SPARK-8300] DataFrame hint for broadcast join.
2015-06-23 01:50:31 -07:00
Davies Liu 6b7f2ceafd [SPARK-8307] [SQL] improve timestamp from parquet
This PR change to convert julian day to unix timestamp directly (without Calendar and Timestamp).

cc adrian-wang rxin

Author: Davies Liu <davies@databricks.com>

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

849e301 [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
b0e4cad [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
8e2d56f [Davies Liu] address comments
634b9f5 [Davies Liu] fix mima
4891efb [Davies Liu] address comment
bfc437c [Davies Liu] fix build
ae5979c [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
602b969 [Davies Liu] remove jodd
2f2e48c [Davies Liu] fix test
8ace611 [Davies Liu] fix mima
212143b [Davies Liu] fix mina
c834108 [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
a3171b8 [Davies Liu] Merge branch 'master' of github.com:apache/spark into improve_ts
5233974 [Davies Liu] fix scala style
361fd62 [Davies Liu] address comments
ea196d4 [Davies Liu] improve timestamp from parquet
2015-06-22 18:03:59 -07:00
BenFradet 50d3242d6a [SPARK-8356] [SQL] Reconcile callUDF and callUdf
Deprecates ```callUdf``` in favor of ```callUDF```.

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

Closes #6902 from BenFradet/SPARK-8356 and squashes the following commits:

ef4e9d8 [BenFradet] deprecated callUDF, use udf instead
9b1de4d [BenFradet] reinstated unit test for the deprecated callUdf
cbd80a5 [BenFradet] deprecated callUdf in favor of callUDF
2015-06-22 15:06:47 -07:00
Wenchen Fan da7bbb9435 [SPARK-8104] [SQL] auto alias expressions in analyzer
Currently we auto alias expression in parser. However, during parser phase we don't have enough information to do the right alias. For example, Generator that has more than 1 kind of element need MultiAlias, ExtractValue don't need Alias if it's in middle of a ExtractValue chain.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

552eba4 [Wenchen Fan] fix python
5b5786d [Wenchen Fan] fix agg
73a90cb [Wenchen Fan] fix case-preserve of ExtractValue
4cfd23c [Wenchen Fan] fix order by
d18f401 [Wenchen Fan] refine
9f07359 [Wenchen Fan] address comments
39c1aef [Wenchen Fan] small fix
33640ec [Wenchen Fan] auto alias expressions in analyzer
2015-06-22 12:13:00 -07:00
Cheng Lian 0818fdec37 [SPARK-8406] [SQL] Adding UUID to output file name to avoid accidental overwriting
This PR fixes a Parquet output file name collision bug which may cause data loss.  Changes made:

1.  Identify each write job issued by `InsertIntoHadoopFsRelation` with a UUID

    All concrete data sources which extend `HadoopFsRelation` (Parquet and ORC for now) must use this UUID to generate task output file path to avoid name collision.

2.  Make `TestHive` use a local mode `SparkContext` with 32 threads to increase parallelism

    The major reason for this is that, the original parallelism of 2 is too low to reproduce the data loss issue.  Also, higher concurrency may potentially caught more concurrency bugs during testing phase. (It did help us spotted SPARK-8501.)

3. `OrcSourceSuite` was updated to workaround SPARK-8501, which we detected along the way.

NOTE: This PR is made a little bit more complicated than expected because we hit two other bugs on the way and have to work them around. See [SPARK-8501] [1] and [SPARK-8513] [2].

[1]: https://github.com/liancheng/spark/tree/spark-8501
[2]: https://github.com/liancheng/spark/tree/spark-8513

----

Some background and a summary of offline discussion with yhuai about this issue for better understanding:

In 1.4.0, we added `HadoopFsRelation` to abstract partition support of all data sources that are based on Hadoop `FileSystem` interface.  Specifically, this makes partition discovery, partition pruning, and writing dynamic partitions for data sources much easier.

To support appending, the Parquet data source tries to find out the max part number of part-files in the destination directory (i.e., `<id>` in output file name `part-r-<id>.gz.parquet`) at the beginning of the write job.  In 1.3.0, this step happens on driver side before any files are written.  However, in 1.4.0, this is moved to task side.  Unfortunately, for tasks scheduled later, they may see wrong max part number generated of files newly written by other finished tasks within the same job.  This actually causes a race condition.  In most cases, this only causes nonconsecutive part numbers in output file names.  But when the DataFrame contains thousands of RDD partitions, it's likely that two tasks may choose the same part number, then one of them gets overwritten by the other.

Before `HadoopFsRelation`, Spark SQL already supports appending data to Hive tables.  From a user's perspective, these two look similar.  However, they differ a lot internally.  When data are inserted into Hive tables via Spark SQL, `InsertIntoHiveTable` simulates Hive's behaviors:

1.  Write data to a temporary location

2.  Move data in the temporary location to the final destination location using

    -   `Hive.loadTable()` for non-partitioned table
    -   `Hive.loadPartition()` for static partitions
    -   `Hive.loadDynamicPartitions()` for dynamic partitions

The important part is that, `Hive.copyFiles()` is invoked in step 2 to move the data to the destination directory (I found the name is kinda confusing since no "copying" occurs here, we are just moving and renaming stuff).  If a file in the source directory and another file in the destination directory happen to have the same name, say `part-r-00001.parquet`, the former is moved to the destination directory and renamed with a `_copy_N` postfix (`part-r-00001_copy_1.parquet`).  That's how Hive handles appending and avoids name collision between different write jobs.

Some alternatives fixes considered for this issue:

1.  Use a similar approach as Hive

    This approach is not preferred in Spark 1.4.0 mainly because file metadata operations in S3 tend to be slow, especially for tables with lots of file and/or partitions.  That's why `InsertIntoHadoopFsRelation` just inserts to destination directory directly, and is often used together with `DirectParquetOutputCommitter` to reduce latency when working with S3.  This means, we don't have the chance to do renaming, and must avoid name collision from the very beginning.

2.  Same as 1.3, just move max part number detection back to driver side

    This isn't doable because unlike 1.3, 1.4 also takes dynamic partitioning into account.  When inserting into dynamic partitions, we don't know which partition directories will be touched on driver side before issuing the write job.  Checking all partition directories is simply too expensive for tables with thousands of partitions.

3.  Add extra component to output file names to avoid name collision

    This seems to be the only reasonable solution for now.  To be more specific, we need a JOB level unique identifier to identify all write jobs issued by `InsertIntoHadoopFile`.  Notice that TASK level unique identifiers can NOT be used.  Because in this way a speculative task will write to a different output file from the original task.  If both tasks succeed, duplicate output will be left behind.  Currently, the ORC data source adds `System.currentTimeMillis` to the output file name for uniqueness.  This doesn't work because of exactly the same reason.

    That's why this PR adds a job level random UUID in `BaseWriterContainer` (which is used by `InsertIntoHadoopFsRelation` to issue write jobs).  The drawback is that record order is not preserved any more (output files of a later job may be listed before those of a earlier job).  However, we never promise to preserve record order when writing data, and Hive doesn't promise this either because the `_copy_N` trick breaks the order.

Author: Cheng Lian <lian@databricks.com>

Closes #6864 from liancheng/spark-8406 and squashes the following commits:

db7a46a [Cheng Lian] More comments
f5c1133 [Cheng Lian] Addresses comments
85c478e [Cheng Lian] Workarounds SPARK-8513
088c76c [Cheng Lian] Adds comment about SPARK-8501
99a5e7e [Cheng Lian] Uses job level UUID in SimpleTextRelation and avoids double task abortion
4088226 [Cheng Lian] Works around SPARK-8501
1d7d206 [Cheng Lian] Adds more logs
8966bbb [Cheng Lian] Fixes Scala style issue
18b7003 [Cheng Lian] Uses job level UUID to take speculative tasks into account
3806190 [Cheng Lian] Lets TestHive use all cores by default
748dbd7 [Cheng Lian] Adding UUID to output file name to avoid accidental overwriting
2015-06-22 10:03:57 -07:00
Nathan Howell 9814b971f0 [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents
Author: Nathan Howell <nhowell@godaddy.com>

Closes #6799 from NathanHowell/spark-8093 and squashes the following commits:

76ac3e8 [Nathan Howell] [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents
2015-06-19 16:19:28 -07:00
Shilei 0c32fc125c [SPARK-8234][SQL] misc function: md5
Author: Shilei <shilei.qian@intel.com>

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

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

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

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

07e1c8f [Liang-Chi Hsieh] Remove AbstractUnaryMathExpression and let BIN inherit UnaryExpression.
0677f1a [Liang-Chi Hsieh] For comments.
cf62b95 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
0cf20f2 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
dea9c12 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
d4f4774 [Liang-Chi Hsieh] Add @ignore_unicode_prefix.
7a0196f [Liang-Chi Hsieh] Fix python style.
ac2bacd [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
a0a2d0f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
4cb764d [Liang-Chi Hsieh] For comments.
0f78682 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
c0c3197 [Liang-Chi Hsieh] Add bin to FunctionRegistry.
824f761 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_bin
50e0c3b [Liang-Chi Hsieh] Add math function bin(a: long): string.
2015-06-19 10:09:31 -07:00
Yu ISHIKAWA 754929b153 [SPARK-8348][SQL] Add in operator to DataFrame Column
I have added it for only Scala.

TODO: we should also support `in` operator in Python.

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #6824 from yu-iskw/SPARK-8348 and squashes the following commits:

e76d02f [Yu ISHIKAWA] Not use infix notation
6f744ac [Yu ISHIKAWA] Fit the test cases because these used the old test data set.
00077d3 [Yu ISHIKAWA] [SPARK-8348][SQL] Add in operator to DataFrame Column
2015-06-18 23:13:05 -07:00
Sandy Ryza 43f50decdd [SPARK-8135] Don't load defaults when reconstituting Hadoop Configurations
Author: Sandy Ryza <sandy@cloudera.com>

Closes #6679 from sryza/sandy-spark-8135 and squashes the following commits:

c5554ff [Sandy Ryza] SPARK-8135. In SerializableWritable, don't load defaults when instantiating Configuration
2015-06-18 19:36:05 -07:00
Liang-Chi Hsieh 31641128b3 [SPARK-8363][SQL] Move sqrt to math and extend UnaryMathExpression
JIRA: https://issues.apache.org/jira/browse/SPARK-8363

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

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

8977e11 [Liang-Chi Hsieh] Remove unnecessary old tests.
d23e79e [Liang-Chi Hsieh] Explicitly indicate sqrt value sequence.
699f48b [Liang-Chi Hsieh] Use correct @since tag.
8dff6d1 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into move_sqrt
bc2ed77 [Liang-Chi Hsieh] Remove/move arithmetic expression test and expression type checking test. Remove unnecessary Sqrt type rule.
d38492f [Liang-Chi Hsieh] Now sqrt accepts boolean because type casting is handled by HiveTypeCoercion.
297cc90 [Liang-Chi Hsieh] Sqrt only accepts double input.
ef4a21a [Liang-Chi Hsieh] Move sqrt to math.
2015-06-18 13:00:31 -07:00
Liang-Chi Hsieh fee3438a32 [SPARK-8218][SQL] Add binary log math function
JIRA: https://issues.apache.org/jira/browse/SPARK-8218

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

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

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

bf96bd9 [Liang-Chi Hsieh] Compare log result in string.
102070d [Liang-Chi Hsieh] Round log result to better comparing in python test.
fd01863 [Liang-Chi Hsieh] For comments.
beed631 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
6089d11 [Liang-Chi Hsieh] Remove unnecessary override.
8cf37b7 [Liang-Chi Hsieh] For comments.
bc89597 [Liang-Chi Hsieh] For comments.
db7dc38 [Liang-Chi Hsieh] Use ctor instead of companion object.
0634ef7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
1750034 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
3d75bfc [Liang-Chi Hsieh] Fix scala style.
5b39c02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
23c54a3 [Liang-Chi Hsieh] Fix scala style.
ebc9929 [Liang-Chi Hsieh] Let Logarithm accept one parameter too.
605574d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
21c3bfd [Liang-Chi Hsieh] Fix scala style.
c6c187f [Liang-Chi Hsieh] For comments.
c795342 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log
f373bac [Liang-Chi Hsieh] Add binary log expression.
2015-06-17 23:31:30 -07:00
zsxwing 78a430ea4d [SPARK-7961][SQL]Refactor SQLConf to display better error message
1. Add `SQLConfEntry` to store the information about a configuration. For those configurations that cannot be found in `sql-programming-guide.md`, I left the doc as `<TODO>`.
2. Verify the value when setting a configuration if this is in SQLConf.
3. Use `SET -v` to display all public configurations.

Author: zsxwing <zsxwing@gmail.com>

Closes #6747 from zsxwing/sqlconf and squashes the following commits:

7d09bad [zsxwing] Use SQLConfEntry in HiveContext
49f6213 [zsxwing] Add getConf, setConf to SQLContext and HiveContext
e014f53 [zsxwing] Merge branch 'master' into sqlconf
93dad8e [zsxwing] Fix the unit tests
cf950c1 [zsxwing] Fix the code style and tests
3c5f03e [zsxwing] Add unsetConf(SQLConfEntry) and fix the code style
a2f4add [zsxwing] getConf will return the default value if a config is not set
037b1db [zsxwing] Add schema to SetCommand
0520c3c [zsxwing] Merge branch 'master' into sqlconf
7afb0ec [zsxwing] Fix the configurations about HiveThriftServer
7e728e3 [zsxwing] Add doc for SQLConfEntry and fix 'toString'
5e95b10 [zsxwing] Add enumConf
c6ba76d [zsxwing] setRawString => setConfString, getRawString => getConfString
4abd807 [zsxwing] Fix the test for 'set -v'
6e47e56 [zsxwing] Fix the compilation error
8973ced [zsxwing] Remove floatConf
1fc3a8b [zsxwing] Remove the 'conf' command and use 'set -v' instead
99c9c16 [zsxwing] Fix tests that use SQLConfEntry as a string
88a03cc [zsxwing] Add new lines between confs and return types
ce7c6c8 [zsxwing] Remove seqConf
f3c1b33 [zsxwing] Refactor SQLConf to display better error message
2015-06-17 23:22:54 -07:00
Lianhui Wang 9db73ec124 [SPARK-8381][SQL]reuse typeConvert when convert Seq[Row] to catalyst type
reuse-typeConvert when convert Seq[Row] to CatalystType

Author: Lianhui Wang <lianhuiwang09@gmail.com>

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

1fec395 [Lianhui Wang] remove CatalystTypeConverters.convertToCatalyst
714462d [Lianhui Wang] add package[sql]
9d1fbf3 [Lianhui Wang] address JoshRosen's comments
768956f [Lianhui Wang] update scala style
4498c62 [Lianhui Wang] reuse typeConvert
2015-06-17 22:52:47 -07:00
Radek Ostrowski 4bd10fd509 [SQL] [DOC] improved a comment
[SQL][DOC] I found it a bit confusing when I came across it for the first time in the docs

Author: Radek Ostrowski <dest.hawaii@gmail.com>
Author: radek <radek@radeks-MacBook-Pro-2.local>

Closes #6332 from radek1st/master and squashes the following commits:

dae3347 [Radek Ostrowski] fixed typo
c76bb3a [radek] improved a comment
2015-06-16 21:04:26 +01:00
Davies Liu bc76a0f750 [SPARK-7184] [SQL] enable codegen by default
In order to have better performance out of box, this PR turn on codegen by default, then codegen can be tested by sql/test and hive/test.

This PR also fix some corner cases for codegen.

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

cc rxin JoshRosen

Author: Davies Liu <davies@databricks.com>

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

f3b25a5 [Davies Liu] fix warning
73750ea [Davies Liu] fix long overflow when compare
3017a47 [Davies Liu] Merge branch 'master' of github.com:apache/spark into enable_codegen
a7d75da [Davies Liu] Merge branch 'master' of github.com:apache/spark into enable_codegen
ff5b75a [Davies Liu] Merge branch 'master' of github.com:apache/spark into enable_codegen
f4cf2c2 [Davies Liu] fix style
99fc139 [Davies Liu] Merge branch 'enable_codegen' of github.com:davies/spark into enable_codegen
91fc7a2 [Davies Liu] disable codegen for ScalaUDF
207e339 [Davies Liu] Update CodeGenerator.scala
44573a3 [Davies Liu] check thread safety of expression
f3886fa [Davies Liu] don't inline primitiveTerm for null literal
c8e7cd2 [Davies Liu] address comment
a8618c9 [Davies Liu] enable codegen by default
2015-06-15 23:03:14 -07:00
Josh Rosen af31335adc [SPARK-8319] [CORE] [SQL] Update logic related to key orderings in shuffle dependencies
This patch updates two pieces of logic that are related to handling of keyOrderings in ShuffleDependencies:

- The Tungsten ShuffleManager falls back to regular SortShuffleManager whenever the shuffle dependency specifies a key ordering, but technically we only need to fall back when an aggregator is also specified. This patch updates the fallback logic to reflect this so that the Tungsten optimizations can apply to more workloads.

- The SQL Exchange operator performs defensive copying of shuffle inputs when a key ordering is specified, but this is unnecessary. The copying was added to guard against cases where ExternalSorter would buffer non-serialized records in memory.  When ExternalSorter is configured without an aggregator, it uses the following logic to determine whether to buffer records in a serialized or deserialized format:

   ```scala
     private val useSerializedPairBuffer =
        ordering.isEmpty &&
        conf.getBoolean("spark.shuffle.sort.serializeMapOutputs", true) &&
        ser.supportsRelocationOfSerializedObjects
   ```

   The `newOrdering.isDefined` branch in `ExternalSorter.needToCopyObjectsBeforeShuffle`, removed by this patch, is not necessary:

   - It was checked even if we weren't using sort-based shuffle, but this was unnecessary because only SortShuffleManager performs map-side sorting.
   - Map-side sorting during shuffle writing is only performed for shuffles that perform map-side aggregation as part of the shuffle (to see this, look at how SortShuffleWriter constructs ExternalSorter).  Since SQL never pushes aggregation into Spark's shuffle, we can guarantee that both the aggregator and ordering will be empty and Spark SQL always uses serializers that support relocation, so sort-shuffle will use the serialized pair buffer unless the user has explicitly disabled it via the SparkConf feature-flag.  Therefore, I think my optimization in Exchange should be safe.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6773 from JoshRosen/SPARK-8319 and squashes the following commits:

7a14129 [Josh Rosen] Revise comments; add handler to guard against future ShuffleManager implementations
07bb2c9 [Josh Rosen] Update comment to clarify circumstances under which shuffle operates on serialized records
269089a [Josh Rosen] Avoid unnecessary copy in SQL Exchange
34e526e [Josh Rosen] Enable Tungsten shuffle for non-agg shuffles w/ key orderings
2015-06-13 16:14:24 -07:00
Davies Liu ce1041c38f [SPARK-8346] [SQL] Use InternalRow instread of catalyst.InternalRow
cc rxin marmbrus

Author: Davies Liu <davies@databricks.com>

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

769d2aa [Davies Liu] remove not needed cast
4acbbe4 [Davies Liu] catalyst.Internal -> InternalRow
2015-06-13 16:13:26 -07:00
Rene Treffer d986fb9a37 [SPARK-7897] Improbe type for jdbc/"unsigned bigint"
The original fix uses DecimalType.Unlimited, which is harder to
handle afterwards. There is no scale and most data should fit into
a long, thus DecimalType(20,0) should be better.

Author: Rene Treffer <treffer@measite.de>

Closes #6789 from rtreffer/spark-7897-unsigned-bigint-as-decimal and squashes the following commits:

2006613 [Rene Treffer] Fix type for "unsigned bigint" jdbc loading.
2015-06-13 11:58:22 -07:00
Michael Armbrust 4aed66f299 [SPARK-8329][SQL] Allow _ in DataSource options
Author: Michael Armbrust <michael@databricks.com>

Closes #6786 from marmbrus/optionsParser and squashes the following commits:

e7d18ef [Michael Armbrust] add dots
99a3452 [Michael Armbrust] [SPARK-8329][SQL] Allow _ in DataSource options
2015-06-12 23:11:16 -07:00
Davies Liu d46f8e5d4b [SPARK-7186] [SQL] Decouple internal Row from external Row
Currently, we use o.a.s.sql.Row both internally and externally. The external interface is wider than what the internal needs because it is designed to facilitate end-user programming. This design has proven to be very error prone and cumbersome for internal Row implementations.

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

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

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

cc rxin marmbrus JoshRosen

Author: Davies Liu <davies@databricks.com>

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

f2abd13 [Davies Liu] fix scalastyle
a7e025c [Davies Liu] move InternalRow into catalyst
30db8ba [Davies Liu] Merge branch 'master' of github.com:apache/spark into internal_row
7cbced8 [Davies Liu] separate Row and InternalRow
2015-06-12 23:06:31 -07:00
akhilthatipamula 19834fa918 [SPARK-7993] [SQL] Improved DataFrame.show() output
Closes #6633

Author: akhilthatipamula <130050068@iitb.ac.in>
Author: zsxwing <zsxwing@gmail.com>

Closes #6784 from zsxwing/pr6633 and squashes the following commits:

5da1c51 [zsxwing] Address comments and add unit tests
17eab7b [akhilthatipamula] refactored code
19874b3 [akhilthatipamula] Update DataFrame.scala
0a76a5e [akhilthatipamula] Optimised showString()
e3dd03f [akhilthatipamula] Modified showString() method
a21012b [akhilthatipamula] improved the show()
4bb742f [akhilthatipamula] Modified dataframe.show() method
2015-06-12 10:40:28 -07:00
Wenchen Fan c19c78577a [SQL] [MINOR] correct semanticEquals logic
It's a follow up of https://github.com/apache/spark/pull/6173, for expressions like `Coalesce` that have a `Seq[Expression]`, when we do semantic equal check for it, we need to do semantic equal check for all of its children.
Also we can just use `Seq[(Expression, NamedExpression)]` instead of `Map[Expression, NamedExpression]` as we only search it with `find`.

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

4daef88 [Wenchen Fan] address comments
dd8fbd9 [Wenchen Fan] correct semanticEquals
2015-06-12 16:38:28 +08:00
Yash Datta e428b3a951 [SPARK-6566] [SQL] Related changes for newer parquet version
This brings in major improvement in that footers are not read on the driver. This also cleans up the code in parquetTableOperations, where we had to override getSplits to eliminate multiple listStatus calls.

cc liancheng

are there any other changes we need for this ?

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

Closes #5889 from saucam/parquet_1.6 and squashes the following commits:

d1bf41e [Yash Datta] SPARK-7340: Fix scalastyle and incorporate review comments
c9aa042 [Yash Datta] SPARK-7340: Use the new user defined filter predicate for pushing down inset into parquet
56bc750 [Yash Datta] SPARK-7340: Change parquet version to latest release
2015-06-12 13:44:09 +08:00
Josh Rosen b9d177c511 [SPARK-8317] [SQL] Do not push sort into shuffle in Exchange operator
In some cases, Spark SQL pushes sorting operations into the shuffle layer by specifying a key ordering as part of the shuffle dependency. I think that we should not do this:

- Since we do not delegate aggregation to Spark's shuffle, specifying the keyOrdering as part of the shuffle has no effect on the shuffle map side.
- By performing the shuffle ourselves (by inserting a sort operator after the shuffle instead), we can use the Exchange planner to choose specialized sorting implementations based on the types of rows being sorted.
- We can remove some complexity from SqlSerializer2 by not requiring it to know about sort orderings, since SQL's own sort operators will already perform the necessary defensive copying.

This patch removes Exchange's `canSortWithShuffle` path and the associated code in `SqlSerializer2`.  Shuffles that used to go through the `canSortWithShuffle` path would always wind up using Spark's `ExternalSorter` (inside of `HashShuffleReader`); to avoid a performance regression as a result of handling these shuffles ourselves, I've changed the SQLConf defaults so that external sorting is enabled by default.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6772 from JoshRosen/SPARK-8317 and squashes the following commits:

ebf9c0f [Josh Rosen] Do not push sort into shuffle in Exchange operator
bf3b4c8 [Josh Rosen] Enable external sort by default
2015-06-11 22:15:15 -07:00
Cheng Hao 767cc94ca6 [SPARK-7158] [SQL] Fix bug of cached data cannot be used in collect() after cache()
When df.cache() method called, the `withCachedData` of `QueryExecution` has been created, which mean it will not look up the cached tables when action method called afterward.

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

Closes #5714 from chenghao-intel/SPARK-7158 and squashes the following commits:

58ea8aa [Cheng Hao] style issue
2bf740f [Cheng Hao] create new QueryExecution instance for CacheManager
a5647d9 [Cheng Hao] hide the queryExecution of DataFrame
fbfd3c5 [Cheng Hao] make the DataFrame.queryExecution mutable for cache/persist/unpersist
2015-06-11 18:01:47 -07:00
Reynold Xin 7d669a56ff [SPARK-8286] Rewrite UTF8String in Java and move it into unsafe package.
Unit test is still in Scala.

Author: Reynold Xin <rxin@databricks.com>

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

562dc6e [Reynold Xin] Flag...
98e600b [Reynold Xin] Another try with encoding setting ..
cfa6bdf [Reynold Xin] Merge branch 'master' into utf8string-java
a3b124d [Reynold Xin] Try different UTF-8 encoded characters.
1ff7c82 [Reynold Xin] Enable UTF-8 encoding.
82d58cc [Reynold Xin] Reset run-tests.
2cb3c69 [Reynold Xin] Use utf-8 encoding in set bytes.
53f8ef4 [Reynold Xin] Hack Jenkins to run one test.
9a48e8d [Reynold Xin] Fixed runtime compilation error.
911c450 [Reynold Xin] Moved unit test also to Java.
4eff7bd [Reynold Xin] Improved unit test coverage.
8e89a3c [Reynold Xin] Fixed tests.
77c64bd [Reynold Xin] Fixed string type codegen.
ffedb62 [Reynold Xin] Code review feedback.
0967ce6 [Reynold Xin] Fixed import ordering.
45a123d [Reynold Xin] [SPARK-8286] Rewrite UTF8String in Java and move it into unsafe package.
2015-06-11 16:07:15 -07:00
Davies Liu 424b0075a1 [SPARK-6411] [SQL] [PySpark] support date/datetime with timezone in Python
Spark SQL does not support timezone, and Pyrolite does not support timezone well. This patch will convert datetime into POSIX timestamp (without confusing of timezone), which is used by SQL. If the datetime object does not have timezone, it's treated as local time.

The timezone in RDD will be lost after one round trip, all the datetime from SQL will be local time.

Because of Pyrolite, datetime from SQL only has precision as 1 millisecond.

This PR also drop the timezone in date, convert it to number of days since epoch (used in SQL).

Author: Davies Liu <davies@databricks.com>

Closes #6250 from davies/tzone and squashes the following commits:

44d8497 [Davies Liu] add timezone support for DateType
99d9d9c [Davies Liu] use int for timestamp
10aa7ca [Davies Liu] Merge branch 'master' of github.com:apache/spark into tzone
6a29aa4 [Davies Liu] support datetime with timezone
2015-06-11 01:00:41 -07:00
Daoyuan Wang 2758ff0a96 [SPARK-8217] [SQL] math function log2
Author: Daoyuan Wang <daoyuan.wang@intel.com>

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

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

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

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

60a1193 [Wenchen Fan] support nested expression sequence and remove GroupExpression
2015-06-10 18:22:47 -07:00
navis.ryu 6a47114bc2 [SPARK-8285] [SQL] CombineSum should be calculated as unlimited decimal first
case cs  CombineSum(expr) =>
        val calcType = expr.dataType
          expr.dataType match {
            case DecimalType.Fixed(_, _) =>
              DecimalType.Unlimited
            case _ =>
              expr.dataType
          }
calcType is always expr.dataType. credits are all belong to IntelliJ

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

Closes #6736 from navis/SPARK-8285 and squashes the following commits:

20382c1 [navis.ryu] [SPARK-8285] [SQL] CombineSum should be calculated as unlimited decimal first
2015-06-10 18:19:12 -07:00
Davies Liu 37719e0cd0 [SPARK-8189] [SQL] use Long for TimestampType in SQL
This PR change to use Long as internal type for TimestampType for efficiency, which means it will the precision below 100ns.

Author: Davies Liu <davies@databricks.com>

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

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

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

e2e8dbd [Daoyuan Wang] move tests
11b351c [Daoyuan Wang] add tests and remove pu
db331c9 [Daoyuan Wang] py style
599ddd8 [Daoyuan Wang] add py
e6783ef [Daoyuan Wang] register function
82d426e [Daoyuan Wang] add function entry
dbf3ab5 [Daoyuan Wang] add PI and E
2015-06-10 09:45:45 -07:00
Reynold Xin 57c60c5be7 [SPARK-7886] Use FunctionRegistry for built-in expressions in HiveContext.
This builds on #6710 and also uses FunctionRegistry for function lookup in HiveContext.

Author: Reynold Xin <rxin@databricks.com>

Closes #6712 from rxin/udf-registry-hive and squashes the following commits:

f4c2df0 [Reynold Xin] Fixed style violation.
0bd4127 [Reynold Xin] Fixed Python UDFs.
f9a0378 [Reynold Xin] Disable one more test.
5609494 [Reynold Xin] Disable some failing tests.
4efea20 [Reynold Xin] Don't check children resolved for UDF resolution.
2ebe549 [Reynold Xin] Removed more hardcoded functions.
aadce78 [Reynold Xin] [SPARK-7886] Use FunctionRegistry for built-in expressions in HiveContext.
2015-06-10 00:36:16 -07:00
Reynold Xin 1b499993ad [SPARK-7886] Add built-in expressions to FunctionRegistry.
This patch switches to using FunctionRegistry for built-in expressions. It is based on #6463, but with some work to simplify it along with unit tests.

TODOs for future pull requests:
- Use static registration so we don't need to register all functions every time we start a new SQLContext
- Switch to using this in HiveContext

Author: Reynold Xin <rxin@databricks.com>
Author: Santiago M. Mola <santi@mola.io>

Closes #6710 from rxin/udf-registry and squashes the following commits:

6930822 [Reynold Xin] Fixed Python test.
b802c9a [Reynold Xin] Made UDF case insensitive.
e60d815 [Reynold Xin] Made UDF case insensitive.
852f9c0 [Reynold Xin] Fixed style violation.
e76a3c1 [Reynold Xin] Fixed parser.
52ddaba [Reynold Xin] Fixed compilation.
ee7854f [Reynold Xin] Improved error reporting.
ff906f2 [Reynold Xin] More robust constructor calling.
77b46f1 [Reynold Xin] Simplified the code.
2a2a149 [Reynold Xin] Merge pull request #6463 from smola/SPARK-7886
8616924 [Santiago M. Mola] [SPARK-7886] Add built-in expressions to FunctionRegistry.
2015-06-09 16:24:38 +08:00
Liang-Chi Hsieh 7658eb28a2 [SPARK-7990][SQL] Add methods to facilitate equi-join on multiple joining keys
JIRA: https://issues.apache.org/jira/browse/SPARK-7990

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

Closes #6616 from viirya/multi_keys_equi_join and squashes the following commits:

cd5c888 [Liang-Chi Hsieh] Import reduce in python3.
c43722c [Liang-Chi Hsieh] For comments.
0400e89 [Liang-Chi Hsieh] Fix scala style.
cc90015 [Liang-Chi Hsieh] Add methods to facilitate equi-join on multiple joining keys.
2015-06-08 23:27:05 -07:00
Reynold Xin 5185389168 [SPARK-8148] Do not use FloatType in partition column inference.
Use DoubleType instead to be more stable and robust.

Author: Reynold Xin <rxin@databricks.com>

Closes #6692 from rxin/SPARK-8148 and squashes the following commits:

6742ecc [Reynold Xin] [SPARK-8148] Do not use FloatType in partition column inference.
2015-06-08 13:15:44 -07:00
Cheng Lian bbdfc0a40f [SPARK-8121] [SQL] Fixes InsertIntoHadoopFsRelation job initialization for Hadoop 1.x
For Hadoop 1.x, `TaskAttemptContext` constructor clones the `Configuration` argument, thus configurations done in `HadoopFsRelation.prepareForWriteJob()` are not populated to *driver* side `TaskAttemptContext` (executor side configurations are properly populated). Currently this should only affect Parquet output committer class configuration.

Author: Cheng Lian <lian@databricks.com>

Closes #6669 from liancheng/spark-8121 and squashes the following commits:

73819e8 [Cheng Lian] Minor logging fix
fce089c [Cheng Lian] Adds more logging
b6f78a6 [Cheng Lian] Fixes compilation error introduced while rebasing
963a1aa [Cheng Lian] Addresses @yhuai's comment
c3a0b1a [Cheng Lian] Fixes InsertIntoHadoopFsRelation job initialization
2015-06-08 11:34:18 -07:00
Daoyuan Wang 49f19b954b [MINOR] change new Exception to IllegalArgumentException
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #6434 from adrian-wang/joinerr and squashes the following commits:

ee1b64f [Daoyuan Wang] break line
f7c53e9 [Daoyuan Wang] to IllegalArgumentException
f8dea2d [Daoyuan Wang] sys.err to IllegalStateException
be82259 [Daoyuan Wang] change new exception to sys.err
2015-06-08 09:41:06 -07:00
Liang-Chi Hsieh 03ef6be9ce [SPARK-7939] [SQL] Add conf to enable/disable partition column type inference
JIRA: https://issues.apache.org/jira/browse/SPARK-7939

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

Closes #6503 from viirya/disable_partition_type_inference and squashes the following commits:

3e90470 [Liang-Chi Hsieh] Default to enable type inference and update docs.
455edb1 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into disable_partition_type_inference
9a57933 [Liang-Chi Hsieh] Add conf to enable/disable partition column type inference.
2015-06-08 17:50:38 +08:00
Reynold Xin f74be744d4 [SPARK-8149][SQL] Break ExpressionEvaluationSuite down to multiple files
Also moved a few files in expressions package around to match test suites.

Author: Reynold Xin <rxin@databricks.com>

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

857599f [Reynold Xin] Fixed style violation.
c0eb74b [Reynold Xin] Fixed compilation.
b3a40f8 [Reynold Xin] Refactored expression test suites.
2015-06-07 18:45:24 -07:00
Reynold Xin d6d601a07b [SPARK-8004][SQL] Quote identifier in JDBC data source.
This is a follow-up patch to #6577 to replace columnEnclosing to quoteIdentifier.

I also did some minor cleanup to the JdbcDialect file.

Author: Reynold Xin <rxin@databricks.com>

Closes #6689 from rxin/jdbc-quote and squashes the following commits:

bad365f [Reynold Xin] Fixed test compilation...
e39e14e [Reynold Xin] Fixed compilation.
db9a8e0 [Reynold Xin] [SPARK-8004][SQL] Quote identifier in JDBC data source.
2015-06-07 10:52:02 -07:00
Cheng Lian 8c321d66d7 [SPARK-8118] [SQL] Mutes noisy Parquet log output reappeared after upgrading Parquet to 1.7.0
Author: Cheng Lian <lian@databricks.com>

Closes #6670 from liancheng/spark-8118 and squashes the following commits:

b6e85a6 [Cheng Lian] Suppresses unnecesary ParquetRecordReader log message (PARQUET-220)
385603c [Cheng Lian] Mutes noisy Parquet log output reappeared after upgrading Parquet to 1.7.0
2015-06-07 16:59:55 +08:00
Liang-Chi Hsieh 26d07f1ece [SPARK-8141] [SQL] Precompute datatypes for partition columns and reuse it
JIRA: https://issues.apache.org/jira/browse/SPARK-8141

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

Closes #6687 from viirya/reuse_partition_column_types and squashes the following commits:

dab0688 [Liang-Chi Hsieh] Reuse partitionColumnTypes.
2015-06-07 15:33:48 +08:00
Liang-Chi Hsieh 901a552c5e [SPARK-8004][SQL] Enclose column names by JDBC Dialect
JIRA: https://issues.apache.org/jira/browse/SPARK-8004

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

Closes #6577 from viirya/enclose_jdbc_columns and squashes the following commits:

614606a [Liang-Chi Hsieh] For comment.
bc50182 [Liang-Chi Hsieh] Enclose column names by JDBC Dialect.
2015-06-06 22:59:31 -07:00
Cheng Lian 16fc49617e [SPARK-8079] [SQL] Makes InsertIntoHadoopFsRelation job/task abortion more robust
As described in SPARK-8079, when writing a DataFrame to a `HadoopFsRelation`, if `HadoopFsRelation.prepareForWriteJob` throws exception, an unexpected NPE will be thrown during job abortion. (This issue doesn't bring much damage since the job is failing anyway.)

This PR makes the job/task abortion logic in `InsertIntoHadoopFsRelation` more robust to avoid such confusing exceptions.

Author: Cheng Lian <lian@databricks.com>

Closes #6612 from liancheng/spark-8079 and squashes the following commits:

87cd81e [Cheng Lian] Addresses @rxin's comment
1864c75 [Cheng Lian] Addresses review comments
9e6dbb3 [Cheng Lian] Makes InsertIntoHadoopFsRelation job/task abortion more robust
2015-06-06 17:23:12 +08:00
Dong Wang eb19d3f75c [SPARK-6964] [SQL] Support Cancellation in the Thrift Server
Support runInBackground in SparkExecuteStatementOperation, and add cancellation

Author: Dong Wang <dong@databricks.com>

Closes #6207 from dongwang218/SPARK-6964-jdbc-cancel and squashes the following commits:

687c113 [Dong Wang] fix 100 characters
7bfa2a7 [Dong Wang] fix merge
380480f [Dong Wang] fix for liancheng's comments
eb3e385 [Dong Wang] small nit
341885b [Dong Wang] small fix
3d8ebf8 [Dong Wang] add spark.sql.hive.thriftServer.async flag
04142c3 [Dong Wang] set SQLSession for async execution
184ec35 [Dong Wang] keep hive conf
819ae03 [Dong Wang] [SPARK-6964][SQL][WIP] Support Cancellation in the Thrift Server
2015-06-05 17:41:12 -07:00
Shivaram Venkataraman 12f5eaeee1 [SPARK-8085] [SPARKR] Support user-specified schema in read.df
cc davies sun-rui

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #6620 from shivaram/sparkr-read-schema and squashes the following commits:

16a6726 [Shivaram Venkataraman] Fix loadDF to pass schema Also add a unit test
a229877 [Shivaram Venkataraman] Use wrapper function to DataFrameReader
ee70ba8 [Shivaram Venkataraman] Support user-specified schema in read.df
2015-06-05 10:19:03 -07:00
Reynold Xin 2bcdf8c239 [SPARK-7440][SQL] Remove physical Distinct operator in favor of Aggregate
This patch replaces Distinct with Aggregate in the optimizer, so Distinct will become
more efficient over time as we optimize Aggregate (via Tungsten).

Author: Reynold Xin <rxin@databricks.com>

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

b3cc50e [Reynold Xin] Mima excludes.
93d6117 [Reynold Xin] Code review feedback.
87e4741 [Reynold Xin] [SPARK-7440][SQL] Remove physical Distinct operator in favor of Aggregate.
2015-06-04 13:52:53 -07:00
Thomas Omans cd3176bd86 [SPARK-7743] [SQL] Parquet 1.7
Resolves [SPARK-7743](https://issues.apache.org/jira/browse/SPARK-7743).

Trivial changes of versions, package names, as well as a small issue in `ParquetTableOperations.scala`

```diff
-    val readContext = getReadSupport(configuration).init(
+    val readContext = ParquetInputFormat.getReadSupportInstance(configuration).init(
```

Since ParquetInputFormat.getReadSupport was made package private in the latest release.

Thanks
-- Thomas Omans

Author: Thomas Omans <tomans@cj.com>

Closes #6597 from eggsby/SPARK-7743 and squashes the following commits:

2df0d1b [Thomas Omans] [SPARK-7743] [SQL] Upgrading parquet version to 1.7.0
2015-06-04 11:32:03 -07:00
Mike Dusenberry df7da07a86 [SPARK-7969] [SQL] Added a DataFrame.drop function that accepts a Column reference.
Added a `DataFrame.drop` function that accepts a `Column` reference rather than a `String`, and added associated unit tests.  Basically iterates through the `DataFrame` to find a column with an expression that is equivalent to that of the `Column` argument supplied to the function.

Author: Mike Dusenberry <dusenberrymw@gmail.com>

Closes #6585 from dusenberrymw/SPARK-7969_Drop_method_on_Dataframes_should_handle_Column and squashes the following commits:

514727a [Mike Dusenberry] Updating the @since tag of the drop(Column) function doc to reflect version 1.4.1 instead of 1.4.0.
2f1bb4e [Mike Dusenberry] Adding an additional assert statement to the 'drop column after join' unit test in order to make sure the correct column was indeed left over.
6bf7c0e [Mike Dusenberry] Minor code formatting change.
e583888 [Mike Dusenberry] Adding more Python doctests for the df.drop with column reference function to test joined datasets that have columns with the same name.
5f74401 [Mike Dusenberry] Updating DataFrame.drop with column reference function to use logicalPlan.output to prevent ambiguities resulting from columns with the same name. Also added associated unit tests for joined datasets with duplicate column names.
4b8bbe8 [Mike Dusenberry] Adding Python support for Dataframe.drop with a Column reference.
986129c [Mike Dusenberry] Added a DataFrame.drop function that accepts a Column reference rather than a String, and added associated unit tests.  Basically iterates through the DataFrame to find a column with an expression that is equivalent to one supplied to the function.
2015-06-04 11:30:07 -07:00
Reynold Xin 2c5a06cafd Update documentation for [SPARK-7980] [SQL] Support SQLContext.range(end) 2015-06-03 14:20:27 -07:00
Reynold Xin 939e4f3d8d [SPARK-8074] Parquet should throw AnalysisException during setup for data type/name related failures.
Author: Reynold Xin <rxin@databricks.com>

Closes #6608 from rxin/parquet-analysis and squashes the following commits:

b5dc8e2 [Reynold Xin] Code review feedback.
5617cf6 [Reynold Xin] [SPARK-8074] Parquet should throw AnalysisException during setup for data type/name related failures.
2015-06-03 13:57:57 -07:00
animesh d053a31be9 [SPARK-7980] [SQL] Support SQLContext.range(end)
1. range() overloaded in SQLContext.scala
2. range() modified in python sql context.py
3. Tests added accordingly in DataFrameSuite.scala and python sql tests.py

Author: animesh <animesh@apache.spark>

Closes #6609 from animeshbaranawal/SPARK-7980 and squashes the following commits:

935899c [animesh] SPARK-7980:python+scala changes
2015-06-03 11:28:18 -07:00
Wenchen Fan d38cf217e0 [SPARK-7562][SPARK-6444][SQL] Improve error reporting for expression data type mismatch
It seems hard to find a common pattern of checking types in `Expression`. Sometimes we know what input types we need(like `And`, we know we need two booleans), sometimes we just have some rules(like `Add`, we need 2 numeric types which are equal). So I defined a general interface `checkInputDataTypes` in `Expression` which returns a `TypeCheckResult`. `TypeCheckResult` can tell whether this expression passes the type checking or what the type mismatch is.

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

TODO: apply type checking interface to more expressions.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

b5ff31b [Wenchen Fan] address comments
b917275 [Wenchen Fan] rebase
39929d9 [Wenchen Fan] add todo
0808fd2 [Wenchen Fan] make constrcutor of TypeCheckResult private
3bee157 [Wenchen Fan] and decimal type coercion rule for binary comparison
8883025 [Wenchen Fan] apply type check interface to CaseWhen
cffb67c [Wenchen Fan] to have resolved call the data type check function
6eaadff [Wenchen Fan] add equal type constraint to EqualTo
3affbd8 [Wenchen Fan] more fixes
654d46a [Wenchen Fan] improve tests
e0a3628 [Wenchen Fan] improve error message
1524ff6 [Wenchen Fan] fix style
69ca3fe [Wenchen Fan] add error message and tests
c71d02c [Wenchen Fan] fix hive tests
6491721 [Wenchen Fan] use value class TypeCheckResult
7ae76b9 [Wenchen Fan] address comments
cb77e4f [Wenchen Fan] Improve error reporting for expression data type mismatch
2015-06-03 00:47:52 -07:00
Cheng Lian 686a45f0b9 [SPARK-8014] [SQL] Avoid premature metadata discovery when writing a HadoopFsRelation with a save mode other than Append
The current code references the schema of the DataFrame to be written before checking save mode. This triggers expensive metadata discovery prematurely. For save mode other than `Append`, this metadata discovery is useless since we either ignore the result (for `Ignore` and `ErrorIfExists`) or delete existing files (for `Overwrite`) later.

This PR fixes this issue by deferring metadata discovery after save mode checking.

Author: Cheng Lian <lian@databricks.com>

Closes #6583 from liancheng/spark-8014 and squashes the following commits:

1aafabd [Cheng Lian] Updates comments
088abaa [Cheng Lian] Avoids schema merging and partition discovery when data schema and partition schema are defined
8fbd93f [Cheng Lian] Fixes SPARK-8014
2015-06-02 13:32:13 -07:00
Cheng Lian 1bb5d716c0 [SPARK-8037] [SQL] Ignores files whose name starts with dot in HadoopFsRelation
Author: Cheng Lian <lian@databricks.com>

Closes #6581 from liancheng/spark-8037 and squashes the following commits:

d08e97b [Cheng Lian] Ignores files whose name starts with dot in HadoopFsRelation
2015-06-03 00:59:50 +08:00
Yin Huai 7b7f7b6c6f [SPARK-8020] [SQL] Spark SQL conf in spark-defaults.conf make metadataHive get constructed too early
https://issues.apache.org/jira/browse/SPARK-8020

Author: Yin Huai <yhuai@databricks.com>

Closes #6571 from yhuai/SPARK-8020-1 and squashes the following commits:

0398f5b [Yin Huai] First populate the SQLConf and then construct executionHive and metadataHive.
2015-06-02 00:16:56 -07:00
Davies Liu bcb47ad771 [SPARK-6917] [SQL] DecimalType is not read back when non-native type exists
cc yhuai

Author: Davies Liu <davies@databricks.com>

Closes #6558 from davies/decimalType and squashes the following commits:

c877ca8 [Davies Liu] Update ParquetConverter.scala
48cc57c [Davies Liu] Update ParquetConverter.scala
b43845c [Davies Liu] add test
3b4a94f [Davies Liu] DecimalType is not read back when non-native type exists
2015-06-01 23:12:29 -07:00
Reynold Xin 75dda33f3e Revert "[SPARK-8020] Spark SQL in spark-defaults.conf make metadataHive get constructed too early"
This reverts commit 91f6be87bc.
2015-06-01 21:35:55 -07:00
Yin Huai 91f6be87bc [SPARK-8020] Spark SQL in spark-defaults.conf make metadataHive get constructed too early
https://issues.apache.org/jira/browse/SPARK-8020

Author: Yin Huai <yhuai@databricks.com>

Closes #6563 from yhuai/SPARK-8020 and squashes the following commits:

4e5addc [Yin Huai] style
bf766c6 [Yin Huai] Failed test.
0398f5b [Yin Huai] First populate the SQLConf and then construct executionHive and metadataHive.
2015-06-01 21:33:57 -07:00
Reynold Xin 4c868b9943 [minor doc] Add exploratory data analysis warning for DataFrame.stat.freqItem API
Author: Reynold Xin <rxin@databricks.com>

Closes #6569 from rxin/freqItemsWarning and squashes the following commits:

7eec145 [Reynold Xin] [minor doc] Add exploratory data analysis warning for DataFrame.stat.freqItem API.
2015-06-01 21:29:39 -07:00
Reynold Xin 89f642a0e8 [SPARK-8026][SQL] Add Column.alias to Scala/Java DataFrame API
Author: Reynold Xin <rxin@databricks.com>

Closes #6565 from rxin/alias and squashes the following commits:

286d880 [Reynold Xin] [SPARK-8026][SQL] Add Column.alias to Scala/Java DataFrame API
2015-06-01 21:13:15 -07:00
Reynold Xin 6396cc0303 [SPARK-7982][SQL] DataFrame.stat.crosstab should use 0 instead of null for pairs that don't appear
Author: Reynold Xin <rxin@databricks.com>

Closes #6566 from rxin/crosstab and squashes the following commits:

e0ace1c [Reynold Xin] [SPARK-7982][SQL] DataFrame.stat.crosstab should use 0 instead of null for pairs that don't appear
2015-06-01 21:11:19 -07:00
Reynold Xin 866652c903 [SPARK-3850] Turn style checker on for trailing whitespaces.
Author: Reynold Xin <rxin@databricks.com>

Closes #6541 from rxin/trailing-whitespace-on and squashes the following commits:

f72ebe4 [Reynold Xin] [SPARK-3850] Turn style checker on for trailing whitespaces.
2015-05-31 14:23:42 -07:00
Reynold Xin 63a50be13d [SPARK-3850] Trim trailing spaces for SQL.
Author: Reynold Xin <rxin@databricks.com>

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

de50316 [Reynold Xin] [SPARK-3850] Trim trailing spaces for SQL.
2015-05-31 00:48:49 -07:00
Reynold Xin 7896e99b2a [SPARK-7975] Add style checker to disallow overriding equals covariantly.
Author: Reynold Xin <rxin@databricks.com>

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

Closes #6527 from rxin/covariant-equals and squashes the following commits:

e7d7784 [Reynold Xin] [SPARK-7975] Enforce CovariantEqualsChecker
2015-05-31 00:05:55 -07:00
Cheng Lian 8764dccebd [SQL] [MINOR] Adds @deprecated Scaladoc entry for SchemaRDD
Author: Cheng Lian <lian@databricks.com>

Closes #6529 from liancheng/schemardd-deprecation-fix and squashes the following commits:

49765c2 [Cheng Lian] Adds @deprecated Scaladoc entry for SchemaRDD
2015-05-30 23:49:42 -07:00
Reynold Xin c63e1a742b [SPARK-7971] Add JavaDoc style deprecation for deprecated DataFrame methods
Scala deprecated annotation actually doesn't show up in JavaDoc.

Author: Reynold Xin <rxin@databricks.com>

Closes #6523 from rxin/df-deprecated-javadoc and squashes the following commits:

26da2b2 [Reynold Xin] [SPARK-7971] Add JavaDoc style deprecation for deprecated DataFrame methods.
2015-05-30 19:51:53 -07:00
Reynold Xin 14b314dc2c [SQL] Tighten up visibility for JavaDoc.
I went through all the JavaDocs and tightened up visibility.

Author: Reynold Xin <rxin@databricks.com>

Closes #6526 from rxin/sql-1.4-visibility-for-docs and squashes the following commits:

bc37d1e [Reynold Xin] Tighten up visibility for JavaDoc.
2015-05-30 19:50:52 -07:00
Reynold Xin 94f62a4979 [SPARK-7940] Enforce whitespace checking for DO, TRY, CATCH, FINALLY, MATCH, LARROW, RARROW in style checker.
…

Author: Reynold Xin <rxin@databricks.com>

Closes #6491 from rxin/more-whitespace and squashes the following commits:

f6e63dc [Reynold Xin] [SPARK-7940] Enforce whitespace checking for DO, TRY, CATCH, FINALLY, MATCH, LARROW, RARROW in style checker.
2015-05-29 13:38:37 -07:00
Reynold Xin ff44c711ab [SPARK-7927] whitespace fixes for SQL core.
So we can enable a whitespace enforcement rule in the style checker to save code review time.

Author: Reynold Xin <rxin@databricks.com>

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

ce6e369 [Reynold Xin] Fixed tests.
6095fed [Reynold Xin] [SPARK-7927] whitespace fixes for SQL core.
2015-05-28 20:10:21 -07:00
Liang-Chi Hsieh a1e092eae5 [SPARK-7897][SQL] Use DecimalType to represent unsigned bigint in JDBCRDD
JIRA: https://issues.apache.org/jira/browse/SPARK-7897

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

Closes #6438 from viirya/jdbc_unsigned_bigint and squashes the following commits:

ccb3c3f [Liang-Chi Hsieh] Use DecimalType to represent unsigned bigint.
2015-05-27 18:51:36 -07:00
Reynold Xin 6fec1a9409 Removed Guava dependency from JavaTypeInference's type signature.
This should also close #6243.

Author: Reynold Xin <rxin@databricks.com>

Closes #6431 from rxin/JavaTypeInference-guava and squashes the following commits:

e58df3c [Reynold Xin] Removed Gauva dependency from JavaTypeInference's type signature.
2015-05-27 11:54:35 -07:00
Cheng Lian 15459db4f6 [SPARK-7847] [SQL] Fixes dynamic partition directory escaping
Please refer to [SPARK-7847] [1] for details.

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

Author: Cheng Lian <lian@databricks.com>

Closes #6389 from liancheng/spark-7847 and squashes the following commits:

935c652 [Cheng Lian] Adds test case for writing various data types as dynamic partition value
f4fc398 [Cheng Lian] Converts partition columns to Scala type when writing dynamic partitions
d0aeca0 [Cheng Lian] Fixes dynamic partition directory escaping
2015-05-27 10:09:12 -07:00
Reynold Xin 9f48bf6b37 [SPARK-7887][SQL] Remove EvaluatedType from SQL Expression.
This type is not really used. Might as well remove it.

Author: Reynold Xin <rxin@databricks.com>

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

51a319a [Reynold Xin] [SPARK-7887][SQL] Remove EvaluatedType from SQL Expression.
2015-05-27 01:12:59 -07:00
Liang-Chi Hsieh 4f98d7a7f1 [SPARK-7697][SQL] Use LongType for unsigned int in JDBCRDD
JIRA: https://issues.apache.org/jira/browse/SPARK-7697

The reported problem case is mysql. But for h2 db, there is no unsigned int. So it is not able to add corresponding test.

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

Closes #6229 from viirya/unsignedint_as_long and squashes the following commits:

dc4b5d8 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into unsignedint_as_long
608695b [Liang-Chi Hsieh] Use LongType for unsigned int in JDBCRDD.
2015-05-27 00:27:39 -07:00
Cheng Lian b463e6d618 [SPARK-7868] [SQL] Ignores _temporary directories in HadoopFsRelation
So that potential partial/corrupted data files left by failed tasks/jobs won't affect normal data scan.

Author: Cheng Lian <lian@databricks.com>

Closes #6411 from liancheng/spark-7868 and squashes the following commits:

273ea36 [Cheng Lian] Ignores _temporary directories
2015-05-26 20:48:56 -07:00
Josh Rosen 0c33c7b4a6 [SPARK-7858] [SQL] Use output schema, not relation schema, for data source input conversion
In `DataSourceStrategy.createPhysicalRDD`, we use the relation schema as the target schema for converting incoming rows into Catalyst rows.  However, we should be using the output schema instead, since our scan might return a subset of the relation's columns.

This patch incorporates #6414 by liancheng, which fixes an issue in `SimpleTestRelation` that prevented this bug from being caught by our old tests:

> In `SimpleTextRelation`, we specified `needsConversion` to `true`, indicating that values produced by this testing relation should be of Scala types, and need to be converted to Catalyst types when necessary. However, we also used `Cast` to convert strings to expected data types. And `Cast` always produces values of Catalyst types, thus no conversion is done at all. This PR makes `SimpleTextRelation` produce Scala values so that data conversion code paths can be properly tested.

Closes #5986.

Author: Josh Rosen <joshrosen@databricks.com>
Author: Cheng Lian <lian@databricks.com>
Author: Cheng Lian <liancheng@users.noreply.github.com>

Closes #6400 from JoshRosen/SPARK-7858 and squashes the following commits:

e71c866 [Josh Rosen] Re-fix bug so that the tests pass again
56b13e5 [Josh Rosen] Add regression test to hadoopFsRelationSuites
2169a0f [Josh Rosen] Remove use of SpecificMutableRow and BufferedIterator
6cd7366 [Josh Rosen] Fix SPARK-7858 by using output types for conversion.
5a00e66 [Josh Rosen] Add assertions in order to reproduce SPARK-7858
8ba195c [Cheng Lian] Merge 9968fba9979287aaa1f141ba18bfb9d4c116a3b3 into 61664732b2
9968fba [Cheng Lian] Tests the data type conversion code paths
2015-05-26 20:24:35 -07:00
Cheng Lian 8af1bf10b7 [SPARK-7842] [SQL] Makes task committing/aborting in InsertIntoHadoopFsRelation more robust
When committing/aborting a write task issued in `InsertIntoHadoopFsRelation`, if an exception is thrown from `OutputWriter.close()`, the committing/aborting process will be interrupted, and leaves messy stuff behind (e.g., the `_temporary` directory created by `FileOutputCommitter`).

This PR makes these two process more robust by catching potential exceptions and falling back to normal task committment/abort.

Author: Cheng Lian <lian@databricks.com>

Closes #6378 from liancheng/spark-7838 and squashes the following commits:

f18253a [Cheng Lian] Makes task committing/aborting in InsertIntoHadoopFsRelation more robust
2015-05-26 00:28:47 +08:00
Yin Huai ed21476bc0 [SPARK-7805] [SQL] Move SQLTestUtils.scala and ParquetTest.scala to src/test
https://issues.apache.org/jira/browse/SPARK-7805

Because `sql/hive`'s tests depend on the test jar of `sql/core`, we do not need to store `SQLTestUtils` and `ParquetTest` in `src/main`. We should only add stuff that will be needed by `sql/console` or Python tests (for Python, we need it in `src/main`, right? davies).

Author: Yin Huai <yhuai@databricks.com>

Closes #6334 from yhuai/SPARK-7805 and squashes the following commits:

af6d0c9 [Yin Huai] mima
b86746a [Yin Huai] Move SQLTestUtils.scala and ParquetTest.scala to src/test.
2015-05-24 09:51:37 -07:00
Yin Huai 2b7e63585d [SPARK-7654] [SQL] Move insertInto into reader/writer interface.
This one continues the work of https://github.com/apache/spark/pull/6216.

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

Closes #6366 from yhuai/insert and squashes the following commits:

3d717fb [Yin Huai] Use insertInto to handle the casue when table exists and Append is used for saveAsTable.
56d2540 [Yin Huai] Add PreWriteCheck to HiveContext's analyzer.
c636e35 [Yin Huai] Remove unnecessary empty lines.
cf83837 [Yin Huai] Move insertInto to write. Also, remove the partition columns from InsertIntoHadoopFsRelation.
0841a54 [Reynold Xin] Removed experimental tag for deprecated methods.
33ed8ef [Reynold Xin] [SPARK-7654][SQL] Move insertInto into reader/writer interface.
2015-05-23 09:48:20 -07:00
Davies Liu efe3bfdf49 [SPARK-7322, SPARK-7836, SPARK-7822][SQL] DataFrame window function related updates
1. ntile should take an integer as parameter.
2. Added Python API (based on #6364)
3. Update documentation of various DataFrame Python functions.

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

Closes #6374 from rxin/window-final and squashes the following commits:

69004c7 [Reynold Xin] Style fix.
288cea9 [Reynold Xin] Update documentaiton.
7cb8985 [Reynold Xin] Merge pull request #6364 from davies/window
66092b4 [Davies Liu] update docs
ed73cb4 [Reynold Xin] [SPARK-7322][SQL] Improve DataFrame window function documentation.
ef55132 [Davies Liu] Merge branch 'master' of github.com:apache/spark into window4
8936ade [Davies Liu] fix maxint in python 3
2649358 [Davies Liu] update docs
778e2c0 [Davies Liu] SPARK-7836 and SPARK-7822: Python API of window functions
2015-05-23 08:30:05 -07:00
Michael Armbrust 3b68cb0430 [SPARK-6743] [SQL] Fix empty projections of cached data
Author: Michael Armbrust <michael@databricks.com>

Closes #6165 from marmbrus/wrongColumn and squashes the following commits:

4fad158 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into wrongColumn
aad7eab [Michael Armbrust] rxins comments
f1e8df1 [Michael Armbrust] [SPARK-6743][SQL] Fix empty projections of cached data
2015-05-22 09:43:46 -07:00
Cheng Hao f6f2eeb179 [SPARK-7322][SQL] Window functions in DataFrame
This closes #6104.

Author: Cheng Hao <hao.cheng@intel.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #6343 from rxin/window-df and squashes the following commits:

026d587 [Reynold Xin] Address code review feedback.
dc448fe [Reynold Xin] Fixed Hive tests.
9794d9d [Reynold Xin] Moved Java test package.
9331605 [Reynold Xin] Refactored API.
3313e2a [Reynold Xin] Merge pull request #6104 from chenghao-intel/df_window
d625a64 [Cheng Hao] Update the dataframe window API as suggsted
c141fb1 [Cheng Hao] hide all of properties of the WindowFunctionDefinition
3b1865f [Cheng Hao] scaladoc typos
f3fd2d0 [Cheng Hao] polish the unit test
6847825 [Cheng Hao] Add additional analystcs functions
57e3bc0 [Cheng Hao] typos
24a08ec [Cheng Hao] scaladoc
28222ed [Cheng Hao] fix bug of range/row Frame
1d91865 [Cheng Hao] style issue
53f89f2 [Cheng Hao] remove the over from the functions.scala
964c013 [Cheng Hao] add more unit tests and window functions
64e18a7 [Cheng Hao] Add Window Function support for DataFrame
2015-05-22 01:00:16 -07:00
Yin Huai 347b50106b [SPARK-7737] [SQL] Use leaf dirs having data files to discover partitions.
https://issues.apache.org/jira/browse/SPARK-7737

cc liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #6329 from yhuai/spark-7737 and squashes the following commits:

7e0dfc7 [Yin Huai] Use leaf dirs having data files to discover partitions.
2015-05-22 07:10:26 +08:00
Andrew Or 5287eec5a6 [SPARK-7718] [SQL] Speed up partitioning by avoiding closure cleaning
According to yhuai we spent 6-7 seconds cleaning closures in a partitioning job that takes 12 seconds. Since we provide these closures in Spark we know for sure they are serializable, so we can bypass the cleaning.

Author: Andrew Or <andrew@databricks.com>

Closes #6256 from andrewor14/sql-partition-speed-up and squashes the following commits:

a82b451 [Andrew Or] Fix style
10f7e3e [Andrew Or] Avoid getting call sites and cleaning closures
17e2943 [Andrew Or] Merge branch 'master' of github.com:apache/spark into sql-partition-speed-up
523f042 [Andrew Or] Skip unnecessary Utils.getCallSites too
f7fe143 [Andrew Or] Avoid unnecessary closure cleaning
2015-05-21 14:33:11 -07:00
Tathagata Das 3d0cccc858 [SPARK-7478] [SQL] Added SQLContext.getOrCreate
Having a SQLContext singleton would make it easier for applications to use a lazily instantiated single shared instance of SQLContext when needed. It would avoid problems like

1. In REPL/notebook environment, rerunning the line {{val sqlContext = new SQLContext}} multiple times created different contexts while overriding the reference to previous context, leading to issues like registered temp tables going missing.

2. In Streaming, creating SQLContext directly leads to serialization/deserialization issues when attempting to recover from DStream checkpoints. See [SPARK-6770]. Also to get around this problem I had to suggest creating a singleton instance - https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala

This can be solved by {{SQLContext.getOrCreate}} which get or creates a new singleton instance of SQLContext using either a given SparkContext or a given SparkConf.

rxin marmbrus

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #6006 from tdas/SPARK-7478 and squashes the following commits:

25f4da9 [Tathagata Das] Addressed comments.
79fe069 [Tathagata Das] Added comments.
c66ca76 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-7478
48adb14 [Tathagata Das] Removed HiveContext.getOrCreate
bf8cf50 [Tathagata Das] Fix more bug
dec5594 [Tathagata Das] Fixed bug
b4e9721 [Tathagata Das] Remove unnecessary import
4ef513b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-7478
d3ea8e4 [Tathagata Das] Added HiveContext
83bc950 [Tathagata Das] Updated tests
f82ae81 [Tathagata Das] Fixed test
bc72868 [Tathagata Das] Added SQLContext.getOrCreate
2015-05-21 14:08:20 -07:00
Yin Huai 30f3f556f7 [SPARK-7763] [SPARK-7616] [SQL] Persists partition columns into metastore
Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Lian <lian@databricks.com>

Closes #6285 from liancheng/spark-7763 and squashes the following commits:

bb2829d [Yin Huai] Fix hashCode.
d677f7d [Cheng Lian] Fixes Scala style issue
44b283f [Cheng Lian] Adds test case for SPARK-7616
6733276 [Yin Huai] Fix a bug that potentially causes https://issues.apache.org/jira/browse/SPARK-7616.
6cabf3c [Yin Huai] Update unit test.
7e02910 [Yin Huai] Use metastore partition columns and do not hijack maybePartitionSpec.
e9a03ec [Cheng Lian] Persists partition columns into metastore
2015-05-21 13:51:40 -07:00
Cheng Lian 8730fbb47b [SPARK-7749] [SQL] Fixes partition discovery for non-partitioned tables
When no partition columns can be found, we should have an empty `PartitionSpec`, rather than a `PartitionSpec` with empty partition columns.

This PR together with #6285 should fix SPARK-7749.

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

Closes #6287 from liancheng/spark-7749 and squashes the following commits:

a799ff3 [Cheng Lian] Adds test cases for SPARK-7749
c4949be [Cheng Lian] Minor refactoring, and tolerant _TEMPORARY directory name
5aa87ea [Yin Huai] Make parsePartitions more robust.
fc56656 [Cheng Lian] Returns empty PartitionSpec if no partition columns can be inferred
19ae41e [Cheng Lian] Don't list base directory as leaf directory
2015-05-21 10:56:17 -07:00
Davies Liu a25c1ab8f0 [SPARK-7565] [SQL] fix MapType in JsonRDD
The key of Map in JsonRDD should be converted into UTF8String (also failed records), Thanks to yhuai viirya

Closes #6084

Author: Davies Liu <davies@databricks.com>

Closes #6299 from davies/string_in_json and squashes the following commits:

0dbf559 [Davies Liu] improve test, fix corrupt record
6836a80 [Davies Liu] move unit tests into Scala
b97af11 [Davies Liu] fix MapType in JsonRDD
2015-05-21 09:58:47 -07:00
Liang-Chi Hsieh d0eb9ffe97 [SPARK-7746][SQL] Add FetchSize parameter for JDBC driver
JIRA: https://issues.apache.org/jira/browse/SPARK-7746

Looks like an easy to add parameter but can show significant performance improvement if the JDBC driver accepts it.

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

Closes #6283 from viirya/jdbc_fetchsize and squashes the following commits:

de47f94 [Liang-Chi Hsieh] Don't keep fetchSize as single parameter.
b7bff2f [Liang-Chi Hsieh] Add FetchSize parameter for JDBC driver.
2015-05-20 22:23:49 -07:00
Cheng Hao 42c592adb3 [SPARK-7320] [SQL] Add Cube / Rollup for dataframe
This is a follow up for #6257, which broke the maven test.

Add cube & rollup for DataFrame
For example:
```scala
testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b"))
testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b"))
```

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

Closes #6304 from chenghao-intel/rollup and squashes the following commits:

04bb1de [Cheng Hao] move the table register/unregister into beforeAll/afterAll
a6069f1 [Cheng Hao] cancel the implicit keyword
ced4b8f [Cheng Hao] remove the unnecessary code changes
9959dfa [Cheng Hao] update the code as comments
e1d88aa [Cheng Hao] update the code as suggested
03bc3d9 [Cheng Hao] Remove the CubedData & RollupedData
5fd62d0 [Cheng Hao] hiden the CubedData & RollupedData
5ffb196 [Cheng Hao] Add Cube / Rollup for dataframe
2015-05-20 19:58:22 -07:00
Patrick Wendell 6338c40da6 Revert "[SPARK-7320] [SQL] Add Cube / Rollup for dataframe"
This reverts commit 10698e1131.
2015-05-20 13:39:04 -07:00
Yin Huai b631bf73b9 [SPARK-7713] [SQL] Use shared broadcast hadoop conf for partitioned table scan.
https://issues.apache.org/jira/browse/SPARK-7713

I tested the performance with the following code:
```scala
import sqlContext._
import sqlContext.implicits._

(1 to 5000).foreach { i =>
  val df = (1 to 1000).map(j => (j, s"str$j")).toDF("a", "b").save(s"/tmp/partitioned/i=$i")
}

sqlContext.sql("""
CREATE TEMPORARY TABLE partitionedParquet
USING org.apache.spark.sql.parquet
OPTIONS (
  path '/tmp/partitioned'
)""")

table("partitionedParquet").explain(true)
```

In our master `explain` takes 40s in my laptop. With this PR, `explain` takes 14s.

Author: Yin Huai <yhuai@databricks.com>

Closes #6252 from yhuai/broadcastHadoopConf and squashes the following commits:

6fa73df [Yin Huai] Address comments of Josh and Andrew.
807fbf9 [Yin Huai] Make the new buildScan and SqlNewHadoopRDD private sql.
e393555 [Yin Huai] Cheng's comments.
2eb53bb [Yin Huai] Use a shared broadcast Hadoop Configuration for partitioned HadoopFsRelations.
2015-05-20 11:23:40 -07:00
Cheng Hao 09265ad7c8 [SPARK-7320] [SQL] Add Cube / Rollup for dataframe
Add `cube` & `rollup` for DataFrame
For example:
```scala
testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b"))
testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b"))
```

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

Closes #6257 from chenghao-intel/rollup and squashes the following commits:

7302319 [Cheng Hao] cancel the implicit keyword
a66e38f [Cheng Hao] remove the unnecessary code changes
a2869d4 [Cheng Hao] update the code as comments
c441777 [Cheng Hao] update the code as suggested
84c9564 [Cheng Hao] Remove the CubedData & RollupedData
279584c [Cheng Hao] hiden the CubedData & RollupedData
ef357e1 [Cheng Hao] Add Cube / Rollup for dataframe
2015-05-20 19:09:47 +08:00
scwf 60336e3bc0 [SPARK-7656] [SQL] use CatalystConf in FunctionRegistry
follow up for #5806

Author: scwf <wangfei1@huawei.com>

Closes #6164 from scwf/FunctionRegistry and squashes the following commits:

15e6697 [scwf] use catalogconf in FunctionRegistry
2015-05-19 17:36:00 -07:00
Josh Rosen c9fa870a6d [SPARK-7687] [SQL] DataFrame.describe() should cast all aggregates to String
In `DataFrame.describe()`, the `count` aggregate produces an integer, the `avg` and `stdev` aggregates produce doubles, and `min` and `max` aggregates can produce varying types depending on what type of column they're applied to.  As a result, we should cast all aggregate results to String so that `describe()`'s output types match its declared output schema.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6218 from JoshRosen/SPARK-7687 and squashes the following commits:

146b615 [Josh Rosen] Fix R test.
2974bd5 [Josh Rosen] Cast to string type instead
f206580 [Josh Rosen] Cast to double to fix SPARK-7687
307ecbf [Josh Rosen] Add failing regression test for SPARK-7687
2015-05-18 21:53:44 -07:00
Daoyuan Wang c2437de189 [SPARK-7150] SparkContext.range() and SQLContext.range()
This PR is based on #6081, thanks adrian-wang.

Closes #6081

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

Closes #6230 from davies/range and squashes the following commits:

d3ce5fe [Davies Liu] add tests
789eda5 [Davies Liu] add range() in Python
4590208 [Davies Liu] Merge commit 'refs/pull/6081/head' of github.com:apache/spark into range
cbf5200 [Daoyuan Wang] let's add python support in a separate PR
f45e3b2 [Daoyuan Wang] remove redundant toLong
617da76 [Daoyuan Wang] fix safe marge for corner cases
867c417 [Daoyuan Wang] fix
13dbe84 [Daoyuan Wang] update
bd998ba [Daoyuan Wang] update comments
d3a0c1b [Daoyuan Wang] add range api()
2015-05-18 21:43:12 -07:00
Davies Liu 32fbd297dd [SPARK-6216] [PYSPARK] check python version of worker with driver
This PR revert #5404, change to pass the version of python in driver into JVM, check it in worker before deserializing closure, then it can works with different major version of Python.

Author: Davies Liu <davies@databricks.com>

Closes #6203 from davies/py_version and squashes the following commits:

b8fb76e [Davies Liu] fix test
6ce5096 [Davies Liu] use string for version
47c6278 [Davies Liu] check python version of worker with driver
2015-05-18 12:55:13 -07:00
Cheng Lian 9dadf019b9 [SPARK-7673] [SQL] WIP: HadoopFsRelation and ParquetRelation2 performance optimizations
This PR introduces several performance optimizations to `HadoopFsRelation` and `ParquetRelation2`:

1.  Moving `FileStatus` listing from `DataSourceStrategy` into a cache within `HadoopFsRelation`.

    This new cache generalizes and replaces the one used in `ParquetRelation2`.

    This also introduces an interface change: to reuse cached `FileStatus` objects, `HadoopFsRelation.buildScan` methods now receive `Array[FileStatus]` instead of `Array[String]`.

1.  When Parquet task side metadata reading is enabled, skip reading row group information when reading Parquet footers.

    This is basically what PR #5334 does. Also, now we uses `ParquetFileReader.readAllFootersInParallel` to read footers in parallel.

Another optimization in question is, instead of asking `HadoopFsRelation.buildScan` to return an `RDD[Row]` for a single selected partition and then union them all, we ask it to return an `RDD[Row]` for all selected partitions. This optimization is based on the fact that Hadoop configuration broadcasting used in `NewHadoopRDD` takes 34% time in the following microbenchmark.  However, this complicates data source user code because user code must merge partition values manually.

To check the cost of broadcasting in `NewHadoopRDD`, I also did microbenchmark after removing the `broadcast` call in `NewHadoopRDD`.  All results are shown below.

### Microbenchmark

#### Preparation code

Generating a partitioned table with 50k partitions, 1k rows per partition:

```scala
import sqlContext._
import sqlContext.implicits._

for (n <- 0 until 500) {
  val data = for {
    p <- (n * 10) until ((n + 1) * 10)
    i <- 0 until 1000
  } yield (i, f"val_$i%04d", f"$p%04d")

  data.
    toDF("a", "b", "p").
    write.
    partitionBy("p").
    mode("append").
    parquet(path)
}
```

#### Benchmarking code

```scala
import sqlContext._
import sqlContext.implicits._

import org.apache.spark.sql.types._
import com.google.common.base.Stopwatch

val path = "hdfs://localhost:9000/user/lian/5k"

def benchmark(n: Int)(f: => Unit) {
  val stopwatch = new Stopwatch()

  def run() = {
    stopwatch.reset()
    stopwatch.start()
    f
    stopwatch.stop()
    stopwatch.elapsedMillis()
  }

  val records = (0 until n).map(_ => run())

  (0 until n).foreach(i => println(s"Round $i: ${records(i)} ms"))
  println(s"Average: ${records.sum / n.toDouble} ms")
}

benchmark(3) { read.parquet(path).explain(extended = true) }
```

#### Results

Before:

```
Round 0: 72528 ms
Round 1: 68938 ms
Round 2: 65372 ms
Average: 68946.0 ms
```

After:

```
Round 0: 59499 ms
Round 1: 53645 ms
Round 2: 53844 ms
Round 3: 49093 ms
Round 4: 50555 ms
Average: 53327.2 ms
```

Also removing Hadoop configuration broadcasting:

(Note that I was testing on a local laptop, thus network cost is pretty low.)

```
Round 0: 15806 ms
Round 1: 14394 ms
Round 2: 14699 ms
Round 3: 15334 ms
Round 4: 14123 ms
Average: 14871.2 ms
```

Author: Cheng Lian <lian@databricks.com>

Closes #6225 from liancheng/spark-7673 and squashes the following commits:

2d58a2b [Cheng Lian] Skips reading row group information when using task side metadata reading
7aa3748 [Cheng Lian] Optimizes FileStatusCache by introducing a map from parent directories to child files
ba41250 [Cheng Lian] Reuses HadoopFsRelation FileStatusCache in ParquetRelation2
3d278f7 [Cheng Lian] Fixes a bug when reading a single Parquet data file
b84612a [Cheng Lian] Fixes Scala style issue
6a08b02 [Cheng Lian] WIP: Moves file status cache into HadoopFSRelation
2015-05-18 12:45:37 -07:00
Yin Huai 530397ba2f [SPARK-7567] [SQL] [follow-up] Use a new flag to set output committer based on mapreduce apis
cc liancheng marmbrus

Author: Yin Huai <yhuai@databricks.com>

Closes #6130 from yhuai/directOutput and squashes the following commits:

312b07d [Yin Huai] A data source can use spark.sql.sources.outputCommitterClass to override the output committer.
2015-05-18 12:17:10 -07:00
Zhan Zhang aa31e431fc [SPARK-2883] [SQL] ORC data source for Spark SQL
This PR updates PR #6135 authored by zhzhan from Hortonworks.

----

This PR implements a Spark SQL data source for accessing ORC files.

> **NOTE**
>
> Although ORC is now an Apache TLP, the codebase is still tightly coupled with Hive.  That's why the new ORC data source is under `org.apache.spark.sql.hive` package, and must be used with `HiveContext`.  However, it doesn't require existing Hive installation to access ORC files.

1.  Saving/loading ORC files without contacting Hive metastore

1.  Support for complex data types (i.e. array, map, and struct)

1.  Aware of common optimizations provided by Spark SQL:

    - Column pruning
    - Partitioning pruning
    - Filter push-down

1.  Schema evolution support
1.  Hive metastore table conversion

This PR also include initial work done by scwf from Huawei (PR #3753).

Author: Zhan Zhang <zhazhan@gmail.com>
Author: Cheng Lian <lian@databricks.com>

Closes #6194 from liancheng/polishing-orc and squashes the following commits:

55ecd96 [Cheng Lian] Reorganizes ORC test suites
d4afeed [Cheng Lian] Addresses comments
21ada22 [Cheng Lian] Adds @since and @Experimental annotations
128bd3b [Cheng Lian] ORC filter bug fix
d734496 [Cheng Lian] Polishes the ORC data source
2650a42 [Zhan Zhang] resolve review comments
3c9038e [Zhan Zhang] resolve review comments
7b3c7c5 [Zhan Zhang] save mode fix
f95abfd [Zhan Zhang] reuse test suite
7cc2c64 [Zhan Zhang] predicate fix
4e61c16 [Zhan Zhang] minor change
305418c [Zhan Zhang] orc data source support
2015-05-18 12:03:40 -07:00
Cheng Lian 010a1c2780 [SPARK-7570] [SQL] Ignores _temporary during partition discovery
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/6091)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #6091 from liancheng/spark-7570 and squashes the following commits:

8ff07e8 [Cheng Lian] Ignores _temporary during partition discovery
2015-05-18 11:59:44 -07:00
Rene Treffer e1ac2a955b [SPARK-6888] [SQL] Make the jdbc driver handling user-definable
Replace the DriverQuirks with JdbcDialect(s) (and MySQLDialect/PostgresDialect)
and allow developers to change the dialects on the fly (for new JDBCRRDs only).

Some types (like an unsigned 64bit number) can be trivially mapped to java.
The status quo is that the RRD will fail to load.
This patch makes it possible to overwrite the type mapping to read e.g.
64Bit numbers as strings and handle them afterwards in software.

JDBCSuite has an example that maps all types to String, which should always
work (at the cost of extra code afterwards).

As a side effect it should now be possible to develop simple dialects
out-of-tree and even with spark-shell.

Author: Rene Treffer <treffer@measite.de>

Closes #5555 from rtreffer/jdbc-dialects and squashes the following commits:

3cbafd7 [Rene Treffer] [SPARK-6888] ignore classes belonging to changed API in MIMA report
fe7e2e8 [Rene Treffer] [SPARK-6888] Make the jdbc driver handling user-definable
2015-05-18 11:55:36 -07:00
Liang-Chi Hsieh e32c0f69f3 [SPARK-7299][SQL] Set precision and scale for Decimal according to JDBC metadata instead of returned BigDecimal
JIRA: https://issues.apache.org/jira/browse/SPARK-7299

When connecting with oracle db through jdbc, the precision and scale of `BigDecimal` object returned by `ResultSet.getBigDecimal` is not correctly matched to the table schema reported by `ResultSetMetaData.getPrecision` and `ResultSetMetaData.getScale`.

So in case you insert a value like `19999` into a column with `NUMBER(12, 2)` type, you get through a `BigDecimal` object with scale as 0. But the dataframe schema has correct type as `DecimalType(12, 2)`. Thus, after you save the dataframe into parquet file and then retrieve it, you will get wrong result `199.99`.

Because it is reported to be problematic on jdbc connection with oracle db. It might be difficult to add test case for it. But according to the user's test on JIRA, it solves this problem.

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

Closes #5833 from viirya/jdbc_decimal_precision and squashes the following commits:

69bc2b5 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into jdbc_decimal_precision
928f864 [Liang-Chi Hsieh] Add comments.
5f9da94 [Liang-Chi Hsieh] Set up Decimal's precision and scale according to table schema instead of returned BigDecimal.
2015-05-18 01:10:55 -07:00
zsxwing ff71d34e00 [SPARK-7693][Core] Remove "import scala.concurrent.ExecutionContext.Implicits.global"
Learnt a lesson from SPARK-7655: Spark should avoid to use `scala.concurrent.ExecutionContext.Implicits.global` because the user may submit blocking actions to `scala.concurrent.ExecutionContext.Implicits.global` and exhaust all threads in it. This could crash Spark. So Spark should always use its own thread pools for safety.

This PR removes all usages of `scala.concurrent.ExecutionContext.Implicits.global` and uses proper thread pools to replace them.

Author: zsxwing <zsxwing@gmail.com>

Closes #6223 from zsxwing/SPARK-7693 and squashes the following commits:

a33ff06 [zsxwing] Decrease the max thread number from 1024 to 128
cf4b3fc [zsxwing] Remove "import scala.concurrent.ExecutionContext.Implicits.global"
2015-05-17 20:37:19 -07:00
Josh Rosen 564562874f [SPARK-7686] [SQL] DescribeCommand is assigned wrong output attributes in SparkStrategies
In `SparkStrategies`, `RunnableDescribeCommand` is called with the output attributes of the table being described rather than the attributes for the `describe` command's output.  I discovered this issue because it caused type conversion errors in some UnsafeRow conversion code that I'm writing.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6217 from JoshRosen/SPARK-7686 and squashes the following commits:

953a344 [Josh Rosen] Fix SPARK-7686 with a simple change in SparkStrategies.
a4eec9f [Josh Rosen] Add failing regression test for SPARK-7686
2015-05-17 11:59:28 -07:00
Liang-Chi Hsieh 3399055787 [SPARK-7447] [SQL] Don't re-merge Parquet schema when the relation is deserialized
JIRA: https://issues.apache.org/jira/browse/SPARK-7447

`MetadataCache` in `ParquetRelation2` is annotated as `transient`. When `ParquetRelation2` is deserialized, we ask `MetadataCache` to refresh and perform schema merging again. It is time-consuming especially for very many parquet files.

With the new `FSBasedParquetRelation`, although `MetadataCache` is not `transient` now, `MetadataCache.refresh()` still performs schema merging again when the relation is deserialized.

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

Closes #6012 from viirya/without_remerge_schema and squashes the following commits:

2663957 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into without_remerge_schema
6ac7d93 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into without_remerge_schema
b0fc09b [Liang-Chi Hsieh] Don't generate and merge parquetSchema multiple times.
2015-05-17 15:42:21 +08:00
Cheng Lian ba4f8ca0d9 [MINOR] [SQL] Removes an unreachable case clause
This case clause is already covered by the one above, and generates a compilation warning.

Author: Cheng Lian <lian@databricks.com>

Closes #6214 from liancheng/remove-unreachable-code and squashes the following commits:

c38ca7c [Cheng Lian] Removes an unreachable case clause
2015-05-16 23:20:09 -07:00
Reynold Xin 517eb37a85 [SPARK-7654][SQL] Move JDBC into DataFrame's reader/writer interface.
Also moved all the deprecated functions into one place for SQLContext and DataFrame, and updated tests to use the new API.

Author: Reynold Xin <rxin@databricks.com>

Closes #6210 from rxin/df-writer-reader-jdbc and squashes the following commits:

7465c2c [Reynold Xin] Fixed unit test.
118e609 [Reynold Xin] Updated tests.
3441b57 [Reynold Xin] Updated javadoc.
13cdd1c [Reynold Xin] [SPARK-7654][SQL] Move JDBC into DataFrame's reader/writer interface.
2015-05-16 22:01:53 -07:00
Cheng Lian ce6391296a [HOTFIX] [SQL] Fixes DataFrameWriter.mode(String)
We forgot an assignment there.

/cc rxin

Author: Cheng Lian <lian@databricks.com>

Closes #6212 from liancheng/fix-df-writer and squashes the following commits:

711fbb0 [Cheng Lian] Adds a test case
3b72d78 [Cheng Lian] Fixes DataFrameWriter.mode(String)
2015-05-16 20:55:17 +08:00
zsxwing 47e7ffe36b [SPARK-7655][Core][SQL] Remove 'scala.concurrent.ExecutionContext.Implicits.global' in 'ask' and 'BroadcastHashJoin'
Because both `AkkaRpcEndpointRef.ask` and `BroadcastHashJoin` uses `scala.concurrent.ExecutionContext.Implicits.global`. However, because the tasks in `BroadcastHashJoin` are usually long-running tasks, which will occupy all threads in `global`. Then `ask` cannot get a chance to process the replies.

For `ask`, actually the tasks are very simple, so we can use `MoreExecutors.sameThreadExecutor()`. For `BroadcastHashJoin`, it's better to use `ThreadUtils.newDaemonCachedThreadPool`.

Author: zsxwing <zsxwing@gmail.com>

Closes #6200 from zsxwing/SPARK-7655-2 and squashes the following commits:

cfdc605 [zsxwing] Remove redundant imort and minor doc fix
cf83153 [zsxwing] Add "sameThread" and "newDaemonCachedThreadPool with maxThreadNumber" to ThreadUtils
08ad0ee [zsxwing] Remove 'scala.concurrent.ExecutionContext.Implicits.global' in 'ask' and 'BroadcastHashJoin'
2015-05-16 00:44:29 -07:00
Reynold Xin 578bfeeff5 [SPARK-7654][SQL] DataFrameReader and DataFrameWriter for input/output API
This patch introduces DataFrameWriter and DataFrameReader.

DataFrameReader interface, accessible through SQLContext.read, contains methods that create DataFrames. These methods used to reside in SQLContext. Example usage:
```scala
sqlContext.read.json("...")
sqlContext.read.parquet("...")
```

DataFrameWriter interface, accessible through DataFrame.write, implements a builder pattern to avoid the proliferation of options in writing DataFrame out. It currently implements:
- mode
- format (e.g. "parquet", "json")
- options (generic options passed down into data sources)
- partitionBy (partitioning columns)
Example usage:
```scala
df.write.mode("append").format("json").partitionBy("date").saveAsTable("myJsonTable")
```

TODO:

- [ ] Documentation update
- [ ] Move JDBC into reader / writer?
- [ ] Deprecate the old interfaces
- [ ] Move the generic load interface into reader.
- [ ] Update example code and documentation

Author: Reynold Xin <rxin@databricks.com>

Closes #6175 from rxin/reader-writer and squashes the following commits:

b146c95 [Reynold Xin] Deprecation of old APIs.
bd8abdf [Reynold Xin] Fixed merge conflict.
26abea2 [Reynold Xin] Added general load methods.
244fbec [Reynold Xin] Added equivalent to example.
4f15d92 [Reynold Xin] Added documentation for partitionBy.
7e91611 [Reynold Xin] [SPARK-7654][SQL] DataFrameReader and DataFrameWriter for input/output API.
2015-05-15 22:00:31 -07:00
Cheng Lian fdf5bba35d [SPARK-7591] [SQL] Partitioning support API tweaks
Please see [SPARK-7591] [1] for the details.

/cc rxin marmbrus yhuai

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

Author: Cheng Lian <lian@databricks.com>

Closes #6150 from liancheng/spark-7591 and squashes the following commits:

af422e7 [Cheng Lian] Addresses @rxin's comments
37d1738 [Cheng Lian] Fixes HadoopFsRelation partition columns initialization
2fc680a [Cheng Lian] Fixes Scala style issue
189ad23 [Cheng Lian] Removes HadoopFsRelation constructor arguments
522c24e [Cheng Lian] Adds OutputWriterFactory
047d40d [Cheng Lian] Renames FSBased* to HadoopFs*, also renamed FSBasedParquetRelation back to ParquetRelation2
2015-05-15 16:20:49 +08:00
Yin Huai e8f0e016ea [SQL] When creating partitioned table scan, explicitly create UnionRDD.
Otherwise, it will cause stack overflow when there are many partitions.

Author: Yin Huai <yhuai@databricks.com>

Closes #6162 from yhuai/partitionUnionedRDD and squashes the following commits:

fa016d8 [Yin Huai] Explicitly create UnionRDD.
2015-05-15 12:04:26 +08:00
Michael Armbrust 6d0633e3ec [SPARK-7548] [SQL] Add explode function for DataFrames
Add an `explode` function for dataframes and modify the analyzer so that single table generating functions can be present in a select clause along with other expressions.   There are currently the following restrictions:
 - only top level TGFs are allowed (i.e. no `select(explode('list) + 1)`)
 - only one may be present in a single select to avoid potentially confusing implicit Cartesian products.

TODO:
 - [ ] Python

Author: Michael Armbrust <michael@databricks.com>

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

7ee2c87 [Michael Armbrust] whitespace
6f80ba3 [Michael Armbrust] Update dataframe.py
c176c89 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explodeFunction
81b5da3 [Michael Armbrust] style
d3faa05 [Michael Armbrust] fix self join case
f9e1e3e [Michael Armbrust] fix python, add since
4f0d0a9 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explodeFunction
e710fe4 [Michael Armbrust] add java and python
52ca0dc [Michael Armbrust] [SPARK-7548][SQL] Add explode function for dataframes.
2015-05-14 19:49:44 -07:00
Wenchen Fan f2cd00be35 [SQL][minor] rename apply for QueryPlanner
A follow-up of https://github.com/apache/spark/pull/5624

Author: Wenchen Fan <cloud0fan@outlook.com>

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

971a92b [Wenchen Fan] use plan instead of execute
24c5ffe [Wenchen Fan] rename apply
2015-05-14 10:25:18 -07:00
zsxwing 728af88cf6 [HOTFIX] Use 'new Job' in fsBasedParquet.scala
Same issue as #6095

cc liancheng

Author: zsxwing <zsxwing@gmail.com>

Closes #6136 from zsxwing/hotfix and squashes the following commits:

4beea54 [zsxwing] Use 'new Job' in fsBasedParquet.scala
2015-05-13 17:58:29 -07:00
Venkata Ramana Gollamudi 59aaa1dad6 [SPARK-7601] [SQL] Support Insert into JDBC Datasource
Supported InsertableRelation for JDBC Datasource JDBCRelation.
Example usage:
sqlContext.sql(
      s"""
        |CREATE TEMPORARY TABLE testram1
        |USING org.apache.spark.sql.jdbc
        |OPTIONS (url '$url', dbtable 'testram1', user 'xx', password 'xx', driver 'com.h2.Driver')
      """.stripMargin.replaceAll("\n", " "))

sqlContext.sql("insert into table testram1 select * from testsrc")
sqlContext.sql("insert overwrite table testram1 select * from testsrc")

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

Closes #6121 from gvramana/JDBCDatasource_insert and squashes the following commits:

f3fb5f1 [Venkata Ramana Gollamudi] Support for JDBC Datasource InsertableRelation
2015-05-13 17:24:04 -07:00
Josh Rosen 73bed408fb [SPARK-7081] Faster sort-based shuffle path using binary processing cache-aware sort
This patch introduces a new shuffle manager that enhances the existing sort-based shuffle with a new cache-friendly sort algorithm that operates directly on binary data. The goals of this patch are to lower memory usage and Java object overheads during shuffle and to speed up sorting. It also lays groundwork for follow-up patches that will enable end-to-end processing of serialized records.

The new shuffle manager, `UnsafeShuffleManager`, can be enabled by setting `spark.shuffle.manager=tungsten-sort` in SparkConf.

The new shuffle manager uses directly-managed memory to implement several performance optimizations for certain types of shuffles. In cases where the new performance optimizations cannot be applied, the new shuffle manager delegates to SortShuffleManager to handle those shuffles.

UnsafeShuffleManager's optimizations will apply when _all_ of the following conditions hold:

 - The shuffle dependency specifies no aggregation or output ordering.
 - The shuffle serializer supports relocation of serialized values (this is currently supported
   by KryoSerializer and Spark SQL's custom serializers).
 - The shuffle produces fewer than 16777216 output partitions.
 - No individual record is larger than 128 MB when serialized.

In addition, extra spill-merging optimizations are automatically applied when the shuffle compression codec supports concatenation of serialized streams. This is currently supported by Spark's LZF serializer.

At a high-level, UnsafeShuffleManager's design is similar to Spark's existing SortShuffleManager.  In sort-based shuffle, incoming records are sorted according to their target partition ids, then written to a single map output file. Reducers fetch contiguous regions of this file in order to read their portion of the map output. In cases where the map output data is too large to fit in memory, sorted subsets of the output can are spilled to disk and those on-disk files are merged to produce the final output file.

UnsafeShuffleManager optimizes this process in several ways:

 - Its sort operates on serialized binary data rather than Java objects, which reduces memory consumption and GC overheads. This optimization requires the record serializer to have certain properties to allow serialized records to be re-ordered without requiring deserialization.  See SPARK-4550, where this optimization was first proposed and implemented, for more details.

 - It uses a specialized cache-efficient sorter (UnsafeShuffleExternalSorter) that sorts arrays of compressed record pointers and partition ids. By using only 8 bytes of space per record in the sorting array, this fits more of the array into cache.

 - The spill merging procedure operates on blocks of serialized records that belong to the same partition and does not need to deserialize records during the merge.

 - When the spill compression codec supports concatenation of compressed data, the spill merge simply concatenates the serialized and compressed spill partitions to produce the final output partition.  This allows efficient data copying methods, like NIO's `transferTo`, to be used and avoids the need to allocate decompression or copying buffers during the merge.

The shuffle read path is unchanged.

This patch is similar to [SPARK-4550](http://issues.apache.org/jira/browse/SPARK-4550) / #4450 but uses a slightly different implementation. The `unsafe`-based implementation featured in this patch lays the groundwork for followup patches that will enable sorting to operate on serialized data pages that will be prepared by Spark SQL's new `unsafe` operators (such as the new aggregation operator introduced in #5725).

### Future work

There are several tasks that build upon this patch, which will be left to future work:

- [SPARK-7271](https://issues.apache.org/jira/browse/SPARK-7271) Redesign / extend the shuffle interfaces to accept binary data as input. The goal here is to let us bypass serialization steps in cases where the sort input is produced by an operator that operates directly on binary data.
- Extension / redesign of the `Serializer` API. We can add new methods which allow serializers to determine the size requirements for serializing objects and for serializing objects directly to a specified memory address (similar to how `UnsafeRowConverter` works in Spark SQL).

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5868 from JoshRosen/unsafe-sort and squashes the following commits:

ef0a86e [Josh Rosen] Fix scalastyle errors
7610f2f [Josh Rosen] Add tests for proper cleanup of shuffle data.
d494ffe [Josh Rosen] Fix deserialization of JavaSerializer instances.
52a9981 [Josh Rosen] Fix some bugs in the address packing code.
51812a7 [Josh Rosen] Change shuffle manager sort name to tungsten-sort
4023fa4 [Josh Rosen] Add @Private annotation to some Java classes.
de40b9d [Josh Rosen] More comments to try to explain metrics code
df07699 [Josh Rosen] Attempt to clarify confusing metrics update code
5e189c6 [Josh Rosen] Track time spend closing / flushing files; split TimeTrackingOutputStream into separate file.
d5779c6 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
c2ce78e [Josh Rosen] Fix a missed usage of MAX_PARTITION_ID
e3b8855 [Josh Rosen] Cleanup in UnsafeShuffleWriter
4a2c785 [Josh Rosen] rename 'sort buffer' to 'pointer array'
6276168 [Josh Rosen] Remove ability to disable spilling in UnsafeShuffleExternalSorter.
57312c9 [Josh Rosen] Clarify fileBufferSize units
2d4e4f4 [Josh Rosen] Address some minor comments in UnsafeShuffleExternalSorter.
fdcac08 [Josh Rosen] Guard against overflow when expanding sort buffer.
85da63f [Josh Rosen] Cleanup in UnsafeShuffleSorterIterator.
0ad34da [Josh Rosen] Fix off-by-one in nextInt() call
56781a1 [Josh Rosen] Rename UnsafeShuffleSorter to UnsafeShuffleInMemorySorter
e995d1a [Josh Rosen] Introduce MAX_SHUFFLE_OUTPUT_PARTITIONS.
e58a6b4 [Josh Rosen] Add more tests for PackedRecordPointer encoding.
4f0b770 [Josh Rosen] Attempt to implement proper shuffle write metrics.
d4e6d89 [Josh Rosen] Update to bit shifting constants
69d5899 [Josh Rosen] Remove some unnecessary override vals
8531286 [Josh Rosen] Add tests that automatically trigger spills.
7c953f9 [Josh Rosen] Add test that covers UnsafeShuffleSortDataFormat.swap().
e1855e5 [Josh Rosen] Fix a handful of misc. IntelliJ inspections
39434f9 [Josh Rosen] Avoid integer multiplication overflow in getMemoryUsage (thanks FindBugs!)
1e3ad52 [Josh Rosen] Delete unused ByteBufferOutputStream class.
ea4f85f [Josh Rosen] Roll back an unnecessary change in Spillable.
ae538dc [Josh Rosen] Document UnsafeShuffleManager.
ec6d626 [Josh Rosen] Add notes on maximum # of supported shuffle partitions.
0d4d199 [Josh Rosen] Bump up shuffle.memoryFraction to make tests pass.
b3b1924 [Josh Rosen] Properly implement close() and flush() in DummySerializerInstance.
1ef56c7 [Josh Rosen] Revise compression codec support in merger; test cross product of configurations.
b57c17f [Josh Rosen] Disable some overly-verbose logs that rendered DEBUG useless.
f780fb1 [Josh Rosen] Add test demonstrating which compression codecs support concatenation.
4a01c45 [Josh Rosen] Remove unnecessary log message
27b18b0 [Josh Rosen] That for inserting records AT the max record size.
fcd9a3c [Josh Rosen] Add notes + tests for maximum record / page sizes.
9d1ee7c [Josh Rosen] Fix MiMa excludes for ShuffleWriter change
fd4bb9e [Josh Rosen] Use own ByteBufferOutputStream rather than Kryo's
67d25ba [Josh Rosen] Update Exchange operator's copying logic to account for new shuffle manager
8f5061a [Josh Rosen] Strengthen assertion to check partitioning
01afc74 [Josh Rosen] Actually read data in UnsafeShuffleWriterSuite
1929a74 [Josh Rosen] Update to reflect upstream ShuffleBlockManager -> ShuffleBlockResolver rename.
e8718dd [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
9b7ebed [Josh Rosen] More defensive programming RE: cleaning up spill files and memory after errors
7cd013b [Josh Rosen] Begin refactoring to enable proper tests for spilling.
722849b [Josh Rosen] Add workaround for transferTo() bug in merging code; refactor tests.
9883e30 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
b95e642 [Josh Rosen] Refactor and document logic that decides when to spill.
1ce1300 [Josh Rosen] More minor cleanup
5e8cf75 [Josh Rosen] More minor cleanup
e67f1ea [Josh Rosen] Remove upper type bound in ShuffleWriter interface.
cfe0ec4 [Josh Rosen] Address a number of minor review comments:
8a6fe52 [Josh Rosen] Rename UnsafeShuffleSpillWriter to UnsafeShuffleExternalSorter
11feeb6 [Josh Rosen] Update TODOs related to shuffle write metrics.
b674412 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
aaea17b [Josh Rosen] Add comments to UnsafeShuffleSpillWriter.
4f70141 [Josh Rosen] Fix merging; now passes UnsafeShuffleSuite tests.
133c8c9 [Josh Rosen] WIP towards testing UnsafeShuffleWriter.
f480fb2 [Josh Rosen] WIP in mega-refactoring towards shuffle-specific sort.
57f1ec0 [Josh Rosen] WIP towards packed record pointers for use in optimized shuffle sort.
69232fd [Josh Rosen] Enable compressible address encoding for off-heap mode.
7ee918e [Josh Rosen] Re-order imports in tests
3aeaff7 [Josh Rosen] More refactoring and cleanup; begin cleaning iterator interfaces
3490512 [Josh Rosen] Misc. cleanup
f156a8f [Josh Rosen] Hacky metrics integration; refactor some interfaces.
2776aca [Josh Rosen] First passing test for ExternalSorter.
5e100b2 [Josh Rosen] Super-messy WIP on external sort
595923a [Josh Rosen] Remove some unused variables.
8958584 [Josh Rosen] Fix bug in calculating free space in current page.
f17fa8f [Josh Rosen] Add missing newline
c2fca17 [Josh Rosen] Small refactoring of SerializerPropertiesSuite to enable test re-use:
b8a09fe [Josh Rosen] Back out accidental log4j.properties change
bfc12d3 [Josh Rosen] Add tests for serializer relocation property.
240864c [Josh Rosen] Remove PrefixComputer and require prefix to be specified as part of insert()
1433b42 [Josh Rosen] Store record length as int instead of long.
026b497 [Josh Rosen] Re-use a buffer in UnsafeShuffleWriter
0748458 [Josh Rosen] Port UnsafeShuffleWriter to Java.
87e721b [Josh Rosen] Renaming and comments
d3cc310 [Josh Rosen] Flag that SparkSqlSerializer2 supports relocation
e2d96ca [Josh Rosen] Expand serializer API and use new function to help control when new UnsafeShuffle path is used.
e267cee [Josh Rosen] Fix compilation of UnsafeSorterSuite
9c6cf58 [Josh Rosen] Refactor to use DiskBlockObjectWriter.
253f13e [Josh Rosen] More cleanup
8e3ec20 [Josh Rosen] Begin code cleanup.
4d2f5e1 [Josh Rosen] WIP
3db12de [Josh Rosen] Minor simplification and sanity checks in UnsafeSorter
767d3ca [Josh Rosen] Fix invalid range in UnsafeSorter.
e900152 [Josh Rosen] Add test for empty iterator in UnsafeSorter
57a4ea0 [Josh Rosen] Make initialSize configurable in UnsafeSorter
abf7bfe [Josh Rosen] Add basic test case.
81d52c5 [Josh Rosen] WIP on UnsafeSorter
2015-05-13 17:07:31 -07:00
Reynold Xin e683182c3e [SQL] Move some classes into packages that are more appropriate.
JavaTypeInference into catalyst
types.DateUtils into catalyst
CacheManager into execution
DefaultParserDialect into catalyst

Author: Reynold Xin <rxin@databricks.com>

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

3fc9613 [Reynold Xin] Fixed import ordering.
83d9ff4 [Reynold Xin] Fixed codegen tests.
e271e86 [Reynold Xin] mima
f4e24a6 [Reynold Xin] [SQL] Move some classes into packages that are more appropriate.
2015-05-13 16:15:31 -07:00
Wenchen Fan 213a6f30fe [SPARK-7551][DataFrame] support backticks for DataFrame attribute resolution
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6074 from cloud-fan/7551 and squashes the following commits:

e6f579e [Wenchen Fan] allow space
2b86699 [Wenchen Fan] handle blank
e218d99 [Wenchen Fan] address comments
54c4209 [Wenchen Fan] fix 7551
2015-05-13 12:47:48 -07:00
Cheng Lian 7ff16e8abe [SPARK-7567] [SQL] Migrating Parquet data source to FSBasedRelation
This PR migrates Parquet data source to the newly introduced `FSBasedRelation`. `FSBasedParquetRelation` is created to replace `ParquetRelation2`. Major differences are:

1. Partition discovery code has been factored out to `FSBasedRelation`
1. `AppendingParquetOutputFormat` is not used now. Instead, an anonymous subclass of `ParquetOutputFormat` is used to handle appending and writing dynamic partitions
1. When scanning partitioned tables, `FSBasedParquetRelation.buildScan` only builds an `RDD[Row]` for a single selected partition
1. `FSBasedParquetRelation` doesn't rely on Catalyst expressions for filter push down, thus it doesn't extend `CatalystScan` anymore

   After migrating `JSONRelation` (which extends `CatalystScan`), we can remove `CatalystScan`.

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

Author: Cheng Lian <lian@databricks.com>

Closes #6090 from liancheng/parquet-migration and squashes the following commits:

6063f87 [Cheng Lian] Casts to OutputCommitter rather than FileOutputCommtter
bfd1cf0 [Cheng Lian] Fixes compilation error introduced while rebasing
f9ea56e [Cheng Lian] Adds ParquetRelation2 related classes to MiMa check whitelist
261d8c1 [Cheng Lian] Minor bug fix and more tests
db65660 [Cheng Lian] Migrates Parquet data source to FSBasedRelation
2015-05-13 11:04:10 -07:00
Cheng Hao 0da254fb29 [SPARK-6734] [SQL] Add UDTF.close support in Generate
Some third-party UDTF extensions generate additional rows in the "GenericUDTF.close()" method, which is supported / documented by Hive.
https://cwiki.apache.org/confluence/display/Hive/DeveloperGuide+UDTF
However, Spark SQL ignores the "GenericUDTF.close()", and it causes bug while porting job from Hive to Spark SQL.

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

Closes #5383 from chenghao-intel/udtf_close and squashes the following commits:

98b4e4b [Cheng Hao] Support UDTF.close
2015-05-14 00:14:59 +08:00
Yin Huai b061bd517a [SQL] In InsertIntoFSBasedRelation.insert, log cause before abort job/task.
We need to add a log entry before calling `abortTask`/`abortJob`. Otherwise, an exception from `abortTask`/`abortJob` will shadow the real cause.

cc liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #6105 from yhuai/logCause and squashes the following commits:

8dfe0d8 [Yin Huai] Log cause.
2015-05-13 23:36:19 +08:00
Cheng Lian 10c546e9d4 [SPARK-7599] [SQL] Don't restrict customized output committers to be subclasses of FileOutputCommitter
Author: Cheng Lian <lian@databricks.com>

Closes #6118 from liancheng/spark-7599 and squashes the following commits:

31e1bd6 [Cheng Lian] Don't restrict customized output committers to be subclasses of FileOutputCommitter
2015-05-13 07:35:55 -07:00
Reynold Xin 97dee313f2 [SPARK-7321][SQL] Add Column expression for conditional statements (when/otherwise)
This builds on https://github.com/apache/spark/pull/5932 and should close https://github.com/apache/spark/pull/5932 as well.

As an example:
```python
df.select(when(df['age'] == 2, 3).otherwise(4).alias("age")).collect()
```

Author: Reynold Xin <rxin@databricks.com>
Author: kaka1992 <kaka_1992@163.com>

Closes #6072 from rxin/when-expr and squashes the following commits:

8f49201 [Reynold Xin] Throw exception if otherwise is applied twice.
0455eda [Reynold Xin] Reset run-tests.
bfb9d9f [Reynold Xin] Updated documentation and test cases.
762f6a5 [Reynold Xin] Merge pull request #5932 from kaka1992/IFCASE
95724c6 [kaka1992] Update
8218d0a [kaka1992] Update
801009e [kaka1992] Update
76d6346 [kaka1992] [SPARK-7321][SQL] Add Column expression for conditional statements (if, case)
2015-05-12 21:43:34 -07:00
Reynold Xin 8fd55358b7 [SPARK-7588] Document all SQL/DataFrame public methods with @since tag
This pull request adds since tag to all public methods/classes in SQL/DataFrame to indicate which version the methods/classes were first added.

Author: Reynold Xin <rxin@databricks.com>

Closes #6101 from rxin/tbc and squashes the following commits:

ed55e11 [Reynold Xin] Add since version to all DataFrame methods.
2015-05-12 18:37:02 -07:00
zsxwing 247b70349c [HOTFIX] Use the old Job API to support old Hadoop versions
#5526 uses `Job.getInstance`, which does not exist in the old Hadoop versions. Just use `new Job` to replace it.

cc liancheng

Author: zsxwing <zsxwing@gmail.com>

Closes #6095 from zsxwing/hotfix and squashes the following commits:

b0c2049 [zsxwing] Use the old Job API to support old Hadoop versions
2015-05-13 08:33:24 +08:00
Venkata Ramana Gollamudi 455551d1c6 [SPARK-7484][SQL]Support jdbc connection properties
Few jdbc drivers like SybaseIQ support passing username and password only through connection properties. So the same needs to be supported for
SQLContext.jdbc, dataframe.createJDBCTable and dataframe.insertIntoJDBC.
Added as default arguments or overrided function to support backward compatability.

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

Closes #6009 from gvramana/add_jdbc_conn_properties and squashes the following commits:

396a0d0 [Venkata Ramana Gollamudi] fixed comments
d66dd8c [Venkata Ramana Gollamudi] fixed comments
1b8cd8c [Venkata Ramana Gollamudi] Support jdbc connection properties
2015-05-12 14:37:23 -07:00
Wenchen Fan 4e290522c2 [SPARK-7276] [DATAFRAME] speed up DataFrame.select by collapsing Project
Author: Wenchen Fan <cloud0fan@outlook.com>

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

ee4a1e1 [Wenchen Fan] fix rebase mistake
a3b565d [Wenchen Fan] refactor
99deb5d [Wenchen Fan] add test
f1f67ad [Wenchen Fan] fix 7276
2015-05-12 11:51:55 -07:00
Wenchen Fan bfcaf8adcd [DataFrame][minor] support column in field accessor
Minor improvement, now we can use `Column` as extraction expression.

Author: Wenchen Fan <cloud0fan@outlook.com>

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

0fdefb7 [Wenchen Fan] support column in field accessor
2015-05-12 10:37:57 -07:00
Cheng Lian 0595b6de8f [SPARK-3928] [SPARK-5182] [SQL] Partitioning support for the data sources API
This PR adds partitioning support for the external data sources API. It aims to simplify development of file system based data sources, and provide first class partitioning support for both read path and write path.  Existing data sources like JSON and Parquet can be simplified with this work.

## New features provided

1. Hive compatible partition discovery

   This actually generalizes the partition discovery strategy used in Parquet data source in Spark 1.3.0.

1. Generalized partition pruning optimization

   Now partition pruning is handled during physical planning phase.  Specific data sources don't need to worry about this harness anymore.

   (This also implies that we can remove `CatalystScan` after migrating the Parquet data source, since now we don't need to pass Catalyst expressions to data source implementations.)

1. Insertion with dynamic partitions

   When inserting data to a `FSBasedRelation`, data can be partitioned dynamically by specified partition columns.

## New structures provided

### Developer API

1. `FSBasedRelation`

   Base abstract class for file system based data sources.

1. `OutputWriter`

   Base abstract class for output row writers, responsible for writing a single row object.

1. `FSBasedRelationProvider`

   A new relation provider for `FSBasedRelation` subclasses. Note that data sources extending `FSBasedRelation` don't need to extend `RelationProvider` and `SchemaRelationProvider`.

### User API

New overloaded versions of

1. `DataFrame.save()`
1. `DataFrame.saveAsTable()`
1. `SQLContext.load()`

are provided to allow users to save/load DataFrames with user defined dynamic partition columns.

### Spark SQL query planning

1. `InsertIntoFSBasedRelation`

   Used to implement write path for `FSBasedRelation`s.

1. New rules for `FSBasedRelation` in `DataSourceStrategy`

   These are added to hook `FSBasedRelation` into physical query plan in read path, and perform partition pruning.

## TODO

- [ ] Use scratch directories when overwriting a table with data selected from itself.

      Currently, this is not supported, because the table been overwritten is always deleted before writing any data to it.

- [ ] When inserting with dynamic partition columns, use external sorter to group the data first.

      This ensures that we only need to open a single `OutputWriter` at a time.  For data sources like Parquet, `OutputWriter`s can be quite memory consuming.  One issue is that, this approach breaks the row distribution in the original DataFrame.  However, we did't promise to preserve data distribution when writing a DataFrame.

- [x] More tests.  Specifically, test cases for

      - [x] Self-join
      - [x] Loading partitioned relations with a subset of partition columns stored in data files.
      - [x] `SQLContext.load()` with user defined dynamic partition columns.

## Parquet data source migration

Parquet data source migration is covered in PR https://github.com/liancheng/spark/pull/6, which is against this PR branch and for preview only. A formal PR need to be made after this one is merged.

Author: Cheng Lian <lian@databricks.com>

Closes #5526 from liancheng/partitioning-support and squashes the following commits:

5351a1b [Cheng Lian] Fixes compilation error introduced while rebasing
1f9b1a5 [Cheng Lian] Tweaks data schema passed to FSBasedRelations
43ba50e [Cheng Lian] Avoids serializing generated projection code
edf49e7 [Cheng Lian] Removed commented stale code block
348a922 [Cheng Lian] Adds projection in FSBasedRelation.buildScan(requiredColumns, inputPaths)
ad4d4de [Cheng Lian] Enables HDFS style globbing
8d12e69 [Cheng Lian] Fixes compilation error
c71ac6c [Cheng Lian] Addresses comments from @marmbrus
7552168 [Cheng Lian] Fixes typo in MimaExclude.scala
0349e09 [Cheng Lian] Fixes compilation error introduced while rebasing
52b0c9b [Cheng Lian] Adjusts project/MimaExclude.scala
c466de6 [Cheng Lian] Addresses comments
bc3f9b4 [Cheng Lian] Uses projection to separate partition columns and data columns while inserting rows
795920a [Cheng Lian] Fixes compilation error after rebasing
0b8cd70 [Cheng Lian] Adds Scala/Catalyst row conversion when writing non-partitioned tables
fa543f3 [Cheng Lian] Addresses comments
5849dd0 [Cheng Lian] Fixes doc typos.  Fixes partition discovery refresh.
51be443 [Cheng Lian] Replaces FSBasedRelation.outputCommitterClass with FSBasedRelation.prepareForWrite
c4ed4fe [Cheng Lian] Bug fixes and a new test suite
a29e663 [Cheng Lian] Bug fix: should only pass actuall data files to FSBaseRelation.buildScan
5f423d3 [Cheng Lian] Bug fixes. Lets data source to customize OutputCommitter rather than OutputFormat
54c3d7b [Cheng Lian] Enforces that FileOutputFormat must be used
be0c268 [Cheng Lian] Uses TaskAttempContext rather than Configuration in OutputWriter.init
0bc6ad1 [Cheng Lian] Resorts to new Hadoop API, and now FSBasedRelation can customize output format class
f320766 [Cheng Lian] Adds prepareForWrite() hook, refactored writer containers
422ff4a [Cheng Lian] Fixes style issue
ce52353 [Cheng Lian] Adds new SQLContext.load() overload with user defined dynamic partition columns
8d2ff71 [Cheng Lian] Merges partition columns when reading partitioned relations
ca1805b [Cheng Lian] Removes duplicated partition discovery code in new Parquet
f18dec2 [Cheng Lian] More strict schema checking
b746ab5 [Cheng Lian] More tests
9b487bf [Cheng Lian] Fixes compilation errors introduced while rebasing
ea6c8dd [Cheng Lian] Removes remote debugging stuff
327bb1d [Cheng Lian] Implements partitioning support for data sources API
3c5073a [Cheng Lian] Fixes SaveModes used in test cases
fb5a607 [Cheng Lian] Fixes compilation error
9d17607 [Cheng Lian] Adds the contract that OutputWriter should have zero-arg constructor
5de194a [Cheng Lian] Forgot Apache licence header
95d0b4d [Cheng Lian] Renames PartitionedSchemaRelationProvider to FSBasedRelationProvider
770b5ba [Cheng Lian] Adds tests for FSBasedRelation
3ba9bbf [Cheng Lian] Adds DataFrame.saveAsTable() overrides which support partitioning
1b8231f [Cheng Lian] Renames FSBasedPrunedFilteredScan to FSBasedRelation
aa8ba9a [Cheng Lian] Javadoc fix
012ed2d [Cheng Lian] Adds PartitioningOptions
7dd8dd5 [Cheng Lian] Adds new interfaces and stub methods for data sources API partitioning support
2015-05-13 01:32:28 +08:00
Reynold Xin 16696759e9 [SQL] Rename Dialect -> ParserDialect.
Author: Reynold Xin <rxin@databricks.com>

Closes #6071 from rxin/parserdialect and squashes the following commits:

ca2eb31 [Reynold Xin] Rename Dialect -> ParserDialect.
2015-05-11 22:06:56 -07:00
Reynold Xin b6bf4f76c7 [SPARK-7324] [SQL] DataFrame.dropDuplicates
This should also close https://github.com/apache/spark/pull/5870

Author: Reynold Xin <rxin@databricks.com>

Closes #6066 from rxin/dropDups and squashes the following commits:

130692f [Reynold Xin] [SPARK-7324][SQL] DataFrame.dropDuplicates
2015-05-11 19:15:14 -07:00
Reynold Xin 87229c95c6 Updated DataFrame.saveAsTable Hive warning to include SPARK-7550 ticket.
So users that are interested in this can track it easily.

Author: Reynold Xin <rxin@databricks.com>

Closes #6067 from rxin/SPARK-7550 and squashes the following commits:

ee0e34c [Reynold Xin] Updated DataFrame.saveAsTable Hive warning to include SPARK-7550 ticket.
2015-05-11 18:10:45 -07:00
Reynold Xin 3a9b6997df [SPARK-7462][SQL] Update documentation for retaining grouping columns in DataFrames.
Author: Reynold Xin <rxin@databricks.com>

Closes #6062 from rxin/agg-retain-doc and squashes the following commits:

43e511e [Reynold Xin] [SPARK-7462][SQL] Update documentation for retaining grouping columns in DataFrames.
2015-05-11 18:07:12 -07:00
madhukar 57255dcd79 [SPARK-7084] improve saveAsTable documentation
Author: madhukar <phatak.dev@gmail.com>

Closes #5654 from phatak-dev/master and squashes the following commits:

386f407 [madhukar] #5654 updated for all the methods
2c997c5 [madhukar] Merge branch 'master' of https://github.com/apache/spark
00bc819 [madhukar] Merge branch 'master' of https://github.com/apache/spark
2a802c6 [madhukar] #5654 updated the doc according to comments
866e8df [madhukar] [SPARK-7084] improve saveAsTable documentation
2015-05-11 17:04:11 -07:00
LCY Vincent a8ea09683a Update Documentation: leftsemi instead of semijoin
should sync up with here?
119f45d61d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala (L26)

Author: LCY Vincent <lauchunyin@gmail.com>

Closes #5944 from vincentlaucy/master and squashes the following commits:

fc0e454 [LCY Vincent] Update DataFrame.scala
2015-05-11 14:48:10 -07:00
Reynold Xin 0a4844f90a [SPARK-7462] By default retain group by columns in aggregate
Updated Java, Scala, Python, and R.

Author: Reynold Xin <rxin@databricks.com>
Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #5996 from rxin/groupby-retain and squashes the following commits:

aac7119 [Reynold Xin] Merge branch 'groupby-retain' of github.com:rxin/spark into groupby-retain
f6858f6 [Reynold Xin] Merge branch 'master' into groupby-retain
5f923c0 [Reynold Xin] Merge pull request #15 from shivaram/sparkr-groupby-retrain
c1de670 [Shivaram Venkataraman] Revert workaround in SparkR to retain grouped cols Based on reverting code added in commit 9a6be746ef
b8b87e1 [Reynold Xin] Fixed DataFrameJoinSuite.
d910141 [Reynold Xin] Updated rest of the files
1e6e666 [Reynold Xin] [SPARK-7462] By default retain group by columns in aggregate
2015-05-11 11:35:16 -07:00
Oleg Sidorkin d7a37bcaf1 [SPARK-7345][SQL] Spark cannot detect renamed columns using JDBC connector
Issue appears when one tries to create DataFrame using sqlContext.load("jdbc"...) statement when "dbtable" contains query with renamed columns.
If original column is used in SQL query once the resulting DataFrame will contain non-renamed column.
If original column is used in SQL query several times with different aliases, sqlContext.load will fail.
Original implementation of JDBCRDD.resolveTable uses getColumnName to detect column names in RDD schema.
Suggested implementation uses getColumnLabel to handle column renames in SQL statement which is aware of SQL "AS" statement.

Readings:
http://stackoverflow.com/questions/4271152/getcolumnlabel-vs-getcolumnname
http://stackoverflow.com/questions/12259829/jdbc-getcolumnname-getcolumnlabel-db2

Official documentation unfortunately a bit misleading in definition of "suggested title" purpose however clearly defines behavior of AS keyword in SQL statement.
http://docs.oracle.com/javase/7/docs/api/java/sql/ResultSetMetaData.html
getColumnLabel - Gets the designated column's suggested title for use in printouts and displays. The suggested title is usually specified by the SQL AS clause. If a SQL AS is not specified, the value returned from getColumnLabel will be the same as the value returned by the getColumnName method.

Author: Oleg Sidorkin <oleg.sidorkin@gmail.com>

Closes #6032 from osidorkin/master and squashes the following commits:

10fc44b [Oleg Sidorkin] [SPARK-7345][SQL] Regression test for JDBCSuite (resolved scala style test error)
2aaf6f7 [Oleg Sidorkin] [SPARK-7345][SQL] Regression test for JDBCSuite (renamed fields in JDBC query)
b7d5b22 [Oleg Sidorkin] [SPARK-7345][SQL] Regression test for JDBCSuite
09559a0 [Oleg Sidorkin] [SPARK-7345][SQL] Spark cannot detect renamed columns using JDBC connector
2015-05-10 01:31:34 -07:00
Josh Rosen cde5483884 [SPARK-7375] [SQL] Avoid row copying in exchange when sort.serializeMapOutputs takes effect
This patch refactors the SQL `Exchange` operator's logic for determining whether map outputs need to be copied before being shuffled. As part of this change, we'll now avoid unnecessary copies in cases where sort-based shuffle operates on serialized map outputs (as in #4450 /
SPARK-4550).

This patch also includes a change to copy the input to RangePartitioner partition bounds calculation, which is necessary because this calculation buffers mutable Java objects.

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5948 from JoshRosen/SPARK-7375 and squashes the following commits:

f305ff3 [Josh Rosen] Reduce scope of some variables in Exchange
899e1d7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-7375
6a6bfce [Josh Rosen] Fix issue related to RangePartitioning:
ad006a4 [Josh Rosen] [SPARK-7375] Avoid defensive copying in exchange operator when sort.serializeMapOutputs takes effect.
2015-05-08 22:09:55 -04:00
Shivaram Venkataraman 0a901dd3a1 [SPARK-7231] [SPARKR] Changes to make SparkR DataFrame dplyr friendly.
Changes include
1. Rename sortDF to arrange
2. Add new aliases `group_by` and `sample_frac`, `summarize`
3. Add more user friendly column addition (mutate), rename
4. Support mean as an alias for avg in Scala and also support n_distinct, n as in dplyr

Using these changes we can pretty much run the examples as described in http://cran.rstudio.com/web/packages/dplyr/vignettes/introduction.html with the same syntax

The only thing missing in SparkR is auto resolving column names when used in an expression i.e. making something like `select(flights, delay)` works in dply but we right now need `select(flights, flights$delay)` or `select(flights, "delay")`. But this is a complicated change and I'll file a new issue for it

cc sun-rui rxin

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #6005 from shivaram/sparkr-df-api and squashes the following commits:

5e0716a [Shivaram Venkataraman] Fix some roxygen bugs
1254953 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/apache/spark into sparkr-df-api
0521149 [Shivaram Venkataraman] Changes to make SparkR DataFrame dplyr friendly. Changes include 1. Rename sortDF to arrange 2. Add new aliases `group_by` and `sample_frac`, `summarize` 3. Add more user friendly column addition (mutate), rename 4. Support mean as an alias for avg in Scala and also support n_distinct, n as in dplyr
2015-05-08 18:29:57 -07:00
Andrew Or bd61f07039 [SPARK-7469] [SQL] DAG visualization: show SQL query operators
The DAG visualization currently displays only low-level Spark primitives (e.g. `map`, `reduceByKey`, `filter` etc.). For SQL, these aren't particularly useful. Instead, we should display higher level physical operators (e.g. `Filter`, `Exchange`, `ShuffleHashJoin`). cc marmbrus

-----------------
**Before**
<img src="https://issues.apache.org/jira/secure/attachment/12731586/before.png" width="600px"/>
-----------------
**After** (Pay attention to the words)
<img src="https://issues.apache.org/jira/secure/attachment/12731587/after.png" width="600px"/>
-----------------

Author: Andrew Or <andrew@databricks.com>

Closes #5999 from andrewor14/dag-viz-sql and squashes the following commits:

0db23a4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-viz-sql
1e211db [Andrew Or] Update comment
0d49fd6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-viz-sql
ffd237a [Andrew Or] Fix style
202dac1 [Andrew Or] Make ignoreParent false by default
e61b1ab [Andrew Or] Visualize SQL operators, not low-level Spark primitives
569034a [Andrew Or] Add a flag to ignore parent settings and scopes
2015-05-08 17:15:10 -07:00
Jacky Li 6dad76e5eb [SPARK-4699] [SQL] Make caseSensitive configurable in spark sql analyzer
based on #3558

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

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

cd51712 [wangfei] fix compile
d4b724f [wangfei] address michael's comment
af512c7 [wangfei] fix conflicts
4ef1be7 [wangfei] fix conflicts
269cf21 [scwf] fix conflicts
b73df6c [scwf] style issue
9e11752 [scwf] improve SimpleCatalystConf
b35529e [scwf] minor style
a3f7659 [scwf] remove unsed imports
2a56515 [scwf] fix conflicts
6db4bf5 [scwf] also fix for HiveContext
7fc4a98 [scwf] fix test case
d5a9933 [wangfei] fix style
eee75ba [wangfei] fix EmptyConf
6ef31cf [wangfei] revert pom changes
5d7c456 [wangfei] set CASE_SENSITIVE false in TestHive
966e719 [wangfei] set CASE_SENSITIVE false in hivecontext
fd30e25 [wangfei] added override
69b3b70 [wangfei] fix AnalysisSuite
5472b08 [wangfei] fix compile issue
56034ca [wangfei] fix conflicts and improve for catalystconf
664d1e9 [Jacky Li] Merge branch 'master' of https://github.com/apache/spark into case
12eca9a [Jacky Li] solve conflict with master
39e369c [Jacky Li] fix confilct after DataFrame PR
dee56e9 [Jacky Li] fix test case failure
05b09a3 [Jacky Li] fix conflict base on the latest master branch
73c16b1 [Jacky Li] fix bug in sql/hive
9bf4cc7 [Jacky Li] fix bug in catalyst
005c56d [Jacky Li] make SQLContext caseSensitivity configurable
6332e0f [Jacky Li] fix bug
fcbf0d9 [Jacky Li] fix scalastyle check
e7bca31 [Jacky Li] make caseSensitive configuration in Analyzer and Catalog
91b1b96 [Jacky Li] make caseSensitive configurable in Analyzer
f57f15c [Jacky Li] add testcase
578d167 [Jacky Li] make caseSensitive configurable
2015-05-08 15:25:54 -07:00
Liang-Chi Hsieh 90527f5604 [SPARK-7390] [SQL] Only merge other CovarianceCounter when its count is greater than zero
JIRA: https://issues.apache.org/jira/browse/SPARK-7390

Also fix a minor typo.

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

Closes #5931 from viirya/fix_covariancecounter and squashes the following commits:

352eda6 [Liang-Chi Hsieh] Only merge other CovarianceCounter when its count is greater than zero.
2015-05-08 14:41:16 -07:00
Wenchen Fan 2d05f325dc [SPARK-7133] [SQL] Implement struct, array, and map field accessor
It's the first step: generalize UnresolvedGetField to support all map, struct, and array
TODO: add `apply` in Scala and `__getitem__` in Python, and unify the `getItem` and `getField` methods to one single API(or should we keep them for compatibility?).

Author: Wenchen Fan <cloud0fan@outlook.com>

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

715c589 [Wenchen Fan] address comments
7ea5b31 [Wenchen Fan] fix python test
4f0833a [Wenchen Fan] add python test
f515d69 [Wenchen Fan] add apply method and test cases
8df6199 [Wenchen Fan] fix python test
239730c [Wenchen Fan] fix test compile
2a70526 [Wenchen Fan] use _bin_op in dataframe.py
6bf72bc [Wenchen Fan] address comments
3f880c3 [Wenchen Fan] add java doc
ab35ab5 [Wenchen Fan] fix python test
b5961a9 [Wenchen Fan] fix style
c9d85f5 [Wenchen Fan] generalize UnresolvedGetField to support all map, struct, and array
2015-05-08 11:49:38 -07:00
Yin Huai 3af423c92f [SPARK-6986] [SQL] Use Serializer2 in more cases.
With 0a2b15ce43, the serialization stream and deserialization stream has enough information to determine it is handling a key-value pari, a key, or a value. It is safe to use `SparkSqlSerializer2` in more cases.

Author: Yin Huai <yhuai@databricks.com>

Closes #5849 from yhuai/serializer2MoreCases and squashes the following commits:

53a5eaa [Yin Huai] Josh's comments.
487f540 [Yin Huai] Use BufferedOutputStream.
8385f95 [Yin Huai] Always create a new row at the deserialization side to work with sort merge join.
c7e2129 [Yin Huai] Update tests.
4513d13 [Yin Huai] Use Serializer2 in more places.
2015-05-07 20:59:42 -07:00
Michael Armbrust cd1d4110cf [SPARK-6908] [SQL] Use isolated Hive client
This PR switches Spark SQL's Hive support to use the isolated hive client interface introduced by #5851, instead of directly interacting with the client.  By using this isolated client we can now allow users to dynamically configure the version of Hive that they are connecting to by setting `spark.sql.hive.metastore.version` without the need recompile.  This also greatly reduces the surface area for our interaction with the hive libraries, hopefully making it easier to support other versions in the future.

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

3ce54e1 [Wenchen Fan] add CaseKeyWhen
2015-05-07 16:26:49 -07:00
Liang-Chi Hsieh ea3077f19c [SPARK-7277] [SQL] Throw exception if the property mapred.reduce.tasks is set to -1
JIRA: https://issues.apache.org/jira/browse/SPARK-7277

As automatically determining the number of reducers is not supported (`mapred.reduce.tasks` is set to `-1`), we should throw exception to users.

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

Closes #5811 from viirya/no_neg_reduce_tasks and squashes the following commits:

e518f96 [Liang-Chi Hsieh] Consider other wrong setting values.
fd9c817 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_neg_reduce_tasks
4ede705 [Liang-Chi Hsieh] Throw exception instead of warning message.
68a1c70 [Liang-Chi Hsieh] Show warning message if mapred.reduce.tasks is set to -1.
2015-05-07 16:22:45 -07:00
ksonj dec8f53719 [SPARK-7116] [SQL] [PYSPARK] Remove cache() causing memory leak
This patch simply removes a `cache()` on an intermediate RDD when evaluating Python UDFs.

Author: ksonj <kson@siberie.de>

Closes #5973 from ksonj/udf and squashes the following commits:

db5b564 [ksonj] removed TODO about cleaning up
fe70c54 [ksonj] Remove cache() causing memory leak
2015-05-07 12:04:43 -07:00
Daoyuan Wang ed9be06a47 [SPARK-7330] [SQL] avoid NPE at jdbc rdd
Thank nadavoosh point this out in #5590

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

Closes #5877 from adrian-wang/jdbcrdd and squashes the following commits:

cc11900 [Daoyuan Wang] avoid NPE in jdbcrdd
2015-05-07 10:05:01 -07:00
Shiti fa8fddffd5 [SPARK-7295][SQL] bitwise operations for DataFrame DSL
Author: Shiti <ssaxena.ece@gmail.com>

Closes #5867 from Shiti/spark-7295 and squashes the following commits:

71a9913 [Shiti] implementation for bitwise and,or, not and xor on Column with tests and docs
2015-05-07 01:00:29 -07:00
Nathan Howell 2d6612cc8b [SPARK-5938] [SPARK-5443] [SQL] Improve JsonRDD performance
This patch comprises of a few related pieces of work:

* Schema inference is performed directly on the JSON token stream
* `String => Row` conversion populate Spark SQL structures without intermediate types
* Projection pushdown is implemented via CatalystScan for DataFrame queries
* Support for the legacy parser by setting `spark.sql.json.useJacksonStreamingAPI` to `false`

Performance improvements depend on the schema and queries being executed, but it should be faster across the board. Below are benchmarks using the last.fm Million Song dataset:

```
Command                                            | Baseline | Patched
---------------------------------------------------|----------|--------
import sqlContext.implicits._                      |          |
val df = sqlContext.jsonFile("/tmp/lastfm.json")   |    70.0s |   14.6s
df.count()                                         |    28.8s |    6.2s
df.rdd.count()                                     |    35.3s |   21.5s
df.where($"artist" === "Robert Hood").collect()    |    28.3s |   16.9s
```

To prepare this dataset for benchmarking, follow these steps:

```
# Fetch the datasets from http://labrosa.ee.columbia.edu/millionsong/lastfm
wget http://labrosa.ee.columbia.edu/millionsong/sites/default/files/lastfm/lastfm_test.zip \
     http://labrosa.ee.columbia.edu/millionsong/sites/default/files/lastfm/lastfm_train.zip

# Decompress and combine, pipe through `jq -c` to ensure there is one record per line
unzip -p lastfm_test.zip lastfm_train.zip  | jq -c . > lastfm.json
```

Author: Nathan Howell <nhowell@godaddy.com>

Closes #5801 from NathanHowell/json-performance and squashes the following commits:

26fea31 [Nathan Howell] Recreate the baseRDD each for each scan operation
a7ebeb2 [Nathan Howell] Increase coverage of inserts into a JSONRelation
e06a1dd [Nathan Howell] Add comments to the `useJacksonStreamingAPI` config flag
6822712 [Nathan Howell] Split up JsonRDD2 into multiple objects
fa8234f [Nathan Howell] Wrap long lines
b31917b [Nathan Howell] Rename `useJsonRDD2` to `useJacksonStreamingAPI`
15c5d1b [Nathan Howell] JSONRelation's baseRDD need not be lazy
f8add6e [Nathan Howell] Add comments on lack of support for precision and scale DecimalTypes
fa0be47 [Nathan Howell] Remove unused default case in the field parser
80dba17 [Nathan Howell] Add comments regarding null handling and empty strings
842846d [Nathan Howell] Point the empty schema inference test at JsonRDD2
ab6ee87 [Nathan Howell] Add projection pushdown support to JsonRDD/JsonRDD2
f636c14 [Nathan Howell] Enable JsonRDD2 by default, add a flag to switch back to JsonRDD
0bbc445 [Nathan Howell] Improve JSON parsing and type inference performance
7ca70c1 [Nathan Howell] Eliminate arrow pattern, replace with pattern matches
2015-05-06 22:56:53 -07:00
Josh Rosen 002c12384d [SPARK-7311] Introduce internal Serializer API for determining if serializers support object relocation
This patch extends the `Serializer` interface with a new `Private` API which allows serializers to indicate whether they support relocation of serialized objects in serializer stream output.

This relocatibilty property is described in more detail in `Serializer.scala`, but in a nutshell a serializer supports relocation if reordering the bytes of serialized objects in serialization stream output is equivalent to having re-ordered those elements prior to serializing them.  The optimized shuffle path introduced in #4450 and #5868 both rely on serializers having this property; this patch just centralizes the logic for determining whether a serializer has this property.  I also added tests and comments clarifying when this works for KryoSerializer.

This change allows the optimizations in #4450 to be applied for shuffles that use `SqlSerializer2`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5924 from JoshRosen/SPARK-7311 and squashes the following commits:

50a68ca [Josh Rosen] Address minor nits
0a7ebd7 [Josh Rosen] Clarify reason why SqlSerializer2 supports this serializer
123b992 [Josh Rosen] Cleanup for submitting as standalone patch.
4aa61b2 [Josh Rosen] Add missing newline
2c1233a [Josh Rosen] Small refactoring of SerializerPropertiesSuite to enable test re-use:
0ba75e6 [Josh Rosen] Add tests for serializer relocation property.
450fa21 [Josh Rosen] Back out accidental log4j.properties change
86d4dcd [Josh Rosen] Flag that SparkSqlSerializer2 supports relocation
b9624ee [Josh Rosen] Expand serializer API and use new function to help control when new UnsafeShuffle path is used.
2015-05-06 10:52:55 -07:00
Yin Huai f2c47082c3 [SPARK-1442] [SQL] Window Function Support for Spark SQL
Adding more information about the implementation...

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

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

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

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

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

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

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

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

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

# Future work #

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

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

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

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

Author: Yin Huai <yhuai@databricks.com>

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

76fe1c8 [Yin Huai] Implementation.
aa2b0ae [Yin Huai] Tests.
2015-05-06 10:43:00 -07:00
Daoyuan Wang 150f671c28 [SPARK-5456] [SQL] fix decimal compare for jdbc rdd
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #5803 from adrian-wang/decimalcompare and squashes the following commits:

aef0e96 [Daoyuan Wang] add null handle
ec455b9 [Daoyuan Wang] fix decimal compare for jdbc rdd
2015-05-06 10:05:10 -07:00
Reynold Xin 322e7e7f68 [SQL] JavaDoc update for various DataFrame functions.
Author: Reynold Xin <rxin@databricks.com>

Closes #5935 from rxin/df-doc1 and squashes the following commits:

aaeaadb [Reynold Xin] [SQL] JavaDoc update for various DataFrame functions.
2015-05-06 08:50:56 -07:00
Burak Yavuz ba2b56614d [SPARK-7358][SQL] Move DataFrame mathfunctions into functions
After a discussion on the user mailing list, it was decided to put all UDF's under `o.a.s.sql.functions`

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

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

a8dc3f7 [Burak Yavuz] address comments
cf7a7bb [Burak Yavuz] [SPARK-7358] Move DataFrame mathfunctions into functions
2015-05-05 22:56:01 -07:00
Reynold Xin 1fd31ba089 [SPARK-6231][SQL/DF] Automatically resolve join condition ambiguity for self-joins.
See the comment in join function for more information.

Author: Reynold Xin <rxin@databricks.com>

Closes #5919 from rxin/self-join-resolve and squashes the following commits:

e2fb0da [Reynold Xin] Updated SQLConf comment.
7233a86 [Reynold Xin] Updated comment.
6be2b4d [Reynold Xin] Removed println
9f6b72f [Reynold Xin] [SPARK-6231][SQL/DF] Automatically resolve ambiguity in join condition for self-joins.
2015-05-05 18:59:46 -07:00
云峤 735bc3d042 [SPARK-7294][SQL] ADD BETWEEN
Author: 云峤 <chensong.cs@alibaba-inc.com>
Author: kaka1992 <kaka_1992@163.com>

Closes #5839 from kaka1992/master and squashes the following commits:

b15360d [kaka1992] Fix python unit test in sql/test. =_= I forget to commit this file last time.
f928816 [kaka1992] Fix python style in sql/test.
d2e7f72 [kaka1992] Fix python style in sql/test.
c54d904 [kaka1992] Fix empty map bug.
7e64d1e [云峤] Update
7b9b858 [云峤] undo
f080f8d [云峤] update pep8
76f0c51 [云峤] Merge remote-tracking branch 'remotes/upstream/master'
7d62368 [云峤] [SPARK-7294] ADD BETWEEN
baf839b [云峤] [SPARK-7294] ADD BETWEEN
d11d5b9 [云峤] [SPARK-7294] ADD BETWEEN
2015-05-05 13:23:53 -07:00
Burak Yavuz 18340d7be5 [SPARK-7243][SQL] Reduce size for Contingency Tables in DataFrames
Reduced take size from 1e8 to 1e6.

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5900 from brkyvz/df-cont-followup and squashes the following commits:

c11e762 [Burak Yavuz] fix grammar
b30ace2 [Burak Yavuz] address comments
a417ba5 [Burak Yavuz] [SPARK-7243][SQL] Reduce  size for Contingency Tables in DataFrames
2015-05-05 11:01:25 -07:00
Burak Yavuz 8055411170 [SPARK-7243][SQL] Contingency Tables for DataFrames
Computes a pair-wise frequency table of the given columns. Also known as cross-tabulation.
cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5842 from brkyvz/df-cont and squashes the following commits:

a07c01e [Burak Yavuz] addressed comments v4.1
ae9e01d [Burak Yavuz] fix test
9106585 [Burak Yavuz] addressed comments v4.0
bced829 [Burak Yavuz] fix merge conflicts
a63ad00 [Burak Yavuz] addressed comments v3.0
a0cad97 [Burak Yavuz] addressed comments v3.0
6805df8 [Burak Yavuz] addressed comments and fixed test
939b7c4 [Burak Yavuz] lint python
7f098bc [Burak Yavuz] add crosstab pyTest
fd53b00 [Burak Yavuz] added python support for crosstab
27a5a81 [Burak Yavuz] implemented crosstab
2015-05-04 17:02:49 -07:00
云峤 f32e69ecc3 [SPARK-7319][SQL] Improve the output from DataFrame.show()
Author: 云峤 <chensong.cs@alibaba-inc.com>

Closes #5865 from kaka1992/df.show and squashes the following commits:

c79204b [云峤] Update
a1338f6 [云峤] Update python dataFrame show test and add empty df unit test.
734369c [云峤] Update python dataFrame show test and add empty df unit test.
84aec3e [云峤] Update python dataFrame show test and add empty df unit test.
159b3d5 [云峤] update
03ef434 [云峤] update
7394fd5 [云峤] update test show
ced487a [云峤] update pep8
b6e690b [云峤] Merge remote-tracking branch 'upstream/master' into df.show
30ac311 [云峤] [SPARK-7294] ADD BETWEEN
7d62368 [云峤] [SPARK-7294] ADD BETWEEN
baf839b [云峤] [SPARK-7294] ADD BETWEEN
d11d5b9 [云峤] [SPARK-7294] ADD BETWEEN
2015-05-04 12:08:38 -07:00
tianyi 343d3bfafd [SPARK-5100] [SQL] add webui for thriftserver
This PR is a rebased version of #3946 , and mainly focused on creating an independent tab for the thrift server in spark web UI.

Features:

1. Session related statistics ( username and IP are only supported in hive-0.13.1 )
2. List all the SQL executing or executed on this server
3. Provide links to the job generated by SQL
4. Provide link to show all SQL executing or executed in a specified session

Prototype snapshots:

This is the main page for thrift server

![image](https://cloud.githubusercontent.com/assets/1411869/7361379/df7dcc64-ed89-11e4-9964-4df0b32f475e.png)

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

Closes #5730 from tianyi/SPARK-5100 and squashes the following commits:

cfd14c7 [tianyi] style fix
0efe3d5 [tianyi] revert part of pom change
c0f2fa0 [tianyi] extends HiveThriftJdbcTest to start/stop thriftserver for UI test
aa20408 [tianyi] fix style problem
c9df6f9 [tianyi] add testsuite for thriftserver ui and fix some style issue
9830199 [tianyi] add webui for thriftserver
2015-05-04 16:59:34 +08:00
Burak Yavuz 9646018bb4 [SPARK-7241] Pearson correlation for DataFrames
submitting this PR from a phone, excuse the brevity.
adds Pearson correlation to Dataframes, reusing the covariance calculation code

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5858 from brkyvz/df-corr and squashes the following commits:

285b838 [Burak Yavuz] addressed comments v2.0
d10babb [Burak Yavuz] addressed comments v0.2
4b74b24 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into df-corr
4fe693b [Burak Yavuz] addressed comments v0.1
a682d06 [Burak Yavuz] ready for PR
2015-05-03 21:44:39 -07:00
Cheng Hao 5d6b90d939 [SPARK-5213] [SQL] Pluggable SQL Parser Support
based on #4015, we should not delete `sqlParser` from sqlcontext, that leads to mima failed. Users implement dialect to give a fallback for `sqlParser`  and we should construct `sqlParser` in sqlcontext according to the dialect
`protected[sql] val sqlParser = new SparkSQLParser(getSQLDialect().parse(_))`

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

Closes #5827 from scwf/sqlparser1 and squashes the following commits:

81b9737 [scwf] comment fix
0878bd1 [scwf] remove comments
c19780b [scwf] fix mima tests
c2895cf [scwf] Merge branch 'master' of https://github.com/apache/spark into sqlparser1
493775c [Cheng Hao] update the code as feedback
81a731f [Cheng Hao] remove the unecessary comment
aab0b0b [Cheng Hao] polish the code a little bit
49b9d81 [Cheng Hao] shrink the comment for rebasing
2015-05-02 15:20:07 -07:00
Burak Yavuz 2e0f3579f1 [SPARK-7242] added python api for freqItems in DataFrames
The python api for DataFrame's plus addressed your comments from previous PR.
rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5859 from brkyvz/df-freq-py2 and squashes the following commits:

f9aa9ce [Burak Yavuz] addressed comments v0.1
4b25056 [Burak Yavuz] added python api for freqItems
2015-05-01 23:43:24 -07:00
Cheng Hao 98e7045805 [SPARK-6999] [SQL] Remove the infinite recursive method (useless)
Remove the method, since it causes infinite recursive calls. And seems it's a dummy method, since we have the API:
`def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame`

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

Closes #5804 from chenghao-intel/spark_6999 and squashes the following commits:

63220a8 [Cheng Hao] remove the infinite recursive method (useless)
2015-05-01 19:39:30 -05:00
Yin Huai 41c6a44b1a [SPARK-7312][SQL] SPARK-6913 broke jdk6 build
JIRA: https://issues.apache.org/jira/browse/SPARK-7312

Author: Yin Huai <yhuai@databricks.com>

Closes #5847 from yhuai/jdbcJava6 and squashes the following commits:

68433a2 [Yin Huai] compile with Java 6
2015-05-01 16:47:00 -07:00
Burak Yavuz 4dc8d74491 [SPARK-7240][SQL] Single pass covariance calculation for dataframes
Added the calculation of covariance between two columns to DataFrames.

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5825 from brkyvz/df-cov and squashes the following commits:

cb18046 [Burak Yavuz] changed to sample covariance
f2e862b [Burak Yavuz] fixed failed test
51e39b8 [Burak Yavuz] moved implementation
0c6a759 [Burak Yavuz] addressed math comments
8456eca [Burak Yavuz] fix pyStyle3
aa2ad29 [Burak Yavuz] fix pyStyle2
4e97a50 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into df-cov
e3b0b85 [Burak Yavuz] addressed comments v0.1
a7115f1 [Burak Yavuz] fix python style
7dc6dbc [Burak Yavuz] reorder imports
408cb77 [Burak Yavuz] initial commit
2015-05-01 13:29:17 -07:00
Reynold Xin 37537760d1 [SPARK-7274] [SQL] Create Column expression for array/struct creation.
Author: Reynold Xin <rxin@databricks.com>

Closes #5802 from rxin/SPARK-7274 and squashes the following commits:

19aecaa [Reynold Xin] Fixed unicode tests.
bfc1538 [Reynold Xin] Export all Python functions.
2517b8c [Reynold Xin] Code review.
23da335 [Reynold Xin] Fixed Python bug.
132002e [Reynold Xin] Fixed tests.
56fce26 [Reynold Xin] Added Python support.
b0d591a [Reynold Xin] Fixed debug error.
86926a6 [Reynold Xin] Added test suite.
7dbb9ab [Reynold Xin] Ok one more.
470e2f5 [Reynold Xin] One more MLlib ...
e2d14f0 [Reynold Xin] [SPARK-7274][SQL] Create Column expression for array/struct creation.
2015-05-01 12:49:02 -07:00
Sandy Ryza 0a2b15ce43 [SPARK-4550] In sort-based shuffle, store map outputs in serialized form
Refer to the JIRA for the design doc and some perf results.

I wanted to call out some of the more possibly controversial changes up front:
* Map outputs are only stored in serialized form when Kryo is in use.  I'm still unsure whether Java-serialized objects can be relocated.  At the very least, Java serialization writes out a stream header which causes problems with the current approach, so I decided to leave investigating this to future work.
* The shuffle now explicitly operates on key-value pairs instead of any object.  Data is written to shuffle files in alternating keys and values instead of key-value tuples.  `BlockObjectWriter.write` now accepts a key argument and a value argument instead of any object.
* The map output buffer can hold a max of Integer.MAX_VALUE bytes.  Though this wouldn't be terribly difficult to change.
* When spilling occurs, the objects that still in memory at merge time end up serialized and deserialized an extra time.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #4450 from sryza/sandy-spark-4550 and squashes the following commits:

8c70dd9 [Sandy Ryza] Fix serialization
9c16fe6 [Sandy Ryza] Fix a couple tests and move getAutoReset to KryoSerializerInstance
6c54e06 [Sandy Ryza] Fix scalastyle
d8462d8 [Sandy Ryza] SPARK-4550
2015-04-30 23:14:14 -07:00
Burak Yavuz b5347a4664 [SPARK-7248] implemented random number generators for DataFrames
Adds the functions `rand` (Uniform Dist) and `randn` (Normal Dist.) as expressions to DataFrames.

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

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

50d69d4 [Burak Yavuz] add seed for test that failed
4234c3a [Burak Yavuz] fix Rand expression
13cad5c [Burak Yavuz] couple fixes
7d53953 [Burak Yavuz] waiting for hive tests
b453716 [Burak Yavuz] move radn with seed down
03637f0 [Burak Yavuz] fix broken hive func
c5909eb [Burak Yavuz] deleted old implementation of Rand
6d43895 [Burak Yavuz] implemented random generators
2015-04-30 21:56:03 -07:00
Patrick Wendell beeafcfd6e Revert "[SPARK-5213] [SQL] Pluggable SQL Parser Support"
This reverts commit 3ba5aaab82.
2015-04-30 20:33:36 -07:00
Cheng Hao 3ba5aaab82 [SPARK-5213] [SQL] Pluggable SQL Parser Support
This PR aims to make the SQL Parser Pluggable, and user can register it's own parser via Spark SQL CLI.

```
# add the jar into the classpath
$hchengmydesktop:spark>bin/spark-sql --jars sql99.jar

-- switch to "hiveql" dialect
   spark-sql>SET spark.sql.dialect=hiveql;
   spark-sql>SELECT * FROM src LIMIT 1;

-- switch to "sql" dialect
   spark-sql>SET spark.sql.dialect=sql;
   spark-sql>SELECT * FROM src LIMIT 1;

-- switch to a custom dialect
   spark-sql>SET spark.sql.dialect=com.xxx.xxx.SQL99Dialect;
   spark-sql>SELECT * FROM src LIMIT 1;

-- register the non-exist SQL dialect
   spark-sql> SET spark.sql.dialect=NotExistedClass;
   spark-sql> SELECT * FROM src LIMIT 1;
-- Exception will be thrown and switch to default sql dialect ("sql" for SQLContext and "hiveql" for HiveContext)
```

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

Closes #4015 from chenghao-intel/sqlparser and squashes the following commits:

493775c [Cheng Hao] update the code as feedback
81a731f [Cheng Hao] remove the unecessary comment
aab0b0b [Cheng Hao] polish the code a little bit
49b9d81 [Cheng Hao] shrink the comment for rebasing
2015-04-30 18:49:06 -07:00
Vyacheslav Baranov e991255e72 [SPARK-6913][SQL] Fixed "java.sql.SQLException: No suitable driver found"
Fixed `java.sql.SQLException: No suitable driver found` when loading DataFrame into Spark SQL if the driver is supplied with `--jars` argument.

The problem is in `java.sql.DriverManager` class that can't access drivers loaded by Spark ClassLoader.

Wrappers that forward requests are created for these drivers.

Also, it's not necessary any more to include JDBC drivers in `--driver-class-path` in local mode, specifying in `--jars` argument is sufficient.

Author: Vyacheslav Baranov <slavik.baranov@gmail.com>

Closes #5782 from SlavikBaranov/SPARK-6913 and squashes the following commits:

510c43f [Vyacheslav Baranov] [SPARK-6913] Fixed review comments
b2a727c [Vyacheslav Baranov] [SPARK-6913] Fixed thread race on driver registration
c8294ae [Vyacheslav Baranov] [SPARK-6913] Fixed "No suitable driver found" when using using JDBC driver added with SparkContext.addJar
2015-04-30 18:45:14 -07:00
scwf 079733817f [SPARK-7093] [SQL] Using newPredicate in NestedLoopJoin to enable code generation
Using newPredicate in NestedLoopJoin instead of InterpretedPredicate to make it can make use of code generation

Author: scwf <wangfei1@huawei.com>

Closes #5665 from scwf/NLP and squashes the following commits:

d19dd31 [scwf] improvement
a887c02 [scwf] improve for NLP boundCondition
2015-04-30 18:15:56 -07:00
rakeshchalasani ee04413935 [SPARK-7280][SQL] Add "drop" column/s on a data frame
Takes a column name/s and returns a new DataFrame that drops a column/s.

Author: rakeshchalasani <vnit.rakesh@gmail.com>

Closes #5818 from rakeshchalasani/SPARK-7280 and squashes the following commits:

ce2ec09 [rakeshchalasani] Minor edit
45c06f1 [rakeshchalasani] Change withColumnRename and format changes
f68945a [rakeshchalasani] Minor fix
0b9104d [rakeshchalasani] Drop one column at a time
289afd2 [rakeshchalasani] [SPARK-7280][SQL] Add "drop" column/s on a data frame
2015-04-30 17:42:50 -07:00
Burak Yavuz 149b3ee2da [SPARK-7242][SQL][MLLIB] Frequent items for DataFrames
Finding frequent items with possibly false positives, using the algorithm described in `http://www.cs.umd.edu/~samir/498/karp.pdf`.
public API under:
```
df.stat.freqItems(cols: Array[String], support: Double = 0.001): DataFrame
```

The output is a local DataFrame having the input column names with `-freqItems` appended to it. This is a single pass algorithm that may return false positives, but no false negatives.

cc mengxr rxin

Let's get the implementations in, I can add python API in a follow up PR.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5799 from brkyvz/freq-items and squashes the following commits:

a6ec82c [Burak Yavuz] addressed comments v?
39b1bba [Burak Yavuz] removed toSeq
0915e23 [Burak Yavuz] addressed comments v2.1
3a5c177 [Burak Yavuz] addressed comments v2.0
482e741 [Burak Yavuz] removed old import
38e784d [Burak Yavuz] addressed comments v1.0
8279d4d [Burak Yavuz] added default value for support
3d82168 [Burak Yavuz] made base implementation
2015-04-30 16:40:32 -07:00
Josh Rosen fa01bec484 [Build] Enable MiMa checks for SQL
Now that 1.3 has been released, we should enable MiMa checks for the `sql` subproject.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5727 from JoshRosen/enable-more-mima-checks and squashes the following commits:

3ad302b [Josh Rosen] Merge remote-tracking branch 'origin/master' into enable-more-mima-checks
0c48e4d [Josh Rosen] Merge remote-tracking branch 'origin/master' into enable-more-mima-checks
e276cee [Josh Rosen] Fix SQL MiMa checks via excludes and private[sql]
44d0d01 [Josh Rosen] Add back 'launcher' exclude
1aae027 [Josh Rosen] Enable MiMa checks for launcher and sql projects.
2015-04-30 16:23:01 -07:00
Liang-Chi Hsieh 6702324b60 [SPARK-7196][SQL] Support precision and scale of decimal type for JDBC
JIRA: https://issues.apache.org/jira/browse/SPARK-7196

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

Closes #5777 from viirya/jdbc_precision and squashes the following commits:

f40f5e6 [Liang-Chi Hsieh] Support precision and scale for NUMERIC type.
49acbf9 [Liang-Chi Hsieh] Add unit test.
a509e19 [Liang-Chi Hsieh] Support precision and scale of decimal type for JDBC.
2015-04-30 15:13:43 -07:00
Burak Yavuz 5553198fe5 [SPARK-7156][SQL] Addressed follow up comments for randomSplit
small fixes regarding comments in PR #5761

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5795 from brkyvz/split-followup and squashes the following commits:

369c522 [Burak Yavuz] changed wording a little
1ea456f [Burak Yavuz] Addressed follow up comments
2015-04-29 19:13:47 -07:00
Burak Yavuz d7dbce8f7d [SPARK-7156][SQL] support RandomSplit in DataFrames
This is built on top of kaka1992 's PR #5711 using Logical plans.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5761 from brkyvz/random-sample and squashes the following commits:

a1fb0aa [Burak Yavuz] remove unrelated file
69669c3 [Burak Yavuz] fix broken test
1ddb3da [Burak Yavuz] copy base
6000328 [Burak Yavuz] added python api and fixed test
3c11d1b [Burak Yavuz] fixed broken test
f400ade [Burak Yavuz] fix build errors
2384266 [Burak Yavuz] addressed comments v0.1
e98ebac [Burak Yavuz] [SPARK-7156][SQL] support RandomSplit in DataFrames
2015-04-29 15:34:05 -07:00
Wenchen Fan 81ea42bf39 [SQL][Minor] fix java doc for DataFrame.agg
Author: Wenchen Fan <cloud0fan@outlook.com>

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

be23064 [Wenchen Fan] fix java doc for DataFrame.agg
2015-04-29 09:49:24 -07:00
Josh Rosen f49284b5bf [SPARK-7076][SPARK-7077][SPARK-7080][SQL] Use managed memory for aggregations
This patch adds managed-memory-based aggregation to Spark SQL / DataFrames. Instead of working with Java objects, this new aggregation path uses `sun.misc.Unsafe` to manipulate raw memory.  This reduces the memory footprint for aggregations, resulting in fewer spills, OutOfMemoryErrors, and garbage collection pauses.  As a result, this allows for higher memory utilization.  It can also result in better cache locality since objects will be stored closer together in memory.

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

### Managing memory with sun.misc.Unsafe

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

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

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

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

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

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

### Compact tuple format

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

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

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

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

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

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

### Hash map for performing aggregations

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

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

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

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

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

Author: Josh Rosen <joshrosen@databricks.com>

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

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

rxin I love Davies.

Author: Burak Yavuz <brkyvz@gmail.com>

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

7c4f563 [Burak Yavuz] removed is_math
3c4adde [Burak Yavuz] cleanup imports
d5dca3f [Burak Yavuz] moved math functions to mathfunctions
25e6534 [Burak Yavuz] addressed comments v2.0
d3f7e0f [Burak Yavuz] addressed comments and added tests
7b7d7c4 [Burak Yavuz] remove tests for removed methods
33c2c15 [Burak Yavuz] fixed python style
3ee0c05 [Burak Yavuz] added python functions
2015-04-29 00:09:24 -07:00
Burak Yavuz 271c4c621d [SPARK-7215] made coalesce and repartition a part of the query plan
Coalesce and repartition now show up as part of the query plan, rather than resulting in a new `DataFrame`.

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5762 from brkyvz/df-repartition and squashes the following commits:

b1e76dd [Burak Yavuz] added documentation on repartitions
5807e35 [Burak Yavuz] renamed coalescepartitions
fa4509f [Burak Yavuz] rename coalesce
2c349b5 [Burak Yavuz] address comments
f2e6af1 [Burak Yavuz] add ticks
686c90b [Burak Yavuz] made coalesce and repartition a part of the query plan
2015-04-28 22:48:04 -07:00
Pei-Lun Lee e13cd86567 [SPARK-6352] [SQL] Custom parquet output committer
Add new config "spark.sql.parquet.output.committer.class" to allow custom parquet output committer and an output committer class specific to use on s3.
Fix compilation error introduced by https://github.com/apache/spark/pull/5042.
Respect ParquetOutputFormat.ENABLE_JOB_SUMMARY flag.

Author: Pei-Lun Lee <pllee@appier.com>

Closes #5525 from ypcat/spark-6352 and squashes the following commits:

54c6b15 [Pei-Lun Lee] error handling
472870e [Pei-Lun Lee] add back custom parquet output committer
ddd0f69 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
9ece5c5 [Pei-Lun Lee] compatibility with hadoop 1.x
8413fcd [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
fe65915 [Pei-Lun Lee] add support for parquet config parquet.enable.summary-metadata
e17bf47 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
9ae7545 [Pei-Lun Lee] [SPARL-6352] [SQL] Change to allow custom parquet output committer.
0d540b9 [Pei-Lun Lee] [SPARK-6352] [SQL] add license
c42468c [Pei-Lun Lee] [SPARK-6352] [SQL] add test case
0fc03ca [Pei-Lun Lee] [SPARK-6532] [SQL] hide class DirectParquetOutputCommitter
769bd67 [Pei-Lun Lee] DirectParquetOutputCommitter
f75e261 [Pei-Lun Lee] DirectParquetOutputCommitter
2015-04-28 16:50:18 +08:00
Reynold Xin d94cd1a733 [SPARK-7135][SQL] DataFrame expression for monotonically increasing IDs.
Author: Reynold Xin <rxin@databricks.com>

Closes #5709 from rxin/inc-id and squashes the following commits:

7853611 [Reynold Xin] private sql.
a9fda0d [Reynold Xin] Missed a few numbers.
343d896 [Reynold Xin] Self review feedback.
a7136cb [Reynold Xin] [SPARK-7135][SQL] DataFrame expression for monotonically increasing IDs.
2015-04-28 00:39:08 -07:00
Burak Yavuz 29576e7860 [SPARK-6829] Added math functions for DataFrames
Implemented almost all math functions found in scala.math (max, min and abs were already present).

cc mengxr marmbrus

Author: Burak Yavuz <brkyvz@gmail.com>

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

fb27153 [Burak Yavuz] reverted exception message
836a098 [Burak Yavuz] fixed test and addressed small comment
e5f0d13 [Burak Yavuz] addressed code review v2.2
b26c5fb [Burak Yavuz] addressed review v2.1
2761f08 [Burak Yavuz] addressed review v2
6588a5b [Burak Yavuz] fixed merge conflicts
b084e10 [Burak Yavuz] Addressed code review
029e739 [Burak Yavuz] fixed atan2 test
534cc11 [Burak Yavuz] added more tests, addressed comments
fa68dbe [Burak Yavuz] added double specific test data
937d5a5 [Burak Yavuz] use doubles instead of ints
8e28fff [Burak Yavuz] Added apache header
7ec8f7f [Burak Yavuz] Added math functions for DataFrames
2015-04-27 23:10:14 -07:00
Sean Owen ab5adb7a97 [SPARK-7145] [CORE] commons-lang (2.x) classes used instead of commons-lang3 (3.x); commons-io used without dependency
Remove use of commons-lang in favor of commons-lang3 classes; remove commons-io use in favor of Guava

Author: Sean Owen <sowen@cloudera.com>

Closes #5703 from srowen/SPARK-7145 and squashes the following commits:

21fbe03 [Sean Owen] Remove use of commons-lang in favor of commons-lang3 classes; remove commons-io use in favor of Guava
2015-04-27 19:50:55 -04:00
wangfei d188b8bad8 [SQL][Minor] rename DataTypeParser.apply to DataTypeParser.parse
rename DataTypeParser.apply to DataTypeParser.parse to make it more clear and readable.
/cc rxin

Author: wangfei <wangfei1@huawei.com>

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

c319977 [wangfei] rename apply to parse
2015-04-26 21:08:47 -07:00
Reynold Xin ca55dc95b7 [SPARK-7152][SQL] Add a Column expression for partition ID.
Author: Reynold Xin <rxin@databricks.com>

Closes #5705 from rxin/df-pid and squashes the following commits:

401018f [Reynold Xin] [SPARK-7152][SQL] Add a Column expression for partition ID.
2015-04-26 11:46:58 -07:00
Vinod K C c1213e6a92 [SPARK-7055][SQL]Use correct ClassLoader for JDBC Driver in JDBCRDD.getConnector
Author: Vinod K C <vinod.kc@huawei.com>

Closes #5633 from vinodkc/use_correct_classloader_driverload and squashes the following commits:

73c5380 [Vinod K C] Use correct ClassLoader for JDBC Driver
2015-04-23 12:00:23 -07:00
Reynold Xin f60bece14f [SPARK-7069][SQL] Rename NativeType -> AtomicType.
Also renamed JvmType to InternalType.

Author: Reynold Xin <rxin@databricks.com>

Closes #5651 from rxin/native-to-atomic-type and squashes the following commits:

cbd4028 [Reynold Xin] [SPARK-7069][SQL] Rename NativeType -> AtomicType.
2015-04-23 01:43:40 -07:00
Reynold Xin 29163c5200 [SPARK-7068][SQL] Remove PrimitiveType
Author: Reynold Xin <rxin@databricks.com>

Closes #5646 from rxin/remove-primitive-type and squashes the following commits:

01b673d [Reynold Xin] [SPARK-7068][SQL] Remove PrimitiveType
2015-04-22 23:55:20 -07:00
Daoyuan Wang 04525c077c [SPARK-6967] [SQL] fix date type convertion in jdbcrdd
This pr convert java.sql.Date type into Int for JDBCRDD.

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

Closes #5590 from adrian-wang/datebug and squashes the following commits:

f897b81 [Daoyuan Wang] add a test case
3c9184c [Daoyuan Wang] fix date type convertion in jdbcrdd
2015-04-22 19:14:28 -07:00
Reynold Xin baf865ddc2 [SPARK-7059][SQL] Create a DataFrame join API to facilitate equijoin.
Author: Reynold Xin <rxin@databricks.com>

Closes #5638 from rxin/joinUsing and squashes the following commits:

13e9cc9 [Reynold Xin] Code review + Python.
b1bd914 [Reynold Xin] [SPARK-7059][SQL] Create a DataFrame join API to facilitate equijoin and self join.
2015-04-22 15:26:58 -07:00
szheng79 fbe7106d75 [SPARK-7039][SQL]JDBCRDD: Add support on type NVARCHAR
Issue:

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

Add support to column type NVARCHAR in Sql Server

java.sql.Types:
http://docs.oracle.com/javase/7/docs/api/java/sql/Types.html

Author: szheng79 <szheng.code@gmail.com>

Closes #5618 from szheng79/patch-1 and squashes the following commits:

10da99c [szheng79] Update JDBCRDD.scala
eab0bd8 [szheng79] Add support on type NVARCHAR
2015-04-22 13:02:55 -07:00
Reynold Xin cdf0328684 [SQL] Rename some apply functions.
I was looking at the code gen code and got confused by a few of use cases of apply, in particular apply on objects. So I went ahead and changed a few of them. Hopefully slightly more clear with a proper verb.

Author: Reynold Xin <rxin@databricks.com>

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

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

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

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

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

c2a5132 [Cheng Hao] add back resolved for Alias
556e982 [Cheng Hao] revert the unncessary change
002c361 [Cheng Hao] change the rule of resolved for Generate
04ae500 [Cheng Hao] add qualifier only for generator output
5ee5d2c [Cheng Hao] prepend the new qualifier
d2e8b43 [Cheng Hao] Update the code as feedback
ca5e7f4 [Cheng Hao] shrink the commits
2015-04-21 15:11:15 -07:00
Punya Biswal 2a24bf92e6 [SPARK-6996][SQL] Support map types in java beans
liancheng mengxr this is similar to #5146.

Author: Punya Biswal <pbiswal@palantir.com>

Closes #5578 from punya/feature/SPARK-6996 and squashes the following commits:

d56c3e0 [Punya Biswal] Fix imports
c7e308b [Punya Biswal] Support java iterable types in POJOs
5e00685 [Punya Biswal] Support map types in java beans
2015-04-21 14:50:02 -07:00
Yin Huai 6265cba00f [SPARK-6969][SQL] Refresh the cached table when REFRESH TABLE is used
https://issues.apache.org/jira/browse/SPARK-6969

Author: Yin Huai <yhuai@databricks.com>

Closes #5583 from yhuai/refreshTableRefreshDataCache and squashes the following commits:

1e5142b [Yin Huai] Add todo.
92b2498 [Yin Huai] Minor updates.
367df92 [Yin Huai] Recache data in the command of REFRESH TABLE.
2015-04-21 14:48:42 -07:00
Liang-Chi Hsieh c736220dac [SPARK-6635][SQL] DataFrame.withColumn should replace columns with identical column names
JIRA https://issues.apache.org/jira/browse/SPARK-6635

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

Closes #5541 from viirya/replace_with_column and squashes the following commits:

b539c7b [Liang-Chi Hsieh] For comment.
72f35b1 [Liang-Chi Hsieh] DataFrame.withColumn can replace original column with identical column name.
2015-04-20 18:54:01 -07:00
Yin Huai ce7ddabbcd [SPARK-6368][SQL] Build a specialized serializer for Exchange operator.
JIRA: https://issues.apache.org/jira/browse/SPARK-6368

Author: Yin Huai <yhuai@databricks.com>

Closes #5497 from yhuai/serializer2 and squashes the following commits:

da562c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2
50e0c3d [Yin Huai] When no filed is emitted to shuffle, use SparkSqlSerializer for now.
9f1ed92 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2
6d07678 [Yin Huai] Address comments.
4273b8c [Yin Huai] Enabled SparkSqlSerializer2.
09e587a [Yin Huai] Remove TODO.
791b96a [Yin Huai] Use UTF8String.
60a1487 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2
3e09655 [Yin Huai] Use getAs for Date column.
43b9fb4 [Yin Huai] Test.
8297732 [Yin Huai] Fix test.
c9373c8 [Yin Huai] Support DecimalType.
2379eeb [Yin Huai] ASF header.
39704ab [Yin Huai] Specialized serializer for Exchange.
2015-04-20 18:42:50 -07:00
cafreeman 59e206deb7 [SPARK-6807] [SparkR] Merge recent SparkR-pkg changes
This PR pulls in recent changes in SparkR-pkg, including

cartesian, intersection, sampleByKey, subtract, subtractByKey, except, and some API for StructType and StructField.

Author: cafreeman <cfreeman@alteryx.com>
Author: Davies Liu <davies@databricks.com>
Author: Zongheng Yang <zongheng.y@gmail.com>
Author: Shivaram Venkataraman <shivaram.venkataraman@gmail.com>
Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>
Author: Sun Rui <rui.sun@intel.com>

Closes #5436 from davies/R3 and squashes the following commits:

c2b09be [Davies Liu] SQLTypes -> schema
a5a02f2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R3
168b7fe [Davies Liu] sort generics
b1fe460 [Davies Liu] fix conflict in README.md
e74c04e [Davies Liu] fix schema.R
4f5ac09 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R5
41f8184 [Davies Liu] rm man
ae78312 [Davies Liu] Merge pull request #237 from sun-rui/SPARKR-154_3
1bdcb63 [Zongheng Yang] Updates to README.md.
5a553e7 [cafreeman] Use object attribute instead of argument
71372d9 [cafreeman] Update docs and examples
8526d2e71 [cafreeman] Remove `tojson` functions
6ef5f2d [cafreeman] Fix spacing
7741d66 [cafreeman] Rename the SQL DataType function
141efd8 [Shivaram Venkataraman] Merge pull request #245 from hqzizania/upstream
9387402 [Davies Liu] fix style
40199eb [Shivaram Venkataraman] Move except into sorted position
07d0dbc [Sun Rui] [SPARKR-244] Fix test failure after integration of subtract() and subtractByKey() for RDD.
7e8caa3 [Shivaram Venkataraman] Merge pull request #246 from hlin09/fixCombineByKey
ed66c81 [cafreeman] Update `subtract` to work with `generics.R`
f3ba785 [cafreeman] Fixed duplicate export
275deb4 [cafreeman] Update `NAMESPACE` and tests
1a3b63d [cafreeman] new version of `CreateDF`
836c4bf [cafreeman] Update `createDataFrame` and `toDF`
be5d5c1 [cafreeman] refactor schema functions
40338a4 [Zongheng Yang] Merge pull request #244 from sun-rui/SPARKR-154_5
20b97a6 [Zongheng Yang] Merge pull request #234 from hqzizania/assist
ba54e34 [Shivaram Venkataraman] Merge pull request #238 from sun-rui/SPARKR-154_4
c9497a3 [Shivaram Venkataraman] Merge pull request #208 from lythesia/master
b317aa7 [Zongheng Yang] Merge pull request #243 from hqzizania/master
136a07e [Zongheng Yang] Merge pull request #242 from hqzizania/stats
cd66603 [cafreeman] new line at EOF
8b76e81 [Shivaram Venkataraman] Merge pull request #233 from redbaron/fail-early-on-missing-dep
7dd81b7 [cafreeman] Documentation
0e2a94f [cafreeman] Define functions for schema and fields
2015-04-17 13:42:19 -07:00
Michael Armbrust 8220d5265f [SPARK-6972][SQL] Add Coalesce to DataFrame
Author: Michael Armbrust <michael@databricks.com>

Closes #5545 from marmbrus/addCoalesce and squashes the following commits:

9fdf3f6 [Michael Armbrust] [SPARK-6972][SQL] Add Coalesce to DataFrame
2015-04-16 21:49:26 -05:00
Michael Armbrust e5949c287e [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver
Otherwise we cannot add jars with drivers after the fact.

Author: Michael Armbrust <michael@databricks.com>

Closes #5543 from marmbrus/jdbcClassloader and squashes the following commits:

d9930f3 [Michael Armbrust] fix imports
73d0614 [Michael Armbrust] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver
2015-04-16 17:59:49 -07:00
Davies Liu 6183b5e2ca [SPARK-6911] [SQL] improve accessor for nested types
Support access columns by index in Python:
```
>>> df[df[0] > 3].collect()
[Row(age=5, name=u'Bob')]
```

Access items in ArrayType or MapType
```
>>> df.select(df.l.getItem(0), df.d.getItem("key")).show()
>>> df.select(df.l[0], df.d["key"]).show()
```

Access field in StructType
```
>>> df.select(df.r.getField("b")).show()
>>> df.select(df.r.a).show()
```

Author: Davies Liu <davies@databricks.com>

Closes #5513 from davies/access and squashes the following commits:

e04d5a0 [Davies Liu] Update run-tests-jenkins
7ada9eb [Davies Liu] update timeout
d125ac4 [Davies Liu] check column name, improve scala tests
6b62540 [Davies Liu] fix test
db15b42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into access
6c32e79 [Davies Liu] add scala tests
11f1df3 [Davies Liu] improve accessor for nested types
2015-04-16 17:33:57 -07:00
Max Seiden 8a53de16fc [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators
[SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators

There were a few places where new SparkSqlSerializer instances were created with new, empty SparkConfs resulting in user specified registrators sometimes not getting initialized.

The fix is to try and pull a conf from the SparkEnv, and construct a new conf (that loads defaults) if one cannot be found.

The changes touched:
    1) SparkSqlSerializer's resource pool (this appears to fix the issue in the comment)
    2) execution.Exchange (for all of the partitioners)
    3) execution.Limit (for the HashPartitioner)

A few tests were added to ColumnTypeSuite, ensuring that a custom registrator and serde is initialized and used when in-memory columns are written.

Author: Max Seiden <max@platfora.com>

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

Closes #5237 from mhseiden/sql_udt_kryo and squashes the following commits:

3175c2f [Max Seiden] [SPARK-5277][SQL] - address code review comments
e5011fb [Max Seiden] [SPARK-5277][SQL] - SparkSqlSerializer does not register user specified KryoRegistrators
2015-04-15 16:15:11 -07:00
Daoyuan Wang 585638e81c [SPARK-2213] [SQL] sort merge join for spark sql
Thanks for the initial work from Ishiihara in #3173

This PR introduce a new join method of sort merge join, which firstly ensure that keys of same value are in the same partition, and inside each partition the Rows are sorted by key. Then we can run down both sides together, find matched rows using [sort merge join](http://en.wikipedia.org/wiki/Sort-merge_join). In this way, we don't have to store the whole hash table of one side as hash join, thus we have less memory usage. Also, this PR would benefit from #3438 , making the sorting phrase much more efficient.

We introduced a new configuration of "spark.sql.planner.sortMergeJoin" to switch between this(`true`) and ShuffledHashJoin(`false`), probably we want the default value of it be `false` at first.

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

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

Closes #5208 from adrian-wang/smj and squashes the following commits:

2493b9f [Daoyuan Wang] fix style
5049d88 [Daoyuan Wang] propagate rowOrdering for RangePartitioning
f91a2ae [Daoyuan Wang] yin's comment: use external sort if option is enabled, add comments
f515cd2 [Daoyuan Wang] yin's comment: outputOrdering, join suite refine
ec8061b [Daoyuan Wang] minor change
413fd24 [Daoyuan Wang] Merge pull request #3 from marmbrus/pr/5208
952168a [Michael Armbrust] add type
5492884 [Michael Armbrust] copy when ordering
7ddd656 [Michael Armbrust] Cleanup addition of ordering requirements
b198278 [Daoyuan Wang] inherit ordering in project
c8e82a3 [Daoyuan Wang] fix style
6e897dd [Daoyuan Wang] hide boundReference from manually construct RowOrdering for key compare in smj
8681d73 [Daoyuan Wang] refactor Exchange and fix copy for sorting
2875ef2 [Daoyuan Wang] fix changed configuration
61d7f49 [Daoyuan Wang] add omitted comment
00a4430 [Daoyuan Wang] fix bug
078d69b [Daoyuan Wang] address comments: add comments, do sort in shuffle, and others
3af6ba5 [Daoyuan Wang] use buffer for only one side
171001f [Daoyuan Wang] change default outputordering
47455c9 [Daoyuan Wang] add apache license ...
a28277f [Daoyuan Wang] fix style
645c70b [Daoyuan Wang] address comments using sort
068c35d [Daoyuan Wang] fix new style and add some tests
925203b [Daoyuan Wang] address comments
07ce92f [Daoyuan Wang] fix ArrayIndexOutOfBound
42fca0e [Daoyuan Wang] code clean
e3ec096 [Daoyuan Wang] fix comment style..
2edd235 [Daoyuan Wang] fix outputpartitioning
57baa40 [Daoyuan Wang] fix sort eval bug
303b6da [Daoyuan Wang] fix several errors
95db7ad [Daoyuan Wang] fix brackets for if-statement
4464f16 [Daoyuan Wang] fix error
880d8e9 [Daoyuan Wang] sort merge join for spark sql
2015-04-15 14:06:10 -07:00
Wenchen Fan 4754e16f47 [SPARK-6898][SQL] completely support special chars in column names
Even if we wrap column names in backticks like `` `a#$b.c` ``,  we still handle the "." inside column name specially. I think it's fragile to use a special char to split name parts, why not put name parts in `UnresolvedAttribute` directly?

Author: Wenchen Fan <cloud0fan@outlook.com>

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

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

48e3e57 [Wenchen Fan] more style fix
820dc45 [Wenchen Fan] do not ignore newName in UnresolvedAttribute
d81ad43 [Wenchen Fan] fix style
11699d6 [Wenchen Fan] completely support special chars in column names
2015-04-15 13:39:12 -07:00
Liang-Chi Hsieh cf38fe04f8 [SPARK-6844][SQL] Clean up accumulators used in InMemoryRelation when it is uncached
JIRA: https://issues.apache.org/jira/browse/SPARK-6844

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

Closes #5475 from viirya/cache_memory_leak and squashes the following commits:

0b41235 [Liang-Chi Hsieh] fix style.
dc1d5d5 [Liang-Chi Hsieh] For comments.
78af229 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cache_memory_leak
26c9bb6 [Liang-Chi Hsieh] Add configuration to enable in-memory table scan accumulators.
1c3b06e [Liang-Chi Hsieh] Clean up accumulators used in InMemoryRelation when it is uncached.
2015-04-15 13:15:58 -07:00
Davies Liu 85842760dc [SPARK-6638] [SQL] Improve performance of StringType in SQL
This PR change the internal representation for StringType from java.lang.String to UTF8String, which is implemented use ArrayByte.

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

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

cc rxin

Author: Davies Liu <davies@databricks.com>

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

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

Author: Yin Huai <yhuai@databricks.com>

Closes #5499 from yhuai/inMemFloat and squashes the following commits:

84cba38 [Yin Huai] Add test.
4b75ba6 [Yin Huai] Add FloatType back.
2015-04-15 13:04:03 -07:00
Liang-Chi Hsieh e3e4e9a38b [SPARK-6800][SQL] Update doc for JDBCRelation's columnPartition
JIRA https://issues.apache.org/jira/browse/SPARK-6800

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

Closes #5488 from viirya/fix_jdbc_where and squashes the following commits:

51386c8 [Liang-Chi Hsieh] Update code comment.
1dcc929 [Liang-Chi Hsieh] Update document.
3eb74d6 [Liang-Chi Hsieh] Revert and modify doc.
df11783 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_jdbc_where
3e7db15 [Liang-Chi Hsieh] Fix wrong logic to generate WHERE clause for JDBC.
2015-04-15 13:02:25 -07:00
Liang-Chi Hsieh b75b307074 [SPARK-6730][SQL] Allow using keyword as identifier in OPTIONS
JIRA: https://issues.apache.org/jira/browse/SPARK-6730

It is very possible that keyword will be used as identifier in `OPTIONS`, this pr makes it works.

However, another approach is that we can request that `OPTIONS` can't include keywords and has to use alternative identifier (e.g. table -> cassandraTable) if needed.

If so, please let me know to close this pr. Thanks.

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

Closes #5520 from viirya/relax_options and squashes the following commits:

339fd68 [Liang-Chi Hsieh] Use regex parser.
92be11c [Liang-Chi Hsieh] Allow using keyword as identifier in OPTIONS.
2015-04-15 13:00:19 -07:00
Josh Rosen a76b921a92 Revert "[SPARK-6352] [SQL] Add DirectParquetOutputCommitter"
This reverts commit b29663eeea.

I'm reverting this because it broke test compilation for the Hadoop 1.x
profiles.
2015-04-14 14:10:15 -07:00
Liang-Chi Hsieh 4898dfa464 [SPARK-6877][SQL] Add code generation support for Min
Currently `min` is not supported in code generation. This pr adds the support for it.

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

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

0ddec23 [Liang-Chi Hsieh] Add code generation support for Min.
2015-04-13 18:16:33 -07:00
Liang-Chi Hsieh 5b8b324f33 [SPARK-6303][SQL] Remove unnecessary Average in GeneratedAggregate
Because `Average` is a `PartialAggregate`, we never get a `Average` node when reaching `HashAggregation` to prepare `GeneratedAggregate`.

That is why in SQLQuerySuite there is already a test for `avg` with codegen. And it works.

But we can find a case in `GeneratedAggregate` to deal with `Average`. Based on the above, we actually never execute this case.

So we can remove this case from `GeneratedAggregate`.

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

Closes #4996 from viirya/add_average_codegened and squashes the following commits:

621c12f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened
368cfbc [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened
74926d1 [Liang-Chi Hsieh] Add Average in canBeCodeGened lists.
2015-04-13 18:15:29 -07:00
Daoyuan Wang e63a86abe2 [SPARK-6872] [SQL] add copy in external sort
We need add copy before call externalsort.

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

Closes #5481 from adrian-wang/extsort and squashes the following commits:

9611586 [Daoyuan Wang] fix bug in external sort
2015-04-13 16:00:58 -07:00
Yash Datta 3a205bbd9e [SQL][SPARK-6742]: Don't push down predicates which reference partition column(s)
cc liancheng

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

Closes #5390 from saucam/fpush and squashes the following commits:

3f026d6 [Yash Datta] SPARK-6742: Fix scalastyle
ce3d702 [Yash Datta] SPARK-6742: Add test case, fix scalastyle
8592acc [Yash Datta] SPARK-6742: Don't push down predicates which reference partition column(s)
2015-04-13 14:43:07 -07:00
Daoyuan Wang 85ee0cabe8 [SPARK-6130] [SQL] support if not exists for insert overwrite into partition in hiveQl
Standard syntax:
INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 FROM from_statement;
INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 FROM from_statement;
 
Hive extension (multiple inserts):
FROM from_statement
INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1
[INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2]
[INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] ...;
FROM from_statement
INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1
[INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2]
[INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] ...;
 
Hive extension (dynamic partition inserts):
INSERT OVERWRITE TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement;
INSERT INTO TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement;

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

Closes #4865 from adrian-wang/insertoverwrite and squashes the following commits:

2fce94f [Daoyuan Wang] add assert
10ea6f3 [Daoyuan Wang] add name for boolean parameter
0bbe9b9 [Daoyuan Wang] fix failure
4391154 [Daoyuan Wang] support if not exists for insert overwrite into partition in hiveQl
2015-04-13 14:29:07 -07:00
Pei-Lun Lee b29663eeea [SPARK-6352] [SQL] Add DirectParquetOutputCommitter
Add a DirectParquetOutputCommitter class that skips _temporary directory when saving to s3. Add new config value "spark.sql.parquet.useDirectParquetOutputCommitter" (default false) to choose between the default output committer.

Author: Pei-Lun Lee <pllee@appier.com>

Closes #5042 from ypcat/spark-6352 and squashes the following commits:

e17bf47 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
9ae7545 [Pei-Lun Lee] [SPARL-6352] [SQL] Change to allow custom parquet output committer.
0d540b9 [Pei-Lun Lee] [SPARK-6352] [SQL] add license
c42468c [Pei-Lun Lee] [SPARK-6352] [SQL] add test case
0fc03ca [Pei-Lun Lee] [SPARK-6532] [SQL] hide class DirectParquetOutputCommitter
769bd67 [Pei-Lun Lee] DirectParquetOutputCommitter
f75e261 [Pei-Lun Lee] DirectParquetOutputCommitter
2015-04-13 21:52:00 +08:00
Reynold Xin 68d1faa3c0 [SPARK-6562][SQL] DataFrame.replace
Supports replacing values with other values in DataFrames.

Python support should be in a separate pull request.

Author: Reynold Xin <rxin@databricks.com>

Closes #5282 from rxin/df-na-replace and squashes the following commits:

4b72434 [Reynold Xin] Removed println.
c8d9946 [Reynold Xin] col -> cols
fbb3c21 [Reynold Xin] [SPARK-6562][SQL] DataFrame.replace
2015-04-12 22:56:12 -07:00
Daoyuan Wang 04bcd67cfc [MINOR] a typo: coalesce
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #5482 from adrian-wang/typo and squashes the following commits:

e65ef6f [Daoyuan Wang] typo
2015-04-12 18:58:53 +01:00
Yin Huai 6d4e854ffb [SPARK-6367][SQL] Use the proper data type for those expressions that are hijacking existing data types.
This PR adds internal UDTs for expressions that are hijacking existing data types.
The following UDTs are added:
* `HyperLogLogUDT` (`BinaryType` as the SQL type) for `ApproxCountDistinctPartition`
* `OpenHashSetUDT` (`ArrayType` as the SQL type) for `CollectHashSet`, `NewSet`, `AddItemToSet`, and `CombineSets`.

I am also adding more unit tests for aggregation with code gen enabled.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #5094 from yhuai/expressionType and squashes the following commits:

8bcd11a [Yin Huai] Return types.
61a1d66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType
e8b4599 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType
2753156 [Yin Huai] Ignore aggregations having sum functions for now.
b5eb259 [Yin Huai] Case object for HyperLogLog type.
00ebdbd [Yin Huai] deserialize/serialize.
54b87ae [Yin Huai] Add UDTs for expressions that return HyperLogLog and OpenHashSet.
2015-04-11 19:26:15 -07:00
Yin Huai d2383fb5ff [SQL] Handle special characters in the authority of a Path's URI.
Author: Yin Huai <yhuai@databricks.com>

Closes #5381 from yhuai/parquetPath2 and squashes the following commits:

fe296b4 [Yin Huai] Create new Path to take care special characters in the authority of a Path's URI.
2015-04-11 18:44:54 -07:00
Takeshi YAMAMURO 352a5da421 [SPARK-6379][SQL] Support a functon to call user-defined functions registered in SQLContext
This is useful for using pre-defined UDFs in SQLContext;

val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value")
val sqlctx = df.sqlContext
sqlctx.udf.register("simpleUdf", (v: Int) => v * v)
df.select($"id", sqlctx.callUdf("simpleUdf", $"value"))

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #5061 from maropu/SupportUDFConversionInSparkContext and squashes the following commits:

f858aff [Takeshi YAMAMURO] Move the function into functions.scala
afd0380 [Takeshi YAMAMURO] Add a return type of callUDF
599b76c [Takeshi YAMAMURO] Remove the implicit conversion and add SqlContext#callUdf
8b56f10 [Takeshi YAMAMURO] Support an implicit conversion from udf"name" to an UDF defined in SQLContext
2015-04-11 18:41:12 -07:00
lazymam500 1f39a61118 [Spark-5068][SQL]Fix bug query data when path doesn't exist for HiveContext
This PR follow up PR #3907 & #3891 & #4356.
According to  marmbrus  liancheng 's comments, I try to use fs.globStatus to retrieve all FileStatus objects under path(s), and then do the filtering locally.

[1]. get pathPattern by path, and put it into pathPatternSet. (hdfs://cluster/user/demo/2016/08/12 -> hdfs://cluster/user/demo/*/*/*)
[2]. retrieve all FileStatus objects ,and cache them by undating existPathSet.
[3]. do the filtering locally
[4]. if we have new pathPattern,do 1,2 step again. (external table maybe have more than one partition pathPattern)

chenghao-intel jeanlyn

Author: lazymam500 <lazyman500@gmail.com>
Author: lazyman <lazyman500@gmail.com>

Closes #5059 from lazyman500/SPARK-5068 and squashes the following commits:

5bfcbfd [lazyman] move spark.sql.hive.verifyPartitionPath to SQLConf,fix scala style
e1d6386 [lazymam500] fix scala style
f23133f [lazymam500] bug fix
47e0023 [lazymam500] fix scala style,add config flag,break the chaining
04c443c [lazyman] SPARK-5068: fix bug when partition path doesn't exists #2
41f60ce [lazymam500] Merge pull request #1 from apache/master
2015-04-11 18:33:14 -07:00
Liang-Chi Hsieh 198cf2a3fa [SPARK-6858][SQL] Register Java HashMap for SparkSqlSerializer
Since now kyro serializer is used for `GeneralHashedRelation` whether kyro is enabled or not, it is better to register Java `HashMap` in `SparkSqlSerializer`.

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

Closes #5465 from viirya/register_hashmap and squashes the following commits:

9062601 [Liang-Chi Hsieh] Register Java HashMap for SparkSqlSerializer.
2015-04-11 14:50:50 -07:00
Volodymyr Lyubinets 67d06880e4 [SQL] [SPARK-6620] Speed up toDF() and rdd() functions by constructing converters in ScalaReflection
cc marmbrus

Author: Volodymyr Lyubinets <vlyubin@gmail.com>

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

e75a387 [Volodymyr Lyubinets] Changes to ScalaUDF
11a20ec [Volodymyr Lyubinets] Avoid creating a tuple
c327bc9 [Volodymyr Lyubinets] Moved the only remaining function from DataTypeConversions to DateUtils
dec6802 [Volodymyr Lyubinets] Addresed review feedback
74301fa [Volodymyr Lyubinets] Addressed review comments
afa3aa5 [Volodymyr Lyubinets] Minor refactoring, added license, removed debug output
881dc60 [Volodymyr Lyubinets] Moved to a separate module; addressed review comments; one extra place of usage; changed behaviour for Java
8cad6e2 [Volodymyr Lyubinets] Addressed review commments
41b2aa9 [Volodymyr Lyubinets] Creating converters for ScalaReflection stuff, and more
2015-04-10 16:27:56 -07:00
Volodymyr Lyubinets b9baa4cd9f [SQL] [SPARK-6794] Use kryo-based SparkSqlSerializer for GeneralHashedRelation
Benchmarking results: http://pastie.org/private/1dneo1mta5zpsw6gmsoeq

Author: Volodymyr Lyubinets <vlyubin@gmail.com>

Closes #5433 from vlyubin/joins and squashes the following commits:

d70c829 [Volodymyr Lyubinets] Addressed review feedback
527eac6 [Volodymyr Lyubinets] Use kryo-based SparkSqlSerializer for GeneralHashedRelation
2015-04-10 12:09:54 -07:00
Shivaram Venkataraman 2fe0a1aaee [SPARK-5654] Integrate SparkR
This pull requests integrates SparkR, an R frontend for Spark. The SparkR package contains both RDD and DataFrame APIs in R and is integrated with Spark's submission scripts to work on different cluster managers.

Some integration points that would be great to get feedback on:

1. Build procedure: SparkR requires R to be installed on the machine to be built. Right now we have a new Maven profile `-PsparkR` that can be used to enable SparkR builds

2. YARN cluster mode: The R package that is built needs to be present on the driver and all the worker nodes during execution. The R package location is currently set using SPARK_HOME, but this might not work on YARN cluster mode.

The SparkR package represents the work of many contributors and attached below is a list of people along with areas they worked on

edwardt (edwart) - Documentation improvements
Felix Cheung (felixcheung) - Documentation improvements
Hossein Falaki (falaki)  - Documentation improvements
Chris Freeman (cafreeman) - DataFrame API, Programming Guide
Todd Gao (7c00) - R worker Internals
Ryan Hafen (hafen) - SparkR Internals
Qian Huang (hqzizania) - RDD API
Hao Lin (hlin09) - RDD API, Closure cleaner
Evert Lammerts (evertlammerts) - DataFrame API
Davies Liu (davies) - DataFrame API, R worker internals, Merging with Spark
Yi Lu (lythesia) - RDD API, Worker internals
Matt Massie (massie) - Jenkins build
Harihar Nahak (hnahak87) - SparkR examples
Oscar Olmedo (oscaroboto) - Spark configuration
Antonio Piccolboni (piccolbo) - SparkR examples, Namespace bug fixes
Dan Putler (dputler) - Dataframe API, SparkR Install Guide
Ashutosh Raina (ashutoshraina) - Build improvements
Josh Rosen (joshrosen) - Travis CI build
Sun Rui (sun-rui)- RDD API, JVM Backend, Shuffle improvements
Shivaram Venkataraman (shivaram) - RDD API, JVM Backend, Worker Internals
Zongheng Yang (concretevitamin) - RDD API, Pipelined RDDs, Examples and EC2 guide

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>
Author: Shivaram Venkataraman <shivaram.venkataraman@gmail.com>
Author: Zongheng Yang <zongheng.y@gmail.com>
Author: cafreeman <cfreeman@alteryx.com>
Author: Shivaram Venkataraman <shivaram@eecs.berkeley.edu>
Author: Davies Liu <davies@databricks.com>
Author: Davies Liu <davies.liu@gmail.com>
Author: hlin09 <hlin09pu@gmail.com>
Author: Sun Rui <rui.sun@intel.com>
Author: lythesia <iranaikimi@gmail.com>
Author: oscaroboto <oscarjr@gmail.com>
Author: Antonio Piccolboni <antonio@piccolboni.info>
Author: root <edward>
Author: edwardt <edwardt.tril@gmail.com>
Author: hqzizania <qian.huang@intel.com>
Author: dputler <dan.putler@gmail.com>
Author: Todd Gao <todd.gao.2013@gmail.com>
Author: Chris Freeman <cfreeman@alteryx.com>
Author: Felix Cheung <fcheung@AVVOMAC-119.local>
Author: Hossein <hossein@databricks.com>
Author: Evert Lammerts <evert@apache.org>
Author: Felix Cheung <fcheung@avvomac-119.t-mobile.com>
Author: felixcheung <felixcheung_m@hotmail.com>
Author: Ryan Hafen <rhafen@gmail.com>
Author: Ashutosh Raina <ashutoshraina@users.noreply.github.com>
Author: Oscar Olmedo <oscarjr@gmail.com>
Author: Josh Rosen <rosenville@gmail.com>
Author: Yi Lu <iranaikimi@gmail.com>
Author: Harihar Nahak <hnahak87@users.noreply.github.com>

Closes #5096 from shivaram/R and squashes the following commits:

da64742 [Davies Liu] fix Date serialization
59266d1 [Davies Liu] check exclusive of primary-py-file and primary-r-file
55808e4 [Davies Liu] fix tests
5581c75 [Davies Liu] update author of SparkR
f731b48 [Shivaram Venkataraman] Only run SparkR tests if R is installed
64eda24 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
d7c3f22 [Shivaram Venkataraman] Address code review comments Changes include 1. Adding SparkR docs to API docs generated 2. Style fixes in SparkR scala files 3. Clean up of shell scripts and explanation of install-dev.sh
377151f [Shivaram Venkataraman] Merge remote-tracking branch 'apache/master' into R
eb5da53 [Shivaram Venkataraman] Merge pull request #3 from davies/R2
a18ff5c [Davies Liu] Update sparkR.R
5133f3a [Shivaram Venkataraman] Merge pull request #7 from hqzizania/R3
940b631 [hqzizania] [SPARKR-92] Phase 2: implement sum(rdd)
0e788c0 [Shivaram Venkataraman] Merge pull request #5 from hlin09/doc-fix
3487461 [hlin09] Add tests log in .gitignore.
1d1802e [Shivaram Venkataraman] Merge pull request #4 from felixcheung/r-require
11981b7 [felixcheung] Update R to fail early if SparkR package is missing
c300e08 [Davies Liu] remove duplicated file
b045701 [Davies Liu] Merge branch 'remote_r' into R
19c9368 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r
f8fa8af [Davies Liu] mute logging when start/stop context
e7104b6 [Davies Liu] remove ::: in SparkR
a1777eb [Davies Liu] move rules into R/.gitignore
e88b649 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
6e20e71 [Davies Liu] address comments
b433817 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R
a1cedad [Shivaram Venkataraman] Merge pull request #228 from felixcheung/doc
e089151 [Davies Liu] Merge pull request #225 from sun-rui/SPARKR-154_2
463e28c [Davies Liu] Merge pull request #2 from shivaram/doc-fixes
bc2d6d8 [Shivaram Venkataraman] Remove arg from sparkR.stop and update docs
d425363 [Shivaram Venkataraman] Some doc fixes for column, generics, group
1f1a7e0 [Shivaram Venkataraman] Some fixes to DataFrame, RDD, SQLContext docs
104ad4e [Shivaram Venkataraman] Check the right env in exists
cf5cd99 [Shivaram Venkataraman] Remove unused numCols argument
85a50ec [Shivaram Venkataraman] Merge pull request #226 from RevolutionAnalytics/master
3eacfc0 [Davies Liu] fix flaky test
733380d [Davies Liu] update R examples (remove master from args)
b21a0da [Davies Liu] Merge pull request #1 from shivaram/log4j-tests
a1493d7 [Shivaram Venkataraman] Address comments
e1f83ab [Shivaram Venkataraman] Send Spark INFO logs to a file in SparkR tests
58276f5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
52cc92d [Shivaram Venkataraman] Add license to create-docs.sh
6ff5ea2 [Shivaram Venkataraman] Add instructions to generate docs
1f478c5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
02b4833 [Shivaram Venkataraman] Add a script to generate R docs (Rd, html) Also fix some issues with our documentation
d6d3729 [Davies Liu] enable spark and pyspark tests
0e5a83f [Davies Liu] fix code style
afd8a77 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
d87a181 [Davies Liu] fix flaky tests
7100fb9 [Shivaram Venkataraman] Fix libPaths in README
bdf3a14 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
05e7375 [Davies Liu] sort generics
b44e371 [Shivaram Venkataraman] Include RStudio instructions in README
855537f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
9fb6af3 [Davies Liu] mark R classes/objects are private
423ea3c [Shivaram Venkataraman] Ignore unknown jobj in cleanup
974e4ea [Davies Liu] fix flaky test
410ec18 [Davies Liu] fix zipRDD() tests
d8b24fc [Davies Liu] disable spark and python tests temporary
ce3ca62 [Davies Liu] fix license check
7da0049 [Davies Liu] fix build
2892e29 [Davies Liu] support R in YARN cluster
ebd4d07 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
38cbf59 [Davies Liu] fix test of zipRDD()
756ece0 [Shivaram Venkataraman] Update README remove outdated TODO
d436f26 [Davies Liu] add missing files
40d193a [Shivaram Venkataraman] Merge pull request #224 from sun-rui/SPARKR-224-new
1a16cd6 [Davies Liu] rm PROJECT_HOME
56670ef [Davies Liu] rm man page
ba4b80b [Davies Liu] Merge branch 'remote_r' into R
f04080c [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r
028cbfb [Davies Liu] fix exit code of sparkr unit test
42d8b4c [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
ef26015 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
a1870e8 [Shivaram Venkataraman] Merge pull request #214 from sun-rui/SPARKR-156_3
cb6e5e3 [Shivaram Venkataraman] Add scripts to start SparkR on windows
8030847 [Shivaram Venkataraman] Set windows file separators, install dirs
05afef0 [Shivaram Venkataraman] Only stop backend JVM if R launched it
95d2de3 [Davies Liu] fix spark-submit with R scripot
baefd9e [Shivaram Venkataraman] Make bin/sparkR use spark-submit As a part of this move the R initialization functions into first.R and first-submit.R
d6f2bdd [Shivaram Venkataraman] Fix run-tests path
ea90fab [Davies Liu] fix spark-submit with R path and sparkR -h
0e2412c [Davies Liu] fix bin/sparkR
9f6aa1f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R
479e3fe [Davies Liu] change println() to logging
52ca6e5 [Shivaram Venkataraman] Add missing comma
716b16f [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R
2d235d4 [Shivaram Venkataraman] Build SparkR with Maven profile
aae881b [Davies Liu] fix rat
ff776aa [Shivaram Venkataraman] Fix style
e4f1937 [Shivaram Venkataraman] Remove DFC example
f7b6936 [Davies Liu] remove Spark prefix for class
043959e [Davies Liu] cleanup
ba53b09 [Davies Liu] support R in spark-submit
f403b4a [Davies Liu] rm .travis.yml
c4a5bdf [Davies Liu] run sparkr tests in Spark
e8fc7ca [Davies Liu] fix .gitignore
35e5755 [Davies Liu] reduce size of example data
50bff63 [Davies Liu] add LICENSE header for R sources
facb6e0 [Davies Liu] add .gitignore for .o, .so, .Rd
18e5eed [Davies Liu] update docs
0a0e632 [Davies Liu] move sparkR into bin/
a76472f [Davies Liu] fix path of assembly jar
df3eeea [Davies Liu] move R/examples into examples/src/main/r
3415cc7 [Davies Liu] move Scala source into core/ and sql/
180fc9c [Davies Liu] move scala
014d253 [Davies Liu] delete man pages
49a8133 [Davies Liu] Merge branch 'remote_r' into R
44994c2 [Davies Liu] Moved files to R/
2fc553f [Shivaram Venkataraman] Merge pull request #222 from davies/column2
b043876 [Davies Liu] fix test
5e610cb [Davies Liu] add more API for Column
6f95d49 [Shivaram Venkataraman] Merge pull request #221 from shivaram/sparkr-stop-start
3214c6d [Shivaram Venkataraman] Merge pull request #217 from hlin09/cleanClosureFix
f5d3355 [Shivaram Venkataraman] Merge pull request #218 from davies/merge
70f620c [Davies Liu] address comments
4b1628d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into merge
3139325 [Shivaram Venkataraman] Merge pull request #212 from davies/toDF
6122e0e [Davies Liu] handle NULL
bc2ff38 [Davies Liu] handle NULL
7f5e70c [Davies Liu] Update SerDe.scala
46454e4 [Davies Liu] address comments
dd52cbc [Shivaram Venkataraman] Merge pull request #220 from shivaram/sparkr-utils-include
662938a [Shivaram Venkataraman] Include utils before SparkR for `head` to work Before this change calling `head` on a DataFrame would not work from the sparkR script as utils would be loaded after SparkR and placed ahead in the search list. This change requires utils to be loaded before SparkR
1bc2998 [Shivaram Venkataraman] Merge pull request #179 from evertlammerts/sparkr-sql
7695d36 [Evert Lammerts] added tests
8190127 [Evert Lammerts] fixed parquetFile signature
d8c8fcc [Shivaram Venkataraman] Merge pull request #219 from shivaram/sparkr-build-final
963c7ee [Davies Liu] Merge branch 'master' into merge
8bff523 [Shivaram Venkataraman] Remove staging repo now that 1.3 is released
e52258f [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into toDF
05b9126 [Shivaram Venkataraman] Merge pull request #215 from davies/agg
8e1497d [Davies Liu] Update DataFrame.R
72adb14 [Davies Liu] Update SQLContext.R
66cc92a [Davies Liu] address commets
55c38bc [Shivaram Venkataraman] Merge pull request #216 from davies/select2
3e0555d [Shivaram Venkataraman] Merge pull request #193 from davies/daemon
0467474 [Davies Liu] add more selecter for DataFrame
9a6be74 [Davies Liu] include grouping columns in agg()
e87bb98 [Davies Liu] improve comment and logging
a6dc435 [Davies Liu] remove dependency of jsonlite
26a3621 [Davies Liu] support date.frame and Date/Time
4e4908a [Davies Liu] createDataFrame from rdd
5757b95 [Shivaram Venkataraman] Merge pull request #196 from davies/die
90f2692 [Shivaram Venkataraman] Merge pull request #211 from hlin09/generics
8583968 [Davies Liu] readFully()
46cea3d [Davies Liu] retry
01aa5ee [Davies Liu] add config for using daemon, refactor
ff948db [hlin09] Remove missingOrInteger.
ecdfda1 [hlin09] Remove duplication.
411b751 [Davies Liu] make RStudio happy
8f8813f [Davies Liu] switch back to use parallel
6bccbbf [hlin09] Move roxygen doc back to implementation.
ffd6e8e [Shivaram Venkataraman] Merge pull request #210 from hlin09/hlin09
471c794 [hlin09] Move getJRDD and broadcast's value to 00-generic.R.
89b886d [hlin09] Move setGeneric() to 00-generics.R.
97dde1a [hlin09] Add a test for access operators.
09ff163 [Shivaram Venkataraman] Merge pull request #204 from cafreeman/sparkr-sql
15a713f [cafreeman] Fix example for `dropTempTable`
dc1291b [hlin09] Add checks for namespace access operators in cleanClosure.
b4c0b2e [Davies Liu] use fork package
3db5649 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql
789be97 [Shivaram Venkataraman] Merge pull request #207 from shivaram/err-remove
e60578a [cafreeman] update tests to guarantee row order
5eec6fc [Shivaram Venkataraman] Merge pull request #206 from sun-rui/SPARKR-156_2
3f7aed6 [Sun Rui] Fix minor typos in the function description.
a8cebf0 [Shivaram Venkataraman] Remove print statement in SparkRBackendHandler This print statement is noisy for SQL methods which have multiple APIs (like loadDF). We already have a better error message when no valid methods are found
5e3a576 [Sun Rui] Fix indentation.
f3d99a6 [Sun Rui] [SPARKR-156] phase 2: implement zipWithIndex() of the RDD class.
a582810 [cafreeman] Merge branch 'dfMethods' into sparkr-sql
7a5d6fd [cafreeman] `withColumn` and `withColumnRenamed`
c5fa3b9 [cafreeman] New `select` method
bcb0bf5 [Shivaram Venkataraman] Merge pull request #180 from davies/group
9dd6a5a [Davies Liu] Update SparkRBackendHandler.scala
e6fb8d8 [Davies Liu] improve logging
428a99a [Davies Liu] remove test, catch exception
fef99de [cafreeman] `intersect`, `subtract`, `unionAll`
befbd32 [cafreeman] `insertInto`
9d01bcd [cafreeman] `dropTempTable`
d8c1c09 [Davies Liu] add test to start and stop context multiple times
18c6004 [Shivaram Venkataraman] Merge pull request #201 from sun-rui/SPARKR-156_1
dfb399a [Davies Liu] address comments
f06ccec [Sun Rui] Use mapply() instead of for statement.
3c7674f [Davies Liu] Merge branch 'die' of github.com:davies/SparkR-pkg into die
ac8a852 [Davies Liu] close monitor connection in sparkR.stop()
4d0fb56 [Shivaram Venkataraman] Merge pull request #203 from shivaram/sparkr-hive-fix
62b0760 [Shivaram Venkataraman] Fix test hive context package name
47a613f [Shivaram Venkataraman] Fix HiveContext package name
fb3b139 [Davies Liu] fix tests
d0d4626 [Shivaram Venkataraman] Merge pull request #199 from davies/load
8b7fb67 [Davies Liu] fix HiveContext
bb46832 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load
e9e2a03 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group
b875b4f [Davies Liu] fix style
de2abfa [Shivaram Venkataraman] Merge pull request #202 from cafreeman/sparkr-sql
3675fcf [cafreeman] Update `explain` and fixed doc for `toJSON`
5fd9575 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load
6fac596 [Davies Liu] support Column expression in agg()
f10a24e [Davies Liu] address comments
ff8b005 [cafreeman] 'saveAsParquetFile`
a5c2887 [cafreeman] fix test
3fab0f8 [cafreeman] `showDF`
779c102 [cafreeman] `isLocal`
68b11cf [cafreeman] `toJSON`
0ac4abc [cafreeman] 'explain`
20242c4 [cafreeman] clean up docs
6a1fe64 [Shivaram Venkataraman] Merge pull request #198 from cafreeman/sparkr-sql
198c130 [Shivaram Venkataraman] Merge pull request #200 from shivaram/sparkr-sql-build
870acd4 [Shivaram Venkataraman] Use rc2 explicitly
8b9a963 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql
bc90115 [cafreeman] Fixed docs
3865f39 [Sun Rui] [SPARKR-156] phase 1: implement zipWithUniqueId() of the RDD class.
a37fd80 [Davies Liu] Update sparkR.R
d18f9d3 [Shivaram Venkataraman] Remove SparkR snapshot build We now have 1.3.0 RC2 on Apache Staging
8de958d [Davies Liu] Update SparkRBackend.scala
4e0becc [Shivaram Venkataraman] Merge pull request #194 from davies/api
197a79b [Davies Liu] add HiveContext (commented)
32aa01d [Shivaram Venkataraman] Merge pull request #191 from felixcheung/doc
5073e07 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load
7918634 [cafreeman] Fix test
acea146 [cafreeman] remove extra line
74269f3 [cafreeman] Merge branch 'dfMethods' into sparkr-sql
cd7ac8a [Shivaram Venkataraman] Merge pull request #197 from cafreeman/sparkr-sql
494a4dd [cafreeman] update export
e14c328 [cafreeman] `selectExpr`
32b37d1 [cafreeman] Fixed indent in `join` test.
2e7b190 [Felix Cheung] small update on yarn deploy mode.
8ff29d6 [Davies Liu] fix tests
12a6db2 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api
294ca4a [cafreeman] `join`, `sort`, and `filter`
4fa6343 [cafreeman] Refactor `join` generic for use with `DataFrame`
3f22c8d [Shivaram Venkataraman] Merge pull request #195 from cafreeman/sparkr-sql
2b6f980 [Davies Liu] shutdown the JVM after R process die
e8639c3 [cafreeman] New 1.3 repo and updates to `column.R`
ed9a89f [Davies Liu] address comments
03bcf20 [Davies Liu] Merge branch 'group' of github.com:davies/SparkR-pkg into group
39c253d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group
98cc97a [Davies Liu] fix test and docs
e2d144a [Felix Cheung] Fixed small typos
3beadcf [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api
06cbc2d [Davies Liu] launch R worker by a daemon
8a676b1 [Shivaram Venkataraman] Merge pull request #188 from davies/column
524c122 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column
f798402 [Davies Liu] Update column.R
1d0f2ae [Davies Liu] Update DataFrame.R
03402eb [Felix Cheung] Updates as per feedback on sparkR-submit
76cf2e0 [Shivaram Venkataraman] Merge pull request #192 from cafreeman/sparkr-sql
1955a09 [cafreeman] return object instead of a list of one object
f585929 [cafreeman] Fix brackets
e998356 [cafreeman] define generic for 'first' in RDD API
71d66a1 [Davies Liu] fix first(0
8ec21af [Davies Liu] fix signature
acae527 [Davies Liu] refactor
d7b17a4 [Davies Liu] fix approxCountDistinct
7dfe27d [Davies Liu] fix cyclic namespace dependency
8caf5bb [Davies Liu] use S4 methods
5c0bb24 [Felix Cheung] Doc updates: build and running on YARN
773baf0 [Zongheng Yang] Merge pull request #178 from davies/random
862f07c [Shivaram Venkataraman] Merge pull request #190 from shivaram/SPARKR-79
b457833 [Shivaram Venkataraman] Merge pull request #189 from shivaram/stdErrFix
f7caeb8 [Davies Liu] Update SparkRBackend.scala
8c4deae [Shivaram Venkataraman] Remove unused function
6e51c7f [Shivaram Venkataraman] Fix stderr redirection on executors
7afa4c9 [Shivaram Venkataraman] Merge pull request #186 from hlin09/funcDep3
4d36ab1 [hlin09] Add tests for broadcast variables.
3f57e56 [hlin09] Fix comments.
7b72487 [hlin09] Fix comments.
ae05bf1 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column
abb4bb9 [Davies Liu] add Column and expression
eb8ac11 [Shivaram Venkataraman] Set Spark version 1.3.0 in Windows build
5c72e73 [Davies Liu] wait atmost 100 seconds
e425437 [Shivaram Venkataraman] Merge pull request #177 from lythesia/master
a00f502 [lythesia] fix indents
0346e5f [Davies Liu] address comment
6134649 [Shivaram Venkataraman] Merge pull request #187 from cafreeman/sparkr-sql
ad0935e [lythesia] minor fixes
b0e7f73 [cafreeman] Update `sampleDF` test
7b0d070 [lythesia] keep partitions check
889c265 [cafreeman] numToInt utility function
27dd3a0 [lythesia] modify tests for repartition
cad0f0c [cafreeman] Fix docs and indents
2808dcf [cafreeman] Three more DataFrame methods
5ef66fb [Davies Liu] send back the port via temporary file
3b46429 [Davies Liu] Merge branch 'master' of github.com:amplab-extras/SparkR-pkg into random
798f453 [cafreeman] Merge branch 'sparkr-sql' into dev
9aa4acf [Shivaram Venkataraman] Merge pull request #184 from davies/socket
020bce8 [Shivaram Venkataraman] Merge pull request #183 from cafreeman/sparkr-sql
222e06b [cafreeman] Lazy evaluation and formatting changes
e776324 [Davies Liu] fix import
211cc15 [cafreeman] Merge branch 'sparkr-sql' into dev
3351afd [hlin09] Replaces getDependencies with cleanClosure, to serialize UDFs to workers.
e7c56d6 [lythesia] fix random partition key
50c74b1 [Davies Liu] address comments
083c89f [cafreeman] Remove commented lines an unused import
dfa119b [hlin09] Improve the coverage of processClosure.
a41c9b9 [cafreeman] Merge branch 'wrapper' into sparkr-sql
1cd714f [cafreeman] Wrapper function docs.
db0cd9e [cafreeman] Clean up for wrapper functions
818c19f [cafreeman] Update schema-related functions
a57884e [cafreeman] Remove unused import
d72e830 [cafreeman] Add wrapper for `StructField` and `StructType`
2ea2ecf [lythesia] use generic arg
09b9512 [hlin09] add docs
f4f077c [hlin09] Add recursive cleanClosure for function access.
f84ad27 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2
5300766 [Shivaram Venkataraman] Merge pull request #185 from hlin09/hlin09
07aa7c0 [hlin09] Unifies the implementation of lapply with lapplyParitionsWithIndex.
f4dbb0b [Davies Liu] use socket in worker
8282c59 [Davies Liu] Update DataFrame.R
ba495a8 [Davies Liu] Update NAMESPACE
36dffb3 [cafreeman] Add 'head` and `first`
534a95f [cafreeman] Schema-related methods
64f488d [cafreeman] Cache and Persist Methods
30d71fd [cafreeman] Standardize method arguments for DataFrame methods
785898b [Shivaram Venkataraman] Merge pull request #182 from cafreeman/sparkr-sql
2619003 [Shivaram Venkataraman] Merge pull request #181 from cafreeman/master
a9bbe0b [cafreeman] Update existing SparkSQL functions
8c241a3 [cafreeman] Merge with master, include changes to method args
68d6de4 [cafreeman] Fix typos
8d2ec6e [Davies Liu] add sum/max/min/avg/mean
774e687 [Davies Liu] add missing API in SQLContext
1e72b4b [Davies Liu] missing API in SQLContext
3294949 [Chris Freeman] Restore `rdd` argument to `getJRDD`
3a58ebc [Davies Liu] rm unrelated file
8bd93b5 [Davies Liu] fix signature
c652b4c [cafreeman] Update method signatures to use generic arg
48c8827 [Davies Liu] update NAMESPACE
84e2d8c [Davies Liu] groupBy and agg()
7c3ddbd [Davies Liu] create jmode in JVM
9465426 [Davies Liu] load and save
982f342 [lythesia] fix numeric issue
7651d84 [lythesia] fix coalesce
4e712e1 [Davies Liu] use random port in backend
041d22b [Shivaram Venkataraman] Merge pull request #172 from cafreeman/sparkr-sql
0d07770 [cafreeman] Added `limit` and updated `take`
301d8e5 [cafreeman] Remove extraneous map functions
0387db2 [cafreeman] Remove colNames
04c4b65 [lythesia] add repartition/coalesce
231deab [cafreeman] Change reserialize to serializeToBytes
acf7e1a [cafreeman] Rework the Scala to R DataFrame Conversion
481ae37 [cafreeman] Updated stale comments and standardized arg names
21d4a97 [hlin09] Adds cleanClosure to capture the function closures.
d24ffb4 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2
8be02de [hlin09] Revert "loop 1-12 test pass."
fddb9cc [hlin09] Revert "add docs"
f8ef0ab [hlin09] Revert "More docs"
8e4b3da [hlin09] Revert "More docs"
57e005b [hlin09] Revert "fix tests."
c10148e [Shivaram Venkataraman] Merge pull request #174 from shivaram/sparkr-runner
910e3be [Shivaram Venkataraman] Add a timeout for initialization Also move sparkRBackend.stop into a finally block
bf52b17 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner
08102b0 [Shivaram Venkataraman] Merge pull request #176 from lythesia/master
9c77b20 [Chris Freeman] Merge pull request #2 from shivaram/sparkr-sql
179ab38 [lythesia] add try counts and increase time interval
71a73b2 [Shivaram Venkataraman] Use a getter for serialization mode This change encapsulates the semantics of serialization mode for RDDs inside a getter function. For PipelinedRDDs if a backing JavaRDD is available we use that else we fall back to a default serialization mode
06bf250 [Shivaram Venkataraman] Merge pull request #173 from shivaram/windows-space-fix
88bf97f [Shivaram Venkataraman] Create SparkContext for R shell launch
f9268d9 [Shivaram Venkataraman] Fix code review comments
e6ad12d [Shivaram Venkataraman] Update comment describing sparkR-submit
17eda4c [Shivaram Venkataraman] Merge pull request #175 from falaki/docfix
ba2b72b [Hossein] Spark 1.1.0 is default
4cd7d3f [lythesia] retry backend connection
749e2d0 [Hossein] Updated README
bc04cf4 [Shivaram Venkataraman] Use SPARKR_BACKEND_PORT in sparkR.R as default Change SparkRRunner to use EXISTING_SPARKR_BACKEND_PORT to differentiate between the two
22a19ac [Shivaram Venkataraman] Use a semaphore to wait for backend to initalize Also pick a random port to avoid collisions
7f1f0f8 [cafreeman] Move comments to fit 100 char line length
8b84e4e [cafreeman] Make if statements more explicit
ce5d5ab [cafreeman] New tests for Union and Object File
b063320 [cafreeman] Changed 'serialized' to 'serializedMode'
0981dff [Zongheng Yang] Merge pull request #168 from sun-rui/SPARKR-153_2
86fc639 [Shivaram Venkataraman] Move sparkR-submit into pkg/inst
fd8f8a9 [Shivaram Venkataraman] Merge branch 'hqzizania-master'
a33dbea [Shivaram Venkataraman] Merge branch 'master' of https://github.com/hqzizania/SparkR-pkg into hqzizania-master
384e6e2 [Shivaram Venkataraman] Merge pull request #171 from hlin09/hlin09
1f5a6ac [hlin09] fixed comments
7f7596a [cafreeman] Additional handling for "row" serialization
8c3b8c5 [cafreeman] Add test for UnionRDD on "row" serialization
b1141f8 [cafreeman] Fixed formatting issues.
5db30bf [cafreeman] Changed serialized from bool to string
2f0c0b8 [cafreeman] Add check for serialized type
d243dfb [cafreeman] Clean up code
5ff63a2 [cafreeman] Change test from boolean to string
77fec1a [cafreeman] Updated .Rd files
9224989 [cafreeman] Various updates for DataFrame to RRDD
26af62b [cafreeman] DataFrame to RRDD
e004481 [cafreeman] Update UnionRDD test
5292be7 [hlin09] Adds support of pipeRDD().
e2a7560 [Shivaram Venkataraman] Merge pull request #170 from cafreeman/sparkr-sql
5d537f4 [cafreeman] Add pairRDD to Description
b6fa88e [cafreeman] Updating to current master
0cda231 [Sun Rui] [SPARKR-153] phase 2: implement aggregateByKey() and foldByKey().
95ee6b4 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner
67fbc60 [Shivaram Venkataraman] Add support for SparkR shell to use spark-submit This ensures that SparkConf options are read in both in batch and interactive modes
2271030 [Shivaram Venkataraman] Merge pull request #167 from sun-rui/removePartionByInRDD
7fcb46a [Sun Rui] Remove partitionBy() in RDD.
52f94c4 [Shivaram Venkataraman] Merge pull request #160 from lythesia/master
59e2d54 [lythesia] merge with upstream
5836650 [Zongheng Yang] Merge pull request #163 from sun-rui/SPARKR-153_1
141723e [Sun Rui] fix comments.
f73a07e [Shivaram Venkataraman] Merge pull request #165 from shivaram/sparkr-sql-build
10ffc6d [Shivaram Venkataraman] Set Spark version to 1.3 using staging dependency Also fix the maven build
c91ede2 [Shivaram Venkataraman] Merge pull request #164 from hlin09/hlin09
9d335a9 [hlin09] Makes git to ignore Eclipse meta files.
94066bf [Sun Rui] [SPARKR-153] phase 1: implement fold() and aggregate().
9c391c7 [hqzizania] Merge remote-tracking branch 'upstream/master'
5f29551 [hqzizania] 	modified:   pkg/R/RDD.R 	modified:   pkg/R/context.R
d968664 [lythesia] fix comment
7972858 [Shivaram Venkataraman] Merge pull request #159 from sun-rui/SPARKR-150_2
7690878 [lythesia] separate out pair RDD functions
f4573c1 [Sun Rui] Use reduce() instead of sortBy().take() to get the ordered elements.
63e62ed [Sun Rui] [SPARKR-150] phase 2: implement takeOrdered() and top().
050390b [Shivaram Venkataraman] Fix bugs in inferring R file
8398f2e [Shivaram Venkataraman] Add sparkR-submit helper script Also adjust R file path for YARN cluster mode
bd6705b [Zongheng Yang] Merge pull request #154 from sun-rui/SPARKR-150
c7964c9 [Sun Rui] Merge with upstream master.
7feac38 [Sun Rui] Use default arguments for sortBy() and sortKeyBy().
de2bfb3 [Sun Rui] Fix minor comments and add more test cases.
0c6e071 [Zongheng Yang] Merge pull request #157 from lythesia/master
f5038c0 [lythesia] pull out anonymous functions in groupByKey
ba6f044 [lythesia] fixes for reduceByKeyLocally
343b6ab [Oscar Olmedo] Export sparkR.stop Closes #156 from oscaroboto/master
25639cf [Shivaram Venkataraman] Replace tabs with spaces
bb25920 [Shivaram Venkataraman] Merge branch 'dputler-master'
fd836db [hlin09] fix tests.
24a7f13 [hlin09] More docs
a465165 [hlin09] More docs
6ad4fc3 [hlin09] add docs
b082a35 [lythesia] add reduceByKeyLocally
7ca6512 [Shivaram Venkataraman] First cut of SparkRRunner
193f5fe [hlin09] loop 1-12 test pass.
345f1b8 [dputler] [SPARKR-195] Implemented project style guidelines for if-else statements
8043559 [Sun Rui] Add a TODO to use binary search in the range partitioner.
91b2fd6 [Sun Rui] Add more test cases.
e8ebbe4 [Shivaram Venkataraman] Merge pull request #152 from cafreeman/sparkr-sql
0c53d6c [dputler] Data frames now coerced to lists, and messages issued for a data frame or matrix on how they are parallelized
6d57ec0 [cafreeman] Remove json test file since we're using a temp
ac1ef09 [cafreeman] Update registerTempTable test
d9da451 [Sun Rui] [SPARKR-150] phase 1: implement sortBy() and sortByKey().
08ff30b [Shivaram Venkataraman] Merge pull request #153 from hqzizania/master
9767e8e [hqzizania] 	modified:   pkg/man/collect-methods.Rd
5d69f0a [hqzizania] 	modified:   pkg/R/RDD.R
4914091 [hqzizania] 	modified:   pkg/inst/tests/test_rdd.R
742a68b [cafreeman] Update test_sparkRSQL.R
a95823e [hqzizania] 	modified:   pkg/R/RDD.R
2d04526 [cafreeman] Formatting
fae9bdd [cafreeman] Renamed to SQLUtils.scala
39888ea [Chris Freeman] Update test_sparkSQL.R
fce2453 [cafreeman] Updated documentation for SQLContext
13fbf12 [cafreeman] Regenerated .Rd files
51ecf41 [cafreeman] Updated Scala object
30d7337 [cafreeman] Added SparkSQL test
74b3ed6 [cafreeman] Incorporate code feedback
554bda0 [Zongheng Yang] Merge pull request #147 from shivaram/sparkr-ec2-fixes
a5f4f8f [cafreeman] Squashed commit of the following:
f34bb88 [Shivaram Venkataraman] Remove profiling information from this PR
c662f29 [Zongheng Yang] Merge pull request #146 from shivaram/spark-1.2-build
21e9b74 [Zongheng Yang] Merge pull request #145 from lythesia/master
76f6b9e [Shivaram Venkataraman] Merge pull request #149 from hqzizania/master
1c2dbec [lythesia] minor fix for refactoring join code
5b380d3 [hqzizania] 	modified:   pkg/man/combineByKey.Rd 	modified:   pkg/man/groupByKey.Rd 	modified:   pkg/man/partitionBy.Rd 	modified:   pkg/man/reduceByKey.Rd
98794fe [hqzizania] 	modified:   pkg/R/RDD.R
b66534d [Zongheng Yang] Merge pull request #144 from shivaram/fix-rd-files
60da1df [Shivaram Venkataraman] Initialize timing variables
179aa75 [Shivaram Venkataraman] Bunch of fixes for longer running jobs 1. Increase the timeout for socket connection to wait for long jobs 2. Add some profiling information in worker.R 3. Put temp file writes before stdin writes in RRDD.scala
06d99f0 [Shivaram Venkataraman] Fix URI to have right number of slashes
add97f5 [Shivaram Venkataraman] Use URL encode to create valid URIs for jars
4eec962 [lythesia] refactor join functions
73430c6 [Shivaram Venkataraman] Make SparkR work on paths with spaces on Windows
aaf8f47 [Shivaram Venkataraman] Exclude hadoop client from Spark dependency
227ee42 [Zongheng Yang] Merge pull request #141 from shivaram/SPARKR-140
ac5ceb1 [Shivaram Venkataraman] Fix code review comments
32394de [Shivaram Venkataraman] Regenerate Rd files for SparkR This fixes a number of issues in SparkR man pages. The main changes are 1. Don't export or generate docs for PipelineRDD 2. Fix variable names for Filter, count to match base methods 3. Document missing arguments for sparkR.init, print.jobj etc.
e157bf6 [Shivaram Venkataraman] Use prev_serialized to track if JRDD is serialized This changes introduces a new variable in PipelineRDD environment to track if the prev_jrdd is serialized or not.
7428a7e [Zongheng Yang] Merge pull request #143 from shivaram/SPARKR-181
7dd1797 [Shivaram Venkataraman] Address code review comments
8f81c45 [Shivaram Venkataraman] Remove roxygen export for PipelinedRDD
0cb90f1 [Zongheng Yang] Merge pull request #142 from shivaram/SPARKR-169
d1c6e6c [Shivaram Venkataraman] Buffer stderr from R and return it on Exception This change buffers the last 100 lines from R process and passes these lines back to the driver if we have an exception. This will help users debug why their tasks failed on the cluster
d6c1393 [Shivaram Venkataraman] Suppress warnings from normalizePath
a382835 [Shivaram Venkataraman] Fix serialization tracking in pipelined RDDs When creating a pipeline RDD, we need to check if the JavaRDD belonging to the parent is serialized.
da39529 [Zongheng Yang] Merge pull request #140 from sun-rui/SPARKR-183
2814caa [Sun Rui] Merge with upstream master.
cd2a5b3 [Sun Rui] Add reference to Nagle's algorithm and clean code.
52356b6 [Shivaram Venkataraman] Merge pull request #139 from shivaram/fix-backend-exit
97e5a1f [Sun Rui] [SPARKR-183] Fix the issue that parallelize collect tests are slow.
a9f8e8e [Shivaram Venkataraman] Merge pull request #138 from concretevitamin/fix-collect-test
125ae43 [Shivaram Venkataraman] Fix SparkR backend to exit in more cases This change has two fixes 1. When the workspace is saved (from R or RStudio) the backend connection seems to be closed before the finalizer is run. In such cases we reopen the connection and stop the backend 2. With RStudio when R is restarted, there are port-conflicts which appear due to a race condition between the JVM and rsession restart. This change adds a 1 sec sleep to avoid this race.
12c102a [Zongheng Yang] Simplify a unit test.
9c0637a [Zongheng Yang] Merge pull request #137 from shivaram/fix-docs
0df0e18 [Shivaram Venkataraman] Fix documentation for includePackage
7549f88 [Zongheng Yang] Merge pull request #136 from shivaram/man-updates
7edbe46 [Shivaram Venkataraman] Add missing man pages
9cb9567 [Shivaram Venkataraman] Merge pull request #131 from shivaram/rJavaExpt
1fa722e [Shivaram Venkataraman] Rename to SerDe now
2fcb051 [Shivaram Venkataraman] Rename to SerDeJVMR
d112cf0 [Shivaram Venkataraman] Style fixes
9fd01cc [Shivaram Venkataraman] Remove unnecessary braces
0881931 [Shivaram Venkataraman] Some more style fixes
f00b531 [Shivaram Venkataraman] Address code review comments. Big changes include style fixes throughout for named arguments
c09ba05 [Shivaram Venkataraman] Change jobj id to be just an integer Add a new print.jobj that gets the class name and prints it Also add a utility function isInstanceOf
be05b16 [Shivaram Venkataraman] Check if context, connection exist before stopping
d596a23 [Shivaram Venkataraman] Address code review comments
396e7ac [Shivaram Venkataraman] Changes to make new backend work on Windows This change uses file.path to construct the Java binary path in a OS agnostic way and uses system2 to handle quoting binary paths correctly. Tests pass on Mac OSX and a Windows EC2 instance.
e7a4e03 [Shivaram Venkataraman] Remove unused file BACKEND.md
62f380b [Shivaram Venkataraman] Update worker.R to use new deserialization call
8b9c4e6 [Shivaram Venkataraman] Change RDD name, setName to use new backend
6dcd5c5 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into rJavaExpt
0873397 [Shivaram Venkataraman] Refactor java object tracking into a new singleton. Also add comments describing each class
95db964 [Shivaram Venkataraman] Add comments, cleanup new R code
bcd4258 [Zongheng Yang] Merge pull request #130 from lythesia/master
74dbc5e [Sun Rui] Match method using parameter types.
7ad4a4d [Sun Rui] Use 1 char to represent types on the backend->client direction.
bace887 [Sun Rui] Use an integer count for the backend java object ID because Uniqueness isn't guaranteed by System.identityHashCode().
b38d04f [Sun Rui] Use 1 char to represent types on the client -> backend direction.
f88bc68 [lythesia] Merge branch 'master' of github.com:lythesia/SparkR-pkg
71d41f5 [lythesia] add test case for fullOuterJoin
eb4f423 [lythesia] --amend
cffecc5 [lythesia] add test case for fullOuterJoin
a547dd2 [Shivaram Venkataraman] Move classTag, rddRef into newJObject call This avoids them getting eagerly garbage collected
1255391 [Shivaram Venkataraman] Add a finalizer for jobj objects This enables Java objects to be garbage collected on the backend when they are no longer referenced in R. Also rename newJava to newJObject to be more consistent with callJMethod
70fa409 [Sun Rui] Add YARN Conf Dir to the class path when launching the backend.
a1108ca [lythesia] add fullOuterJoin in RDD.R
2152727 [Shivaram Venkataraman] Remove empty file
cd08bee [Shivaram Venkataraman] Update all functions to use new backend All unit tests pass.
9de49b7 [Shivaram Venkataraman] Add high level calls for methods, constructors Also update BACKEND.md
5a97ea4 [Shivaram Venkataraman] Add jobj S3 class that holds backend refs
e071d3e [Shivaram Venkataraman] Change SparkRBackend to use general method calls This change uses a custom protocl + JNI to invoke any method on a given object type. Also update serializers, deserializers to make code more concise
49f0404 [Shivaram Venkataraman] Merge pull request #129 from lythesia/master
7f8cd82 [lythesia] update man
4715ed2 [Yi Lu] Update RDD.R
5a53801 [lythesia] fix name,setName
4f3870b [lythesia] add name,setName in RDD.R
1c25700 [Shivaram Venkataraman] Merge pull request #128 from sun-rui/SPARKR-165
c8507d8 [Sun Rui] [SPARKR-165] IS_SCALAR is not present in R before 3.1
2cff2bd [Sun Rui] Add function to invoke Java method.
7a31da1 [Shivaram Venkataraman] Merge branch 'dputler-master'. Closes #119
0ceba82 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/dputler/SparkR-pkg into dputler-master
735f70c [Shivaram Venkataraman] Merge pull request #125 from 7c00/rawcon
fccfe6c [Shivaram Venkataraman] Merge pull request #127 from sun-rui/SPARKR-164
387bd57 [Sun Rui] [SPARKR-164] Temporary files used by SparkR accumulat as time goes on.
5f2268f [Shivaram Venkataraman] Add support to stop backend
5f745c0 [Shivaram Venkataraman] Update notes in backend
22015c1 [Shivaram Venkataraman] Add first cut of SparkR Backend
52821da [Todd Gao] switch the order of packages and function deps
d7b0007 [Todd Gao] remove memCompress
cb6873e [Shivaram Venkataraman] Merge pull request #126 from sun-rui/SPARKR-147
c5962eb [Todd Gao] further optimize using rawConnection
f04c6e0 [Sun Rui] [SPARKR-147] Support multiple directories as input to textFile.
b7de604 [Todd Gao] optimize execFunctionDeps loading in worker.R
4d4fc30 [Shivaram Venkataraman] Merge pull request #122 from cafreeman/master
b508877 [cafreeman] Update SparkR_IDE_Setup.sh
21ed9d7 [cafreeman] Update build.sbt
f73ec16 [cafreeman] Delete SparkR_IDE_Setup_Guide.md
d63b026 [cafreeman] Delete SparkR_Quick_Start_Guide.md
6e6cb62 [cafreeman] Update SparkR_IDE_Setup.sh
bc6042b [cafreeman] Update build.sbt
a8197d5 [cafreeman] Merge remote-tracking branch 'upstream/master'
d671564 [Zongheng Yang] Merge pull request #123 from shivaram/jcheck-void
76b8d00 [Zongheng Yang] Merge pull request #124 from shivaram/master
b690d58 [Shivaram Venkataraman] Specify how to change Spark versions in README
0fb003d [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into jcheck-void
1c227b4 [Shivaram Venkataraman] Also add a check in context.R
96812b6 [Shivaram Venkataraman] Check for exceptions after void method calls
f5c216d [cafreeman] Merge remote-tracking branch 'upstream/master'
90c8933 [Zongheng Yang] Merge pull request #121 from shivaram/fix-sort-order
bd0e3b4 [Shivaram Venkataraman] Fix saveAsTextFile test case
2e55f67 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into fix-sort-order
f10c607 [Shivaram Venkataraman] Merge pull request #118 from sun-rui/saveAsTextFile
6c9bfc0 [Sun Rui] Merge remote-tracking branch 'SparkR_upstream/master' into saveAsTextFile
6faedbe [cafreeman] Update SparkR_IDE_Setup_Guide.md
57008bc [cafreeman] Update SparkR_IDE_Setup.sh
bb1c17d [cafreeman] Update SparkR_IDE_Setup.sh
538bfdb [cafreeman] Update SparkR_Quick_Start_Guide.md
31322c6 [cafreeman] Update SparkR_IDE_Setup.sh
ca3f593 [Sun Rui] Refactor RRDD code.
df58d95 [cafreeman] Update SparkR_Quick_Start_Guide.md
b488c88 [cafreeman] Rename Spark_IDE_Setup.sh to SparkR_IDE_Setup.sh
b2545a4 [cafreeman] Added IDE Setup Guide
0ffb5de [cafreeman] Merge branch 'master' of https://github.com/cafreeman/SparkR-pkg
bd8fbfb [cafreeman] Merge remote-tracking branch 'upstream/master'
98efa5b [cafreeman] Added Quick Start Guide
3cf88f2 [Shivaram Venkataraman] Sort lists before comparing in unit tests Since Spark doesn't guarantee that shuffle results will always be in the same order, we need to sort the results before comparing for deterministic behavior
d621dbc [Shivaram Venkataraman] Merge pull request #120 from sun-rui/objectFile
c4a44d7 [Sun Rui] Add @seealso in comments and extract some common code into a function.
724e3a4 [cafreeman] Update Spark_IDE_Setup.sh
8153e5a [Sun Rui] [SPARKR-146] Support read/save object files in SparkR.
17f9909 [cafreeman] Update Spark_IDE_Setup.sh
a9eb080 [cafreeman] IDE Shell Script
64d800c [dputler] Merge remote branch 'upstream/master'
1fbdb2e [dputler] Added the ability for the user to specify a text file location throught the use of tilde expansion or just the file name if it is in the working directory.
d83c017 [Shivaram Venkataraman] Merge pull request #113 from sun-rui/stringHashCodeInC
a7d9cdb [Sun Rui] Fix build on Windows.
7d81b05 [Shivaram Venkataraman] Merge pull request #114 from hlin09/hlin09
47c4bb7 [hlin09] fix reviews
a457f7f [Shivaram Venkataraman] Merge pull request #116 from dputler/master
0fa48d1 [Shivaram Venkataraman] Merge pull request #117 from sun-rui/keyBy
85cfeb4 [Sun Rui] [SPARKR-144] Implement saveAsTextFile() in the RDD class.
09083d9 [Sun Rui] Add keyBy() to the RDD class.
caad5d7 [dputler] Adding the script to install software on the Cloudera Quick Start VM.
dca3d05 [hlin09] Minor fix.
ece5f7d [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09
a40874b [hlin09] Use extendible accumulators aggregate the cogroup values.
d0347ce [Zongheng Yang] Merge pull request #112 from sun-rui/outer_join
492f76e [Sun Rui] Refine code and add description.
ba01358 [Shivaram Venkataraman] Merge pull request #115 from sun-rui/SPARKR-130
5c8e46e [Sun Rui] Fix per the review comments.
7190a2c [Sun Rui] Update comment to add a reference to storage levels.
1da705e [hlin09] Fix the review comments.
c4b77be [Sun Rui] [SPARKR-130] Add persist(storageLevel) API to RDD.
b424a1a [hlin09] Add function cogroup().
9770312 [Shivaram Venkataraman] Merge pull request #111 from hlin09/hlin09
cead7df [hlin09] fix review comments.
54f712e [Sun Rui] Implement string hash code in C.
425f0c6 [Sun Rui] Add leftOuterJoin() and rightOuterJoin() to the RDD class.
39509c7 [hlin09] add Rd file for foreach and foreachPartition.
63d6ac7 [hlin09] Adds function foreach() and foreachPartition().
9c954df [Zongheng Yang] Merge pull request #105 from sun-rui/join
c71228d [Sun Rui] Pre-allocate list with fixed length. Add test case for join() using string key.
bc3e9f6 [Shivaram Venkataraman] Merge pull request #108 from concretevitamin/take-optimize
c06fc90 [Zongheng Yang] Fix: only optimize for unserialized dataset case.
d399aeb [Zongheng Yang] Apply size-capping on logical representation instead of physical.
e4217dd [Zongheng Yang] Merge pull request #107 from shivaram/master
7952180 [Shivaram Venkataraman] Copy, use getLocalDirs from Spark Utils.scala
08e24c3 [Zongheng Yang] Merge pull request #109 from hlin09/hlin09
97d4e02 [Zongheng Yang] Min() upper-bound size with actual size.
bb779bf [hlin09] Rename the filter function to filterRDD to follow the API consistency. Filter() is also kept.
ce1661f [Zongheng Yang] Fix slow take(): deserialize only up to necessary # of elements.
4dca9b1 [Shivaram Venkataraman] Merge pull request #106 from hlin09/hlin09
1220d92 [hlin09] Adds function numPartitions().
2326a65 [Shivaram Venkataraman] Use SPARK_LOCAL_DIRS to create tmp files
e119757 [hlin09] Minor fix.
9c24c8b [hlin09] Adds function countByKey().
48fce67 [hlin09] Adds countByValue().
6679eef [Sun Rui] Update documentation for join().
70586b4 [Sun Rui] Add join() to the RDD class.
e6fb999 [Zongheng Yang] Merge pull request #103 from shivaram/rlibdir-fix
a21f146 [Shivaram Venkataraman] Merge pull request #102 from hlin09/hlin09
32eb619 [Shivaram Venkataraman] Merge pull request #104 from sun-rui/add_keys_values
d8692e9 [Sun Rui] Add keys() and values() for the RDD class.
18b9be1 [Shivaram Venkataraman] Allow users to set where SparkR is installed This also adds a warning if somebody tries to call sparkR.init multiple times.
a17f135 [hlin09] Adds tests for flatMap and flatMapValues.
4bcf59b [hlin09] Adds function flatMapValues.
4a193ef [Zongheng Yang] Merge pull request #101 from ashutoshraina/master
60d22f2 [Ashutosh Raina] changed sbt version
5400793 [Zongheng Yang] Merge pull request #98 from shivaram/windows-fixes-build
36d61a7 [Shivaram Venkataraman] Merge pull request #97 from hlin09/hlin09
f7d7d89 [hlin09] Remove redundant code in test.
6bbe823 [hlin09] minor style fix.
9b47f3a [Shivaram Venkataraman] Merge pull request #100 from hnahak87/patch-1
7f6e4ea [Harihar Nahak] Update logistic_regression.R
a605047 [Shivaram Venkataraman] Merge pull request #99 from hlin09/makefile
323151d [hlin09] Fix yar flag in Makefile to remove build error in Maven.
8911897 [hlin09] Make reserialize() private function in package.
79aee73 [Shivaram Venkataraman] Add notes on how to build SparkR on windows
49a99e7 [Shivaram Venkataraman] Clean up some commented code
ddc271b [Shivaram Venkataraman] Only append file:/// to non empty jar paths
a53952e [Shivaram Venkataraman] Add windows build scripts
325b179 [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09
daf5040 [hlin09] Add reserialize() before union if two RDDs are not both serialized.
536afb1 [hlin09] Add new function of union().
7044677 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes
d22a02d [Zongheng Yang] Merge pull request #94 from shivaram/windows-fixes-stdin
51924f7 [Shivaram Venkataraman] Merge pull request #90 from oscaroboto/master
eb97d85 [Shivaram Venkataraman] Merge pull request #96 from sun-rui/add_clarification_readme
5a128f4 [Sun Rui] Add clarification on setting Spark master when launching the SparkR shell.
187526a [oscaroboto] Update sparkR.R
32c567b [Shivaram Venkataraman] Merge pull request #95 from concretevitamin/master
4cd2d5e [Zongheng Yang] Notes about spark-ec2.
1c28e3b [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes
8e8a029 [Zongheng Yang] Merge pull request #92 from shivaram/sparkr-yarn
721043b [Zongheng Yang] Update README.md with YARN instructions.
1681f58 [Shivaram Venkataraman] Use temporary files for input instead of stdin This fixes a bug for Windows where stdin would get truncated
b084314 [oscaroboto] removed ... from example
44c93d4 [oscaroboto] Added example to SparkR.R
be82dcc [Shivaram Venkataraman] Merge pull request #93 from hlin09/hlin09
868554d [oscaroboto] Update sparkR.R
488ac47 [hlin09] Add generated Rd file of previous added functions, distinct() and mapValues().
b2740ad [hlin09] Add test for filter all elements. Add filter() as alias.
08d3631 [hlin09] Minor style fixes.
2c0e34f [hlin09] Adds function Filter(), which extracts the elements that satisfy a predicate.
5951d3b [Shivaram Venkataraman] Remove SBT plugin
4e70ced [oscaroboto] changed ExecutorEnv to sparkExecutorEnvMap, to make it consistent with sparkEnvirMap
903d18a [oscaroboto] changed executorEnv to sparkExecutorEnvMap,  will do the same in R
f97346e [oscaroboto] executorEnv to lower-case e
88a524e [oscaroboto] Added LD_LIBRARY_PATH to the ExecutorEnv. This is need so that the nodes can find libjvm.so, or if the master has a different LD_LIBRARY_PATH then the nodes. Make sure to export LD_LIBRARY_PATH  that includes the path to libjvm.so in the nodes.
1d208ae [oscaroboto] added the YARN_CONF_DIR to the classpath
8a9b75c [oscaroboto] forgot to change hm and ee inside the for loops
579db58 [Shivaram Venkataraman] Merge pull request #91 from sun-rui/add_max_min
4381efa [Sun Rui] use reduce() to implemement max() and min().
a5459c5 [Shivaram Venkataraman] Consolidate yarn flags
86b04eb [Shivaram Venkataraman] Don't use quotes around yarn
bf0797f [Shivaram Venkataraman] Add dependency on spark yarn module
af5fe77 [Shivaram Venkataraman] Fix SBT build, add dependency tree plugin
4917607 [Sun Rui] Add maximum() and minimum() API to RDD.
51bbbe4 [Shivaram Venkataraman] Changes to make SparkR work with YARN
9d5e3ab [oscaroboto] a few stylistic changes. Also change vars to sparkEnvirMap and eevars to ExecutorEnv, to match sparkR.R
578f545 [oscaroboto] a few stylistic changes
39eea2f [oscaroboto] Modification to dynamically create a sparkContext with YARN. Added .setExecutorEnv to the sparkConf in createSparkContext within the RRDD object. This modification was made together with sparkR.R
17ec42e [oscaroboto] A modification to dynamically create a sparkContext with YARN. sparkR.R modified to pass custom Jar file names and EnvironmentEnv to the sparkConf. RRDD.scala was also modified to accept the new inputs to creatSparkContext.
624ac9d [Shivaram Venkataraman] Merge pull request #87 from sun-rui/SPARKR-125
4f213db [Shivaram Venkataraman] Merge pull request #89 from sun-rui/SPARKR-108
eb833c5 [Shivaram Venkataraman] Merge pull request #88 from hlin09/hlin09
07bf971 [Sun Rui] [SPARKR-108] Implement map-side reduction for reduceByKey().
4accba1 [hlin09] Fixes style and adds an optional param 'numPartition' in distinct().
80d303a [hlin09] typo fixed.
e37a9b5 [hlin09] Adds function distinct() and mapValues().
08dac06 [Sun Rui] [SPARKR-125] Get the iterator of the parent RDD before launching a R worker process in compute() of RRDD/PairwiseRRDD
c4ba53c [Shivaram Venkataraman] Merge pull request #85 from edwardt/master
72a9d27 [root] reorder to keep relative ordering the same
f3fcb10 [root] fix up build.sbt also to match pom.xml
5ecbe3e [root] Make spark verison configurable in build script per ISSUE122
a44e63d [Shivaram Venkataraman] Merge pull request #84 from sun-rui/SPARKR-94
fbb5663 [Sun Rui] Add {} to one-line functions and add a test case for lookup where no match is found.
95beb4e [Shivaram Venkataraman] Merge pull request #82 from edwardt/master
36776c5 [edwardt] missed one 0.9.0 revert
b26deec [Sun Rui] [SPARKR-94] Add a  method to get an element of a pair RDD object by key.
1ba256e [edwardt] Keep 0.9.0 and says uses 1.1.0 by default
5380c43 [root] missed one version
21f74da [root] upgrade to spark version 1.1.0 to match lastest merge list
ddfcde9 [root] merge
67d067a [Shivaram Venkataraman] Merge pull request #81 from sun-rui/SparkR-117
993868f [Sun Rui] [SPARKR-117] Update Spark dependency to 1.1.0
d20661a [Zongheng Yang] Merge pull request #80 from sun-rui/master
0b2da9f [Sun Rui] Update Rd file and add a test case for mapPartitions.
5879648 [Sun Rui] Add mapPartitions() method to RDD for API consistency.
c033461 [Shivaram Venkataraman] Merge pull request #79 from sun-rui/fix-kmeans
f62b77e [Sun Rui] Adjust coding style.
b40911d [Sun Rui] Fix syntax error in examples/kmeans.R.
5304451 [Shivaram Venkataraman] Merge pull request #78 from sun-rui/master
70ffbfb [Sun Rui] Fix a bug that modifications to build.sbt won't trigger rebuilding.
a25696c [Shivaram Venkataraman] Merge pull request #76 from edwardt/addjira
b8bbd93 [edwardt] Update README.md
615d930 [edwardt] Update README.md
e522e69 [edwardt] Update README.md
03e6ced [edwardt] Update README.md
3007015 [root] don't check in gedit buffer file'
c35c9a6 [root] Add where to enter bugs ad feeback
469eae3 [edwardt] Update README.md
61b4a43 [edwardt] Update Makefile (style uniformity)
ce3337d [edwardt] Update README.md
7ff68fc [root] Merge branch 'master' of https://github.com/edwardt/SparkR-pkg
16353f5 [root] add links to devtools and install_github
513b9e5 [Shivaram Venkataraman] Merge pull request #72 from edwardt/master
31608a4 [edwardt] Update Makefile (style uniformity)
4ffe146 [root] Makefile: factor out SPARKR_VERSION to reduce potential copy&paste error; cp & rm called with -f in build/clean phase; .gitignore includes checkpoints and unit test log generated by run-tests.sh
715275f [Zongheng Yang] Merge pull request #68 from shivaram/master
90e2083 [Shivaram Venkataraman] Add return type to hasNext
8eb983d [Shivaram Venkataraman] Fix up comment
2206164 [Shivaram Venkataraman] Delete temporary files after they are read This change deletes temporary files used for communication between Rscript and the JVM once they have been completely read.
5881da7 [Zongheng Yang] Merge pull request #67 from shivaram/improve-shuffle
81251e2 [Shivaram Venkataraman] Address code review comments
a5f573f [Shivaram Venkataraman] Use a better list append in shuffles This is helpful in scenarios where we have a large number of values in a bucket
388e64d [Shivaram Venkataraman] Merge pull request #55 from RevolutionAnalytics/master
e1f95b6 [Zongheng Yang] Merge pull request #65 from concretevitamin/parallelize-fix
fc1a71a [Zongheng Yang] Fix that collect(parallelize(sc,1:72,15)) drops elements.
b8204c5 [Zongheng Yang] Minor: update a URL in README.
86f30c3 [Antonio Piccolboni] better fix for amplab-extras/SparkR-pkg#53
b3c318d [Antonio Piccolboni] delayed loading to have all namespaces available.
f323e97 [Antonio Piccolboni] tentative fix for amplab-extras/SparkR-pkg#53
6f82269 [Zongheng Yang] Merge pull request #48 from shivaram/master
8f433e5 [Shivaram Venkataraman] Move up Hadoop in pom.xml and add back protobufs As Hadoop 1.0.4 doesn't use protobufs, we can't exclude protobufs from Spark always. This change tries to order the dependencies so that the shader first picks up Hadoop's protobufs over Mesos.
bfe7e26 [Shivaram Venkataraman] Merge pull request #36 from RevolutionAnalytics/vectorize-examples
059ae41 [Antonio Piccolboni] and more formatting
9dbd531 [Antonio Piccolboni] more formatting per committer request
948738a [Antonio Piccolboni] converted tabs to spaces per project request
49f5f5a [Shivaram Venkataraman] Merge pull request #35 from shivaram/master
3eb5ad3 [Shivaram Venkataraman] on_failure -> after_failure in travis.yml
139bdee [Shivaram Venkataraman] Cache sbt, maven, ivy dependencies
4ebced2 [Shivaram Venkataraman] Merge pull request #34 from shivaram/master
8437061 [Shivaram Venkataraman] Exclude protobuf from Spark dependency in Maven This avoids pulling in multiple versions of protobuf from Mesos and Hadoop.
91aa527 [Antonio Piccolboni] vectorized version, 36s 10 slices 10^6 per slice. The older version takes 30 sec on 1/10th of data.
f137a57 [Antonio Piccolboni] for rstudio users
1f7ffb0 [Antonio Piccolboni] implemented using matrices and vectorized calls wherever possible
46b23df [Antonio Piccolboni] replace require with library
b15d7db [Antonio Piccolboni] faster parsing
8b7aeb3 [Antonio Piccolboni] 22x speed improvement, 3X mem impovement
c5bce07 [Zongheng Yang] Merge pull request #30 from shivaram/string-tests
21fa2d8 [Shivaram Venkataraman] Fix bug where serialized was not changed for RRRD Reason: When an RRDD is created in getJRDD we have converted any possibly unserialized RDD to a serialized RDD.
9d1ea20 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into string-tests
7b9348c [Shivaram Venkataraman] Add tests for partition with string keys Add two tests one with a string array and one from a textFile to test both codepaths
aacd726 [Shivaram Venkataraman] Update README with maven proxy instructions
803e62c [Shivaram Venkataraman] Merge pull request #28 from concretevitamin/master
7c093e6 [Zongheng Yang] Use inherits() to test an object's class.
061c591 [Shivaram Venkataraman] Merge pull request #26 from hafen/master
90f9fda [Ryan Hafen] Fix isRdd() to properly check for class
5b10cc7 [Zongheng Yang] Merge pull request #24 from shivaram/master
7014f83 [Shivaram Venkataraman] Remove unused transformers in maven's pom.xml
b00cea5 [Shivaram Venkataraman] Add support for a Maven build
11ec9b2 [Shivaram Venkataraman] Merge pull request #12 from concretevitamin/pipelined
6b18a90 [Zongheng Yang] Merge branch 'master' into pipelined
57127b8 [Zongheng Yang] Merge pull request #23 from shivaram/master
1ac3940 [Zongheng Yang] Review feedback.
a06fb34 [Zongheng Yang] Remove outdated comment.
0a1fc13 [Shivaram Venkataraman] Fixes for using SparkR with Hadoop2. 1. Exclude ASM, Netty from Hadoop similar to Spark. 2. Concat services files to ensure HDFS filesystems work. 3. Update README with an example
9a1db44 [Zongheng Yang] Merge pull request #22 from shivaram/master
e462448 [Shivaram Venkataraman] Use `$` for calling `put` instead of .jrcall
ed4559a [Shivaram Venkataraman] Add support for passing Spark environment vars This change creates a new `createSparkContext` method in RRDD as we can't pass Map<String, String> through rJava. Also use SPARK_MEM in local mode to increase heap size and update the README with some examples.
10228fb [Shivaram Venkataraman] Merge pull request #20 from concretevitamin/digit-ex
1398d9f [Zongheng Yang] Add linear_solver_mnist to examples/.
d484c2a [Zongheng Yang] Add tests for actions on PipelinedRDD.
d9cb95c [Zongheng Yang] Add setCheckpointDir() to context.R; comment fix.
f8bc8a9 [Zongheng Yang] Minor edits per Shivaram's comments.
8cd67f7 [Shivaram Venkataraman] Merge pull request #15 from shivaram/master
d4468a9 [Shivaram Venkataraman] Remove trailing comma
e2714b8 [Shivaram Venkataraman] Remove Apache Staging repo and update README
334eace [Zongheng Yang] Add a multi-transformation test to benchmark on pipelining.
5650ad7 [Zongheng Yang] Put serialized field inside env for both RDD and PipelinedRDD.
0b9e8bb [Zongheng Yang] First cut at PipelinedRDD.
a4c431e [Zongheng Yang] Add `isCheckpointed` field and checkpoint().
dac0795 [Zongheng Yang] Minor inline comment style fix.
bfb8e26 [Zongheng Yang] Add isCached field (inside an env) and unpersist().
295bff6 [Zongheng Yang] Merge pull request #11 from shivaram/master
4cb209c [Shivaram Venkataraman] Search rLibDir in worker before libPaths This ensures we pick up the SparkR intended and not an older version installed on the same machine
ef198ff [Zongheng Yang] Merge pull request #10 from shivaram/unit-tests
e0557a8 [Shivaram Venkataraman] Update travis to install plyr
8b18bc1 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into unit-tests
4a9ca31 [Shivaram Venkataraman] Use smaller broadcast and plyr instead of Matrix Matrix package takes around 2s to load and slows down unit tests.
21c6a61 [Zongheng Yang] Merge pull request #8 from shivaram/master
08c2947 [Shivaram Venkataraman] Move dev install directory to front of libPaths
bda42ee [Shivaram Venkataraman] Merge pull request #7 from JoshRosen/travis
cc5f5c0 [Josh Rosen] Add Travis CI integration (using craigcitro/r-travis)
b6c864b [Shivaram Venkataraman] Merge pull request #6 from concretevitamin/env-style-fix
4fcef22 [Zongheng Yang] Use one style ($) for accessing names in environments.
8a948c6 [Shivaram Venkataraman] Merge pull request #4 from shivaram/master
24978eb [Shivaram Venkataraman] Update README to use install_github
8899db4 [Shivaram Venkataraman] Update TODO.md
91792de [Shivaram Venkataraman] Update Spark requirements
f34f4bf [Shivaram Venkataraman] Check tests for failures and output error msg
cd750d3 [Shivaram Venkataraman] Update run-tests to use new path
1877b7c [Shivaram Venkataraman] Unset R_TESTS to make tests work with R CMD check Also silence Akka remoting logs and update Makefile to build on log4j changes
e60e18a [Shivaram Venkataraman] Update README to remove Spark installation notes
4450189 [Shivaram Venkataraman] Add Spark 0.9 dependency from Apache Staging Also clean up assembly jar from inst on make clean
5eb2131 [Shivaram Venkataraman] Update repo path in README
ec8210e [Shivaram Venkataraman] Remove broadcastId hack as it is public in Spark
9f0e080 [Shivaram Venkataraman] Merge branch 'install-github'
5c88fbd [Shivaram Venkataraman] Add helper script to run tests
77450a1 [Shivaram Venkataraman] Remove dependency on Spark Logging
6cb00d1 [Shivaram Venkataraman] Update README and add helper script install-dev.sh
28346ca [Shivaram Venkataraman] Only normalize if SPARK_HOME is not empty
0fd6571 [Shivaram Venkataraman] Normalize SPARK_HOME before passing it
ff96d5c [Shivaram Venkataraman] Pass in SPARK_HOME and jar file path
34c4dce [Shivaram Venkataraman] Move src into pkg and update Makefile This enables the package to be installed using install_github using devtools and automates the build procedure.
b25afed [Shivaram Venkataraman] Change package name to edu.berkeley.cs.amplab
c691464 [Shivaram Venkataraman] Add Apache 2.0 License file
27a4a4b [Shivaram Venkataraman] Add notes on how to compile roxygen2 docs
ca63844 [Shivaram Venkataraman] Add broadcast documentation Also generate documentation for sample, takeSample etc.
e4dd976 [Shivaram Venkataraman] Update TODO.md
e42d435 [Shivaram Venkataraman] Add support for broadcast variables
6b638e7 [Shivaram Venkataraman] Add the assembly jar to SparkContext
bf24e32 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg
43c05ce [Zongheng Yang] Fix a flaky/incorrect test for sampleRDD().
c6a9dfc [Zongheng Yang] Initial port of the kmeans example.
6885581 [Zongheng Yang] Implement element-level sampleRDD() and takeSample() with tests.
d3a4987 [Zongheng Yang] Add a test for lapplyPartitionsWithIndex on pairwise RDD.
c7899c1 [Zongheng Yang] Add lapplyPartitionsWithIndex, with a test and an alias function.
a9a7436 [Shivaram Venkataraman] Add DFC example from Tselil, Benjamin and Jonah
fbc5a95 [Zongheng Yang] Implement take() and takeSample().
c4a3409 [Shivaram Venkataraman] Use RDD instead of RRDD
dfad3f5 [Zongheng Yang] Add test_utils.R: a unit test for convertJListToRList().
a45227d [Zongheng Yang] Update .gitignore.
238fe6e [Zongheng Yang] Add a unit test for textFile().
a88898b [Zongheng Yang] Rename test_rrd to test_rrdd
10c8baa [Shivaram Venkataraman] Make SparkR work as a standalone package. Changes include: 1. Adding a new `sbt` project that builds RRDD.scala 2. Change the onLoad functions to load the assembly jar for SparkR 3. Set rLibDir in RRDD.scala and worker.R to load things correctly
78adcd8 [Shivaram Venkataraman] Add a gitignore
ca6108f [Shivaram Venkataraman] Merge branch 'SparkR-scalacode' of ../SparkR
999bd61 [Shivaram Venkataraman] Update collectPartition in R and use ClassTag
c58f63e [Shivaram Venkataraman] Update collectPartition in R and use ClassTag
48265fd [Shivaram Venkataraman] Use new version of collectPartitions in take
d4fe086 [Shivaram Venkataraman] Move collectPartitions to JavaRDDLike Also remove numPartitions in JavaRDD and update R code
bfecd7b [Shivaram Venkataraman] Scala 2.10 changes 1. Update sparkR script 2. Use classTag instead of classManifest
092a4b3 [Shivaram Venkataraman] Add combineByKey, update TODO
ac0d81d [Shivaram Venkataraman] Add more documentation
d1dc3fa [Shivaram Venkataraman] Add more documentation
c515e3a [Shivaram Venkataraman] Update TODO
db56a34 [Shivaram Venkataraman] Add a test case for include package
41cea51 [Shivaram Venkataraman] Ensure all parent environments are serialized. Also add a test case with an inline function
a978e84 [Shivaram Venkataraman] Add support to include packages in the worker
12bf8ce [Shivaram Venkataraman] Add support to include packages in the worker
fb7e72c [Shivaram Venkataraman] Cleanup TODO
16ac314 [Shivaram Venkataraman] Add documentation for functions in context, sparkR
85b1d25 [Shivaram Venkataraman] Set license to Apache
88f1101 [Shivaram Venkataraman] Add unit test running instructions
c40768e [Shivaram Venkataraman] Update TODO
0c7efbf [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions
5880d42 [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions
2dee36c [Shivaram Venkataraman] Remove empty test file
a82219b [Shivaram Venkataraman] Update TODOs
5db00dc [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR    package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD.
f196479 [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR    package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD.
987e36f [Shivaram Venkataraman] Add perf todo
0b03265 [Shivaram Venkataraman] Update TODO with testing, docs todo
685aaad [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R.
95b9ddc [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R.
4f00895 [Zongheng Yang] Remove the unnecessary `pairwise' flag in RRDD class. Reasons:
75d36d9 [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD.
e3fbd9d [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD.
67a4335 [Zongheng Yang] Add unit test for parallelize() and collect() pairwise data.
100ae65 [Zongheng Yang] Properly parallelize() and collect() pairwise data.
cd0a5e2 [Zongheng Yang] Properly parallelize() and collect() pairwise data.
aea16c3 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues.
45eb943 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues.
11c893b [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func
82c201a [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func
b3bfad2 [Zongheng Yang] Update TODO: take() done.
0e45293 [Zongheng Yang] Add ability to parallelize key-val collections in R.
f60406a [Zongheng Yang] Add ability to parallelize key-val collections in R.
7d7fe3b [Zongheng Yang] Re-implement take(): take a partition at a time and append.
a054e55 [Zongheng Yang] Fix take() tests(): mode difference.
9de0935 [Zongheng Yang] Implement take() for RRDD.
1e4427e [Zongheng Yang] Implement take() for RRDD.
ec3cd67 [Shivaram Venkataraman] Use temp file in Spark to pipe output
417aaed [Shivaram Venkataraman] Use temp file in Spark to pipe output
bb0a3c3 [Shivaram Venkataraman] Add conf directory to classpath
9594d8a [Shivaram Venkataraman] Clean up LR example
3b26b58 [Shivaram Venkataraman] Add a list of things to do.
cabce68 [Shivaram Venkataraman] Fix warnings from package check
fde3f9c [Shivaram Venkataraman] Flatten by default and disable recursive unlist
ab2e061 [Shivaram Venkataraman] Create LIB_DIR before installing SparkR package
555220a [Shivaram Venkataraman] Add readme and update Makefile
1319cda [Shivaram Venkataraman] Make standalone programs run with sparkR
ae19fa8 [Shivaram Venkataraman] Add support for cache and use `tempfile`
4e89ca4 [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples
25a0bea [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples
f50223f [Zongheng Yang] Make parallelize() and collect() use lists. Add a few more tests for them.
fc7693f [Zongheng Yang] Refactor and enhance the previously added unit test a little bit.
6de9b81 [Zongheng Yang] Add a simple unit test for parallelize().
8b95155 [Zongheng Yang] Add testthat skeleton infrastructure
ef305bf [Zongheng Yang] parallelize() followed by collect() now work for vectors/lists of strings and numerics (should work for other primitives as well).
dc16af4 [Zongheng Yang] Comment: toArray() allocates memory for a copy
f50121e [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object.
46eb063 [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object.
6b4938a [Zongheng Yang] parallelize(): a raw can be parallelized by JavaSparkContext and get back JavaRDD
978aa0f [Zongheng Yang] Add parallelize() skeleton: only return serialized slices now
84c1fd2 [Zongheng Yang] Use .jsimplify() to get around generic List's get() type erasure problem
f16b891 [Zongheng Yang] Convert a few reflectionc alls to .jcall
1284c13 [Zongheng Yang] WIP on collect(): JavaListToRList() failed with errors.
4c2e516 [Zongheng Yang] Add simple prototype of S4 class RRDD. Make TextFile() returns an RRDD.
82aa17a [Zongheng Yang] Add textFile()
83ce63f [Zongheng Yang] Create a JavaSparkContext and save it in .sparkEnv using sparkR.init()
01cdf0e [Zongheng Yang] Add Makefile for SparkR
fc9cae2 [Shivaram Venkataraman] Add skeleton R package
2015-04-08 22:45:40 -07:00
Venkata Ramana Gollamudi 7d7384c781 [SPARK-6451][SQL] supported code generation for CombineSum
Author: Venkata Ramana Gollamudi <ramana.gollamudi@huawei.com>

Closes #5138 from gvramana/sum_fix_codegen and squashes the following commits:

95f5fe4 [Venkata Ramana Gollamudi] rebase merge changes
12f45a5 [Venkata Ramana Gollamudi] Combined and added code generations tests as per comment
d6a76ac [Venkata Ramana Gollamudi] added support for codegeneration for CombineSum and tests
2015-04-08 18:42:34 -07:00
Volodymyr Lyubinets e40ea8742a [Minor] [SQL] [SPARK-6729] Minor fix for DriverQuirks get
The function uses .substring(0, X), which will trigger OutOfBoundsException if string length is less than X. A better way to do this is to use startsWith, which won't error out in this case.

Author: Volodymyr Lyubinets <vlyubin@gmail.com>

Closes #5378 from vlyubin/quirks and squashes the following commits:

504e8e0 [Volodymyr Lyubinets] Minor fix for DriverQuirks get
2015-04-06 18:00:51 -07:00
Liang-Chi Hsieh 7bca62f790 [SPARK-6607][SQL] Check invalid characters for Parquet schema and show error messages
'(' and ')' are special characters used in Parquet schema for type annotation. When we run an aggregation query, we will obtain attribute name such as "MAX(a)".

If we directly store the generated DataFrame as Parquet file, it causes failure when reading and parsing the stored schema string.

Several methods can be adopted to solve this. This pr uses a simplest one to just replace attribute names before generating Parquet schema based on these attributes.

Another possible method might be modifying all aggregation expression names from "func(column)" to "func[column]".

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

Closes #5263 from viirya/parquet_aggregation_name and squashes the following commits:

2d70542 [Liang-Chi Hsieh] Address comment.
463dff4 [Liang-Chi Hsieh] Instead of replacing special chars, showing error message to user to suggest using Alias.
1de001d [Liang-Chi Hsieh] Replace special characters '(' and ')' of Parquet schema.
2015-04-05 00:20:43 +08:00
Yin Huai da25c86d64 [SQL] Use path.makeQualified in newParquet.
Author: Yin Huai <yhuai@databricks.com>

Closes #5353 from yhuai/wrongFS and squashes the following commits:

849603b [Yin Huai] Not use deprecated method.
6d6ae34 [Yin Huai] Use path.makeQualified.
2015-04-04 23:26:10 +08:00
Reynold Xin 82701ee25f [SPARK-6428] Turn on explicit type checking for public methods.
This builds on my earlier pull requests and turns on the explicit type checking in scalastyle.

Author: Reynold Xin <rxin@databricks.com>

Closes #5342 from rxin/SPARK-6428 and squashes the following commits:

7b531ab [Reynold Xin] import ordering
2d9a8a5 [Reynold Xin] jl
e668b1c [Reynold Xin] override
9b9e119 [Reynold Xin] Parenthesis.
82e0cf5 [Reynold Xin] [SPARK-6428] Turn on explicit type checking for public methods.
2015-04-03 01:25:02 -07:00
Michael Armbrust 052dee0707 [SPARK-6686][SQL] Use resolved output instead of names for toDF rename
This is a workaround for a problem reported on the user list.  This doesn't fix the core problem, but in general is a more robust way to do renames.

Author: Michael Armbrust <michael@databricks.com>

Closes #5337 from marmbrus/toDFrename and squashes the following commits:

6a3159d [Michael Armbrust] [SPARK-6686][SQL] Use resolved output instead of names for toDF rename
2015-04-02 18:30:55 -07:00
Cheng Lian d3944b6f2a [Minor] [SQL] Follow-up of PR #5210
This PR addresses rxin's comments in PR #5210.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5219 from liancheng/spark-6554-followup and squashes the following commits:

41f3a09 [Cheng Lian] Addresses comments in #5210
2015-04-02 16:15:34 -07:00
Xiangrui Meng 424e987dfe [SPARK-6672][SQL] convert row to catalyst in createDataFrame(RDD[Row], ...)
We assume that `RDD[Row]` contains Scala types. So we need to convert them into catalyst types in createDataFrame. liancheng

Author: Xiangrui Meng <meng@databricks.com>

Closes #5329 from mengxr/SPARK-6672 and squashes the following commits:

2d52644 [Xiangrui Meng] set needsConversion = false in jsonRDD
06896e4 [Xiangrui Meng] add createDataFrame without conversion
4a3767b [Xiangrui Meng] convert Row to catalyst
2015-04-02 17:57:01 +08:00
Davies Liu 40df5d49bb [SPARK-6663] [SQL] use Literal.create instread of constructor
In order to do inbound checking and type conversion, we should use Literal.create() instead of  constructor.

Author: Davies Liu <davies@databricks.com>

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

1667604 [Davies Liu] fix style and add comment
5f8c0fd [Davies Liu] use Literal.create instread of constructor
2015-04-01 23:11:38 -07:00
Chet Mancini 191524e740 [SPARK-6658][SQL] Update DataFrame documentation to fix type references.
First contribution here; would love to be getting some code contributions in soon. Let me know if there's anything about contribution process I should improve.

Author: Chet Mancini <chetmancini@gmail.com>

Closes #5316 from chetmancini/SPARK_6658_dataframe_doc and squashes the following commits:

53b627a [Chet Mancini] [SQL] SPARK-6658: Update DataFrame documentation to refer to correct types
2015-04-01 21:39:46 -07:00
Cheng Lian d36c5fca7b [SPARK-6608] [SQL] Makes DataFrame.rdd a lazy val
Before 1.3.0, `SchemaRDD.id` works as a unique identifier of each `SchemaRDD`. In 1.3.0, unlike `SchemaRDD`, `DataFrame` is no longer an RDD, and `DataFrame.rdd` is actually a function which always returns a new RDD instance. Making `DataFrame.rdd` a lazy val should bring the unique identifier back.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5265 from liancheng/spark-6608 and squashes the following commits:

7500968 [Cheng Lian] Updates javadoc
7f37d21 [Cheng Lian] Makes DataFrame.rdd a lazy val
2015-04-01 21:34:45 +08:00
Reynold Xin 305abe1e57 [Doc] Improve Python DataFrame documentation
Author: Reynold Xin <rxin@databricks.com>

Closes #5287 from rxin/pyspark-df-doc-cleanup-context and squashes the following commits:

1841b60 [Reynold Xin] Lint.
f2007f1 [Reynold Xin] functions and types.
bc3b72b [Reynold Xin] More improvements to DataFrame Python doc.
ac1d4c0 [Reynold Xin] Bug fix.
b163365 [Reynold Xin] Python fix. Added Experimental flag to DataFrameNaFunctions.
608422d [Reynold Xin] [Doc] Cleanup context.py Python docs.
2015-03-31 18:31:36 -07:00
Liang-Chi Hsieh 2036bc5993 [SPARK-6633][SQL] Should be "Contains" instead of "EndsWith" when constructing sources.StringContains
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #5299 from viirya/stringcontains and squashes the following commits:

c1ece4c [Liang-Chi Hsieh] Should be Contains instead of EndsWith.
2015-03-31 13:18:07 -07:00
Michael Armbrust cd48ca5012 [SPARK-6145][SQL] fix ORDER BY on nested fields
This PR is based on work by cloud-fan in #4904, but with two differences:
 - We isolate the logic for Sort's special handling into `ResolveSortReferences`
 - We avoid creating UnresolvedGetField expressions during resolution.  Instead we either resolve GetField or we return None.  This avoids us going down the wrong path early on.

Author: Michael Armbrust <michael@databricks.com>

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

b8cae45 [Michael Armbrust] fix another test
0f36a11 [Michael Armbrust] WIP
91820cd [Michael Armbrust] Fix bug.
2015-03-31 11:23:18 -07:00
Reynold Xin f07e714062 [SPARK-6625][SQL] Add common string filters to data sources.
Filters such as startsWith, endsWith, contains will be very useful for data sources that provide search functionality, e.g. Succinct, Elastic Search, Solr.

I also took this chance to improve documentation for the data source filters.

Author: Reynold Xin <rxin@databricks.com>

Closes #5285 from rxin/ds-string-filters and squashes the following commits:

f021727 [Reynold Xin] Fixed grammar.
7695a52 [Reynold Xin] [SPARK-6625][SQL] Add common string filters to data sources.
2015-03-31 00:19:51 -07:00
Reynold Xin b8ff2bc61c [SPARK-6119][SQL] DataFrame support for missing data handling
This pull request adds variants of DataFrame.na.drop and DataFrame.na.fill to the Scala/Java API, and DataFrame.fillna and DataFrame.dropna to the Python API.

Author: Reynold Xin <rxin@databricks.com>

Closes #5274 from rxin/df-missing-value and squashes the following commits:

4ee1b98 [Reynold Xin] Improve error reporting in Python.
33a330c [Reynold Xin] Remove replace for now.
bc4fdbb [Reynold Xin] Added documentation for replace.
d56f5a5 [Reynold Xin] Added replace for Scala/Java.
2385d00 [Reynold Xin] Feedback from Xiangrui on "how".
914a374 [Reynold Xin] fill with map.
185c67e [Reynold Xin] Allow specifying column subsets in fill.
749eb47 [Reynold Xin] fillna
249b94e [Reynold Xin] Removing undefined functions.
6a73c68 [Reynold Xin] Missing file.
67d7003 [Reynold Xin] [SPARK-6119][SQL] DataFrame.na.drop (Scala/Java) and DataFrame.dropna (Python)
2015-03-30 20:47:10 -07:00
Cheng Lian fde6945417 [SPARK-6369] [SQL] Uses commit coordinator to help committing Hive and Parquet tables
This PR leverages the output commit coordinator introduced in #4066 to help committing Hive and Parquet tables.

This PR extracts output commit code in `SparkHadoopWriter.commit` to `SparkHadoopMapRedUtil.commitTask`, and reuses it for committing Parquet and Hive tables on executor side.

TODO

- [ ] Add tests

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

Author: Cheng Lian <lian@databricks.com>

Closes #5139 from liancheng/spark-6369 and squashes the following commits:

72eb628 [Cheng Lian] Fixes typo in javadoc
9a4b82b [Cheng Lian] Adds javadoc and addresses @aarondav's comments
dfdf3ef [Cheng Lian] Uses commit coordinator to help committing Hive and Parquet tables
2015-03-31 07:48:37 +08:00
Adam Budde 5909f0973d [SPARK-6538][SQL] Add missing nullable Metastore fields when merging a Parquet schema
Opening to replace #5188.

When Spark SQL infers a schema for a DataFrame, it will take the union of all field types present in the structured source data (e.g. an RDD of JSON data). When the source data for a row doesn't define a particular field on the DataFrame's schema, a null value will simply be assumed for this field. This workflow makes it very easy to construct tables and query over a set of structured data with a nonuniform schema. However, this behavior is not consistent in some cases when dealing with Parquet files and an external table managed by an external Hive metastore.

In our particular usecase, we use Spark Streaming to parse and transform our input data and then apply a window function to save an arbitrary-sized batch of data as a Parquet file, which itself will be added as a partition to an external Hive table via an *"ALTER TABLE... ADD PARTITION..."* statement. Since our input data is nonuniform, it is expected that not every partition batch will contain every field present in the table's schema obtained from the Hive metastore. As such, we expect that the schema of some of our Parquet files may not contain the same set fields present in the full metastore schema.

In such cases, it seems natural that Spark SQL would simply assume null values for any missing fields in the partition's Parquet file, assuming these fields are specified as nullable by the metastore schema. This is not the case in the current implementation of ParquetRelation2. The **mergeMetastoreParquetSchema()** method used to reconcile differences between a Parquet file's schema and a schema retrieved from the Hive metastore will raise an exception if the Parquet file doesn't match the same set of fields specified by the metastore.

This pull requests alters the behavior of **mergeMetastoreParquetSchema()** by having it first add any nullable fields from the metastore schema to the Parquet file schema if they aren't already present there.

Author: Adam Budde <budde@amazon.com>

Closes #5214 from budde/nullable-fields and squashes the following commits:

a52d378 [Adam Budde] Refactor ParquetSchemaSuite.scala for cases now permitted by SPARK-6471 and SPARK-6538
9041bfa [Adam Budde] Add missing nullable Metastore fields when merging a Parquet schema
2015-03-28 09:14:09 +08:00
Reynold Xin 3af7334304 [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 row, not 1 row
Author: Reynold Xin <rxin@databricks.com>

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

1306d88 [Reynold Xin] Proper fix.
e135bb9 [Reynold Xin] [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 rows, not 1 row.
2015-03-27 14:56:57 -07:00
Michael Armbrust 5d9c37c23d [SPARK-6550][SQL] Use analyzed plan in DataFrame
This is based on bug and test case proposed by viirya.  See #5203 for a excellent description of the problem.

TLDR; The problem occurs because the function `groupBy(String)` calls `resolve`, which returns an `AttributeReference`.  However, this `AttributeReference` is based on an analyzed plan which is thrown away.  At execution time, we once again analyze the plan.  However, in the case of self-joins, each call to analyze will produce a new tree for the left side of the join, rendering the previously returned `AttributeReference` invalid.

As a fix, I propose we keep the analyzed plan instead of the unresolved plan inside of a `DataFrame`.

Author: Michael Armbrust <michael@databricks.com>

Closes #5217 from marmbrus/preanalyzer and squashes the following commits:

1f98e2d [Michael Armbrust] revert change
dd4dec1 [Michael Armbrust] Use the analyzed plan in DataFrame
089c52e [Michael Armbrust] WIP
2015-03-27 11:40:00 -07:00
Cheng Lian 71a0d40ebd [SPARK-6554] [SQL] Don't push down predicates which reference partition column(s)
There are two cases for the new Parquet data source:

1. Partition columns exist in the Parquet data files

   We don't need to push-down these predicates since partition pruning already handles them.

1. Partition columns don't exist in the Parquet data files

   We can't push-down these predicates since they are considered as invalid columns by Parquet.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5210 from liancheng/spark-6554 and squashes the following commits:

4f7ec03 [Cheng Lian] Adds comments
e134ced [Cheng Lian] Don't push down predicates which reference partition column(s)
2015-03-26 13:11:37 -07:00
Reynold Xin 784fcd5327 [SPARK-6117] [SQL] Improvements to DataFrame.describe()
1. Slightly modifications to the code to make it more readable.
2. Added Python implementation.
3. Updated the documentation to state that we don't guarantee the output schema for this function and it should only be used for exploratory data analysis.

Author: Reynold Xin <rxin@databricks.com>

Closes #5201 from rxin/df-describe and squashes the following commits:

25a7834 [Reynold Xin] Reset run-tests.
6abdfee [Reynold Xin] [SPARK-6117] [SQL] Improvements to DataFrame.describe()
2015-03-26 12:26:13 -07:00
Yash Datta 1c05027a14 [SQL][SPARK-6471]: Metastore schema should only be a subset of parquet schema to support dropping of columns using replace columns
Currently in the parquet relation 2 implementation, error is thrown in case merged schema is not exactly the same as metastore schema.
But to support cases like deletion of column using replace column command, we can relax the restriction so that even if metastore schema is a subset of merged parquet schema, the query will work.

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

Closes #5141 from saucam/replace_col and squashes the following commits:

e858d5b [Yash Datta] SPARK-6471: Fix test cases, add a new test case for metastore schema to be subset of parquet schema
5f2f467 [Yash Datta] SPARK-6471: Metastore schema should only be a subset of parquet schema to support dropping of columns using replace columns
2015-03-26 21:13:38 +08:00
Michael Armbrust f88f51bbd4 [SPARK-6465][SQL] Fix serialization of GenericRowWithSchema using kryo
Author: Michael Armbrust <michael@databricks.com>

Closes #5191 from marmbrus/kryoRowsWithSchema and squashes the following commits:

bb83522 [Michael Armbrust] Fix serialization of GenericRowWithSchema using kryo
f914f16 [Michael Armbrust] Add no arg constructor to GenericRowWithSchema
2015-03-26 18:46:57 +08:00
azagrebin 5bbcd1304c [SPARK-6117] [SQL] add describe function to DataFrame for summary statis...
Please review my solution for SPARK-6117

Author: azagrebin <azagrebin@gmail.com>

Closes #5073 from azagrebin/SPARK-6117 and squashes the following commits:

f9056ac [azagrebin] [SPARK-6117] [SQL] create one aggregation and split it locally into resulting DF, colocate test data with test case
ddb3950 [azagrebin] [SPARK-6117] [SQL] simplify implementation, add test for DF without numeric columns
9daf31e [azagrebin] [SPARK-6117] [SQL] add describe function to DataFrame for summary statistics
2015-03-26 00:25:04 -07:00
Michael Armbrust a8f51b8296 [SPARK-6458][SQL] Better error messages for invalid data sources
Avoid unclear match errors and use `AnalysisException`.

Author: Michael Armbrust <michael@databricks.com>

Closes #5158 from marmbrus/dataSourceError and squashes the following commits:

af9f82a [Michael Armbrust] Yins comment
90c6ba4 [Michael Armbrust] Better error messages for invalid data sources
2015-03-24 14:10:56 -07:00
Michael Armbrust 26c6ce3d29 [SPARK-6437][SQL] Use completion iterator to close external sorter
Otherwise we will leak files when spilling occurs.

Author: Michael Armbrust <michael@databricks.com>

Closes #5161 from marmbrus/cleanupAfterSort and squashes the following commits:

cb13d3c [Michael Armbrust] hint to inferencer
cdebdf5 [Michael Armbrust] Use completion iterator to close external sorter
2015-03-24 12:10:30 -07:00
Michael Armbrust 32efadd050 [SPARK-6459][SQL] Warn when constructing trivially true equals predicate
For example, one might expect the following code to work, but it does not.  Now you will at least get a warning with a suggestion to use aliases.

```scala
val df = sqlContext.load(path, "parquet")
val txns = df.groupBy("cust_id").agg($"cust_id", countDistinct($"day_num").as("txns"))
val spend = df.groupBy("cust_id").agg($"cust_id", sum($"extended_price").as("spend"))
val rmJoin = txns.join(spend, txns("cust_id") === spend("cust_id"), "inner")
```

Author: Michael Armbrust <michael@databricks.com>

Closes #5163 from marmbrus/selfJoinError and squashes the following commits:

16c1f0b [Michael Armbrust] fix visibility
1b57e8d [Michael Armbrust] Warn when constructing trivially true equals predicate
2015-03-24 12:09:02 -07:00
Xiangrui Meng 6bdddb6f6f [SPARK-6361][SQL] support adding a column with metadata in DF
This is used by ML pipelines to embed ML attributes in columns created by ML transformers/estimators. marmbrus

Author: Xiangrui Meng <meng@databricks.com>

Closes #5151 from mengxr/SPARK-6361 and squashes the following commits:

bb30de3 [Xiangrui Meng] support adding a column with metadata in DF
2015-03-24 12:08:19 -07:00
Xiangrui Meng a1d1529dae [SPARK-6475][SQL] recognize array types when infer data types from JavaBeans
Right now if there is a array field in a JavaBean, the user wold see an exception in `createDataFrame`. liancheng

Author: Xiangrui Meng <meng@databricks.com>

Closes #5146 from mengxr/SPARK-6475 and squashes the following commits:

51e87e5 [Xiangrui Meng] validate schemas
4f2df5e [Xiangrui Meng] recognize array types when infer data types from JavaBeans
2015-03-24 10:11:27 -07:00
Volodymyr Lyubinets bfd3ee9f76 [SPARK-6124] Support jdbc connection properties in OPTIONS part of the query
One more thing if this PR is considered to be OK - it might make sense to add extra .jdbc() API's that take Properties to SQLContext.

Author: Volodymyr Lyubinets <vlyubin@gmail.com>

Closes #4859 from vlyubin/jdbcProperties and squashes the following commits:

7a8cfda [Volodymyr Lyubinets] Support jdbc connection properties in OPTIONS part of the query
2015-03-23 17:00:27 -07:00
Daoyuan Wang 4659468f36 [SPARK-4985] [SQL] parquet support for date type
This PR might have some issues with #3732 ,
and this would have merge conflicts with #3820 so the review can be delayed till that 2 were merged.

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

Closes #3822 from adrian-wang/parquetdate and squashes the following commits:

2c5d54d [Daoyuan Wang] add a test case
faef887 [Daoyuan Wang] parquet support for primitive date
97e9080 [Daoyuan Wang] parquet support for date type
2015-03-23 11:46:16 +08:00
vinodkc 2bf40c58e6 [SPARK-6337][Documentation, SQL]Spark 1.3 doc fixes
Author: vinodkc <vinod.kc.in@gmail.com>

Closes #5112 from vinodkc/spark_1.3_doc_fixes and squashes the following commits:

2c6aee6 [vinodkc] Spark 1.3 doc fixes
2015-03-22 20:00:08 +00:00
ypcat 9b1e1f20d4 [SPARK-6408] [SQL] Fix JDBCRDD filtering string literals
Author: ypcat <ypcat6@gmail.com>
Author: Pei-Lun Lee <pllee@appier.com>

Closes #5087 from ypcat/spark-6408 and squashes the following commits:

1becc16 [ypcat] [SPARK-6408] [SQL] styling
1bc4455 [ypcat] [SPARK-6408] [SQL] move nested function outside
e57fa4a [ypcat] [SPARK-6408] [SQL] fix test case
245ab6f [ypcat] [SPARK-6408] [SQL] add test cases for filtering quoted strings
8962534 [Pei-Lun Lee] [SPARK-6408] [SQL] Fix filtering string literals
2015-03-22 15:49:13 +08:00
Yin Huai 94a102acb8 [SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser.
This PR creates a trait `DataTypeParser` used to parse data types. This trait aims to be single place to provide the functionality of parsing data types' string representation. It is currently mixed in with `DDLParser` and `SqlParser`. It is also used to parse the data type for `DataFrame.cast` and to convert Hive metastore's data type string back to a `DataType`.

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

Author: Yin Huai <yhuai@databricks.com>

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

0e66097 [Yin Huai] Special handle struct<>.
fea6012 [Yin Huai] Style.
c9733fb [Yin Huai] Create a trait to parse data types.
2015-03-21 13:27:53 -07:00
Yanbo Liang e5d2c37c68 [SPARK-5821] [SQL] JSON CTAS command should throw error message when delete path failure
When using "CREATE TEMPORARY TABLE AS SELECT" to create JSON table, we first delete the path file or directory and then generate a new directory with the same name. But if only read permission was granted, the delete failed.
Here we just throwing an error message to let users know what happened.
ParquetRelation2 may also hit this problem. I think to restrict JSONRelation and ParquetRelation2 must base on directory is more reasonable for access control. Maybe I can do it in follow up works.

Author: Yanbo Liang <ybliang8@gmail.com>
Author: Yanbo Liang <yanbohappy@gmail.com>

Closes #4610 from yanboliang/jsonInsertImprovements and squashes the following commits:

c387fce [Yanbo Liang] fix typos
42d7fb6 [Yanbo Liang] add unittest & fix output format
46f0d9d [Yanbo Liang] Update JSONRelation.scala
e2df8d5 [Yanbo Liang] check path exisit when write
79f7040 [Yanbo Liang] Update JSONRelation.scala
e4bc229 [Yanbo Liang] Update JSONRelation.scala
5a42d83 [Yanbo Liang] JSONRelation CTAS should check if delete is successful
2015-03-21 11:23:28 +08:00
Cheng Lian 937c1e5503 [SPARK-6315] [SQL] Also tries the case class string parser while reading Parquet schema
When writing Parquet files, Spark 1.1.x persists the schema string into Parquet metadata with the result of `StructType.toString`, which was then deprecated in Spark 1.2 by a schema string in JSON format. But we still need to take the old schema format into account while reading Parquet files.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5034 from liancheng/spark-6315 and squashes the following commits:

a182f58 [Cheng Lian] Adds a regression test
b9c6dbe [Cheng Lian] Also tries the case class string parser while reading Parquet schema
2015-03-21 11:18:45 +08:00
Yanbo Liang bc37c9743e [SPARK-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful
Do the same check as #4610 for ParquetRelation2.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #5107 from yanboliang/spark-5821-parquet and squashes the following commits:

7092c8d [Yanbo Liang] ParquetRelation2 CTAS should check if delete is successful
2015-03-21 10:53:04 +08:00
Reynold Xin a95043b178 [SPARK-6428][SQL] Added explicit type for all public methods in sql/core
Also implemented equals/hashCode when they are missing.

This is done in order to enable automatic public method type checking.

Author: Reynold Xin <rxin@databricks.com>

Closes #5104 from rxin/sql-hashcode-explicittype and squashes the following commits:

ffce6f3 [Reynold Xin] Code review feedback.
8b36733 [Reynold Xin] [SPARK-6428][SQL] Added explicit type for all public methods.
2015-03-20 15:47:07 -07:00
Sean Owen 6f80c3e888 SPARK-6338 [CORE] Use standard temp dir mechanisms in tests to avoid orphaned temp files
Use `Utils.createTempDir()` to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify

Author: Sean Owen <sowen@cloudera.com>

Closes #5029 from srowen/SPARK-6338 and squashes the following commits:

27b740a [Sean Owen] Fix hive-thriftserver tests that don't expect an existing dir
4a212fa [Sean Owen] Standardize a bit more temp dir management
9004081 [Sean Owen] Revert some added recursive-delete calls
57609e4 [Sean Owen] Use Utils.createTempDir() to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify
2015-03-20 14:16:21 +00:00
Lomig Mégard 68707225f1 [SQL][docs][minor] Fixed sample code in SQLContext scaladoc
Error in the code sample of the `implicits` object in `SQLContext`.

Author: Lomig Mégard <lomig.megard@gmail.com>

Closes #5051 from tarfaa/simple and squashes the following commits:

5a88acc [Lomig Mégard] [docs][minor] Fixed sample code in SQLContext scaladoc
2015-03-16 23:52:42 -07:00
Volodymyr Lyubinets d19efeddc0 [SPARK-6330] Fix filesystem bug in newParquet relation
If I'm running this locally and my path points to S3, this would currently error out because of incorrect FS.
I tested this in a scenario that previously didn't work, this change seemed to fix the issue.

Author: Volodymyr Lyubinets <vlyubin@gmail.com>

Closes #5020 from vlyubin/parquertbug and squashes the following commits:

a645ad5 [Volodymyr Lyubinets] Fix filesystem bug in newParquet relation
2015-03-16 12:13:18 -07:00
Cheng Hao 12a345adcb [SPARK-2087] [SQL] Multiple thriftserver sessions with single HiveContext instance
Still, we keep only a single HiveContext within ThriftServer, and we also create a object called `SQLSession` for isolating the different user states.

Developers can obtain/release a new user session via `openSession` and `closeSession`, and `SQLContext` and `HiveContext` will also provide a default session if no `openSession` called, for backward-compatibility.

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

Closes #4885 from chenghao-intel/multisessions_singlecontext and squashes the following commits:

1c47b2a [Cheng Hao] rename the tss => tlSession
815b27a [Cheng Hao] code style issue
57e3fa0 [Cheng Hao] openSession is not compatible between Hive0.12 & 0.13.1
4665b0d [Cheng Hao] thriftservice with single context
2015-03-17 01:09:27 +08:00
Cheng Lian 5be6b0e4f4 [SPARK-6195] [SQL] Adds in-memory column type for fixed-precision decimals
This PR adds a specialized in-memory column type for fixed-precision decimals.

For all other column types, a single integer column type ID is enough to determine which column type to use. However, this doesn't apply to fixed-precision decimal types with different precision and scale parameters. Moreover, according to the previous design, there seems no trivial way to encode precision and scale information into the columnar byte buffer. On the other hand, considering we always know the data type of the column to be built / scanned ahead of time. This PR no longer use column type ID to construct `ColumnBuilder`s and `ColumnAccessor`s, but resorts to the actual column data type. In this way, we can pass precision / scale information along the way.

The column type ID is now not used anymore and can be removed in a future PR.

### Micro benchmark result

The following micro benchmark builds a simple table with 2 million decimals (precision = 10, scale = 0), cache it in memory, then count all the rows. Code (simply paste it into Spark shell):

```scala
import sc._
import sqlContext._
import sqlContext.implicits._
import org.apache.spark.sql.types._
import com.google.common.base.Stopwatch

def benchmark(n: Int)(f: => Long) {
  val stopwatch = new Stopwatch()

  def run() = {
    stopwatch.reset()
    stopwatch.start()
    f
    stopwatch.stop()
    stopwatch.elapsedMillis()
  }

  val records = (0 until n).map(_ => run())

  (0 until n).foreach(i => println(s"Round $i: ${records(i)} ms"))
  println(s"Average: ${records.sum / n.toDouble} ms")
}

// Explicit casting is required because ScalaReflection can't inspect decimal precision
parallelize(1 to 2000000)
  .map(i => Tuple1(Decimal(i, 10, 0)))
  .toDF("dec")
  .select($"dec" cast DecimalType(10, 0))
  .registerTempTable("dec")

sql("CACHE TABLE dec")
val df = table("dec")

// Warm up
df.count()
df.count()

benchmark(5) {
  df.count()
}
```

With `FIXED_DECIMAL` column type:

- Round 0: 75 ms
- Round 1: 97 ms
- Round 2: 75 ms
- Round 3: 70 ms
- Round 4: 72 ms
- Average: 77.8 ms

Without `FIXED_DECIMAL` column type:

- Round 0: 1233 ms
- Round 1: 1170 ms
- Round 2: 1171 ms
- Round 3: 1141 ms
- Round 4: 1141 ms
- Average: 1171.2 ms

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

Author: Cheng Lian <lian@databricks.com>

Closes #4938 from liancheng/decimal-column-type and squashes the following commits:

fef5338 [Cheng Lian] Updates fixed decimal column type related test cases
e08ab5b [Cheng Lian] Only resorts to FIXED_DECIMAL when the value can be held in a long
4db713d [Cheng Lian] Adds in-memory column type for fixed-precision decimals
2015-03-14 19:53:54 +08:00
Davies Liu b38e073fee [SPARK-6210] [SQL] use prettyString as column name in agg()
use prettyString instead of toString() (which include id of expression) as column name in agg()

Author: Davies Liu <davies@databricks.com>

Closes #5006 from davies/prettystring and squashes the following commits:

cb1fdcf [Davies Liu] use prettyString as column name in agg()
2015-03-14 00:43:33 -07:00
Cheng Lian cdc34ed910 [SPARK-6285] [SQL] Removes unused ParquetTestData and duplicated TestGroupWriteSupport
All the contents in this file are not referenced anywhere and should have been removed in #4116 when I tried to get rid of the old Parquet test suites.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5010 from liancheng/spark-6285 and squashes the following commits:

06ed057 [Cheng Lian] Removes unused ParquetTestData and duplicated TestGroupWriteSupport
2015-03-14 07:09:53 +08:00
Volodymyr Lyubinets 25b71d8c15 [SPARK-6296] [SQL] Added equals to Column
Author: Volodymyr Lyubinets <vlyubin@gmail.com>

Closes #4988 from vlyubin/columncomp and squashes the following commits:

92d7c8f [Volodymyr Lyubinets] Added equals to Column
2015-03-12 00:55:26 -07:00
Sean Owen 55c4831d68 SPARK-6245 [SQL] jsonRDD() of empty RDD results in exception
Avoid `UnsupportedOperationException` from JsonRDD.inferSchema on empty RDD.

Not sure if this is supposed to be an error (but a better one), but it seems like this case can come up if the input is down-sampled so much that nothing is sampled.

Now stuff like this:
```
sqlContext.jsonRDD(sc.parallelize(List[String]()))
```
just results in
```
org.apache.spark.sql.DataFrame = []
```

Author: Sean Owen <sowen@cloudera.com>

Closes #4971 from srowen/SPARK-6245 and squashes the following commits:

3699964 [Sean Owen] Set() -> Set.empty
3c619e1 [Sean Owen] Avoid UnsupportedOperationException from JsonRDD.inferSchema on empty RDD
2015-03-11 14:09:09 +00:00
Sean Owen 6e94c4eadf SPARK-6225 [CORE] [SQL] [STREAMING] Resolve most build warnings, 1.3.0 edition
Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.

Author: Sean Owen <sowen@cloudera.com>

Closes #4950 from srowen/SPARK-6225 and squashes the following commits:

3080972 [Sean Owen] Ordered imports: Java, Scala, 3rd party, Spark
c67985b [Sean Owen] Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.
2015-03-11 13:15:19 +00:00
Michael Armbrust eb48fd6e9d [SQL] Make Strategies a public developer API
Author: Michael Armbrust <michael@databricks.com>

Closes #4920 from marmbrus/openStrategies and squashes the following commits:

cbc35c0 [Michael Armbrust] [SQL] Make Strategies a public developer API
2015-03-05 14:50:25 -08:00
Yin Huai 1b4bb25c10 [SPARK-6163][SQL] jsonFile should be backed by the data source API
jira: https://issues.apache.org/jira/browse/SPARK-6163

Author: Yin Huai <yhuai@databricks.com>

Closes #4896 from yhuai/SPARK-6163 and squashes the following commits:

45e023e [Yin Huai] Address @chenghao-intel's comment.
2e8734e [Yin Huai] Use JSON data source for jsonFile.
92a4a33 [Yin Huai] Test.
2015-03-05 14:49:44 -08:00
Reynold Xin 54d19689ff [SPARK-5310][SQL] Fixes to Docs and Datasources API
- Various Fixes to docs
 - Make data source traits actually interfaces

Based on #4862 but with fixed conflicts.

Author: Reynold Xin <rxin@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #4868 from marmbrus/pr/4862 and squashes the following commits:

fe091ea [Michael Armbrust] Merge remote-tracking branch 'origin/master' into pr/4862
0208497 [Reynold Xin] Test fixes.
34e0a28 [Reynold Xin] [SPARK-5310][SQL] Various fixes to Spark SQL docs.
2015-03-02 22:14:08 -08:00
Yin Huai 12599942e6 [SPARK-5950][SQL]Insert array into a metastore table saved as parquet should work when using datasource api
This PR contains the following changes:
1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However,  the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values).
2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types.
3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings.
4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust.
5. Update the equality check of JSON relation. Since JSON does not really cares nullability,  `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables.

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

Thanks viirya for the initial work in #4729.

cc marmbrus liancheng

Author: Yin Huai <yhuai@databricks.com>

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

3b61a04 [Yin Huai] Revert change on equals.
80e487e [Yin Huai] asNullable in UDT.
587d88b [Yin Huai] Make methods private.
0cb7ea2 [Yin Huai] marmbrus's comments.
3cec464 [Yin Huai] Cheng's comments.
486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
d3747d1 [Yin Huai] Remove unnecessary change.
8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check.
0eb5578 [Yin Huai] Fix tests.
f6ed813 [Yin Huai] Update old parquet path.
e4f397c [Yin Huai] Unit tests.
b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check.
8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data.
bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data.
0a703e7 [Yin Huai] Test failed again since we cannot read correct content.
9a26611 [Yin Huai] Make InsertIntoTable happy.
8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability
4ec17fd [Yin Huai] Failed test.
2015-03-02 19:31:55 -08:00
Cheng Lian 1a49496b4a [SPARK-6082] [SQL] Provides better error message for malformed rows when caching tables
Constructs like Hive `TRANSFORM` may generate malformed rows (via badly authored external scripts for example). I'm a bit hesitant to have this feature, since it introduces per-tuple cost when caching tables. However, considering caching tables is usually a one-time cost, this is probably worth having.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4842 from liancheng/spark-6082 and squashes the following commits:

b05dbff [Cheng Lian] Provides better error message for malformed rows when caching tables
2015-03-02 16:18:00 -08:00
Paul Power d9a8bae778 [DOCS] Refactored Dataframe join comment to use correct parameter ordering
The API signatire for join requires the JoinType to be the third parameter. The code examples provided for join show JoinType being provided as the 2nd parater resuling in errors (i.e. "df1.join(df2, "outer", $"df1Key" === $"df2Key") ). The correct sample code is df1.join(df2, $"df1Key" === $"df2Key", "outer")

Author: Paul Power <paul.power@peerside.com>

Closes #4847 from peerside/master and squashes the following commits:

ebc1efa [Paul Power] Merge pull request #1 from peerside/peerside-patch-1
e353340 [Paul Power] Updated comments use correct sample code for Dataframe joins
2015-03-02 13:09:35 -08:00
Yin Huai 3efd8bb6cf [SPARK-6052][SQL]In JSON schema inference, we should always set containsNull of an ArrayType to true
Always set `containsNull = true` when infer the schema of JSON datasets. If we set `containsNull` based on records we scanned, we may miss arrays with null values when we do sampling. Also, because future data can have arrays with null values, if we convert JSON data to parquet, always setting `containsNull = true` is a more robust way to go.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #4806 from yhuai/jsonArrayContainsNull and squashes the following commits:

05eab9d [Yin Huai] Change containsNull to true.
2015-03-02 23:18:07 +08:00
Cheng Lian e6003f0a57 [SPARK-5775] [SQL] BugFix: GenericRow cannot be cast to SpecificMutableRow when nested data and partitioned table
This PR adapts anselmevignon's #4697 to master and branch-1.3. Please refer to PR description of #4697 for details.

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

Author: Cheng Lian <lian@databricks.com>
Author: Cheng Lian <liancheng@users.noreply.github.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #4792 from liancheng/spark-5775 and squashes the following commits:

538f506 [Cheng Lian] Addresses comments
cee55cf [Cheng Lian] Merge pull request #4 from yhuai/spark-5775-yin
b0b74fb [Yin Huai] Remove runtime pattern matching.
ca6e038 [Cheng Lian] Fixes SPARK-5775
2015-02-28 21:15:43 +08:00
Yin Huai 5e5ad6558d [SPARK-6024][SQL] When a data source table has too many columns, it's schema cannot be stored in metastore.
JIRA: https://issues.apache.org/jira/browse/SPARK-6024

Author: Yin Huai <yhuai@databricks.com>

Closes #4795 from yhuai/wideSchema and squashes the following commits:

4882e6f [Yin Huai] Address comments.
73e71b4 [Yin Huai] Address comments.
143927a [Yin Huai] Simplify code.
cc1d472 [Yin Huai] Make the schema wider.
12bacae [Yin Huai] If the JSON string of a schema is too large, split it before storing it in metastore.
e9b4f70 [Yin Huai] Failed test.
2015-02-26 20:46:05 -08:00
Liang-Chi Hsieh 4ad5153f54 [SPARK-6037][SQL] Avoiding duplicate Parquet schema merging
`FilteringParquetRowInputFormat` manually merges Parquet schemas before computing splits. However, it is duplicate because the schemas are already merged in `ParquetRelation2`. We don't need to re-merge them at `InputFormat`.

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

Closes #4786 from viirya/dup_parquet_schemas_merge and squashes the following commits:

ef78a5a [Liang-Chi Hsieh] Avoiding duplicate Parquet schema merging.
2015-02-27 11:06:47 +08:00
Jacky Li 2358657547 [SPARK-6007][SQL] Add numRows param in DataFrame.show()
It is useful to let the user decide the number of rows to show in DataFrame.show

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

Closes #4767 from jackylk/show and squashes the following commits:

a0e0f4b [Jacky Li] fix testcase
7cdbe91 [Jacky Li] modify according to comment
bb54537 [Jacky Li] for Java compatibility
d7acc18 [Jacky Li] modify according to comments
981be52 [Jacky Li] add numRows param in DataFrame.show()
2015-02-26 10:40:58 -08:00
Yin Huai 192e42a293 [SPARK-6016][SQL] Cannot read the parquet table after overwriting the existing table when spark.sql.parquet.cacheMetadata=true
Please see JIRA (https://issues.apache.org/jira/browse/SPARK-6016) for details of the bug.

Author: Yin Huai <yhuai@databricks.com>

Closes #4775 from yhuai/parquetFooterCache and squashes the following commits:

78787b1 [Yin Huai] Remove footerCache in FilteringParquetRowInputFormat.
dff6fba [Yin Huai] Failed unit test.
2015-02-27 01:01:32 +08:00
Yanbo Liang 41e2e5acb7 [SPARK-5926] [SQL] make DataFrame.explain leverage queryExecution.logical
DataFrame.explain return wrong result when the query is DDL command.

For example, the following two queries should print out the same execution plan, but it not.
sql("create table tb as select * from src where key > 490").explain(true)
sql("explain extended create table tb as select * from src where key > 490")

This is because DataFrame.explain leverage logicalPlan which had been forced executed, we should use  the unexecuted plan queryExecution.logical.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #4707 from yanboliang/spark-5926 and squashes the following commits:

fa6db63 [Yanbo Liang] logicalPlan is not lazy
0e40a1b [Yanbo Liang] make DataFrame.explain leverage queryExecution.logical
2015-02-25 15:37:13 -08:00
Liang-Chi Hsieh 12dbf98c5d [SPARK-5999][SQL] Remove duplicate Literal matching block
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #4760 from viirya/dup_literal and squashes the following commits:

06e7516 [Liang-Chi Hsieh] Remove duplicate Literal matching block.
2015-02-25 15:22:33 -08:00
Cheng Lian e0fdd467e2 [SPARK-6010] [SQL] Merging compatible Parquet schemas before computing splits
`ReadContext.init` calls `InitContext.getMergedKeyValueMetadata`, which doesn't know how to merge conflicting user defined key-value metadata and throws exception. In our case, when dealing with different but compatible schemas, we have different Spark SQL schema JSON strings in different Parquet part-files, thus causes this problem. Reading similar Parquet files generated by Hive doesn't suffer from this issue.

In this PR, we manually merge the schemas before passing it to `ReadContext` to avoid the exception.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4768 from liancheng/spark-6010 and squashes the following commits:

9002f0a [Cheng Lian] Fixes SPARK-6010
2015-02-25 15:15:22 -08:00
Michael Armbrust f84c799ea0 [SPARK-5996][SQL] Fix specialized outbound conversions
Author: Michael Armbrust <michael@databricks.com>

Closes #4757 from marmbrus/udtConversions and squashes the following commits:

3714aad [Michael Armbrust] [SPARK-5996][SQL] Fix specialized outbound conversions
2015-02-25 10:13:40 -08:00
Reynold Xin fba11c2f55 [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.
Also added desc/asc function for constructing sorting expressions more conveniently. And added a small fix to lift alias out of cast expression.

Author: Reynold Xin <rxin@databricks.com>

Closes #4752 from rxin/SPARK-5985 and squashes the following commits:

aeda5ae [Reynold Xin] Added Experimental flag to ColumnName.
047ad03 [Reynold Xin] Lift alias out of cast.
c9cf17c [Reynold Xin] [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python.
2015-02-24 18:59:23 -08:00
Michael Armbrust 201236628a [SPARK-5532][SQL] Repartition should not use external rdd representation
Author: Michael Armbrust <michael@databricks.com>

Closes #4738 from marmbrus/udtRepart and squashes the following commits:

c06d7b5 [Michael Armbrust] fix compilation
91c8829 [Michael Armbrust] [SQL][SPARK-5532] Repartition should not use external rdd representation
2015-02-24 10:52:18 -08:00
Cheng Lian 8403331333 [SPARK-5968] [SQL] Suppresses ParquetOutputCommitter WARN logs
Please refer to the [JIRA ticket] [1] for the motivation.

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

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

Author: Cheng Lian <lian@databricks.com>

Closes #4744 from liancheng/spark-5968 and squashes the following commits:

caac6a8 [Cheng Lian] Suppresses ParquetOutputCommitter WARN logs
2015-02-24 10:45:38 -08:00
Michael Armbrust 1ed57086d4 [SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution
Author: Michael Armbrust <michael@databricks.com>

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

afbaa19 [Michael Armbrust] fix python
d93278c [Michael Armbrust] fix hive
e5fa0a4 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis
52119f2 [Michael Armbrust] more tests
82a5431 [Michael Armbrust] fix tests
25753d2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis
aee1e6a [Michael Armbrust] fix hive
b23a844 [Michael Armbrust] newline
de8dc51 [Michael Armbrust] more comments
acf620a [Michael Armbrust] [SPARK-5873][SQL] Show partially analyzed plans in query execution
2015-02-23 17:34:54 -08:00
Yin Huai 48376bfe9c [SPARK-5935][SQL] Accept MapType in the schema provided to a JSON dataset.
JIRA: https://issues.apache.org/jira/browse/SPARK-5935

Author: Yin Huai <yhuai@databricks.com>
Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #4710 from yhuai/jsonMapType and squashes the following commits:

3e40390 [Yin Huai] Remove unnecessary changes.
f8e6267 [Yin Huai] Fix test.
baa36e3 [Yin Huai] Accept MapType in the schema provided to jsonFile/jsonRDD.
2015-02-23 17:16:34 -08:00
Cheng Hao 275b1bef89 [DataFrame] [Typo] Fix the typo
Author: Cheng Hao <hao.cheng@intel.com>

Closes #4717 from chenghao-intel/typo1 and squashes the following commits:

858d7b0 [Cheng Hao] update the typo
2015-02-22 08:56:30 +00:00
Yin Huai 70bfb5c728 [SPARK-5909][SQL] Add a clearCache command to Spark SQL's cache manager
JIRA: https://issues.apache.org/jira/browse/SPARK-5909

Author: Yin Huai <yhuai@databricks.com>

Closes #4694 from yhuai/clearCache and squashes the following commits:

397ecc4 [Yin Huai] Address comments.
a2702fc [Yin Huai] Update parser.
3a54506 [Yin Huai] add isEmpty to CacheManager.
6d14460 [Yin Huai] Python clearCache.
f7b8dbd [Yin Huai] Add clear cache command.
2015-02-20 16:20:02 +08:00
Reynold Xin 8ca3418e1b [SPARK-5904][SQL] DataFrame API fixes.
1. Column is no longer a DataFrame to simplify class hierarchy.
2. Don't use varargs on abstract methods (see Scala compiler bug SI-9013).

Author: Reynold Xin <rxin@databricks.com>

Closes #4686 from rxin/SPARK-5904 and squashes the following commits:

fd9b199 [Reynold Xin] Fixed Python tests.
df25cef [Reynold Xin] Non final.
5221530 [Reynold Xin] [SPARK-5904][SQL] DataFrame API fixes.
2015-02-19 12:09:44 -08:00
Davies Liu aa8f10e82a [SPARK-5722] [SQL] [PySpark] infer int as LongType
The `int` is 64-bit on 64-bit machine (very common now), we should infer it as LongType for it in Spark SQL.

Also, LongType in SQL will come back as `int`.

Author: Davies Liu <davies@databricks.com>

Closes #4666 from davies/long and squashes the following commits:

6bc6cc4 [Davies Liu] infer int as LongType
2015-02-18 14:17:04 -08:00
Cheng Lian 61ab08549c [Minor] [SQL] Cleans up DataFrame variable names and toDF() calls
Although we've migrated to the DataFrame API, lots of code still uses `rdd` or `srdd` as local variable names. This PR tries to address these naming inconsistencies and some other minor DataFrame related style issues.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4670 from liancheng/df-cleanup and squashes the following commits:

3e14448 [Cheng Lian] Cleans up DataFrame variable names and toDF() calls
2015-02-17 23:36:20 -08:00
Yin Huai 117121a4ec [SPARK-5852][SQL]Fail to convert a newly created empty metastore parquet table to a data source parquet table.
The problem is that after we create an empty hive metastore parquet table (e.g. `CREATE TABLE test (a int) STORED AS PARQUET`), Hive will create an empty dir for us, which cause our data source `ParquetRelation2` fail to get the schema of the table. See JIRA for the case to reproduce the bug and the exception.

This PR is based on #4562 from chenghao-intel.

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

Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Hao <hao.cheng@intel.com>

Closes #4655 from yhuai/CTASParquet and squashes the following commits:

b8b3450 [Yin Huai] Update tests.
2ac94f7 [Yin Huai] Update tests.
3db3d20 [Yin Huai] Minor update.
d7e2308 [Yin Huai] Revert changes in HiveMetastoreCatalog.scala.
36978d1 [Cheng Hao] Update the code as feedback
a04930b [Cheng Hao] fix bug of scan an empty parquet based table
442ffe0 [Cheng Hao] passdown the schema for Parquet File in HiveContext
2015-02-17 15:47:59 -08:00
Michael Armbrust de4836f8f1 [SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext
Author: Michael Armbrust <michael@databricks.com>

Closes #4657 from marmbrus/pythonUdfs and squashes the following commits:

a7823a8 [Michael Armbrust] [SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext
2015-02-17 13:23:45 -08:00
Liang-Chi Hsieh ac506b7c28 [Minor][SQL] Use same function to check path parameter in JSONRelation
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #4649 from viirya/use_checkpath and squashes the following commits:

0f9a1a1 [Liang-Chi Hsieh] Use same function to check path parameter.
2015-02-17 12:24:13 -08:00
Michael Armbrust c74b07fa94 [SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation
Author: Michael Armbrust <michael@databricks.com>

Closes #4642 from marmbrus/docs and squashes the following commits:

d291c34 [Michael Armbrust] python tests
9be66e3 [Michael Armbrust] comments
d56afc2 [Michael Armbrust] fix style
f004747 [Michael Armbrust] fix build
c4a907b [Michael Armbrust] fix tests
42e2b73 [Michael Armbrust] [SQL] Documentation / API Clean-up.
2015-02-17 10:21:17 -08:00
Reynold Xin 0e180bfc3c [SQL] Various DataFrame doc changes.
Added a bunch of tags.

Also changed parquetFile to take varargs rather than a string followed by varargs.

Author: Reynold Xin <rxin@databricks.com>

Closes #4636 from rxin/df-doc and squashes the following commits:

651f80c [Reynold Xin] Fixed parquetFile in PySpark.
8dc3024 [Reynold Xin] [SQL] Various DataFrame doc changes.
2015-02-16 19:00:30 -08:00
Yin Huai e189cbb052 [SPARK-4865][SQL]Include temporary tables in SHOW TABLES
This PR adds a `ShowTablesCommand` to support `SHOW TABLES [IN databaseName]` SQL command. The result of `SHOW TABLE` has two columns, `tableName` and `isTemporary`. For temporary tables, the value of `isTemporary` column will be `false`.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #4618 from yhuai/showTablesCommand and squashes the following commits:

0c09791 [Yin Huai] Use ShowTablesCommand.
85ee76d [Yin Huai] Since SHOW TABLES is not a Hive native command any more and we will not see "OK" (originally generated by Hive's driver), use SHOW DATABASES in the test.
94bacac [Yin Huai] Add SHOW TABLES to the list of noExplainCommands.
d71ed09 [Yin Huai] Fix test.
a4a6ec3 [Yin Huai] Add SHOW TABLE command.
2015-02-16 15:59:23 -08:00
kai cb6c48c874 [SQL] Optimize arithmetic and predicate operators
Existing implementation of arithmetic operators and BinaryComparison operators have redundant type checking codes, e.g.:
Expression.n2 is used by Add/Subtract/Multiply.
(1) n2 always checks left.dataType == right.dataType. However, this checking should be done once when we resolve expression types;
(2) n2 requires dataType is a NumericType. This can be done once.

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

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

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

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

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

fef0cf1 [kai] Merge branch 'master' of github.com:apache/spark into arithmetic-optimize
4b3a1bb [kai] chmod a-x
5a41e49 [kai] chmod a-x Expression.scala
cb37c94 [kai] rebase onto spark master
7f6e968 [kai] chmod 100755 -> 100644
6cddb46 [kai] format
7490dbc [kai] fix unresolved-expression exception for EqualTo
9c40bc0 [kai] fix bitwisenot
3cbd363 [kai] clean up test code
ca47801 [kai] override evalInternal for bitwise ops
8fa84a1 [kai] add bitwise or and xor
6892fc4 [kai] revert override evalInternal
f8eba24 [kai] override evalInternal
31ccdd4 [kai] rewrite all bitwise op and remove evalInternal
86297e2 [kai] generalized
cb92ae1 [kai] bitwise-and: override eval
97a7d6c [kai] bitwise-and: override evalInternal using and func
0906c39 [kai] add bitwise test
62abbbc [kai] clean up predicate and arithmetic
b34d58d [kai] add caching and benmark option
12c5b32 [kai] override eval
1cd7571 [kai] fix sqrt and maxof
03fd0c3 [kai] fix predicate
16fd84c [kai] optimize + - * / % -(unary) abs < > <= >=
fd95823 [kai] remove unnecessary type checking
24d062f [kai] test suite
2015-02-16 15:58:05 -08:00
Yin Huai 5b6cd65cd6 [SPARK-5746][SQL] Check invalid cases for the write path of data source API
JIRA: https://issues.apache.org/jira/browse/SPARK-5746

liancheng marmbrus

Author: Yin Huai <yhuai@databricks.com>

Closes #4617 from yhuai/insertOverwrite and squashes the following commits:

8e3019d [Yin Huai] Fix compilation error.
499e8e7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite
e76e85a [Yin Huai] Address comments.
ac31b3c [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite
f30bdad [Yin Huai] Use toDF.
99da57e [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite
6b7545c [Yin Huai] Add a pre write check to the data source API.
a88c516 [Yin Huai] DDLParser will take a parsering function to take care CTAS statements.
2015-02-16 15:51:59 -08:00
Cheng Lian c51ab37fad [SPARK-5833] [SQL] Adds REFRESH TABLE command
Lifts `HiveMetastoreCatalog.refreshTable` to `Catalog`. Adds `RefreshTable` command to refresh (possibly cached) metadata in external data sources tables.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4624 from liancheng/refresh-table and squashes the following commits:

8d1aa4c [Cheng Lian] Adds REFRESH TABLE command
2015-02-16 12:52:05 -08:00
Cheng Lian 6f54dee661 [SPARK-5296] [SQL] Add more filter types for data sources API
This PR adds the following filter types for data sources API:

- `IsNull`
- `IsNotNull`
- `Not`
- `And`
- `Or`

The code which converts Catalyst predicate expressions to data sources filters is very similar to filter conversion logics in `ParquetFilters` which converts Catalyst predicates to Parquet filter predicates. In this way we can support nested AND/OR/NOT predicates without changing current `BaseScan` type hierarchy.

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

Author: Cheng Lian <lian@databricks.com>

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

Closes #4623 from liancheng/more-fiters and squashes the following commits:

1b296f4 [Cheng Lian] Add more filter types for data sources API
2015-02-16 12:48:55 -08:00
Reynold Xin 9baac56ccd Minor fixes for commit https://github.com/apache/spark/pull/4592. 2015-02-16 10:09:55 -08:00
Liang-Chi Hsieh 5c78be7a51 [SPARK-5799][SQL] Compute aggregation function on specified numeric columns
Compute aggregation function on specified numeric columns. For example:

    val df = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")).toDataFrame("key", "value1", "value2", "rest")
    df.groupBy("key").min("value2")

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

Closes #4592 from viirya/specific_cols_agg and squashes the following commits:

9446896 [Liang-Chi Hsieh] For comments.
314c4cd [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into specific_cols_agg
353fad7 [Liang-Chi Hsieh] For python unit tests.
54ed0c4 [Liang-Chi Hsieh] Address comments.
b079e6b [Liang-Chi Hsieh] Remove duplicate codes.
55100fb [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into specific_cols_agg
880c2ac [Liang-Chi Hsieh] Fix Python style checks.
4c63a01 [Liang-Chi Hsieh] Fix pyspark.
b1a24fc [Liang-Chi Hsieh] Address comments.
2592f29 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into specific_cols_agg
27069c3 [Liang-Chi Hsieh] Combine functions and add varargs annotation.
371a3f7 [Liang-Chi Hsieh] Compute aggregation function on specified numeric columns.
2015-02-16 10:06:11 -08:00
Cheng Lian 3ce58cf9c0 [SPARK-4553] [SPARK-5767] [SQL] Wires Parquet data source with the newly introduced write support for data source API
This PR migrates the Parquet data source to the new data source write support API.  Now users can also overwriting and appending to existing tables. Notice that inserting into partitioned tables is not supported yet.

When Parquet data source is enabled, insertion to Hive Metastore Parquet tables is also fullfilled by the Parquet data source. This is done by the newly introduced `HiveMetastoreCatalog.ParquetConversions` rule, which is a "proper" implementation of the original hacky `HiveStrategies.ParquetConversion`. The latter is still preserved, and can be removed together with the old Parquet support in the future.

TODO:

- [x] Update outdated comments in `newParquet.scala`.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4563 from liancheng/parquet-refining and squashes the following commits:

fa98d27 [Cheng Lian] Fixes test cases which should disable off Parquet data source
2476e82 [Cheng Lian] Fixes compilation error introduced during rebasing
a83d290 [Cheng Lian] Passes Hive Metastore partitioning information to ParquetRelation2
2015-02-16 01:38:31 -08:00
Cheng Lian 199a9e8027 [Minor] [SQL] Renames stringRddToDataFrame to stringRddToDataFrameHolder for consistency
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4613)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #4613 from liancheng/df-implicit-rename and squashes the following commits:

db8bdd3 [Cheng Lian] Renames stringRddToDataFrame to stringRddToDataFrameHolder for consistency
2015-02-16 01:33:37 -08:00
Takeshi Yamamuro c771e475c4 [SPARK-5827][SQL] Add missing import in the example of SqlContext
If one tries an example by using copy&paste, throw an exception.

Author: Takeshi Yamamuro <linguin.m.s@gmail.com>

Closes #4615 from maropu/AddMissingImportInSqlContext and squashes the following commits:

ab21b66 [Takeshi Yamamuro] Add missing import in the example of SqlContext
2015-02-15 14:42:20 +00:00
Reynold Xin e98dfe627c [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
- The old implicit would convert RDDs directly to DataFrames, and that added too many methods.
- toDataFrame -> toDF
- Dsl -> functions
- implicits moved into SQLContext.implicits
- addColumn -> withColumn
- renameColumn -> withColumnRenamed

Python changes:
- toDataFrame -> toDF
- Dsl -> functions package
- addColumn -> withColumn
- renameColumn -> withColumnRenamed
- add toDF functions to RDD on SQLContext init
- add flatMap to DataFrame

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

Closes #4556 from rxin/SPARK-5752 and squashes the following commits:

5ef9910 [Reynold Xin] More fix
61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752
ff5832c [Reynold Xin] Fix python
749c675 [Reynold Xin] count(*) fixes.
5806df0 [Reynold Xin] Fix build break again.
d941f3d [Reynold Xin] Fixed explode compilation break.
fe1267a [Davies Liu] flatMap
c4afb8e [Reynold Xin] style
d9de47f [Davies Liu] add comment
b783994 [Davies Liu] add comment for toDF
e2154e5 [Davies Liu] schema() -> schema
3a1004f [Davies Liu] Dsl -> functions, toDF()
fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed
0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
97dd47c [Davies Liu] fix mistake
6168f74 [Davies Liu] fix test
1fc0199 [Davies Liu] fix test
a075cd5 [Davies Liu] clean up, toPandas
663d314 [Davies Liu] add test for agg('*')
9e214d5 [Reynold Xin] count(*) fixes.
1ed7136 [Reynold Xin] Fix build break again.
921b2e3 [Reynold Xin] Fixed explode compilation break.
14698d4 [Davies Liu] flatMap
ba3e12d [Reynold Xin] style
d08c92d [Davies Liu] add comment
5c8b524 [Davies Liu] add comment for toDF
a4e5e66 [Davies Liu] schema() -> schema
d377fc9 [Davies Liu] Dsl -> functions, toDF()
6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed
807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-13 23:03:22 -08:00
Yin Huai 2e0c084528 [SPARK-5789][SQL]Throw a better error message if JsonRDD.parseJson encounters unrecoverable parsing errors.
Author: Yin Huai <yhuai@databricks.com>

Closes #4582 from yhuai/jsonErrorMessage and squashes the following commits:

152dbd4 [Yin Huai] Update error message.
1466256 [Yin Huai] Throw a better error message when a JSON object in the input dataset span multiple records (lines for files or strings for an RDD of strings).
2015-02-13 13:51:06 -08:00
Yin Huai 2aea892ebd [SQL] Fix docs of SQLContext.tables
Author: Yin Huai <yhuai@databricks.com>

Closes #4579 from yhuai/tablesDoc and squashes the following commits:

7f8964c [Yin Huai] Fix doc.
2015-02-12 20:37:55 -08:00
Yin Huai 1d0596a16e [SPARK-3299][SQL]Public API in SQLContext to list tables
https://issues.apache.org/jira/browse/SPARK-3299

Author: Yin Huai <yhuai@databricks.com>

Closes #4547 from yhuai/tables and squashes the following commits:

6c8f92e [Yin Huai] Add tableNames.
acbb281 [Yin Huai] Update Python test.
7793dcb [Yin Huai] Fix scala test.
572870d [Yin Huai] Address comments.
aba2e88 [Yin Huai] Format.
12c86df [Yin Huai] Add tables() to SQLContext to return a DataFrame containing existing tables.
2015-02-12 18:08:01 -08:00
Yin Huai c025a46882 [SQL] Move SaveMode to SQL package.
Author: Yin Huai <yhuai@databricks.com>

Closes #4542 from yhuai/moveSaveMode and squashes the following commits:

65a4425 [Yin Huai] Move SaveMode to sql package.
2015-02-12 15:32:17 -08:00
Michael Armbrust ee04a8b19b [SPARK-5573][SQL] Add explode to dataframes
Author: Michael Armbrust <michael@databricks.com>

Closes #4546 from marmbrus/explode and squashes the following commits:

eefd33a [Michael Armbrust] whitespace
a8d496c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into explode
4af740e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explode
dc86a5c [Michael Armbrust] simple version
d633d01 [Michael Armbrust] add scala specific
950707a [Michael Armbrust] fix comments
ba8854c [Michael Armbrust] [SPARK-5573][SQL] Add explode to dataframes
2015-02-12 15:19:19 -08:00
Yin Huai c352ffbdb9 [SPARK-5758][SQL] Use LongType as the default type for integers in JSON schema inference.
Author: Yin Huai <yhuai@databricks.com>

Closes #4544 from yhuai/jsonUseLongTypeByDefault and squashes the following commits:

6e2ffc2 [Yin Huai] Use LongType as the default type for integers in JSON schema inference.
2015-02-12 15:17:25 -08:00
Michael Armbrust a38e23c30f [SQL] Make dataframe more tolerant of being serialized
Eases use in the spark-shell.

Author: Michael Armbrust <michael@databricks.com>

Closes #4545 from marmbrus/serialization and squashes the following commits:

04748e6 [Michael Armbrust] @scala.annotation.varargs
b36e219 [Michael Armbrust] moreFixes
2015-02-11 19:05:49 -08:00
Reynold Xin d931b01dca [SQL] Two DataFrame fixes.
- Removed DataFrame.apply for projection & filtering since they are extremely confusing.
- Added implicits for RDD[Int], RDD[Long], and RDD[String]

Author: Reynold Xin <rxin@databricks.com>

Closes #4543 from rxin/df-cleanup and squashes the following commits:

81ec915 [Reynold Xin] [SQL] More DataFrame fixes.
2015-02-11 18:32:48 -08:00
Michael Armbrust a60d2b70ad [SPARK-5454] More robust handling of self joins
Also I fix a bunch of bad output in test cases.

Author: Michael Armbrust <michael@databricks.com>

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

4f4a85c [Michael Armbrust] comments
49c8e26 [Michael Armbrust] fix tests
6fc38de [Michael Armbrust] fix style
55d64b3 [Michael Armbrust] fix dataframe selfjoins
2015-02-11 12:31:56 -08:00
Davies Liu b694eb9c2f [SPARK-5677] [SPARK-5734] [SQL] [PySpark] Python DataFrame API remaining tasks
1. DataFrame.renameColumn

2. DataFrame.show() and _repr_

3. Use simpleString() rather than jsonValue in DataFrame.dtypes

4. createDataFrame from local Python data, including pandas.DataFrame

Author: Davies Liu <davies@databricks.com>

Closes #4528 from davies/df3 and squashes the following commits:

014acea [Davies Liu] fix typo
6ba526e [Davies Liu] fix tests
46f5f95 [Davies Liu] address comments
6cbc154 [Davies Liu] dataframe.show() and improve dtypes
6f94f25 [Davies Liu] create DataFrame from local Python data
2015-02-11 12:13:16 -08:00
Reynold Xin b8f88d3272 [SPARK-5702][SQL] Allow short names for built-in data sources.
Also took the chance to fixed up some style ...

Author: Reynold Xin <rxin@databricks.com>

Closes #4489 from rxin/SPARK-5702 and squashes the following commits:

74f42e3 [Reynold Xin] [SPARK-5702][SQL] Allow short names for built-in data sources.
2015-02-10 20:40:21 -08:00
Reynold Xin 7e24249af1 [SQL][DataFrame] Fix column computability bug.
Do not recursively strip out projects. Only strip the first level project.

```scala
df("colA") + df("colB").as("colC")
```

Previously, the above would construct an invalid plan.

Author: Reynold Xin <rxin@databricks.com>

Closes #4519 from rxin/computability and squashes the following commits:

87ff763 [Reynold Xin] Code review feedback.
015c4fc [Reynold Xin] [SQL][DataFrame] Fix column computability.
2015-02-10 19:50:44 -08:00
Cheng Hao 45df77b841 [SPARK-5709] [SQL] Add EXPLAIN support in DataFrame API for debugging purpose
Author: Cheng Hao <hao.cheng@intel.com>

Closes #4496 from chenghao-intel/df_explain and squashes the following commits:

552aa58 [Cheng Hao] Add explain support for DF
2015-02-10 19:40:51 -08:00
Davies Liu ea60284095 [SPARK-5704] [SQL] [PySpark] createDataFrame from RDD with columns
Deprecate inferSchema() and applySchema(), use createDataFrame() instead, which could take an optional `schema` to create an DataFrame from an RDD. The `schema` could be StructType or list of names of columns.

Author: Davies Liu <davies@databricks.com>

Closes #4498 from davies/create and squashes the following commits:

08469c1 [Davies Liu] remove Scala/Java API for now
c80a7a9 [Davies Liu] fix hive test
d1bd8f2 [Davies Liu] cleanup applySchema
9526e97 [Davies Liu] createDataFrame from RDD with columns
2015-02-10 19:40:12 -08:00
Cheng Hao a60aea86b4 [SPARK-5683] [SQL] Avoid multiple json generator created
Author: Cheng Hao <hao.cheng@intel.com>

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

aeb7801 [Cheng Hao] avoid multiple json generator created
2015-02-10 18:19:56 -08:00
Yin Huai aaf50d05c7 [SPARK-5658][SQL] Finalize DDL and write support APIs
https://issues.apache.org/jira/browse/SPARK-5658

Author: Yin Huai <yhuai@databricks.com>

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

Closes #4446 from yhuai/writeSupportFollowup and squashes the following commits:

f3a96f7 [Yin Huai] davies's comments.
225ff71 [Yin Huai] Use Scala TestHiveContext to initialize the Python HiveContext in Python tests.
2306f93 [Yin Huai] Style.
2091fcd [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
537e28f [Yin Huai] Correctly clean up temp data.
ae4649e [Yin Huai] Fix Python test.
609129c [Yin Huai] Doc format.
92b6659 [Yin Huai] Python doc and other minor updates.
cbc717f [Yin Huai] Rename dataSourceName to source.
d1c12d3 [Yin Huai] No need to delete the duplicate rule since it has been removed in master.
22cfa70 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
d91ecb8 [Yin Huai] Fix test.
4c76d78 [Yin Huai] Simplify APIs.
3abc215 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
0832ce4 [Yin Huai] Fix test.
98e7cdb [Yin Huai] Python style.
2bf44ef [Yin Huai] Python APIs.
c204967 [Yin Huai] Format
a10223d [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
9ff97d8 [Yin Huai] Add SaveMode to saveAsTable.
9b6e570 [Yin Huai] Update doc.
c2be775 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
99950a2 [Yin Huai] Use Java enum for SaveMode.
4679665 [Yin Huai] Remove duplicate rule.
77d89dc [Yin Huai] Update doc.
e04d908 [Yin Huai] Move import and add (Scala-specific) to scala APIs.
cf5703d [Yin Huai] Add checkAnswer to Java tests.
7db95ff [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
6dfd386 [Yin Huai] Add java test.
f2f33ef [Yin Huai] Fix test.
e702386 [Yin Huai] Apache header.
b1e9b1b [Yin Huai] Format.
ed4e1b4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
af9e9b3 [Yin Huai] DDL and write support API followup.
2a6213a [Yin Huai] Update API names.
e6a0b77 [Yin Huai] Update test.
43bae01 [Yin Huai] Remove createTable from HiveContext.
5ffc372 [Yin Huai] Add more load APIs to SQLContext.
5390743 [Yin Huai] Add more save APIs to DataFrame.
2015-02-10 17:29:52 -08:00
Yin Huai e28b6bdbb5 [SQL] Make Options in the data source API CREATE TABLE statements optional.
Users will not need to put `Options()` in a CREATE TABLE statement when there is not option provided.

Author: Yin Huai <yhuai@databricks.com>

Closes #4515 from yhuai/makeOptionsOptional and squashes the following commits:

1a898d3 [Yin Huai] Make options optional.
2015-02-10 17:06:12 -08:00
Cheng Lian 2d50a010ff [SPARK-5725] [SQL] Fixes ParquetRelation2.equals
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4513)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #4513 from liancheng/spark-5725 and squashes the following commits:

bf6a087 [Cheng Lian] Fixes ParquetRelation2.equals
2015-02-10 17:02:44 -08:00
Sheng, Li 91e3512544 [SQL][Minor] correct some comments
Author: Sheng, Li <OopsOutOfMemory@users.noreply.github.com>
Author: OopsOutOfMemory <victorshengli@126.com>

Closes #4508 from OopsOutOfMemory/cmt and squashes the following commits:

d8a68c6 [Sheng, Li] Update ddl.scala
f24aeaf [OopsOutOfMemory] correct style
2015-02-11 00:59:46 +00:00
Michael Armbrust de80b1ba4d [SQL] Add toString to DataFrame/Column
Author: Michael Armbrust <michael@databricks.com>

Closes #4436 from marmbrus/dfToString and squashes the following commits:

8a3c35f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into dfToString
b72a81b [Michael Armbrust] add toString
2015-02-10 13:14:01 -08:00
Sean Owen 2d1e916730 SPARK-5239 [CORE] JdbcRDD throws "java.lang.AbstractMethodError: oracle.jdbc.driver.xxxxxx.isClosed()Z"
This is a completion of https://github.com/apache/spark/pull/4033 which was withdrawn for some reason.

Author: Sean Owen <sowen@cloudera.com>

Closes #4470 from srowen/SPARK-5239.2 and squashes the following commits:

2398bde [Sean Owen] Avoid use of JDBC4-only isClosed()
2015-02-10 09:19:01 +00:00
Cheng Hao bd0b5ea708 [SQL] Remove the duplicated code
Author: Cheng Hao <hao.cheng@intel.com>

Closes #4494 from chenghao-intel/tiny_code_change and squashes the following commits:

450dfe7 [Cheng Hao] remove the duplicated code
2015-02-09 21:33:34 -08:00
Davies Liu 08488c175f [SPARK-5469] restructure pyspark.sql into multiple files
All the DataTypes moved into pyspark.sql.types

The changes can be tracked by `--find-copies-harder -M25`
```
davieslocalhost:~/work/spark/python$ git diff --find-copies-harder -M25 --numstat master..
2       5       python/docs/pyspark.ml.rst
0       3       python/docs/pyspark.mllib.rst
10      2       python/docs/pyspark.sql.rst
1       1       python/pyspark/mllib/linalg.py
21      14      python/pyspark/{mllib => sql}/__init__.py
14      2108    python/pyspark/{sql.py => sql/context.py}
10      1772    python/pyspark/{sql.py => sql/dataframe.py}
7       6       python/pyspark/{sql_tests.py => sql/tests.py}
8       1465    python/pyspark/{sql.py => sql/types.py}
4       2       python/run-tests
1       1       sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
```

Also `git blame -C -C python/pyspark/sql/context.py` to track the history.

Author: Davies Liu <davies@databricks.com>

Closes #4479 from davies/sql and squashes the following commits:

1b5f0a5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sql
2b2b983 [Davies Liu] restructure pyspark.sql
2015-02-09 20:49:22 -08:00
Michael Armbrust 68b25cf695 [SQL] Add some missing DataFrame functions.
- as with a `Symbol`
- distinct
- sqlContext.emptyDataFrame
- move add/remove col out of RDDApi section

Author: Michael Armbrust <michael@databricks.com>

Closes #4437 from marmbrus/dfMissingFuncs and squashes the following commits:

2004023 [Michael Armbrust] Add missing functions
2015-02-09 16:02:56 -08:00
Hung Lin 4575c5643a [SPARK-5472][SQL] Fix Scala code style
Fix Scala code style.

Author: Hung Lin <hung@zoomdata.com>

Closes #4464 from hunglin/SPARK-5472 and squashes the following commits:

ef7a3b3 [Hung Lin] SPARK-5472: fix scala style
2015-02-08 22:36:42 -08:00
Reynold Xin a052ed4250 [SPARK-5643][SQL] Add a show method to print the content of a DataFrame in tabular format.
An example:
```
year  month AVG('Adj Close) MAX('Adj Close)
1980  12    0.503218        0.595103
1981  01    0.523289        0.570307
1982  02    0.436504        0.475256
1983  03    0.410516        0.442194
1984  04    0.450090        0.483521
```

Author: Reynold Xin <rxin@databricks.com>

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

a6857b5 [Wenchen Fan] fix import order
8411c40 [Wenchen Fan] use UnresolvedGetField
2015-02-06 13:08:09 -08:00
Yin Huai 3eccf29ce0 [SPARK-5595][SPARK-5603][SQL] Add a rule to do PreInsert type casting and field renaming and invalidating in memory cache after INSERT
This PR adds a rule to Analyzer that will add preinsert data type casting and field renaming to the select clause in an `INSERT INTO/OVERWRITE` statement. Also, with the change of this PR, we always invalidate our in memory data cache after inserting into a BaseRelation.

cc marmbrus liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #4373 from yhuai/insertFollowUp and squashes the following commits:

08237a7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertFollowUp
316542e [Yin Huai] Doc update.
c9ccfeb [Yin Huai] Revert a unnecessary change.
84aecc4 [Yin Huai] Address comments.
1951fe1 [Yin Huai] Merge remote-tracking branch 'upstream/master'
c18da34 [Yin Huai] Invalidate cache after insert.
727f21a [Yin Huai] Preinsert casting and renaming.
2015-02-06 12:38:07 -08:00
OopsOutOfMemory 0b7eb3f3b7 [SPARK-5324][SQL] Results of describe can't be queried
Make below code works.
```
sql("DESCRIBE test").registerTempTable("describeTest")
sql("SELECT * FROM describeTest").collect()
```

Author: OopsOutOfMemory <victorshengli@126.com>
Author: Sheng, Li <OopsOutOfMemory@users.noreply.github.com>

Closes #4249 from OopsOutOfMemory/desc_query and squashes the following commits:

6fee13d [OopsOutOfMemory] up-to-date
e71430a [Sheng, Li] Update HiveOperatorQueryableSuite.scala
3ba1058 [OopsOutOfMemory] change to default argument
aac7226 [OopsOutOfMemory] Merge branch 'master' into desc_query
68eb6dd [OopsOutOfMemory] Merge branch 'desc_query' of github.com:OopsOutOfMemory/spark into desc_query
354ad71 [OopsOutOfMemory] query describe command
d541a35 [OopsOutOfMemory] refine test suite
e1da481 [OopsOutOfMemory] refine test suite
a780539 [OopsOutOfMemory] Merge branch 'desc_query' of github.com:OopsOutOfMemory/spark into desc_query
0015f82 [OopsOutOfMemory] code style
dd0aaef [OopsOutOfMemory] code style
c7d606d [OopsOutOfMemory] rename test suite
75f2342 [OopsOutOfMemory] refine code and test suite
f942c9b [OopsOutOfMemory] initial
11559ae [OopsOutOfMemory] code style
c5fdecf [OopsOutOfMemory] code style
aeaea5f [OopsOutOfMemory] rename test suite
ac2c3bb [OopsOutOfMemory] refine code and test suite
544573e [OopsOutOfMemory] initial
2015-02-06 12:33:20 -08:00
Reynold Xin 7dc4965f34 [SPARK-5639][SQL] Support DataFrame.renameColumn.
Author: Reynold Xin <rxin@databricks.com>

Closes #4410 from rxin/df-renameCol and squashes the following commits:

a6a796e [Reynold Xin] [SPARK-5639][SQL] Support DataFrame.renameColumn.
2015-02-05 23:02:40 -08:00
Reynold Xin e8a5d50a96 [SPARK-5638][SQL] Add a config flag to disable eager analysis of DataFrames
Author: Reynold Xin <rxin@databricks.com>

Closes #4408 from rxin/df-config-eager and squashes the following commits:

c0204cf [Reynold Xin] [SPARK-5638][SQL] Add a config flag to disable eager analysis of DataFrames.
2015-02-05 18:07:10 -08:00
Cheng Lian a9ed51178c [SPARK-5182] [SPARK-5528] [SPARK-5509] [SPARK-3575] [SQL] Parquet data source improvements
This PR adds three major improvements to Parquet data source:

1.  Partition discovery

    While reading Parquet files resides in Hive style partition directories, `ParquetRelation2` automatically discovers partitioning information and infers partition column types.

    This is also a partial work for [SPARK-5182] [1], which aims to provide first class partitioning support for the data source API.  Related code in this PR can be easily extracted to the data source API level in future versions.

1.  Schema merging

    When enabled, Parquet data source collects schema information from all Parquet part-files and tries to merge them.  Exceptions are thrown when incompatible schemas are detected.  This feature is controlled by data source option `parquet.mergeSchema`, and is enabled by default.

1.  Metastore Parquet table conversion moved to analysis phase

    This greatly simplifies the conversion logic.  `ParquetConversion` strategy can be removed once the old Parquet implementation is removed in the future.

This version of Parquet data source aims to entirely replace the old Parquet implementation.  However, the old version hasn't been removed yet.  Users can fall back to the old version by turning off SQL configuration `spark.sql.parquet.useDataSourceApi`.

Other JIRA tickets fixed as side effects in this PR:

- [SPARK-5509] [3]: `EqualTo` now uses a proper `Ordering` to compare binary types.

- [SPARK-3575] [4]: Metastore schema is now preserved and passed to `ParquetRelation2` via data source option `parquet.metastoreSchema`.

TODO:

- [ ] More test cases for partition discovery
- [x] Fix write path after data source write support (#4294) is merged

      It turned out to be non-trivial to fall back to old Parquet implementation on the write path when Parquet data source is enabled.  Since we're planning to include data source write support in 1.3.0, I simply ignored two test cases involving Parquet insertion for now.

- [ ] Fix outdated comments and documentations

PS: This PR looks big, but more than a half of the changed lines in this PR are trivial changes to test cases. To test Parquet with and without the new data source, almost all Parquet test cases are moved into wrapper driver functions. This introduces hundreds of lines of changes.

[1]: https://issues.apache.org/jira/browse/SPARK-5182
[2]: https://issues.apache.org/jira/browse/SPARK-5528
[3]: https://issues.apache.org/jira/browse/SPARK-5509
[4]: https://issues.apache.org/jira/browse/SPARK-3575

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

Author: Cheng Lian <lian@databricks.com>

Closes #4308 from liancheng/parquet-partition-discovery and squashes the following commits:

b6946e6 [Cheng Lian] Fixes MiMA issues, addresses comments
8232e17 [Cheng Lian] Write support for Parquet data source
a49bd28 [Cheng Lian] Fixes spelling typo in trait name "CreateableRelationProvider"
808380f [Cheng Lian] Fixes issues introduced while rebasing
50dd8d1 [Cheng Lian] Addresses @rxin's comment, fixes UDT schema merging
adf2aae [Cheng Lian] Fixes compilation error introduced while rebasing
4e0175f [Cheng Lian] Fixes Python Parquet API, we need Py4J array to call varargs method
0d8ec1d [Cheng Lian] Adds more test cases
b35c8c6 [Cheng Lian] Fixes some typos and outdated comments
dd704fd [Cheng Lian] Fixes Python Parquet API
596c312 [Cheng Lian] Uses switch to control whether use Parquet data source or not
7d0f7a2 [Cheng Lian] Fixes Metastore Parquet table conversion
a1896c7 [Cheng Lian] Fixes all existing Parquet test suites except for ParquetMetastoreSuite
5654c9d [Cheng Lian] Draft version of Parquet partition discovery and schema merging
2015-02-05 15:29:56 -08:00
OopsOutOfMemory 4d8d070c4f [SPARK-5135][SQL] Add support for describe table to DDL in SQLContext
Hi, rxin marmbrus
I considered your suggestion (in #4127) and now re-write it. This is now up-to-date.
Could u please review it ?

Author: OopsOutOfMemory <victorshengli@126.com>

Closes #4227 from OopsOutOfMemory/describe and squashes the following commits:

053826f [OopsOutOfMemory] describe
2015-02-05 13:07:48 -08:00
Reynold Xin 7d789e117d [SPARK-5612][SQL] Move DataFrame implicit functions into SQLContext.implicits.
Author: Reynold Xin <rxin@databricks.com>

Closes #4386 from rxin/df-implicits and squashes the following commits:

9d96606 [Reynold Xin] style fix
edd296b [Reynold Xin] ReplSuite
1c946ab [Reynold Xin] [SPARK-5612][SQL] Move DataFrame implicit functions into SQLContext.implicits.
2015-02-04 23:44:34 -08:00
Reynold Xin 84acd08e08 [SPARK-5602][SQL] Better support for creating DataFrame from local data collection
1. Added methods to create DataFrames from Seq[Product]
2. Added executeTake to avoid running a Spark job on LocalRelations.

Author: Reynold Xin <rxin@databricks.com>

Closes #4372 from rxin/localDataFrame and squashes the following commits:

f696858 [Reynold Xin] style checker.
839ef7f [Reynold Xin] [SPARK-5602][SQL] Better support for creating DataFrame from local data collection.
2015-02-04 19:53:57 -08:00
Reynold Xin 6b4c7f0806 [SQL][DataFrame] Minor cleanup.
1. Removed LocalHiveContext in Python.
2. Reduced DSL UDF support from 22 arguments to 10 arguments so JavaDoc/ScalaDoc look nicer.

Author: Reynold Xin <rxin@databricks.com>

Closes #4374 from rxin/df-style and squashes the following commits:

e493342 [Reynold Xin] [SQL][DataFrame] Minor cleanup.
2015-02-04 19:51:48 -08:00
Sadhan Sood dba98bf698 [SPARK-4520] [SQL] This pr fixes the ArrayIndexOutOfBoundsException as r...
...aised in SPARK-4520.

The exception is thrown only for a thrift generated parquet file. The array element schema name is assumed as "array" as per ParquetAvro but for thrift generated parquet files, it is array_name + "_tuple". This leads to missing child of array group type and hence when the parquet rows are being materialized leads to the exception.

Author: Sadhan Sood <sadhan@tellapart.com>

Closes #4148 from sadhan/SPARK-4520 and squashes the following commits:

c5ccde8 [Sadhan Sood] [SPARK-4520] [SQL] This pr fixes the ArrayIndexOutOfBoundsException as raised in SPARK-4520.
2015-02-04 19:18:06 -08:00
Reynold Xin 1fbd124b1b [SPARK-5605][SQL][DF] Allow using String to specify colum name in DSL aggregate functions
Author: Reynold Xin <rxin@databricks.com>

Closes #4376 from rxin/SPARK-5605 and squashes the following commits:

c55f5fa [Reynold Xin] Added a Python test.
f4b8dbb [Reynold Xin] [SPARK-5605][SQL][DF] Allow using String to specify colum name in DSL aggregate functions.
2015-02-04 18:35:51 -08:00