Commit graph

4509 commits

Author SHA1 Message Date
Tathagata Das 407c3cedf2 [SPARK-17624][SQL][STREAMING][TEST] Fixed flaky StateStoreSuite.maintenance
## What changes were proposed in this pull request?

The reason for the flakiness was follows. The test starts the maintenance background thread, and then writes 20 versions of the state store. The maintenance thread is expected to create snapshots in the middle, and clean up old files that are not needed any more. The earliest delta file (1.delta) is expected to be deleted as snapshots will ensure that the earliest delta would not be needed.

However, the default configuration for the maintenance thread is to retain files such that last 2 versions can be recovered, and delete the rest. Now while generating the versions, the maintenance thread can kick in and create snapshots anywhere between version 10 and 20 (at least 10 deltas needed for snapshot). Then later it will choose to retain only version 20 and 19 (last 2). There are two cases.

- Common case: One of the version between 10 and 19 gets snapshotted. Then recovering versions 19 and 20 just needs 19.snapshot and 20.delta, so 1.delta gets deleted.

- Uncommon case (reason for flakiness): Only version 20 gets snapshotted. Then recovering versoin 20 requires 20.snapshot, and recovering version 19 all the previous 19...1.delta. So 1.delta does not get deleted.

This PR rearranges the checks such that it create 20 versions, and then waits that there is at least one snapshot, then creates another 20. This will ensure that the latest 2 versions cannot require anything older than the first snapshot generated, and therefore will 1.delta will be deleted.

In addition, I have added more logs, and comments that I felt would help future debugging and understanding what is going on.

## How was this patch tested?

Ran the StateStoreSuite > 6K times in a heavily loaded machine (10 instances of tests running in parallel). No failures.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #15592 from tdas/SPARK-17624.
2016-10-24 17:21:16 -07:00
Sean Owen 4ecbe1b92f
[SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but can resolve as HDFS path
## What changes were proposed in this pull request?

Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15382 from srowen/SPARK-17810.
2016-10-24 10:44:45 +01:00
CodingCat a81fba048f [SPARK-18058][SQL] Comparing column types ignoring Nullability in Union and SetOperation
## What changes were proposed in this pull request?

The PR tries to fix [SPARK-18058](https://issues.apache.org/jira/browse/SPARK-18058) which refers to a bug that the column types are compared with the extra care about Nullability in Union and SetOperation.

This PR converts the columns types by setting all fields as nullable before comparison

## How was this patch tested?

regular unit test cases

Author: CodingCat <zhunansjtu@gmail.com>

Closes #15595 from CodingCat/SPARK-18058.
2016-10-23 19:42:11 +02:00
jiangxingbo b158256c2e [SPARK-18045][SQL][TESTS] Move HiveDataFrameAnalyticsSuite to package sql
## What changes were proposed in this pull request?

The testsuite `HiveDataFrameAnalyticsSuite` has nothing to do with HIVE, we should move it to package `sql`.
The original test cases in that suite are splited into two existing testsuites: `DataFrameAggregateSuite` tests for the functions and ~~`SQLQuerySuite`~~`SQLQueryTestSuite` tests for the SQL statements.

## How was this patch tested?
~~Modified `SQLQuerySuite` in package `sql`.~~
Add query file for `SQLQueryTestSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15582 from jiangxb1987/group-analytics-test.
2016-10-23 13:28:35 +02:00
Tejas Patil 21c7539a52 [SPARK-18038][SQL] Move output partitioning definition from UnaryNodeExec to its children
## What changes were proposed in this pull request?

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

This was a suggestion by rxin over one of the dev list discussion : http://apache-spark-developers-list.1001551.n3.nabble.com/Project-not-preserving-child-partitioning-td19417.html

His words:

>> It would be better (safer) to move the output partitioning definition into each of the operator and remove it from UnaryExecNode.

With this PR, following is the output partitioning and ordering for all the impls of `UnaryExecNode`.

UnaryExecNode's impl | outputPartitioning | outputOrdering | comment
------------ | ------------- | ------------ | ------------
AppendColumnsExec | child's | Nil | child's ordering can be used
AppendColumnsWithObjectExec | child's | Nil | child's ordering can be used
BroadcastExchangeExec | BroadcastPartitioning | Nil | -
CoalesceExec | UnknownPartitioning | Nil | -
CollectLimitExec | SinglePartition | Nil | -
DebugExec | child's | Nil | child's ordering can be used
DeserializeToObjectExec | child's | Nil | child's ordering can be used
ExpandExec | UnknownPartitioning | Nil | -
FilterExec | child's | child's | -
FlatMapGroupsInRExec | child's | Nil | child's ordering can be used
GenerateExec | child's | Nil | need to dig more
GlobalLimitExec | child's | child's | -
HashAggregateExec | child's | Nil | -
InputAdapter | child's | child's | -
InsertIntoHiveTable | child's | Nil | terminal node, doesn't need partitioning
LocalLimitExec | child's | child's | -
MapElementsExec | child's | child's | -
MapGroupsExec | child's | Nil | child's ordering can be used
MapPartitionsExec | child's | Nil | child's ordering can be used
ProjectExec | child's | child's | -
SampleExec | child's | Nil | child's ordering can be used
ScriptTransformation | child's | Nil | child's ordering can be used
SerializeFromObjectExec | child's | Nil | child's ordering can be used
ShuffleExchange | custom | Nil | -
SortAggregateExec | child's | sort over grouped exprs | -
SortExec | child's | custom | -
StateStoreRestoreExec  | child's | Nil | child's ordering can be used
StateStoreSaveExec | child's | Nil | child's ordering can be used
SubqueryExec | child's | child's | -
TakeOrderedAndProjectExec | SinglePartition | custom | -
WholeStageCodegenExec | child's | child's | -
WindowExec | child's | child's | -

## How was this patch tested?

This does NOT change any existing functionality so relying on existing tests

Author: Tejas Patil <tejasp@fb.com>

Closes #15575 from tejasapatil/SPARK-18038_UnaryNodeExec_output_partitioning.
2016-10-23 13:25:47 +02:00
Tejas Patil eff4aed1ac [SPARK-18035][SQL] Introduce performant and memory efficient APIs to create ArrayBasedMapData
## What changes were proposed in this pull request?

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

In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658

The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323

This copy is not needed as we get rid of it once we extract the key and value arrays.

Here is the call trace:

```
org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664)
scala.collection.AbstractMap.toSeq(Map.scala:59)
scala.collection.MapLike$class.toSeq(MapLike.scala:323)
scala.collection.AbstractMap.toBuffer(Map.scala:59)
scala.collection.MapLike$class.toBuffer(MapLike.scala:326)
scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104)
scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
scala.collection.AbstractIterable.foreach(Iterable.scala:54)
scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
```

Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration.

EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient

## Performance gains

The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query.

## How was this patch tested?

This does not change the end result produced so relying on existing tests.

Author: Tejas Patil <tejasp@fb.com>

Closes #15573 from tejasapatil/SPARK-18035_avoid_toSeq.
2016-10-22 20:43:43 -07:00
hyukjinkwon 5fa9f8795a [SPARK-17123][SQL] Use type-widened encoder for DataFrame rather than existing encoder to allow type-widening from set operations
# What changes were proposed in this pull request?

This PR fixes set operations in `DataFrame` to be performed fine without exceptions when the types are non-scala native types. (e.g, `TimestampType`, `DateType` and `DecimalType`).

The problem is, it seems set operations such as `union`, `intersect` and `except` uses the encoder belonging to the `Dataset` in caller.

So, `Dataset` of the caller holds `ExpressionEncoder[Row]` as it is when the set operations are performed. However, the return types can be actually widen. So, we should use `ExpressionEncoder[Row]` constructed from executed plan rather than using existing one. Otherwise, this will generate some codes wrongly via `StaticInvoke`.

Running the codes below:

```scala
val dates = Seq(
  (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)),
  (new Date(3), BigDecimal.valueOf(4), new Timestamp(5))
).toDF("date", "timestamp", "decimal")

val widenTypedRows = Seq(
  (new Timestamp(2), 10.5D, "string")
).toDF("date", "timestamp", "decimal")

