Commit graph

5020 commits

Author SHA1 Message Date
Wenchen Fan 7c7fc30b4a [SPARK-19678][SQL] remove MetastoreRelation
## What changes were proposed in this pull request?

`MetastoreRelation` is used to represent table relation for hive tables, and provides some hive related information. We will resolve `SimpleCatalogRelation` to `MetastoreRelation` for hive tables, which is unnecessary as these 2 are the same essentially. This PR merges `SimpleCatalogRelation` and `MetastoreRelation`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17015 from cloud-fan/table-relation.
2017-02-28 09:24:36 -08:00
Yuming Wang 9b8eca65dc [SPARK-19660][CORE][SQL] Replace the configuration property names that are deprecated in the version of Hadoop 2.6
## What changes were proposed in this pull request?

Replace all the Hadoop deprecated configuration property names according to [DeprecatedProperties](https://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-common/DeprecatedProperties.html).

except:
https://github.com/apache/spark/blob/v2.1.0/python/pyspark/sql/tests.py#L1533
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala#L987
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala#L45
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L614

## How was this patch tested?

Existing tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16990 from wangyum/HadoopDeprecatedProperties.
2017-02-28 10:13:42 +00:00
windpiger a350bc16d3 [SPARK-19748][SQL] refresh function has a wrong order to do cache invalidate and regenerate the inmemory var for InMemoryFileIndex with FileStatusCache
## What changes were proposed in this pull request?

If we refresh a InMemoryFileIndex with a FileStatusCache, it will first use the FileStatusCache to re-generate the cachedLeafFiles etc, then call FileStatusCache.invalidateAll.

While the order to do these two actions is wrong, this lead to the refresh action does not take effect.

```
  override def refresh(): Unit = {
    refresh0()
    fileStatusCache.invalidateAll()
  }

  private def refresh0(): Unit = {
    val files = listLeafFiles(rootPaths)
    cachedLeafFiles =
      new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f)
    cachedLeafDirToChildrenFiles = files.toArray.groupBy(_.getPath.getParent)
    cachedPartitionSpec = null
  }
```
## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #17079 from windpiger/fixInMemoryFileIndexRefresh.
2017-02-28 00:16:49 -08:00
uncleGen 7353038353 [SPARK-19749][SS] Name socket source with a meaningful name
## What changes were proposed in this pull request?

Name socket source with a meaningful name

## How was this patch tested?

Jenkins

Author: uncleGen <hustyugm@gmail.com>

Closes #17082 from uncleGen/SPARK-19749.
2017-02-27 18:02:45 -08:00
hyukjinkwon 8a5a58506c [SPARK-15615][SQL][BUILD][FOLLOW-UP] Replace deprecated usage of json(RDD[String]) API
## What changes were proposed in this pull request?

This PR proposes to replace the deprecated `json(RDD[String])` usage to `json(Dataset[String])`.

This currently produces so many warnings.

## How was this patch tested?

Fixed tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17071 from HyukjinKwon/SPARK-15615-followup.
2017-02-27 14:33:02 -08:00
hyukjinkwon 4ba9c6c453 [MINOR][BUILD] Fix lint-java breaks in Java
## What changes were proposed in this pull request?

This PR proposes to fix the lint-breaks as below:

```
[ERROR] src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.network.buffer.ManagedBuffer.
[ERROR] src/main/java/org/apache/spark/unsafe/types/UTF8String.java:[156,10] (modifier) ModifierOrder: 'Nonnull' annotation modifier does not precede non-annotation modifiers.
[ERROR] src/main/java/org/apache/spark/SparkFirehoseListener.java:[122] (sizes) LineLength: Line is longer than 100 characters (found 105).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[164,78] (coding) OneStatementPerLine: Only one statement per line allowed.
[ERROR] src/test/java/test/org/apache/spark/JavaAPISuite.java:[1157] (sizes) LineLength: Line is longer than 100 characters (found 121).
[ERROR] src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java:[149] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/test/java/test/org/apache/spark/streaming/Java8APISuite.java:[146] (sizes) LineLength: Line is longer than 100 characters (found 122).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[32,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.Time.
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[611] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[1317] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java:[91] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[113] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[164] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[212] (sizes) LineLength: Line is longer than 100 characters (found 114).
[ERROR] src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java:[36] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java:[26,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[20,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[94] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.api.java.UDF1.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[72] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java:[121] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaRDD.
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaSparkContext.
```

## How was this patch tested?

Manually via

```bash
./dev/lint-java
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17072 from HyukjinKwon/java-lint.
2017-02-27 08:44:26 +00:00
Eyal Zituny 9f8e392159 [SPARK-19594][STRUCTURED STREAMING] StreamingQueryListener fails to handle QueryTerminatedEvent if more then one listeners exists
## What changes were proposed in this pull request?

currently if multiple streaming queries listeners exists, when a QueryTerminatedEvent is triggered, only one of the listeners will be invoked while the rest of the listeners will ignore the event.
this is caused since the the streaming queries listeners bus holds a set of running queries ids and when a termination event is triggered, after the first listeners is handling the event, the terminated query id is being removed from the set.
in this PR, the query id will be removed from the set only after all the listeners handles the event

## How was this patch tested?

a test with multiple listeners has been added to StreamingQueryListenerSuite

Author: Eyal Zituny <eyal.zituny@equalum.io>

Closes #16991 from eyalzit/master.
2017-02-26 15:57:32 -08:00
Dilip Biswal 68f2142cfd [SQL] Duplicate test exception in SQLQueryTestSuite due to meta files(.DS_Store) on Mac
## What changes were proposed in this pull request?
After adding the tests for subquery, we now have multiple level of directories under "sql-tests/inputs".  Some times on Mac while using Finder application it creates the meta data files called ".DS_Store". When these files are present at different levels in directory hierarchy, we get duplicate test exception while running the tests  as we just use the file name as the test case name. In this PR, we use the relative file path from the base directory along with the test file as the test name. Also after this change, we can have the same test file name under different directory like exists/basic.sql , in/basic.sql. Here is the truncated output of the test run after the change.

```SQL
info] SQLQueryTestSuite:
[info] - arithmetic.sql (5 seconds, 235 milliseconds)
[info] - array.sql (536 milliseconds)
[info] - blacklist.sql !!! IGNORED !!!
[info] - cast.sql (550 milliseconds)
....
....
....
[info] - union.sql (315 milliseconds)
[info] - subquery/.DS_Store !!! IGNORED !!!
[info] - subquery/exists-subquery/.DS_Store !!! IGNORED !!!
[info] - subquery/exists-subquery/exists-aggregate.sql (2 seconds, 451 milliseconds)
....
....
[info] - subquery/in-subquery/in-group-by.sql (12 seconds, 264 milliseconds)
....
....
[info] - subquery/scalar-subquery/scalar-subquery-predicate.sql (7 seconds, 769 milliseconds)
[info] - subquery/scalar-subquery/scalar-subquery-select.sql (4 seconds, 119 milliseconds)
```
Since this is a simple change, i haven't created a JIRA for it.
## How was this patch tested?
Manually verified. This is change to test infrastructure

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

Closes #17060 from dilipbiswal/sqlquerytestsuite.
2017-02-25 23:56:57 -08:00
Wenchen Fan 89608cf262 [SPARK-17075][SQL][FOLLOWUP] fix some minor issues and clean up the code
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/16395. It fixes some code style issues, naming issues, some missing cases in pattern match, etc.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17065 from cloud-fan/follow-up.
2017-02-25 23:01:44 -08:00
lvdongr fe07de9566 [SPARK-19673][SQL] "ThriftServer default app name is changed wrong"
## What changes were proposed in this pull request?
In spark 1.x ,the name of ThriftServer is SparkSQL:localHostName. While the ThriftServer default name is changed to the className of HiveThfift2 , which is not appropriate.

## How was this patch tested?
manual tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: lvdongr <lv.dongdong@zte.com.cn>

Closes #17010 from lvdongr/ThriftserverName.
2017-02-25 21:47:02 +00:00
Herman van Hovell 8f0511ed49 [SPARK-19650] Commands should not trigger a Spark job
Spark executes SQL commands eagerly. It does this by creating an RDD which contains the command's results. The downside to this is that any action on this RDD triggers a Spark job which is expensive and is unnecessary.

This PR fixes this by avoiding the materialization of an `RDD` for `Command`s; it just materializes the result and puts them in a `LocalRelation`.

Added a regression test to `SQLQuerySuite`.

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

Closes #17027 from hvanhovell/no-job-command.
2017-02-24 23:05:59 -08:00
Xiao Li 4cb025afaf [SPARK-19735][SQL] Remove HOLD_DDLTIME from Catalog APIs
### What changes were proposed in this pull request?
As explained in Hive JIRA https://issues.apache.org/jira/browse/HIVE-12224, HOLD_DDLTIME was broken as soon as it landed. Hive 2.0 removes HOLD_DDLTIME from the API. In Spark SQL, we always set it to FALSE. Like Hive, we should also remove it from our Catalog APIs.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17063 from gatorsmile/removalHoldDDLTime.
2017-02-24 23:03:59 -08:00
wangzhenhua 69d0da6373 [SPARK-17078][SQL] Show stats when explain
## What changes were proposed in this pull request?

Currently we can only check the estimated stats in logical plans by debugging. We need to provide an easier and more efficient way for developers/users.

In this pr, we add EXPLAIN COST command to show stats in the optimized logical plan.
E.g.
```
spark-sql> EXPLAIN COST select count(1) from store_returns;