val results = dates.union(widenTypedRows).collect()
results.foreach(println)
```

prints below:

**Before**

```java
23:08:54.490 ERROR org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 28, Column 107: No applicable constructor/method found for actual parameters "long"; candidates are: "public static java.sql.Date org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(int)"
/* 001 */ public java.lang.Object generate(Object[] references) {
/* 002 */   return new SpecificSafeProjection(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificSafeProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private MutableRow mutableRow;
/* 009 */   private Object[] values;
/* 010 */   private org.apache.spark.sql.types.StructType schema;
/* 011 */
/* 012 */
/* 013 */   public SpecificSafeProjection(Object[] references) {
/* 014 */     this.references = references;
/* 015 */     mutableRow = (MutableRow) references[references.length - 1];
/* 016 */
/* 017 */     this.schema = (org.apache.spark.sql.types.StructType) references[0];
/* 018 */   }
/* 019 */
/* 020 */   public java.lang.Object apply(java.lang.Object _i) {
/* 021 */     InternalRow i = (InternalRow) _i;
/* 022 */
/* 023 */     values = new Object[3];
/* 024 */
/* 025 */     boolean isNull2 = i.isNullAt(0);
/* 026 */     long value2 = isNull2 ? -1L : (i.getLong(0));
/* 027 */     boolean isNull1 = isNull2;
/* 028 */     final java.sql.Date value1 = isNull1 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(value2);
/* 029 */     isNull1 = value1 == null;
/* 030 */     if (isNull1) {
/* 031 */       values[0] = null;
/* 032 */     } else {
/* 033 */       values[0] = value1;
/* 034 */     }
/* 035 */
/* 036 */     boolean isNull4 = i.isNullAt(1);
/* 037 */     double value4 = isNull4 ? -1.0 : (i.getDouble(1));
/* 038 */
/* 039 */     boolean isNull3 = isNull4;
/* 040 */     java.math.BigDecimal value3 = null;
/* 041 */     if (!isNull3) {
/* 042 */
/* 043 */       Object funcResult = null;
/* 044 */       funcResult = value4.toJavaBigDecimal();
/* 045 */       if (funcResult == null) {
/* 046 */         isNull3 = true;
/* 047 */       } else {
/* 048 */         value3 = (java.math.BigDecimal) funcResult;
/* 049 */       }
/* 050 */
/* 051 */     }
/* 052 */     isNull3 = value3 == null;
/* 053 */     if (isNull3) {
/* 054 */       values[1] = null;
/* 055 */     } else {
/* 056 */       values[1] = value3;
/* 057 */     }
/* 058 */
/* 059 */     boolean isNull6 = i.isNullAt(2);
/* 060 */     UTF8String value6 = isNull6 ? null : (i.getUTF8String(2));
/* 061 */     boolean isNull5 = isNull6;
/* 062 */     final java.sql.Timestamp value5 = isNull5 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaTimestamp(value6);
/* 063 */     isNull5 = value5 == null;
/* 064 */     if (isNull5) {
/* 065 */       values[2] = null;
/* 066 */     } else {
/* 067 */       values[2] = value5;
/* 068 */     }
/* 069 */
/* 070 */     final org.apache.spark.sql.Row value = new org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(values, schema);
/* 071 */     if (false) {
/* 072 */       mutableRow.setNullAt(0);
/* 073 */     } else {
/* 074 */
/* 075 */       mutableRow.update(0, value);
/* 076 */     }
/* 077 */
/* 078 */     return mutableRow;
/* 079 */   }
/* 080 */ }
```

**After**

```bash
[1969-12-31 00:00:00.0,1.0,1969-12-31 16:00:00.002]
[1969-12-31 00:00:00.0,4.0,1969-12-31 16:00:00.005]
[1969-12-31 16:00:00.002,10.5,string]
```

## How was this patch tested?

Unit tests in `DataFrameSuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15072 from HyukjinKwon/SPARK-17123.
2016-10-22 20:09:04 +02:00
Eric Liang 3eca283aca [SPARK-17994][SQL] Add back a file status cache for catalog tables
## What changes were proposed in this pull request?

In SPARK-16980, we removed the full in-memory cache of table partitions in favor of loading only needed partitions from the metastore. This greatly improves the initial latency of queries that only read a small fraction of table partitions.

However, since the metastore does not store file statistics, we need to discover those from remote storage. With the loss of the in-memory file status cache this has to happen on each query, increasing the latency of repeated queries over the same partitions.

The proposal is to add back a per-table cache of partition contents, i.e. Map[Path, Array[FileStatus]]. This cache would be retained per-table, and can be invalidated through refreshTable() and refreshByPath(). Unlike the prior cache, it can be incrementally updated as new partitions are read.

## How was this patch tested?

Existing tests and new tests in `HiveTablePerfStatsSuite`.

cc mallman

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15539 from ericl/meta-cache.
2016-10-22 22:08:28 +08:00
Sean Owen 7178c56433 [SPARK-16606][MINOR] Tiny follow-up to , to correct more instances of the same log message typo
## What changes were proposed in this pull request?

Tiny follow-up to SPARK-16606 / https://github.com/apache/spark/pull/14533 , to correct more instances of the same log message typo

## How was this patch tested?

Existing tests (no functional change anyway)

Author: Sean Owen <sowen@cloudera.com>

Closes #15586 from srowen/SPARK-16606.2.
2016-10-21 22:20:52 -07:00
Reynold Xin 3fbf5a58c2 [SPARK-18042][SQL] OutputWriter should expose file path written
## What changes were proposed in this pull request?
This patch adds a new "path" method on OutputWriter that returns the path of the file written by the OutputWriter. This is part of the necessary work to consolidate structured streaming and batch write paths.

The batch write path has a nice feature that each data source can define the extension of the files, and allow Spark to specify the staging directory and the prefix for the files. However, in the streaming path we need to collect the list of files written, and there is no interface right now to do that.

## How was this patch tested?
N/A - there is no behavior change and this should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15580 from rxin/SPARK-18042.
2016-10-21 17:27:18 -07:00
Wenchen Fan 140570252f [SPARK-18044][STREAMING] FileStreamSource should not infer partitions in every batch
## What changes were proposed in this pull request?

In `FileStreamSource.getBatch`, we will create a `DataSource` with specified schema, to avoid inferring the schema again and again. However, we don't pass the partition columns, and will infer the partition again and again.

This PR fixes it by keeping the partition columns in `FileStreamSource`, like schema.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15581 from cloud-fan/stream.
2016-10-21 15:28:16 -07:00
Tathagata Das 7a531e3054 [SPARK-17926][SQL][STREAMING] Added json for statuses
## What changes were proposed in this pull request?

StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`.

## How was this patch tested?
New unit tests

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #15476 from tdas/SPARK-17926.
2016-10-21 13:07:29 -07:00
Zheng RuiFeng a8ea4da8d0
[SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.

cc srowen

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15564 from zhengruifeng/avoid_0_length_array.
2016-10-21 09:49:37 +01:00
Wenchen Fan 57e97fcbd6 [SPARK-18029][SQL] PruneFileSourcePartitions should not change the output of LogicalRelation
## What changes were proposed in this pull request?

In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it.

## How was this patch tested?

the new `PruneFileSourcePartitionsSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15569 from cloud-fan/partition-bug.
2016-10-21 12:27:53 +08:00
Shixiong Zhu 1bb99c4887 [SPARK-18030][TESTS] Adds more checks to collect more info about FileStreamSourceSuite failure
## What changes were proposed in this pull request?

My hunch is `mkdirs` fails. Just add more checks to collect more info.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15577 from zsxwing/SPARK-18030-debug.
2016-10-20 20:44:32 -07:00
Reynold Xin 7f9ec19eae [SPARK-18021][SQL] Refactor file name specification for data sources
## What changes were proposed in this pull request?
Currently each data source OutputWriter is responsible for specifying the entire file name for each file output. This, however, does not make any sense because we rely on file naming schemes for certain behaviors in Spark SQL, e.g. bucket id. The current approach allows individual data sources to break the implementation of bucketing.

On the flip side, we also don't want to move file naming entirely out of data sources, because different data sources do want to specify different extensions.

This patch divides file name specification into two parts: the first part is a prefix specified by the caller of OutputWriter (in WriteOutput), and the second part is the suffix that can be specified by the OutputWriter itself. Note that a side effect of this change is that now all file based data sources also support bucketing automatically.

There are also some other minor cleanups:

- Removed the UUID passed through generic Configuration string
- Some minor rewrites for better clarity
- Renamed "path" in multiple places to "stagingDir", to more accurately reflect its meaning

## How was this patch tested?
This should be covered by existing data source tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15562 from rxin/SPARK-18021.
2016-10-20 12:18:56 -07:00
Koert Kuipers 84b245f2dd [SPARK-15780][SQL] Support mapValues on KeyValueGroupedDataset
## What changes were proposed in this pull request?

Add mapValues to KeyValueGroupedDataset

## How was this patch tested?

New test in DatasetSuite for groupBy function, mapValues, flatMap

Author: Koert Kuipers <koert@tresata.com>

Closes #13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues.
2016-10-20 10:08:12 -07:00
Tejas Patil fb0894b3a8 [SPARK-17698][SQL] Join predicates should not contain filter clauses
## What changes were proposed in this pull request?

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

`ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below:

[0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91

eg.

```
val df = (1 until 10).toDF("id").coalesce(1)
hc.sql("DROP TABLE IF EXISTS table1").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1")
hc.sql("DROP TABLE IF EXISTS table2").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2")

sqlContext.sql("""
  SELECT a.id, b.id
  FROM table1 a
  FULL OUTER JOIN table2 b
  ON a.id = b.id AND a.id='1' AND b.id='1'
""").explain(true)
```

BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job.

```
SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter
:- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200)
:     +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200)
      +- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

AFTER :

```
SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0))
:- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

## How was this patch tested?

- Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses`
- Ran all the tests in `BucketedReadSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause.
2016-10-20 09:50:55 -07:00
Dilip Biswal e895bc2548 [SPARK-17860][SQL] SHOW COLUMN's database conflict check should respect case sensitivity configuration
## What changes were proposed in this pull request?
SHOW COLUMNS command validates the user supplied database
name with database name from qualified table name name to make
sure both of them are consistent. This comparison should respect
case sensitivity.

## How was this patch tested?
Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure.

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

Closes #15423 from dilipbiswal/dkb_show_column_fix.
2016-10-20 19:39:25 +08:00
Dongjoon Hyun 986a3b8b5b
[SPARK-17796][SQL] Support wildcard character in filename for LOAD DATA LOCAL INPATH
## What changes were proposed in this pull request?

Currently, Spark 2.0 raises an `input path does not exist` AnalysisException if the file name contains '*'. It is misleading since it occurs when there exist some matched files. Also, it was a supported feature in Spark 1.6.2. This PR aims to support wildcard characters in filename for `LOAD DATA LOCAL INPATH` SQL command like Spark 1.6.2.

**Reported Error Scenario**
```scala
scala> sql("CREATE TABLE t(a string)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("LOAD DATA LOCAL INPATH '/tmp/x*' INTO TABLE t")
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /tmp/x*;
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15376 from dongjoon-hyun/SPARK-17796.
2016-10-20 09:53:12 +01:00
Eric Liang 4bd17c4606 [SPARK-17991][SQL] Enable metastore partition pruning by default.
## What changes were proposed in this pull request?

This should apply to non-converted metastore relations. WIP to see if this causes any test failures.

## How was this patch tested?

Existing tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15475 from ericl/try-enabling-pruning.
2016-10-19 23:55:05 -07:00
Reynold Xin f313117bc9 [SPARK-18012][SQL] Simplify WriterContainer
## What changes were proposed in this pull request?
This patch refactors WriterContainer to simplify the logic and make control flow more obvious.The previous code setup made it pretty difficult to track the actual dependencies on variables and setups because the driver side and the executor side were using the same set of variables.

## How was this patch tested?
N/A - this should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15551 from rxin/writercontainer-refactor.
2016-10-19 22:22:35 -07:00
hyukjinkwon 4b2011ec9d [SPARK-17989][SQL] Check ascendingOrder type in sort_array function rather than throwing ClassCastException
## What changes were proposed in this pull request?

This PR proposes to check the second argument, `ascendingOrder`  rather than throwing `ClassCastException` exception message.

```sql
select sort_array(array('b', 'd'), '1');
```

**Before**

```
16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')]
java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean
	at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85)
	at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185)
	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297)
```

**After**

```
Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7;
```

## How was this patch tested?

Unit test in `DataFrameFunctionsSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15532 from HyukjinKwon/SPARK-17989.
2016-10-19 19:36:21 -07:00
Wenchen Fan 4329c5cea4 [SPARK-17873][SQL] ALTER TABLE RENAME TO should allow users to specify database in destination table name(but have to be same as source table)
## What changes were proposed in this pull request?

Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`.

However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO.

This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc.

## How was this patch tested?

The added back tests and some new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15434 from cloud-fan/revert.
2016-10-18 20:23:13 -07:00
Eric Liang 5f20ae0394 [SPARK-17980][SQL] Fix refreshByPath for converted Hive tables
## What changes were proposed in this pull request?

There was a bug introduced in https://github.com/apache/spark/pull/14690 which broke refreshByPath with converted hive tables (though, it turns out it was very difficult to refresh converted hive tables anyways, since you had to specify the exact path of one of the partitions).

This changes refreshByPath to invalidate by prefix instead of exact match, and fixes the issue.

cc sameeragarwal for refreshByPath changes
mallman

## How was this patch tested?

Extended unit test.

Author: Eric Liang <ekl@databricks.com>

Closes #15521 from ericl/fix-caching.
2016-10-19 10:20:12 +08:00
Tathagata Das 941b3f9aca [SPARK-17731][SQL][STREAMING][FOLLOWUP] Refactored StreamingQueryListener APIs
## What changes were proposed in this pull request?

As per rxin request, here are further API changes
- Changed `Stream(Started/Progress/Terminated)` events to `Stream*Event`
- Changed the fields in `StreamingQueryListener.on***` from `query*` to `event`

## How was this patch tested?
Existing unit tests.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #15530 from tdas/SPARK-17731-1.
2016-10-18 17:32:16 -07:00
hyukjinkwon b3130c7b6a [SPARK-17955][SQL] Make DataFrameReader.jdbc call DataFrameReader.format("jdbc").load
## What changes were proposed in this pull request?

This PR proposes to make `DataFrameReader.jdbc` call `DataFrameReader.format("jdbc").load` consistently with other APIs in `DataFrameReader`/`DataFrameWriter` and avoid calling `sparkSession.baseRelationToDataFrame(..)` here and there.

The changes were mostly copied from `DataFrameWriter.jdbc()` which was recently updated.

```diff
-    val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
-    val options = new JDBCOptions(url, table, params)
-    val relation = JDBCRelation(parts, options)(sparkSession)
-    sparkSession.baseRelationToDataFrame(relation)
+    this.extraOptions = this.extraOptions ++ connectionProperties.asScala
+    // explicit url and dbtable should override all
+    this.extraOptions += ("url" -> url, "dbtable" -> table)
+    format("jdbc").load()
```

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15499 from HyukjinKwon/SPARK-17955.
2016-10-18 13:49:02 -07:00
Eric Liang 4ef39c2f44 [SPARK-17974] try 2) Refactor FileCatalog classes to simplify the inheritance tree
## What changes were proposed in this pull request?

This renames `BasicFileCatalog => FileCatalog`, combines  `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.

In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```

(note that this is a re-submission of https://github.com/apache/spark/pull/15518 which got reverted)

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #15533 from ericl/fix-scalastyle-revert.
2016-10-18 13:33:46 -07:00
hyukjinkwon 37686539f5 [SPARK-17388] [SQL] Support for inferring type date/timestamp/decimal for partition column
## What changes were proposed in this pull request?

Currently, Spark only supports to infer `IntegerType`, `LongType`, `DoubleType` and `StringType`.

`DecimalType` is being tried but it seems it never infers type as `DecimalType` as `DoubleType` is being tried first. Also, it seems `DateType` and `TimestampType` could be inferred.

As far as I know, it is pretty common to use both for a partition column.

This PR fixes the incorrect `DecimalType` try and also adds the support for both `DateType` and `TimestampType` for inferring partition column type.

## How was this patch tested?

Unit tests in `ParquetPartitionDiscoverySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14947 from HyukjinKwon/SPARK-17388.
2016-10-18 13:20:42 -07:00
Wenchen Fan e59df62e62 [SPARK-17899][SQL][FOLLOW-UP] debug mode should work for corrupted table
## What changes were proposed in this pull request?

Debug mode should work for corrupted table, so that we can really debug

## How was this patch tested?

new test in `MetastoreDataSourcesSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15528 from cloud-fan/debug.
2016-10-18 11:03:10 -07:00
Tathagata Das a9e79a41ee [SQL][STREAMING][TEST] Follow up to remove Option.contains for Scala 2.10 compatibility
## What changes were proposed in this pull request?

Scala 2.10 does not have Option.contains, which broke Scala 2.10 build.

## How was this patch tested?
Locally compiled and ran sql/core unit tests in 2.10

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #15531 from tdas/metrics-flaky-test-fix-1.
2016-10-18 02:29:55 -07:00
Liwei Lin 7d878cf2da [SQL][STREAMING][TEST] Fix flaky tests in StreamingQueryListenerSuite
This work has largely been done by lw-lin in his PR #15497. This is a slight refactoring of it.

## What changes were proposed in this pull request?
There were two sources of flakiness in StreamingQueryListener test.

- When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock.
```
+-----------------------------------+--------------------------------+
|      StreamExecution thread       |         testing thread         |
+-----------------------------------+--------------------------------+
|  ManualClock.waitTillTime(100) {  |                                |
|        _isWaiting = true          |                                |
|            wait(10)               |                                |
|        still in wait(10)          |  if (_isWaiting) advance(100)  |
|        still in wait(10)          |  if (_isWaiting) advance(200)  | <- this should be disallowed !
|        still in wait(10)          |  if (_isWaiting) advance(300)  | <- this should be disallowed !
|      wake up from wait(10)        |                                |
|       current time is 600         |                                |
|       _isWaiting = false          |                                |
|  }                                |                                |
+-----------------------------------+--------------------------------+
```

- Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger.

My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(200)` (see above) will wait for stream execution thread to complete the wait that started at time 0, and start a new wait at time 200 (i.e. time stamp after the previous `advance(100)`).

In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest.

## How was this patch tested?
Ran existing unit test MANY TIME in Jenkins

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Liwei Lin <lwlin7@gmail.com>

Closes #15519 from tdas/metrics-flaky-test-fix.
2016-10-18 00:49:57 -07:00
Reynold Xin 1c5a7d7f64 Revert "[SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree"
This reverts commit 8daa1a29b6.
2016-10-17 21:26:28 -07:00
Eric Liang 8daa1a29b6 [SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree
## What changes were proposed in this pull request?

This renames `BasicFileCatalog => FileCatalog`, combines  `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.

In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```

cc cloud-fan mallman

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #15518 from ericl/refactor-session-file-catalog.
2016-10-17 21:01:22 -07:00
Dilip Biswal 813ab5e025 [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables
## What changes were proposed in this pull request?
Reopens the closed PR https://github.com/apache/spark/pull/15190
(Please refer to the above link for review comments on the PR)

Make sure the hive.default.fileformat is used to when creating the storage format metadata.

Output
``` SQL
scala> spark.sql("SET hive.default.fileformat=orc")
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]