...
== Optimized Logical Plan ==
Aggregate [count(1) AS count(1)#24L], Statistics(sizeInBytes=16.0 B, rowCount=1, isBroadcastable=false)
+- Project, Statistics(sizeInBytes=4.3 GB, rowCount=5.76E+8, isBroadcastable=false)
   +- Relation[sr_returned_date_sk#3,sr_return_time_sk#4,sr_item_sk#5,sr_customer_sk#6,sr_cdemo_sk#7,sr_hdemo_sk#8,sr_addr_sk#9,sr_store_sk#10,sr_reason_sk#11,sr_ticket_number#12,sr_return_quantity#13,sr_return_amt#14,sr_return_tax#15,sr_return_amt_inc_tax#16,sr_fee#17,sr_return_ship_cost#18,sr_refunded_cash#19,sr_reversed_charge#20,sr_store_credit#21,sr_net_loss#22] parquet, Statistics(sizeInBytes=28.6 GB, rowCount=5.76E+8, isBroadcastable=false)
...
```

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16594 from wzhfy/showStats.
2017-02-24 10:24:59 -08:00
Shuai Lin 05954f32e9 [SPARK-17075][SQL] Follow up: fix file line ending and improve the tests
## What changes were proposed in this pull request?

Fixed the line ending of `FilterEstimation.scala` (It's still using `\n\r`). Also improved the tests to cover the cases where the literals are on the left side of a binary operator.

## How was this patch tested?

Existing unit tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #17051 from lins05/fix-cbo-filter-file-encoding.
2017-02-24 10:24:01 -08:00
Tejas Patil 3e40f6c3d6 [SPARK-17495][SQL] Add more tests for hive hash
## What changes were proposed in this pull request?

This PR adds tests hive-hash by comparing the outputs generated against Hive 1.2.1. Following datatypes are covered by this PR:
- null
- boolean
- byte
- short
- int
- long
- float
- double
- string
- array
- map
- struct

Datatypes that I have _NOT_ covered but I will work on separately are:
- Decimal (handled separately in https://github.com/apache/spark/pull/17056)
- TimestampType
- DateType
- CalendarIntervalType

## How was this patch tested?

NA

Author: Tejas Patil <tejasp@fb.com>

Closes #17049 from tejasapatil/SPARK-17495_remaining_types.
2017-02-24 09:46:42 -08:00
jerryshao a920a43694 [SPARK-19038][YARN] Avoid overwriting keytab configuration in yarn-client
## What changes were proposed in this pull request?

Because yarn#client will reset the `spark.yarn.keytab` configuration to point to the location in distributed file, so if user still uses the old `SparkConf` to create `SparkSession` with Hive enabled, it will read keytab from the path in distributed cached. This is OK for yarn cluster mode, but in yarn client mode where driver is running out of container, it will be failed to fetch the keytab.

So here we should avoid reseting this configuration in the `yarn#client` and only overwriting it for AM, so using `spark.yarn.keytab` could get correct keytab path no matter running in client (keytab in local fs) or cluster (keytab in distributed cache) mode.

## How was this patch tested?

Verified in security cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #16923 from jerryshao/SPARK-19038.
2017-02-24 09:31:52 -08:00
windpiger 8f33731e79 [SPARK-19664][SQL] put hive.metastore.warehouse.dir in hadoopconf to overwrite its original value
## What changes were proposed in this pull request?

In [SPARK-15959](https://issues.apache.org/jira/browse/SPARK-15959), we bring back the `hive.metastore.warehouse.dir` , while in the logic, when use the value of  `spark.sql.warehouse.dir` to overwrite `hive.metastore.warehouse.dir` , it set it to `sparkContext.conf` which does not overwrite the value is hadoopConf, I think it should put in `sparkContext.hadoopConfiguration` and overwrite the original value of hadoopConf

https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L64

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #16996 from windpiger/hivemetawarehouseConf.
2017-02-23 22:57:23 -08:00
Ron Hu d7e43b613a [SPARK-17075][SQL] implemented filter estimation
## What changes were proposed in this pull request?

We traverse predicate and evaluate the logical expressions to compute the selectivity of a FILTER operator.

## How was this patch tested?

We add a new test suite to test various logical operators.

Author: Ron Hu <ron.hu@huawei.com>

Closes #16395 from ron8hu/filterSelectivity.
2017-02-23 20:18:21 -08:00
Carson Wang eff7b40890 [SPARK-19674][SQL] Ignore driver accumulator updates don't belong to the execution when merging all accumulator updates
## What changes were proposed in this pull request?
In SQLListener.getExecutionMetrics, driver accumulator updates don't belong to the execution should be ignored when merging all accumulator updates to prevent NoSuchElementException.

## How was this patch tested?
Updated unit test.

Author: Carson Wang <carson.wang@intel.com>

Closes #17009 from carsonwang/FixSQLMetrics.
2017-02-23 14:31:16 -08:00
Takeshi Yamamuro 09ed6e7711 [SPARK-18699][SQL] Put malformed tokens into a new field when parsing CSV data
## What changes were proposed in this pull request?
This pr added a logic to put malformed tokens into a new field when parsing CSV data  in case of permissive modes. In the current master, if the CSV parser hits these malformed ones, it throws an exception below (and then a job fails);
```
Caused by: java.lang.IllegalArgumentException
	at java.sql.Date.valueOf(Date.java:143)
	at org.apache.spark.sql.catalyst.util.DateTimeUtils$.stringToTime(DateTimeUtils.scala:137)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply$mcJ$sp(CSVInferSchema.scala:272)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
	at scala.util.Try.getOrElse(Try.scala:79)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:269)
	at
```
In case that users load large CSV-formatted data, the job failure makes users get some confused. So, this fix set NULL for original columns and put malformed tokens in a new field.

## How was this patch tested?
Added tests in `CSVSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #16928 from maropu/SPARK-18699-2.
2017-02-23 12:09:36 -08:00
Shixiong Zhu 9bf4e2baad [SPARK-19497][SS] Implement streaming deduplication
## What changes were proposed in this pull request?

This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.

The following cases are supported:

- one or multiple `dropDuplicates()` without aggregation (with or without watermark)
- `dropDuplicates` before aggregation

Not supported cases:

- `dropDuplicates` after aggregation

Breaking changes:
- `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16970 from zsxwing/dedup.
2017-02-23 11:25:39 -08:00
Herman van Hovell 78eae7e67f [SPARK-19459] Support for nested char/varchar fields in ORC
## What changes were proposed in this pull request?
This PR is a small follow-up on https://github.com/apache/spark/pull/16804. This PR also adds support for nested char/varchar fields in orc.

## How was this patch tested?
I have added a regression test to the OrcSourceSuite.

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

Closes #17030 from hvanhovell/SPARK-19459-follow-up.
2017-02-23 10:25:18 -08:00
Takeshi Yamamuro 93aa427159 [SPARK-19691][SQL] Fix ClassCastException when calculating percentile of decimal column
## What changes were proposed in this pull request?
This pr fixed a class-cast exception below;
```
scala> spark.range(10).selectExpr("cast (id as decimal) as x").selectExpr("percentile(x, 0.5)").collect()
 java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to java.lang.Number
	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:141)
	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:58)
	at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.update(interfaces.scala:514)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:187)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:181)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:151)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:109)
	at
```
This fix simply converts catalyst values (i.e., `Decimal`) into scala ones by using `CatalystTypeConverters`.

## How was this patch tested?
Added a test in `DataFrameSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17028 from maropu/SPARK-19691.
2017-02-23 16:28:36 +01:00
Takeshi Yamamuro 769aa0f1d2 [SPARK-19695][SQL] Throw an exception if a columnNameOfCorruptRecord field violates requirements in json formats
## What changes were proposed in this pull request?
This pr comes from #16928 and fixed a json behaviour along with the CSV one.

## How was this patch tested?
Added tests in `JsonSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17023 from maropu/SPARK-19695.
2017-02-22 21:39:20 -08:00
pj.fanning d3147502e7 [SPARK-15615][SQL] Add an API to load DataFrame from Dataset[String] storing JSON
## What changes were proposed in this pull request?

SPARK-15615 proposes replacing the sqlContext.read.json(rdd) with a dataset equivalent.
SPARK-15463 adds a CSV API for reading from Dataset[String] so this keeps the API consistent.
I am deprecating the existing RDD based APIs.

## How was this patch tested?

There are existing tests. I left most tests to use the existing APIs as they delegate to the new json API.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: pj.fanning <pj.fanning@workday.com>
Author: PJ Fanning <pjfanning@users.noreply.github.com>

Closes #16895 from pjfanning/SPARK-15615.
2017-02-22 18:03:25 -08:00
Xiao Li dc005ed53c [SPARK-19658][SQL] Set NumPartitions of RepartitionByExpression In Parser
### What changes were proposed in this pull request?

Currently, if `NumPartitions` is not set in RepartitionByExpression, we will set it using `spark.sql.shuffle.partitions` during Planner. However, this is not following the general resolution process. This PR is to set it in `Parser` and then `Optimizer` can use the value for plan optimization.

### How was this patch tested?

Added a test case.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16988 from gatorsmile/resolveRepartition.
2017-02-22 17:26:56 -08:00
hyukjinkwon 37112fcfcd [SPARK-19666][SQL] Skip a property without getter in Java schema inference and allow empty bean in encoder creation
## What changes were proposed in this pull request?

This PR proposes to fix two.

**Skip a property without a getter in beans**

Currently, if we use a JavaBean without the getter as below:

```java
public static class BeanWithoutGetter implements Serializable {
  private String a;

  public void setA(String a) {
    this.a = a;
  }
}

BeanWithoutGetter bean = new BeanWithoutGetter();
List<BeanWithoutGetter> data = Arrays.asList(bean);
spark.createDataFrame(data, BeanWithoutGetter.class).show();
```

- Before

It throws an exception as below:

```
java.lang.NullPointerException
	at org.spark_project.guava.reflect.TypeToken.method(TypeToken.java:465)
	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:126)
	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:125)
```

- After

```
++
||
++
||
++
```

**Supports empty bean in encoder creation**

```java
public static class EmptyBean implements Serializable {}

EmptyBean bean = new EmptyBean();
List<EmptyBean> data = Arrays.asList(bean);
spark.createDataset(data, Encoders.bean(EmptyBean.class)).show();
```

- Before

throws an exception as below:

```
java.lang.UnsupportedOperationException: Cannot infer type for class EmptyBean because it is not bean-compliant
	at org.apache.spark.sql.catalyst.JavaTypeInference$.org$apache$spark$sql$catalyst$JavaTypeInference$$serializerFor(JavaTypeInference.scala:436)
	at org.apache.spark.sql.catalyst.JavaTypeInference$.serializerFor(JavaTypeInference.scala:341)
```

- After

```
++
||
++
||
++
```

## How was this patch tested?

Unit test in `JavaDataFrameSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17013 from HyukjinKwon/SPARK-19666.
2017-02-22 12:42:23 -08:00
Bogdan Raducanu 10c566cc3b [SPARK-13721][SQL] Make GeneratorOuter unresolved.
## What changes were proposed in this pull request?

This is a small change to make GeneratorOuter always unresolved. It is mostly no-op change but makes it more clear since GeneratorOuter shouldn't survive analysis phase.
This requires also handling in ResolveAliases rule.

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

Author: Bogdan Raducanu <bogdan@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #17026 from bogdanrdc/PR16958.
2017-02-22 15:42:40 +01:00
Xiao Li 1a45d2b2cc [SPARK-19670][SQL][TEST] Enable Bucketed Table Reading and Writing Testing Without Hive Support
### What changes were proposed in this pull request?
Bucketed table reading and writing does not need Hive support. We can move the test cases from `sql/hive` to `sql/core`. After this PR, we can improve the test case coverage. Bucket table reading and writing can be tested with and without Hive support.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17004 from gatorsmile/mvTestCaseForBuckets.
2017-02-21 19:30:36 -08:00
hyukjinkwon 17b93b5feb
[SPARK-18922][TESTS] Fix new test failures on Windows due to path and resource not closed
## What changes were proposed in this pull request?

This PR proposes to fix new test failures on WIndows as below:

**Before**

```
KafkaRelationSuite:
 - test late binding start offsets *** FAILED *** (7 seconds, 679 milliseconds)
   Cause: java.nio.file.FileSystemException: C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log -> C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log.deleted: The process cannot access the file because it is being used by another process.

KafkaSourceSuite:
 - deserialization of initial offset with Spark 2.1.0 *** FAILED *** (3 seconds, 542 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-97ef64fc-ae61-4ce3-ac59-287fd38bd824

 - deserialization of initial offset written by Spark 2.1.0 *** FAILED *** (60 milliseconds)
   java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/external/kafka-0-10-sql/target/scala-2.11/test-classes/kafka-source-initial-offset-version-2.1.0.b

HiveDDLSuite:
 - partitioned table should always put partition columns at the end of table schema *** FAILED *** (657 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-f1b83d09-850a-4bba-8e43-a2a28dfaa757;

DDLSuite:
 - create a data source table without schema *** FAILED *** (94 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-a3f3c161-afae-4d6f-9182-e8642f77062b;

 - SET LOCATION for managed table *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
 Exchange SinglePartit
 +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#99367L])
    +- *FileScan parquet default.tbl[] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/C:projectsspark	arget	mpspark-15be2f2f-4ea9-4c47-bfee-1b7b49363033], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<>

 - insert data to a data source table which has a not existed location should succeed *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-34987671-e8d1-4624-ba5b-db1012e1246b;

 - insert into a data source table with no existed partition location should succeed *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4c6ccfbf-4091-4032-9fbc-3d40c58267d5;

 - read data from a data source table which has a not existed location should succeed *** FAILED *** (0 milliseconds)

 - read data from a data source table with no existed partition location should succeed *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-6af39e37-abd1-44e8-ac68-e2dfcf67a2f3;

InputOutputMetricsSuite:
 - output metrics on records written *** FAILED *** (0 milliseconds)
   java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-cd69ee77-88f2-4202-bed6-19c0ee05ef55\InputOutputMetricsSuite, expected: file:///

 - output metrics on records written - new Hadoop API *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-b69e8fcb-047b-4de8-9cdf-5f026efb6762\InputOutputMetricsSuite, expected: file:///
```

**After**

```
KafkaRelationSuite:
 - test late binding start offsets !!! CANCELED !!! (62 milliseconds)

KafkaSourceSuite:
 - deserialization of initial offset with Spark 2.1.0 (5 seconds, 341 milliseconds)
 - deserialization of initial offset written by Spark 2.1.0 (910 milliseconds)

HiveDDLSuite:
 - partitioned table should always put partition columns at the end of table schema (2 seconds)

DDLSuite:
 - create a data source table without schema (828 milliseconds)
 - SET LOCATION for managed table (406 milliseconds)
 - insert data to a data source table which has a not existed location should succeed (406 milliseconds)
 - insert into a data source table with no existed partition location should succeed (453 milliseconds)
 - read data from a data source table which has a not existed location should succeed (94 milliseconds)
 - read data from a data source table with no existed partition location should succeed (265 milliseconds)

InputOutputMetricsSuite:
 - output metrics on records written (172 milliseconds)
 - output metrics on records written - new Hadoop API (297 milliseconds)
```

## How was this patch tested?

Fixed tests in `InputOutputMetricsSuite`, `KafkaRelationSuite`,  `KafkaSourceSuite`, `DDLSuite.scala` and `HiveDDLSuite`.

Manually tested via AppVeyor as below:

`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/ex8nvwa6tsh7rmto
`KafkaRelationSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/h8dlcowew52y8ncw
`KafkaSourceSuite`: https://ci.appveyor.com/project/spark-test/spark/build/634-20170219-windows-test/job/9ybgjl7yeubxcre4
`DDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/635-20170219-windows-test
`HiveDDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/up6o9n47er087ltb

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16999 from HyukjinKwon/windows-fix.
2017-02-20 21:26:54 -08:00
windpiger 73f065569d [SPARK-19669][HOTFIX][SQL] sessionState access privileges compiled failed in TestSQLContext
## What changes were proposed in this pull request?

In [SPARK-19669](0733a54a45) change the sessionState access privileges from private to public, this lead to the compile failed in TestSQLContext

this pr is a hotfix for this.

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #17008 from windpiger/hotfixcompile.
2017-02-20 19:20:23 -08:00
Reynold Xin 0733a54a45 [SPARK-19669][SQL] Open up visibility for sharedState, sessionState, and a few other functions
## What changes were proposed in this pull request?
To ease debugging, most of Spark SQL internals have public level visibility. Two of the most important internal states, sharedState and sessionState, however, are package private. It would make more sense to open these up as well with clear documentation that they are internal.

In addition, users currently have way to set active/default SparkSession, but no way to actually get them back. We should open those up as well.

## How was this patch tested?
N/A - only visibility change.

Author: Reynold Xin <rxin@databricks.com>

Closes #17002 from rxin/SPARK-19669.
2017-02-20 12:21:07 -08:00
Xiao Li ead4ba0eb5 [SPARK-15453][SQL][FOLLOW-UP] FileSourceScanExec to extract outputOrdering information
### What changes were proposed in this pull request?
`outputOrdering` is also dependent on whether the bucket has more than one files. The test cases fail when we try to move them to sql/core. This PR is to fix the test cases introduced in https://github.com/apache/spark/pull/14864 and add a test case to verify [the related logics](070c249947/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala (L197-L206)).

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16994 from gatorsmile/bucketingTS.
2017-02-20 09:04:22 -08:00
Wenchen Fan 776b8f17cf [SPARK-19563][SQL] avoid unnecessary sort in FileFormatWriter
## What changes were proposed in this pull request?

In `FileFormatWriter`, we will sort the input rows by partition columns and bucket id and sort columns, if we want to write data out partitioned or bucketed.

However, if the data is already sorted, we will sort it again, which is unnecssary.

This PR removes the sorting logic in `FileFormatWriter` and use `SortExec` instead. We will not add `SortExec` if the data is already sorted.

## How was this patch tested?

I did a micro benchmark manually
```
val df = spark.range(10000000).select($"id", $"id" % 10 as "part").sort("part")
spark.time(df.write.partitionBy("part").parquet("/tmp/test"))
```
The result was about 6.4 seconds before this PR, and is 5.7 seconds afterwards.

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16898 from cloud-fan/writer.
2017-02-19 18:13:12 -08:00
windpiger 65fe902e13 [SPARK-19598][SQL] Remove the alias parameter in UnresolvedRelation
## What changes were proposed in this pull request?

Remove the alias parameter in `UnresolvedRelation`, and use `SubqueryAlias` to replace it.
This can simplify some `match case` situations.

For example, the broadcast hint pull request can have one fewer case https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala#L57-L61

## How was this patch tested?
add some unit tests

Author: windpiger <songjun@outlook.com>

Closes #16956 from windpiger/removeUnresolveTableAlias.
2017-02-19 16:50:16 -08:00
Sean Owen 1487c9af20
[SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features
## What changes were proposed in this pull request?

Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code.

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16964 from srowen/SPARK-19534.
2017-02-19 09:42:50 -08:00
jinxing ba8912e5f3
[SPARK-19450] Replace askWithRetry with askSync.
## What changes were proposed in this pull request?

`askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated.
As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218):

>askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it.

Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`.

## How was this patch tested?
This PR doesn't change code logic, existing unit test can cover.

Author: jinxing <jinxing@meituan.com>

Closes #16790 from jinxing64/SPARK-19450.
2017-02-19 04:34:07 -08:00
Ala Luszczak b486ffc86d [SPARK-19447] Make Range operator generate "recordsRead" metric
## What changes were proposed in this pull request?

The Range was modified to produce "recordsRead" metric instead of "generated rows". The tests were updated and partially moved to SQLMetricsSuite.

## How was this patch tested?

Unit tests.

Author: Ala Luszczak <ala@databricks.com>

Closes #16960 from ala/range-records-read.
2017-02-18 07:51:41 -08:00
Shixiong Zhu 15b144d2bf [SPARK-19617][SS] Fix the race condition when starting and stopping a query quickly
## What changes were proposed in this pull request?

The streaming thread in StreamExecution uses the following ways to check if it should exit:
- Catch an InterruptException.
- `StreamExecution.state` is TERMINATED.

When starting and stopping a query quickly, the above two checks may both fail:
- Hit [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) and swallow InterruptException
- StreamExecution.stop is called before `state` becomes `ACTIVE`. Then [runBatches](dcc2d540a5/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala (L252)) changes the state from `TERMINATED` to `ACTIVE`.

If the above cases both happen, the query will hang forever.

This PR changes `state` to `AtomicReference` and uses`compareAndSet` to make sure we only change the state from `INITIALIZING` to `ACTIVE`. It also removes the `runUninterruptibly` hack from ``HDFSMetadata`, because HADOOP-14084 won't cause any problem after we fix the race condition.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16947 from zsxwing/SPARK-19617.
2017-02-17 19:04:45 -08:00
Davies Liu 3d0c3af0a7 [SPARK-19500] [SQL] Fix off-by-one bug in BytesToBytesMap
## What changes were proposed in this pull request?

Radix sort require that half of array as free (as temporary space), so we use 0.5 as the scale factor to make sure that BytesToBytesMap will not have more items than 1/2 of capacity. Turned out this is not true, the current implementation of append() could leave 1 more item than the threshold (1/2 of capacity) in the array, which break the requirement of radix sort (fail the assert in 2.2, or fail to insert into InMemorySorter in 2.1).

This PR fix the off-by-one bug in BytesToBytesMap.

This PR also fix a bug that the array will never grow if it fail to grow once (stay as initial capacity), introduced by #15722 .

## How was this patch tested?

Added regression test.

Author: Davies Liu <davies@databricks.com>

Closes #16844 from davies/off_by_one.
2017-02-17 09:38:06 -08:00
Wenchen Fan 54d23599df [SPARK-18120][SPARK-19557][SQL] Call QueryExecutionListener callback methods for DataFrameWriter methods
## What changes were proposed in this pull request?

We only notify `QueryExecutionListener` for several `Dataset` operations, e.g. collect, take, etc. We should also do the notification for `DataFrameWriter` operations.

## How was this patch tested?

new regression test

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16962 from cloud-fan/insert.
2017-02-16 21:09:14 -08:00
Nathan Howell 21fde57f15 [SPARK-18352][SQL] Support parsing multiline json files
## What changes were proposed in this pull request?

If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory.

Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired.

These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing.

I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one.

## How was this patch tested?

New and existing unit tests. No performance or load tests have been run.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16386 from NathanHowell/SPARK-18352.
2017-02-16 20:51:19 -08:00
Zheng RuiFeng 54a30c8a70 [SPARK-19436][SQL] Add missing tests for approxQuantile
## What changes were proposed in this pull request?
1, check the behavior with illegal `quantiles` and `relativeError`
2, add tests for `relativeError` > 1
3, update tests for `null` data
4, update some docs for javadoc8

## How was this patch tested?
local test in spark-shell

Author: Zheng RuiFeng <ruifengz@foxmail.com>
Author: Ruifeng Zheng <ruifengz@foxmail.com>

Closes #16776 from zhengruifeng/fix_approxQuantile.
2017-02-16 09:42:13 -08:00
hyukjinkwon 3b4376876f
[MINOR][BUILD] Fix javadoc8 break
## What changes were proposed in this pull request?

These error below seems caused by unidoc that does not understand double commented block.

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected
[error]  * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction =
[error]                                  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected
[error]  * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction =
[error]                                                                       ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected
[error]  *    new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() {
[error]                                         ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected
[error]  *    new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() {
[error]                                                                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: illegal character: '#'
[error]  *      &#64;Override
[error]          ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: class, interface, or enum expected
[error]  *      &#64;Override
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                    ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                                ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                                                     ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                                                                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:76: error: class, interface, or enum expected
[error]  *          boolean shouldRemove = ...; // Decide whether to remove the state
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:77: error: class, interface, or enum expected
[error]  *          if (shouldRemove) {
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:79: error: class, interface, or enum expected
[error]  *          } else {
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:81: error: class, interface, or enum expected
[error]  *            state.update(newState); // Set the new state
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:82: error: class, interface, or enum expected
[error]  *          }
[error]  ^
[error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:85: error: class, interface, or enum expected
[error]  *          state.update(initialState);
[error]  ^
[error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:86: error: class, interface, or enum expected
[error]  *        }
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:90: error: class, interface, or enum expected
[error]  * </code></pre>
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:92: error: class, interface, or enum expected
[error]  * tparam S User-defined type of the state to be stored for each key. Must be encodable into
[error]            ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:93: error: class, interface, or enum expected
[error]  *           Spark SQL types (see {link Encoder} for more details).
[error]                                          ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:94: error: class, interface, or enum expected
[error]  * since 2.1.1
[error]           ^
```

And another link seems unrecognisable.

```
.../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:16: error: reference not found
[error]  * That is, in every batch of the {link streaming.StreamingQuery StreamingQuery},
[error]
```

Note that this PR does not fix the two breaks as below:

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:43: error: unexpected content
[error]    * see {link DataFrameStatsFunctions.approxQuantile(col:Str* approxQuantile} for
[error]      ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:52: error: bad use of '>'
[error]    * param relativeError The relative target precision to achieve (>= 0).
[error]                                                                     ^
[error]
```

because these seem probably fixed soon in https://github.com/apache/spark/pull/16776 and I intended to avoid potential conflicts.

## How was this patch tested?

Manually via `jekyll build`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16926 from HyukjinKwon/javadoc-break.
2017-02-16 12:35:43 +00:00
Sean Owen 0e2405490f
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings

For the future:

- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16871 from srowen/SPARK-19493.
2017-02-16 12:32:45 +00:00
Kevin Yu 3871d94a69 [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 3rd batch
## What changes were proposed in this pull request?

This is 3ird batch of test case for IN/NOT IN subquery. In this PR, it has these test files:

`in-having.sql`
`in-joins.sql`
`in-multiple-columns.sql`

These are the queries and results from running on DB2.
[in-having DB2 version](https://github.com/apache/spark/files/772668/in-having.sql.db2.txt)
[output of in-having](https://github.com/apache/spark/files/772670/in-having.sql.db2.out.txt)
[in-joins DB2 version](https://github.com/apache/spark/files/772672/in-joins.sql.db2.txt)
[output of in-joins](https://github.com/apache/spark/files/772673/in-joins.sql.db2.out.txt)
[in-multiple-columns DB2 version](https://github.com/apache/spark/files/772678/in-multiple-columns.sql.db2.txt)
[output of in-multiple-columns](https://github.com/apache/spark/files/772680/in-multiple-columns.sql.db2.out.txt)

## How was this patch tested?
This pr is adding new test cases. We compare the result from spark with the result from another RDBMS(We used DB2 LUW). If the results are the same, we assume the result is correct.

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

Closes #16841 from kevinyu98/spark-18871-33.
2017-02-16 00:02:15 -08:00
Tejas Patil f041e55eef [SPARK-19618][SQL] Inconsistency wrt max. buckets allowed from Dataframe API vs SQL
## What changes were proposed in this pull request?

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

Moved the check for validating number of buckets from `DataFrameWriter` to `BucketSpec` creation

## How was this patch tested?

- Added more unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #16948 from tejasapatil/SPARK-19618_max_buckets.
2017-02-15 22:45:58 -08:00
Kevin Yu 8487902a98 [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 4th batch
## What changes were proposed in this pull request?

This is 4th batch of test case for IN/NOT IN subquery. In this PR, it has these test files:

`in-set-operations.sql`
`in-with-cte.sql`
`not-in-joins.sql`

Here are the queries and results from running on DB2.

[in-set-operations DB2 version](https://github.com/apache/spark/files/772846/in-set-operations.sql.db2.txt)
[Output of in-set-operations](https://github.com/apache/spark/files/772848/in-set-operations.sql.db2.out.txt)
[in-with-cte DB2 version](https://github.com/apache/spark/files/772849/in-with-cte.sql.db2.txt)
[Output of in-with-cte](https://github.com/apache/spark/files/772856/in-with-cte.sql.db2.out.txt)
[not-in-joins DB2 version](https://github.com/apache/spark/files/772851/not-in-joins.sql.db2.txt)
[Output of not-in-joins](https://github.com/apache/spark/files/772852/not-in-joins.sql.db2.out.txt)

## How was this patch tested?

This pr is adding new test cases. We compare the result from spark with the result from another RDBMS(We used DB2 LUW). If the results are the same, we assume the result is correct.

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

Closes #16915 from kevinyu98/spark-18871-44.
2017-02-15 21:29:28 -08:00
Shixiong Zhu fc02ef95cd [SPARK-19603][SS] Fix StreamingQuery explain command
## What changes were proposed in this pull request?

`StreamingQuery.explain` doesn't show the correct streaming physical plan right now because `ExplainCommand` receives a runtime batch plan and its `logicalPlan.isStreaming` is always false.

This PR adds `streaming` parameter to `ExplainCommand` to allow `StreamExecution` to specify that it's a streaming plan.

Examples of the explain outputs:

- streaming DataFrame.explain()
```
== Physical Plan ==
*HashAggregate(keys=[value#518], functions=[count(1)])
+- StateStoreSave [value#518], OperatorStateId(<unknown>,0,0), Append, 0
   +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
      +- StateStoreRestore [value#518], OperatorStateId(<unknown>,0,0)
         +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
            +- Exchange hashpartitioning(value#518, 5)
               +- *HashAggregate(keys=[value#518], functions=[partial_count(1)])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
                     +- *MapElements <function1>, obj#517: java.lang.String
                        +- *DeserializeToObject value#513.toString, obj#516: java.lang.String
                           +- StreamingRelation MemoryStream[value#513], [value#513]
```

- StreamingQuery.explain(extended = false)
```
== Physical Plan ==
*HashAggregate(keys=[value#518], functions=[count(1)])
+- StateStoreSave [value#518], OperatorStateId(...,0,0), Complete, 0
   +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
      +- StateStoreRestore [value#518], OperatorStateId(...,0,0)
         +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
            +- Exchange hashpartitioning(value#518, 5)
               +- *HashAggregate(keys=[value#518], functions=[partial_count(1)])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
                     +- *MapElements <function1>, obj#517: java.lang.String
                        +- *DeserializeToObject value#543.toString, obj#516: java.lang.String
                           +- LocalTableScan [value#543]
```

- StreamingQuery.explain(extended = true)
```
== Parsed Logical Plan ==
Aggregate [value#518], [value#518, count(1) AS count(1)#524L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String
      +- DeserializeToObject cast(value#543 as string).toString, obj#516: java.lang.String
         +- LocalRelation [value#543]

== Analyzed Logical Plan ==
value: string, count(1): bigint
Aggregate [value#518], [value#518, count(1) AS count(1)#524L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String
      +- DeserializeToObject cast(value#543 as string).toString, obj#516: java.lang.String
         +- LocalRelation [value#543]

== Optimized Logical Plan ==
Aggregate [value#518], [value#518, count(1) AS count(1)#524L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String
      +- DeserializeToObject value#543.toString, obj#516: java.lang.String
         +- LocalRelation [value#543]

== Physical Plan ==
*HashAggregate(keys=[value#518], functions=[count(1)], output=[value#518, count(1)#524L])
+- StateStoreSave [value#518], OperatorStateId(...,0,0), Complete, 0
   +- *HashAggregate(keys=[value#518], functions=[merge_count(1)], output=[value#518, count#530L])
      +- StateStoreRestore [value#518], OperatorStateId(...,0,0)
         +- *HashAggregate(keys=[value#518], functions=[merge_count(1)], output=[value#518, count#530L])
            +- Exchange hashpartitioning(value#518, 5)
               +- *HashAggregate(keys=[value#518], functions=[partial_count(1)], output=[value#518, count#530L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
                     +- *MapElements <function1>, obj#517: java.lang.String
                        +- *DeserializeToObject value#543.toString, obj#516: java.lang.String
                           +- LocalTableScan [value#543]
```

## How was this patch tested?

The updated unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16934 from zsxwing/SPARK-19603.
2017-02-15 20:51:33 -08:00
Shixiong Zhu 21b4ba2d6f [SPARK-19599][SS] Clean up HDFSMetadataLog
## What changes were proposed in this pull request?

SPARK-19464 removed support for Hadoop 2.5 and earlier, so we can do some cleanup for HDFSMetadataLog.

This PR includes the following changes:
- ~~Remove the workaround codes for HADOOP-10622.~~ Unfortunately, there is another issue [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) that prevents us from removing the workaround codes.
- Remove unnecessary `writer: (T, OutputStream) => Unit` and just call `serialize` directly.
- Remove catching FileNotFoundException.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16932 from zsxwing/metadata-cleanup.
2017-02-15 16:21:43 -08:00