scala> spark.sql("CREATE TABLE tmp_default(id INT)")
res2: org.apache.spark.sql.DataFrame = []
```
Before
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]
```
After
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]

```
## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Added new tests to HiveDDLCommandSuite, SQLQuerySuite

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

Closes #15495 from dilipbiswal/orc2.
2016-10-17 20:46:30 -07:00
gatorsmile d88a1bae6a [SPARK-17751][SQL] Remove spark.sql.eagerAnalysis and Output the Plan if Existed in AnalysisException
### What changes were proposed in this pull request?
Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it.

This PR also outputs the plan. Without the fix, the analysis error is like
```
cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12
```

After the fix, the analysis error becomes:
```
org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12;
'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6]
+- SubqueryAlias t
   +- Project [_1#2 AS k#5, _2#3 AS v#6]
      +- LocalRelation [_1#2, _2#3]
```

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15316 from gatorsmile/eagerAnalysis.
2016-10-17 11:33:06 -07:00
Sital Kedia c7ac027d5f [SPARK-17839][CORE] Use Nio's directbuffer instead of BufferedInputStream in order to avoid additional copy from os buffer cache to user buffer
## What changes were proposed in this pull request?

Currently we use BufferedInputStream to read the shuffle file which copies the file content from os buffer cache to the user buffer. This adds additional latency in reading the spill files. We made a change to use java nio's direct buffer to read the spill files and for certain pipelines spilling significant amount of data, we see up to 7% speedup for the entire pipeline.

## How was this patch tested?
Tested by running the job in the cluster and observed up to 7% speedup.

Author: Sital Kedia <skedia@fb.com>

Closes #15408 from sitalkedia/skedia/nio_spill_read.
2016-10-17 11:03:04 -07:00
Weiqing Yang 56b0f5f4d1 [MINOR][SQL] Add prettyName for current_database function
## What changes were proposed in this pull request?
Added a `prettyname` for current_database function.

## How was this patch tested?
Manually.

Before:
```
scala> sql("select current_database()").show
+-----------------+
|currentdatabase()|
+-----------------+
|          default|
+-----------------+
```

After:
```
scala> sql("select current_database()").show
+------------------+
|current_database()|
+------------------+
|           default|
+------------------+
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15506 from weiqingy/prettyName.
2016-10-16 22:38:30 -07:00
gatorsmile e18d02c5a8 [SPARK-17947][SQL] Add Doc and Comment about spark.sql.debug
### What changes were proposed in this pull request?
Just document the impact of `spark.sql.debug`:

When enabling the debug, Spark SQL internal table properties are not filtered out; however, some related DDL commands (e.g., Analyze Table and CREATE TABLE LIKE) might not work properly.

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15494 from gatorsmile/addDocForSQLDebug.
2016-10-17 12:08:25 +08:00
Dongjoon Hyun 59e3eb5af8 [SPARK-17819][SQL] Support default database in connection URIs for Spark Thrift Server
## What changes were proposed in this pull request?

Currently, Spark Thrift Server ignores the default database in URI. This PR supports that like the following.

```sql
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "create database testdb"
$ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "create table t(a int)"
$ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "show tables"
...
+------------+--------------+--+
| tableName  | isTemporary  |
+------------+--------------+--+
| t          | false        |
+------------+--------------+--+
1 row selected (0.347 seconds)
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "show tables"
...
+------------+--------------+--+
| tableName  | isTemporary  |
+------------+--------------+--+
+------------+--------------+--+
No rows selected (0.098 seconds)
```

## How was this patch tested?

Manual.

Note: I tried to add a test case for this, but I cannot found a suitable testsuite for this. I'll add the testcase if some advice is given.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15399 from dongjoon-hyun/SPARK-17819.
2016-10-16 20:15:32 -07:00
Jun Kim 36d81c2c68 [SPARK-17953][DOCUMENTATION] Fix typo in SparkSession scaladoc
## What changes were proposed in this pull request?

### Before:
```scala
SparkSession.builder()
     .master("local")
     .appName("Word Count")
     .config("spark.some.config.option", "some-value").
     .getOrCreate()
```

### After:
```scala
SparkSession.builder()
     .master("local")
     .appName("Word Count")
     .config("spark.some.config.option", "some-value")
     .getOrCreate()
```

There was one unexpected dot!

Author: Jun Kim <i2r.jun@gmail.com>

Closes #15498 from tae-jun/SPARK-17953.
2016-10-15 00:36:55 -07:00
Michael Allman 6ce1b675ee [SPARK-16980][SQL] Load only catalog table partition metadata required to answer a query
(This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.)

## What changes were proposed in this pull request?

In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference.

If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild.

In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space.

This PR proposes an alternative approach. Basically, it makes four changes:

1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates.
1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates.
1. It removes partition loading and caching from `HiveMetastoreCatalog`.
1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog.

The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters.

As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted.

## Open Issues

1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR.
1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue.
1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`.
1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly.

## How was this patch tested?

The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded.

Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #14690 from mallman/spark-16980-lazy_partition_fetching.
2016-10-14 18:26:18 -07:00
Srinath Shankar 2d96d35dc0 [SPARK-17946][PYSPARK] Python crossJoin API similar to Scala
## What changes were proposed in this pull request?

Add a crossJoin function to the DataFrame API similar to that in Scala. Joins with no condition (cartesian products) must be specified with the crossJoin API

## How was this patch tested?
Added python tests to ensure that an AnalysisException if a cartesian product is specified without crossJoin(), and that cartesian products can execute if specified via crossJoin()

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

Author: Srinath Shankar <srinath@databricks.com>

Closes #15493 from srinathshankar/crosspython.
2016-10-14 18:24:47 -07:00
Reynold Xin 72adfbf94a [SPARK-17900][SQL] Graduate a list of Spark SQL APIs to stable
## What changes were proposed in this pull request?
This patch graduates a list of Spark SQL APIs and mark them stable.

The following are marked stable:

Dataset/DataFrame
- functions, since 1.3
- ColumnName, since 1.3
- DataFrameNaFunctions, since 1.3.1
- DataFrameStatFunctions, since 1.4
- UserDefinedFunction, since 1.3
- UserDefinedAggregateFunction, since 1.5
- Window and WindowSpec, since 1.4

Data sources:
- DataSourceRegister, since 1.5
- RelationProvider, since 1.3
- SchemaRelationProvider, since 1.3
- CreatableRelationProvider, since 1.3
- BaseRelation, since 1.3
- TableScan, since 1.3
- PrunedScan, since 1.3
- PrunedFilteredScan, since 1.3
- InsertableRelation, since 1.3

The following are kept experimental / evolving:

Data sources:
- CatalystScan (tied to internal logical plans so it is not stable by definition)

Structured streaming:
- all classes (introduced new in 2.0 and will likely change)

Dataset typed operations (introduced in 1.6 and 2.0 and might change, although probability is low)
- all typed methods on Dataset
- KeyValueGroupedDataset
- o.a.s.sql.expressions.javalang.typed
- o.a.s.sql.expressions.scalalang.typed
- methods that return typed Dataset in SparkSession

We should discuss more whether we want to mark Dataset typed operations stable in 2.1.

## How was this patch tested?
N/A - just annotation changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #15469 from rxin/SPARK-17900.
2016-10-14 16:13:42 -07:00
Jeff Zhang f00df40cfe [SPARK-11775][PYSPARK][SQL] Allow PySpark to register Java UDF
Currently pyspark can only call the builtin java UDF, but can not call custom java UDF. It would be better to allow that. 2 benefits:
* Leverage the power of rich third party java library
* Improve the performance. Because if we use python UDF, python daemons will be started on worker which will affect the performance.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #9766 from zjffdu/SPARK-11775.
2016-10-14 15:50:35 -07:00
Nick Pentreath 5aeb7384c7 [SPARK-16063][SQL] Add storageLevel to Dataset
[SPARK-11905](https://issues.apache.org/jira/browse/SPARK-11905) added support for `persist`/`cache` for `Dataset`. However, there is no user-facing API to check if a `Dataset` is cached and if so what the storage level is. This PR adds `getStorageLevel` to `Dataset`, analogous to `RDD.getStorageLevel`.

Updated `DatasetCacheSuite`.

Author: Nick Pentreath <nickp@za.ibm.com>

Closes #13780 from MLnick/ds-storagelevel.

Signed-off-by: Michael Armbrust <michael@databricks.com>
2016-10-14 15:09:49 -07:00
Davies Liu da9aeb0fde [SPARK-17863][SQL] should not add column into Distinct
## What changes were proposed in this pull request?

We are trying to resolve the attribute in sort by pulling up some column for grandchild into child, but that's wrong when the child is Distinct, because the added column will change the behavior of Distinct, we should not do that.

## How was this patch tested?

Added regression test.

Author: Davies Liu <davies@databricks.com>

Closes #15489 from davies/order_distinct.
2016-10-14 14:45:20 -07:00
Yin Huai 522dd0d0e5 Revert "[SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables"
This reverts commit 7ab86244e3.
2016-10-14 14:09:35 -07:00
Dilip Biswal 7ab86244e3 [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables
## What changes were proposed in this pull request?
Make sure the hive.default.fileformat is used to when creating the storage format metadata.

Output
``` SQL
scala> spark.sql("SET hive.default.fileformat=orc")
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]

scala> spark.sql("CREATE TABLE tmp_default(id INT)")
res2: org.apache.spark.sql.DataFrame = []
```
Before
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]
```
After
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]

```

## How was this patch tested?
Added new tests to HiveDDLCommandSuite

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

Closes #15190 from dilipbiswal/orc.
2016-10-14 13:22:59 -07:00
Tathagata Das 05800b4b4e [TEST] Ignore flaky test in StreamingQueryListenerSuite
## What changes were proposed in this pull request?

Ignoring the flaky test introduced in #15307

https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/1736/testReport/junit/org.apache.spark.sql.streaming/StreamingQueryListenerSuite/single_listener__check_trigger_statuses/

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #15491 from tdas/metrics-flaky-test.
2016-10-14 12:39:25 -07:00
Andrew Ash fa37877af0
Typo: form -> from
## What changes were proposed in this pull request?

Minor typo fix

## How was this patch tested?

Existing unit tests on Jenkins

Author: Andrew Ash <andrew@andrewash.com>

Closes #15486 from ash211/patch-8.
2016-10-14 18:13:19 +01:00
wangzhenhua 7486442fe0 [SPARK-17073][SQL][FOLLOWUP] generate column-level statistics
## What changes were proposed in this pull request?
This pr adds some test cases for statistics: case sensitive column names, non ascii column names, refresh table, and also improves some documentation.

## How was this patch tested?
add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15360 from wzhfy/colStats2.
2016-10-14 21:18:49 +08:00
Wenchen Fan 2fb12b0a33 [SPARK-17903][SQL] MetastoreRelation should talk to external catalog instead of hive client
## What changes were proposed in this pull request?

`HiveExternalCatalog` should be the only interface to talk to the hive metastore. In `MetastoreRelation` we can just use `ExternalCatalog` instead of `HiveClient` to interact with hive metastore,  and add missing API in `ExternalCatalog`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15460 from cloud-fan/relation.
2016-10-14 15:53:50 +08:00
Reynold Xin 6c29b3de76 [SPARK-17925][SQL] Break fileSourceInterfaces.scala into multiple pieces
## What changes were proposed in this pull request?
This patch does a few changes to the file structure of data sources:

- Break fileSourceInterfaces.scala into multiple pieces (HadoopFsRelation, FileFormat, OutputWriter)
- Move ParquetOutputWriter into its own file

I created this as a separate patch so it'd be easier to review my future PRs that focus on refactoring this internal logic. This patch only moves code around, and has no logic changes.

## How was this patch tested?
N/A - should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15473 from rxin/SPARK-17925.
2016-10-14 14:14:52 +08:00
Reynold Xin 8543996c3f [SPARK-17927][SQL] Remove dead code in WriterContainer.
## What changes were proposed in this pull request?
speculationEnabled and DATASOURCE_OUTPUTPATH seem like just dead code.

## How was this patch tested?
Tests should fail if they are not dead code.

Author: Reynold Xin <rxin@databricks.com>

Closes #15477 from rxin/SPARK-17927.
2016-10-14 12:35:59 +08:00
Jakob Odersky 9dc0ca060d [SPARK-17368][SQL] Add support for value class serialization and deserialization
## What changes were proposed in this pull request?
Value classes were unsupported because catalyst data types were
obtained through reflection on erased types, which would resolve to a
value class' wrapped type and hence lead to unavailable methods during
code generation.

E.g. the following class
```scala
case class Foo(x: Int) extends AnyVal
```
would be seen as an `int` in catalyst and will cause instance cast failures when generated java code tries to treat it as a `Foo`.

This patch simply removes the erasure step when getting data types for
catalyst.

## How was this patch tested?
Additional tests in `ExpressionEncoderSuite`.

Author: Jakob Odersky <jakob@odersky.com>

Closes #15284 from jodersky/value-classes.
2016-10-13 17:48:09 -07:00
petermaxlee adc112429d [SPARK-17661][SQL] Consolidate various listLeafFiles implementations
## What changes were proposed in this pull request?
There are 4 listLeafFiles-related functions in Spark:

- ListingFileCatalog.listLeafFiles (which calls HadoopFsRelation.listLeafFilesInParallel if the number of paths passed in is greater than a threshold; if it is lower, then it has its own serial version implemented)
- HadoopFsRelation.listLeafFiles (called only by HadoopFsRelation.listLeafFilesInParallel)
- HadoopFsRelation.listLeafFilesInParallel (called only by ListingFileCatalog.listLeafFiles)

It is actually very confusing and error prone because there are effectively two distinct implementations for the serial version of listing leaf files. As an example, SPARK-17599 updated only one of the code path and ignored the other one.

This code can be improved by:

- Move all file listing code into ListingFileCatalog, since it is the only class that needs this.
- Keep only one function for listing files in serial.

## How was this patch tested?
This change should be covered by existing unit and integration tests. I also moved a test case for HadoopFsRelation.shouldFilterOut from HadoopFsRelationSuite to ListingFileCatalogSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #15235 from petermaxlee/SPARK-17661.
2016-10-13 14:16:39 -07:00
Tathagata Das 7106866c22 [SPARK-17731][SQL][STREAMING] Metrics for structured streaming
## What changes were proposed in this pull request?

Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics.
https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing

Specifically, this PR adds the following public APIs changes.

### New APIs
- `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later)

- `StreamingQueryStatus` has the following important fields
  - inputRate - Current rate (rows/sec) at which data is being generated by all the sources
  - processingRate - Current rate (rows/sec) at which the query is processing data from
                                  all the sources
  - ~~outputRate~~ - *Does not work with wholestage codegen*
  - latency - Current average latency between the data being available in source and the sink writing the corresponding output
  - sourceStatuses: Array[SourceStatus] - Current statuses of the sources
  - sinkStatus: SinkStatus - Current status of the sink
  - triggerStatus - Low-level detailed status of the last completed/currently active trigger
    - latencies - getOffset, getBatch, full trigger, wal writes
    - timestamps - trigger start, finish, after getOffset, after getBatch
    - numRows - input, output, state total/updated rows for aggregations

- `SourceStatus` has the following important fields
  - inputRate - Current rate (rows/sec) at which data is being generated by the source
  - processingRate - Current rate (rows/sec) at which the query is processing data from the source
  - triggerStatus - Low-level detailed status of the last completed/currently active trigger

- Python API for `StreamingQuery.status()`

### Breaking changes to existing APIs
**Existing direct public facing APIs**
- Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`.
  - Branch 2.0 should have it deprecated, master should have it removed.

**Existing advanced listener APIs**
- `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus`
   - Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status)

- Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`.

- Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`.

- For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java.

## How was this patch tested?

Old and new unit tests.
- Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite.
- New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite.
- New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite.
- Source-specific tests for making sure input rows are counted are is source-specific test suites.
- Additional tests to test minor additions in LocalTableScanExec, StateStore, etc.

Metrics also manually tested using Ganglia sink

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #15307 from tdas/SPARK-17731.
2016-10-13 13:36:26 -07:00
Pete Robbins 84f149e414 [SPARK-17827][SQL] maxColLength type should be Int for String and Binary
## What changes were proposed in this pull request?
correct the expected type from Length function to be Int

## How was this patch tested?
Test runs on little endian and big endian platforms

Author: Pete Robbins <robbinspg@gmail.com>

Closes #15464 from robbinspg/SPARK-17827.
2016-10-13 11:26:30 -07:00
Reynold Xin 04d417a7ca [SPARK-17830][SQL] Annotate remaining SQL APIs with InterfaceStability
## What changes were proposed in this pull request?
This patch annotates all the remaining APIs in SQL (excluding streaming) with InterfaceStability.

## How was this patch tested?
N/A - just annotation change.

Author: Reynold Xin <rxin@databricks.com>

Closes #15457 from rxin/SPARK-17830-2.
2016-10-13 11:12:30 -07:00
gatorsmile 0a8e51a5e4 [SPARK-17657][SQL] Disallow Users to Change Table Type
### What changes were proposed in this pull request?
Hive allows users to change the table type from `Managed` to `External` or from `External` to `Managed` by altering table's property `EXTERNAL`. See the JIRA: https://issues.apache.org/jira/browse/HIVE-1329

So far, Spark SQL does not correctly support it, although users can do it. Many assumptions are broken in the implementation. Thus, this PR is to disallow users to change it.

In addition, we also do not allow users to set the property `EXTERNAL` when creating a table.

### How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15230 from gatorsmile/alterTableSetExternal.
2016-10-13 21:36:39 +08:00
Wenchen Fan db8784feaa [SPARK-17899][SQL] add a debug mode to keep raw table properties in HiveExternalCatalog
## What changes were proposed in this pull request?

Currently `HiveExternalCatalog` will filter out the Spark SQL internal table properties, e.g. `spark.sql.sources.provider`, `spark.sql.sources.schema`, etc. This is reasonable for external users as they don't want to see these internal properties in `DESC TABLE`.

However, as a Spark developer, sometimes we do wanna see the raw table properties. This PR adds a new internal SQL conf, `spark.sql.debug`, to enable debug mode and keep these raw table properties.

This config can also be used in similar places where we wanna retain debug information in the future.

## How was this patch tested?

new test in MetastoreDataSourcesSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15458 from cloud-fan/debug.
2016-10-13 03:26:29 -04:00
buzhihuojie 7222a25a11 minor doc fix for Row.scala
## What changes were proposed in this pull request?

minor doc fix for "getAnyValAs" in class Row

## How was this patch tested?

None.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: buzhihuojie <ren.weiluo@gmail.com>

Closes #15452 from david-weiluo-ren/minorDocFixForRow.
2016-10-12 22:51:54 -07:00
Liang-Chi Hsieh 064d6650e9 [SPARK-17866][SPARK-17867][SQL] Fix Dataset.dropduplicates
## What changes were proposed in this pull request?

Two issues regarding Dataset.dropduplicates:

1. Dataset.dropDuplicates should consider the columns with same column name

    We find and get the first resolved attribute from output with the given column name in `Dataset.dropDuplicates`. When we have the more than one columns with the same name. Other columns are put into aggregation columns, instead of grouping columns.

2. Dataset.dropDuplicates should not change the output of child plan

    We create new `Alias` with new exprId in `Dataset.dropDuplicates` now. However it causes problem when we want to select the columns as follows:

        val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS()
        // ds("_2") will cause analysis exception
        ds.dropDuplicates("_1").select(ds("_1").as[String], ds("_2").as[Int])

Because the two issues are both related to `Dataset.dropduplicates` and the code changes are not big, so submitting them together as one PR.

## How was this patch tested?

Jenkins tests.

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

Closes #15427 from viirya/fix-dropduplicates.
2016-10-13 13:27:57 +08:00
Burak Yavuz edeb51a39d [SPARK-17876] Write StructuredStreaming WAL to a stream instead of materializing all at once
## What changes were proposed in this pull request?

The CompactibleFileStreamLog materializes the whole metadata log in memory as a String. This can cause issues when there are lots of files that are being committed, especially during a compaction batch.
You may come across stacktraces that look like:
```
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
at java.lang.StringCoding.encode(StringCoding.java:350)
at java.lang.String.getBytes(String.java:941)
at org.apache.spark.sql.execution.streaming.FileStreamSinkLog.serialize(FileStreamSinkLog.scala:127)

```
The safer way is to write to an output stream so that we don't have to materialize a huge string.

## How was this patch tested?

Existing unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15437 from brkyvz/ser-to-stream.
2016-10-12 21:40:45 -07:00
Reynold Xin 6f20a92ca3 [SPARK-17845] [SQL] More self-evident window function frame boundary API
## What changes were proposed in this pull request?
This patch improves the window function frame boundary API to make it more obvious to read and to use. The two high level changes are:

1. Create Window.currentRow, Window.unboundedPreceding, Window.unboundedFollowing to indicate the special values in frame boundaries. These methods map to the special integral values so we are not breaking backward compatibility here. This change makes the frame boundaries more self-evident (instead of Long.MinValue, it becomes Window.unboundedPreceding).

2. In Python, for any value less than or equal to JVM's Long.MinValue, treat it as Window.unboundedPreceding. For any value larger than or equal to JVM's Long.MaxValue, treat it as Window.unboundedFollowing. Before this change, if the user specifies any value that is less than Long.MinValue but not -sys.maxsize (e.g. -sys.maxsize + 1), the number we pass over to the JVM would overflow, resulting in a frame that does not make sense.

Code example required to specify a frame before this patch:
```
Window.rowsBetween(-Long.MinValue, 0)
```

While the above code should still work, the new way is more obvious to read:
```
Window.rowsBetween(Window.unboundedPreceding, Window.currentRow)
```

## How was this patch tested?
- Updated DataFrameWindowSuite (for Scala/Java)
- Updated test_window_functions_cumulative_sum (for Python)
- Renamed DataFrameWindowSuite DataFrameWindowFunctionsSuite to better reflect its purpose

Author: Reynold Xin <rxin@databricks.com>

Closes #15438 from rxin/SPARK-17845.
2016-10-12 16:45:10 -07:00
Imran Rashid 9ce7d3e542 [SPARK-17675][CORE] Expand Blacklist for TaskSets
## What changes were proposed in this pull request?

This is a step along the way to SPARK-8425.

To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)

Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.

Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).

## How was this patch tested?

Added unit tests, run tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>

Closes #15249 from squito/taskset_blacklist_only.
2016-10-12 16:43:03 -05:00
Shixiong Zhu 47776e7c0c [SPARK-17850][CORE] Add a flag to ignore corrupt files
## What changes were proposed in this pull request?

Add a flag to ignore corrupt files. For Spark core, the configuration is `spark.files.ignoreCorruptFiles`. For Spark SQL, it's `spark.sql.files.ignoreCorruptFiles`.

## How was this patch tested?

The added unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15422 from zsxwing/SPARK-17850.
2016-10-12 13:51:53 -07:00
prigarg d5580ebaa0 [SPARK-17884][SQL] To resolve Null pointer exception when casting from empty string to interval type.
## What changes were proposed in this pull request?
This change adds a check in castToInterval method of Cast expression , such that if converted value is null , then isNull variable should be set to true.

Earlier, the expression Cast(Literal(), CalendarIntervalType) was throwing NullPointerException because of the above mentioned reason.

## How was this patch tested?
Added test case in CastSuite.scala

jira entry for detail: https://issues.apache.org/jira/browse/SPARK-17884

Author: prigarg <prigarg@adobe.com>

Closes #15449 from priyankagargnitk/SPARK-17884.
2016-10-12 10:14:45 -07:00
Wenchen Fan b9a147181d [SPARK-17720][SQL] introduce static SQL conf
## What changes were proposed in this pull request?

SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897.

Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.

## How was this patch tested?

new tests in SQLConfSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15295 from cloud-fan/global-conf.
2016-10-11 20:27:08 -07:00
Liang-Chi Hsieh c8c090640a [SPARK-17821][SQL] Support And and Or in Expression Canonicalize
## What changes were proposed in this pull request?

Currently `Canonicalize` object doesn't support `And` and `Or`. So we can compare canonicalized form of predicates consistently. We should add the support.

## How was this patch tested?

Jenkins tests.

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

Closes #15388 from viirya/canonicalize-and-or.
2016-10-11 16:06:40 +08:00
Reynold Xin 3694ba48f0 [SPARK-17864][SQL] Mark data type APIs as stable (not DeveloperApi)
## What changes were proposed in this pull request?
The data type API has not been changed since Spark 1.3.0, and is ready for graduation. This patch marks them as stable APIs using the new InterfaceStability annotation.

This patch also looks at the various files in the catalyst module (not the "package") and marks the remaining few classes appropriately as well.

## How was this patch tested?
This is an annotation change. No functional changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #15426 from rxin/SPARK-17864.
2016-10-11 15:35:52 +08:00
Wenchen Fan 7388ad94d7 [SPARK-17338][SQL][FOLLOW-UP] add global temp view
## What changes were proposed in this pull request?

address post hoc review comments for https://github.com/apache/spark/pull/14897

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15424 from cloud-fan/global-temp-view.
2016-10-11 15:21:28 +08:00
Reynold Xin b515768f26 [SPARK-17844] Simplify DataFrame API for defining frame boundaries in window functions
## What changes were proposed in this pull request?
When I was creating the example code for SPARK-10496, I realized it was pretty convoluted to define the frame boundaries for window functions when there is no partition column or ordering column. The reason is that we don't provide a way to create a WindowSpec directly with the frame boundaries. We can trivially improve this by adding rowsBetween and rangeBetween to Window object.

As an example, to compute cumulative sum using the natural ordering, before this pr:
```
df.select('key, sum("value").over(Window.partitionBy(lit(1)).rowsBetween(Long.MinValue, 0)))
```

After this pr:
```
df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0)))
```

Note that you could argue there is no point specifying a window frame without partitionBy/orderBy -- but it is strange that only rowsBetween and rangeBetween are not the only two APIs not available.

This also fixes https://issues.apache.org/jira/browse/SPARK-17656 (removing _root_.scala).

## How was this patch tested?
Added test cases to compute cumulative sum in DataFrameWindowSuite for Scala/Java and tests.py for Python.

Author: Reynold Xin <rxin@databricks.com>

Closes #15412 from rxin/SPARK-17844.
2016-10-10 22:33:20 -07:00
hyukjinkwon 0c0ad436ad [SPARK-17719][SPARK-17776][SQL] Unify and tie up options in a single place in JDBC datasource package
## What changes were proposed in this pull request?

This PR proposes to fix arbitrary usages among `Map[String, String]`, `Properties` and `JDBCOptions` instances for options in `execution/jdbc` package and make the connection properties exclude Spark-only options.

This PR includes some changes as below:

  - Unify `Map[String, String]`, `Properties` and `JDBCOptions` in `execution/jdbc` package to `JDBCOptions`.

- Move `batchsize`, `fetchszie`, `driver` and `isolationlevel` options into `JDBCOptions` instance.

- Document `batchSize` and `isolationlevel` with marking both read-only options and write-only options. Also, this includes minor types and detailed explanation for some statements such as url.

- Throw exceptions fast by checking arguments first rather than in execution time (e.g. for `fetchsize`).

- Exclude Spark-only options in connection properties.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15292 from HyukjinKwon/SPARK-17719.
2016-10-10 22:22:41 -07:00
hyukjinkwon 90217f9dee [SPARK-16896][SQL] Handle duplicated field names in header consistently with null or empty strings in CSV
## What changes were proposed in this pull request?

Currently, CSV datasource allows to load duplicated empty string fields or fields having `nullValue` in the header. It'd be great if this can deal with normal fields as well.

This PR proposes handling the duplicates consistently with the existing behaviour with considering case-sensitivity (`spark.sql.caseSensitive`) as below:

data below:

```
fieldA,fieldB,,FIELDA,fielda,,
1,2,3,4,5,6,7
```

is parsed as below:

```scala
spark.read.format("csv").option("header", "true").load("test.csv").show()
```

- when `spark.sql.caseSensitive` is `false` (by default).

  ```
  +-------+------+---+-------+-------+---+---+
  |fieldA0|fieldB|_c2|FIELDA3|fieldA4|_c5|_c6|
  +-------+------+---+-------+-------+---+---+
  |      1|     2|  3|      4|      5|  6|  7|
  +-------+------+---+-------+-------+---+---+
  ```

- when `spark.sql.caseSensitive` is `true`.

  ```
  +-------+------+---+-------+-------+---+---+
  |fieldA0|fieldB|_c2| FIELDA|fieldA4|_c5|_c6|
  +-------+------+---+-------+-------+---+---+
  |      1|     2|  3|      4|      5|  6|  7|
  +-------+------+---+-------+-------+---+---+
  ```

**In more details**,

There is a good reference about this problem, `read.csv()` in R. So, I initially wanted to propose the similar behaviour.

In case of R,  the CSV data below:

```
fieldA,fieldB,,fieldA,fieldA,,
1,2,3,4,5,6,7
```

is parsed as below:

```r
test <- read.csv(file="test.csv",header=TRUE,sep=",")
> test
  fieldA fieldB X fieldA.1 fieldA.2 X.1 X.2
1      1      2 3        4        5   6   7
```

However, Spark CSV datasource already is handling duplicated empty strings and `nullValue` as field names. So the data below:

```
,,,fieldA,,fieldB,
1,2,3,4,5,6,7
```

is parsed as below:

```scala
spark.read.format("csv").option("header", "true").load("test.csv").show()
```
```
+---+---+---+------+---+------+---+
|_c0|_c1|_c2|fieldA|_c4|fieldB|_c6|
+---+---+---+------+---+------+---+
|  1|  2|  3|     4|  5|     6|  7|
+---+---+---+------+---+------+---+
```

R starts the number for each duplicate but Spark adds the number for its position for all fields for `nullValue` and empty strings.

In terms of case-sensitivity, it seems R is case-sensitive as below: (it seems it is not configurable).

```
a,a,a,A,A
1,2,3,4,5
```

is parsed as below:

```r
test <- read.csv(file="test.csv",header=TRUE,sep=",")
> test
  a a.1 a.2 A A.1
1 1   2   3 4   5
```

## How was this patch tested?

Unit test in `CSVSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14745 from HyukjinKwon/SPARK-16896.
2016-10-11 10:21:22 +08:00
Davies Liu d5ec4a3e01 [SPARK-17738][TEST] Fix flaky test in ColumnTypeSuite
## What changes were proposed in this pull request?

The default buffer size is not big enough for randomly generated MapType.

## How was this patch tested?

Ran the tests in 100 times, it never fail (it fail 8 times before the patch).

Author: Davies Liu <davies@databricks.com>

Closes #15395 from davies/flaky_map.
2016-10-10 19:14:01 -07:00
Reynold Xin 689de92005 [SPARK-17830] Annotate spark.sql package with InterfaceStability
## What changes were proposed in this pull request?
This patch annotates the InterfaceStability level for top level classes in o.a.spark.sql and o.a.spark.sql.util packages, to experiment with this new annotation.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #15392 from rxin/SPARK-17830.
2016-10-10 11:29:09 -07:00
jiangxingbo 7e16c94f18
[HOT-FIX][SQL][TESTS] Remove unused function in SparkSqlParserSuite
## What changes were proposed in this pull request?

The function `SparkSqlParserSuite.createTempViewUsing` is not used for now and causes build failure, this PR simply removes it.

## How was this patch tested?
N/A

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15418 from jiangxb1987/parserSuite.
2016-10-10 13:49:25 +01:00
Wenchen Fan 23ddff4b2b [SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?

Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

changes for `SessionCatalog`:

1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.

changes for SQL commands:

1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.

changes for other public API

1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`

## How was this patch tested?

new tests in `SQLViewSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 15:48:57 +08:00
jiangxingbo 16590030c1 [SPARK-17741][SQL] Grammar to parse top level and nested data fields separately
## What changes were proposed in this pull request?

Currently we use the same rule to parse top level and nested data fields. For example:
```
create table tbl_x(
  id bigint,
  nested struct<col1:string,col2:string>
)
```
Shows both syntaxes. In this PR we split this rule in a top-level and nested rule.

Before this PR,
```
sql("CREATE TABLE my_tab(column1: INT)")
```
works fine.
After this PR, it will throw a `ParseException`:
```
scala> sql("CREATE TABLE my_tab(column1: INT)")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27)
```

## How was this patch tested?
Add new testcases in `SparkSqlParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15346 from jiangxb1987/cdt.
2016-10-09 22:00:54 -07:00
jiangxingbo 26fbca4806 [SPARK-17832][SQL] TableIdentifier.quotedString creates un-parseable names when name contains a backtick
## What changes were proposed in this pull request?

The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example:
```
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1"))
parseTableIdentifier(complexName.unquotedString) // Does not work
parseTableIdentifier(complexName.quotedString) // Does not work
parseExpression(complexName.unquotedString) // Does not work
parseExpression(complexName.quotedString) // Does not work
```
We should handle the backtick properly to make `quotedString` parseable.

## How was this patch tested?
Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15403 from jiangxb1987/backtick.
2016-10-09 21:52:46 -07:00
Weiqing Yang 8a6bbe095b
[MINOR][SQL] Use resource path for test_script.sh
## What changes were proposed in this pull request?
This PR modified the test case `test("script")` to use resource path for `test_script.sh`. Make the test case portable (even in IntelliJ).

## How was this patch tested?
Passed the test case.
Before:
Run `test("script")` in IntelliJ:
```
Caused by: org.apache.spark.SparkException: Subprocess exited with status 127. Error: bash: src/test/resources/test_script.sh: No such file or directory
```
After:
Test passed.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15246 from weiqingy/hivetest.
2016-10-08 12:12:35 +01:00
hyukjinkwon 24850c9415 [HOTFIX][BUILD] Do not use contains in Option in JdbcRelationProvider
## What changes were proposed in this pull request?

This PR proposes the fix the use of `contains` API which only exists from Scala 2.11.

## How was this patch tested?

Manually checked:

```scala
scala> val o: Option[Boolean] = None
o: Option[Boolean] = None

scala> o == Some(false)
res17: Boolean = false

scala> val o: Option[Boolean] = Some(true)
o: Option[Boolean] = Some(true)

scala> o == Some(false)
res18: Boolean = false

scala> val o: Option[Boolean] = Some(false)
o: Option[Boolean] = Some(false)

scala> o == Some(false)
res19: Boolean = true
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15393 from HyukjinKwon/hotfix.
2016-10-07 17:59:24 -07:00
Davies Liu 94b24b84a6 [SPARK-17806] [SQL] fix bug in join key rewritten in HashJoin
## What changes were proposed in this pull request?

In HashJoin, we try to rewrite the join key as Long to improve the performance of finding a match. The rewriting part is not well tested, has a bug that could cause wrong result when there are at least three integral columns in the joining key also the total length of the key exceed 8 bytes.

## How was this patch tested?

Added unit test to covering the rewriting with different number of columns and different data types. Manually test the reported case and confirmed that this PR fix the bug.

Author: Davies Liu <davies@databricks.com>

Closes #15390 from davies/rewrite_key.
2016-10-07 15:03:47 -07:00
Herman van Hovell 97594c29b7 [SPARK-17761][SQL] Remove MutableRow
## What changes were proposed in this pull request?
In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`.

The code below illustrates the immutability issue with InternalRow:
```scala
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
val struct = new GenericMutableRow(1)
val row = InternalRow(struct, 1)
println(row)
scala> [[null], 1]
struct.setInt(0, 42)
println(row)
scala> [[42], 1]
```

This might be somewhat controversial, so feedback is appreciated.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15333 from hvanhovell/SPARK-17761.
2016-10-07 14:03:45 -07:00
Davies Liu 2badb58cdd [SPARK-15621][SQL] Support spilling for Python UDF
## What changes were proposed in this pull request?

When execute a Python UDF, we buffer the input row into as queue, then pull them out to join with the result from Python UDF. In the case that Python UDF is slow or the input row is too wide, we could ran out of memory because of the queue. Since we can't flush all the buffers (sockets) between JVM and Python process from JVM side, we can't limit the rows in the queue, otherwise it could deadlock.

This PR will manage the memory used by the queue, spill that into disk when there is no enough memory (also release the memory and disk space as soon as possible).

## How was this patch tested?

Added unit tests. Also manually ran a workload with large input row and slow python UDF (with  large broadcast) like this:

```
b = range(1<<24)
add = udf(lambda x: x + len(b), IntegerType())
df = sqlContext.range(1, 1<<26, 1, 4)
print df.select(df.id, lit("adf"*10000).alias("s"), add(df.id).alias("add")).groupBy(length("s")).sum().collect()
```

It ran out of memory (hang because of full GC) before the patch, ran smoothly after the patch.

Author: Davies Liu <davies@databricks.com>

Closes #15089 from davies/spill_udf.
2016-10-07 13:45:00 -07:00
Prashant Sharma bb1aaf28ec [SPARK-16411][SQL][STREAMING] Add textFile to Structured Streaming.
## What changes were proposed in this pull request?

Adds the textFile API which exists in DataFrameReader and serves same purpose.

## How was this patch tested?

Added corresponding testcase.

Author: Prashant Sharma <prashsh1@in.ibm.com>

Closes #14087 from ScrapCodes/textFile.
2016-10-07 11:16:24 -07:00
hyukjinkwon aa3a6841eb [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvider
## What changes were proposed in this pull request?

This PR proposes cleaning up the confusing part in `createRelation` as discussed in https://github.com/apache/spark/pull/12601/files#r80627940

Also, this PR proposes the changes below:

 - Add documentation for `batchsize` and `isolationLevel`.
 - Move property names into `JDBCOptions` so that they can be managed in a single place. which were, `fetchsize`, `batchsize`, `isolationLevel` and `driver`.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15263 from HyukjinKwon/SPARK-14525.
2016-10-07 10:52:32 -07:00
Sean Owen cff5607552 [SPARK-17707][WEBUI] Web UI prevents spark-submit application to be finished
## What changes were proposed in this pull request?

This expands calls to Jetty's simple `ServerConnector` constructor to explicitly specify a `ScheduledExecutorScheduler` that makes daemon threads. It should otherwise result in exactly the same configuration, because the other args are copied from the constructor that is currently called.

(I'm not sure we should change the Hive Thriftserver impl, but I did anyway.)

This also adds `sc.stop()` to the quick start guide example.

## How was this patch tested?

Existing tests; _pending_ at least manual verification of the fix.

Author: Sean Owen <sowen@cloudera.com>

Closes #15381 from srowen/SPARK-17707.
2016-10-07 10:31:41 -07:00
hyukjinkwon 2b01d3c701
[SPARK-16960][SQL] Deprecate approxCountDistinct, toDegrees and toRadians according to FunctionRegistry
## What changes were proposed in this pull request?

It seems `approxCountDistinct`, `toDegrees` and `toRadians` are also missed while matching the names to the ones in `FunctionRegistry`. (please see [approx_count_distinct](5c2ae79bfc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L244)), [degrees](5c2ae79bfc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L203)) and [radians](5c2ae79bfc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L222)) in `FunctionRegistry`).

I took a scan between `functions.scala` and `FunctionRegistry` and it seems these are all left. For `countDistinct` and `sumDistinct`, they are not registered in `FunctionRegistry`.

This PR deprecates `approxCountDistinct`, `toDegrees` and `toRadians` and introduces `approx_count_distinct`, `degrees` and `radians`.

## How was this patch tested?

Existing tests should cover this.

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

Closes #14538 from HyukjinKwon/SPARK-16588-followup.
2016-10-07 11:49:34 +01:00
Shixiong Zhu 9a48e60e63 [SPARK-17780][SQL] Report Throwable to user in StreamExecution
## What changes were proposed in this pull request?

When using an incompatible source for structured streaming, it may throw NoClassDefFoundError. It's better to just catch Throwable and report it to the user since the streaming thread is dying.

## How was this patch tested?

`test("NoClassDefFoundError from an incompatible source")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15352 from zsxwing/SPARK-17780.
2016-10-06 12:51:12 -07:00
Reynold Xin 79accf45ac [SPARK-17798][SQL] Remove redundant Experimental annotations in sql.streaming
## What changes were proposed in this pull request?
I was looking through API annotations to catch mislabeled APIs, and realized DataStreamReader and DataStreamWriter classes are already annotated as Experimental, and as a result there is no need to annotate each method within them.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #15373 from rxin/SPARK-17798.
2016-10-06 10:33:45 -07:00
Dongjoon Hyun 92b7e57280 [SPARK-17750][SQL] Fix CREATE VIEW with INTERVAL arithmetic.
## What changes were proposed in this pull request?

Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions.

```scala
scala> sql("CREATE TABLE dates (ts TIMESTAMP)")

scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates")
java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ...
```

## How was this patch tested?

Pass Jenkins with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15318 from dongjoon-hyun/SPARK-17750.
2016-10-06 09:42:30 -07:00
Shixiong Zhu b678e465af [SPARK-17346][SQL][TEST-MAVEN] Generate the sql test jar to fix the maven build
## What changes were proposed in this pull request?

Generate the sql test jar to fix the maven build

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15368 from zsxwing/sql-test-jar.
2016-10-05 18:11:31 -07:00
Shixiong Zhu 9293734d35 [SPARK-17346][SQL] Add Kafka source for Structured Streaming
## What changes were proposed in this pull request?

This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source.

It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing

tdas did most of work and part of them was inspired by koeninger's work.

### Introduction

The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows:

Column | Type
---- | ----
key | binary
value | binary
topic | string
partition | int
offset | long
timestamp | long
timestampType | int

The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic.

### Configuration

The user can use `DataStreamReader.option` to set the following configurations.

Kafka Source's options | value | default | meaning
------ | ------- | ------ | -----
startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off.
failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected.
subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors
fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets.
fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets

Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")`

### Usage

* Subscribe to 1 topic
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1")
  .load()
```

* Subscribe to multiple topics
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1,topic2")
  .load()
```

* Subscribe to a pattern
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribePattern", "topic.*")
  .load()
```

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Author: cody koeninger <cody@koeninger.org>

Closes #15102 from zsxwing/kafka-source.
2016-10-05 16:45:45 -07:00
Herman van Hovell 5fd54b994e [SPARK-17758][SQL] Last returns wrong result in case of empty partition
## What changes were proposed in this pull request?
The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order:
```
- Partition 1 [Row1, Row2]
- Partition 2 [Row3]
- Partition 3 []
```
In this case the `Last` function will currently return a null, instead of the value of `Row3`.

This PR fixes this by adding a `valueSet` flag to the `Last` function.

## How was this patch tested?
We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15348 from hvanhovell/SPARK-17758.
2016-10-05 16:05:30 -07:00
Dongjoon Hyun 6a05eb24d0 [SPARK-17328][SQL] Fix NPE with EXPLAIN DESCRIBE TABLE
## What changes were proposed in this pull request?

This PR fixes the following NPE scenario in two ways.

**Reported Error Scenario**
```scala
scala> sql("EXPLAIN DESCRIBE TABLE x").show(truncate = false)
INFO SparkSqlParser: Parsing command: EXPLAIN DESCRIBE TABLE x
java.lang.NullPointerException
```

- **DESCRIBE**: Extend `DESCRIBE` syntax to accept `TABLE`.
- **EXPLAIN**: Prevent NPE in case of the parsing failure of target statement, e.g., `EXPLAIN DESCRIBE TABLES x`.

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15357 from dongjoon-hyun/SPARK-17328.
2016-10-05 10:52:43 -07:00
Herman van Hovell 89516c1c4a [SPARK-17258][SQL] Parse scientific decimal literals as decimals
## What changes were proposed in this pull request?
Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals.

This implications in tests are limited to a single Hive compatibility test.

## How was this patch tested?
Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14828 from hvanhovell/SPARK-17258.
2016-10-04 23:48:26 -07:00
Tejas Patil a99743d053 [SPARK-17495][SQL] Add Hash capability semantically equivalent to Hive's
## What changes were proposed in this pull request?

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

Spark internally uses Murmur3Hash for partitioning. This is different from the one used by Hive. For queries which use bucketing this leads to different results if one tries the same query on both engines. For us, we want users to have backward compatibility to that one can switch parts of applications across the engines without observing regressions.

This PR includes `HiveHash`, `HiveHashFunction`, `HiveHasher` which mimics Hive's hashing at https://github.com/apache/hive/blob/master/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java#L638

I am intentionally not introducing any usages of this hash function in rest of the code to keep this PR small. My eventual goal is to have Hive bucketing support in Spark. Once this PR gets in, I will make hash function pluggable in relevant areas (eg. `HashPartitioning`'s `partitionIdExpression` has Murmur3 hardcoded : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala#L265)

## How was this patch tested?

Added `HiveHashSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15047 from tejasapatil/SPARK-17495_hive_hash.
2016-10-04 18:59:31 -07:00