Commit graph

20173 commits

Author SHA1 Message Date
Devaraj K e16e8c7ad3 [SPARK-21146][CORE] Master/Worker should handle and shutdown when any thread gets UncaughtException
## What changes were proposed in this pull request?

Adding the default UncaughtExceptionHandler to the Worker.

## How was this patch tested?

I verified it manually, when any of the worker thread gets uncaught exceptions then the default UncaughtExceptionHandler will handle those exceptions.

Author: Devaraj K <devaraj@apache.org>

Closes #18357 from devaraj-kavali/SPARK-21146.
2017-07-12 00:14:58 -07:00
liuzhaokun 24367f23f7 [SPARK-21382] The note about Scala 2.10 in building-spark.md is wrong.
[https://issues.apache.org/jira/browse/SPARK-21382](https://issues.apache.org/jira/browse/SPARK-21382)
There should be "Note that support for Scala 2.10 is deprecated as of Spark 2.1.0 and may be removed in Spark 2.3.0",right?

Author: liuzhaokun <liu.zhaokun@zte.com.cn>

Closes #18606 from liu-zhaokun/new07120923.
2017-07-11 23:02:20 -07:00
Jane Wang 2cbfc975ba [SPARK-12139][SQL] REGEX Column Specification
## What changes were proposed in this pull request?
Hive interprets regular expression, e.g., `(a)?+.+` in query specification. This PR enables spark to support this feature when hive.support.quoted.identifiers is set to true.

## How was this patch tested?

- Add unittests in SQLQuerySuite.scala
- Run spark-shell tested the original failed query:
scala> hc.sql("SELECT `(a|b)?+.+` from test1").collect.foreach(println)

Author: Jane Wang <janewang@fb.com>

Closes #18023 from janewangfb/support_select_regex.
2017-07-11 22:00:36 -07:00
gatorsmile d3e071658f [SPARK-19285][SQL] Implement UDF0
### What changes were proposed in this pull request?
This PR is to implement UDF0. `UDF0` is needed when users need to implement a JAVA UDF with no argument.

### How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18598 from gatorsmile/udf0.
2017-07-11 15:44:29 -07:00
Marcelo Vanzin 1cad31f006 [SPARK-16019][YARN] Use separate RM poll interval when starting client AM.
Currently the code monitoring the launch of the client AM uses the value of
spark.yarn.report.interval as the interval for polling the RM; if someone
has that value to a really large interval, it would take that long to detect
that the client AM has started, which is not expected.

Instead, have a separate config for the interval to use when the client AM is
starting. The other config is still used in cluster mode, and to detect the
status of the client AM after it is already running.

Tested by running client and cluster mode apps with a modified value of
spark.yarn.report.interval, verifying client AM launch is detected before
that interval elapses.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18380 from vanzin/SPARK-16019.
2017-07-11 11:25:40 -07:00
hyukjinkwon ebc124d4c4 [SPARK-21365][PYTHON] Deduplicate logics parsing DDL type/schema definition
## What changes were proposed in this pull request?

This PR deals with four points as below:

- Reuse existing DDL parser APIs rather than reimplementing within PySpark

- Support DDL formatted string, `field type, field type`.

- Support case-insensitivity for parsing.

- Support nested data types as below:

  **Before**
  ```
  >>> spark.createDataFrame([[[1]]], "struct<a: struct<b: int>>").show()
  ...
  ValueError: The strcut field string format is: 'field_name:field_type', but got: a: struct<b: int>
  ```

  ```
  >>> spark.createDataFrame([[[1]]], "a: struct<b: int>").show()
  ...
  ValueError: The strcut field string format is: 'field_name:field_type', but got: a: struct<b: int>
  ```

  ```
  >>> spark.createDataFrame([[1]], "a int").show()
  ...
  ValueError: Could not parse datatype: a int
  ```

  **After**
  ```
  >>> spark.createDataFrame([[[1]]], "struct<a: struct<b: int>>").show()
  +---+
  |  a|
  +---+
  |[1]|
  +---+
  ```

  ```
  >>> spark.createDataFrame([[[1]]], "a: struct<b: int>").show()
  +---+
  |  a|
  +---+
  |[1]|
  +---+
  ```

  ```
  >>> spark.createDataFrame([[1]], "a int").show()
  +---+
  |  a|
  +---+
  |  1|
  +---+
  ```

## How was this patch tested?

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18590 from HyukjinKwon/deduplicate-python-ddl.
2017-07-11 22:03:10 +08:00
Xingbo Jiang 66d2168655 [SPARK-21366][SQL][TEST] Add sql test for window functions
## What changes were proposed in this pull request?

Add sql test for window functions, also remove uncecessary test cases in `WindowQuerySuite`.

## How was this patch tested?

Added `window.sql` and the corresponding output file.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #18591 from jiangxb1987/window.
2017-07-11 21:52:54 +08:00
hyukjinkwon 7514db1dec [SPARK-21263][SQL] Do not allow partially parsing double and floats via NumberFormat in CSV
## What changes were proposed in this pull request?

This PR proposes to remove `NumberFormat.parse` use to disallow a case of partially parsed data. For example,

```
scala> spark.read.schema("a DOUBLE").option("mode", "FAILFAST").csv(Seq("10u12").toDS).show()
+----+
|   a|
+----+
|10.0|
+----+
```

## How was this patch tested?

Unit tests added in `UnivocityParserSuite` and `CSVSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18532 from HyukjinKwon/SPARK-21263.
2017-07-11 11:11:08 +01:00
Michael Allman a4baa8f48f [SPARK-20331][SQL] Enhanced Hive partition pruning predicate pushdown
(Link to Jira: https://issues.apache.org/jira/browse/SPARK-20331)

## What changes were proposed in this pull request?

Spark 2.1 introduced scalable support for Hive tables with huge numbers of partitions. Key to leveraging this support is the ability to prune unnecessary table partitions to answer queries. Spark supports a subset of the class of partition pruning predicates that the Hive metastore supports. If a user writes a query with a partition pruning predicate that is *not* supported by Spark, Spark falls back to loading all partitions and pruning client-side. We want to broaden Spark's current partition pruning predicate pushdown capabilities.

One of the key missing capabilities is support for disjunctions. For example, for a table partitioned by date, writing a query with a predicate like

    date = 20161011 or date = 20161014

will result in Spark fetching all partitions. For a table partitioned by date and hour, querying a range of hours across dates can be quite difficult to accomplish without fetching all partition metadata.

The current partition pruning support supports only comparisons against literals. We can expand that to foldable expressions by evaluating them at planning time.

We can also implement support for the "IN" comparison by expanding it to a sequence of "OR"s.

## How was this patch tested?

The `HiveClientSuite` and `VersionsSuite` were refactored and simplified to make Hive client-based, version-specific testing more modular and conceptually simpler. There are now two Hive test suites: `HiveClientSuite` and `HivePartitionFilteringSuite`. These test suites have a single-argument constructor taking a `version` parameter. As such, these test suites cannot be run by themselves. Instead, they have been bundled into "aggregation" test suites which run each suite for each Hive client version. These aggregation suites are called `HiveClientSuites` and `HivePartitionFilteringSuites`. The `VersionsSuite` and `HiveClientSuite` have been refactored into each of these aggregation suites, respectively.

`HiveClientSuite` and `HivePartitionFilteringSuite` subclass a new abstract class, `HiveVersionSuite`. `HiveVersionSuite` collects functionality related to testing a single Hive version and overrides relevant test suite methods to display version-specific information.

A new trait, `HiveClientVersions`, has been added with a sequence of Hive test versions.

Author: Michael Allman <michael@videoamp.com>

Closes #17633 from mallman/spark-20331-enhanced_partition_pruning_pushdown.
2017-07-11 14:50:11 +08:00
hyukjinkwon d4d9e17b31 [SPARK-20456][PYTHON][FOLLOWUP] Fix timezone-dependent doctests in unix_timestamp and from_unixtime
## What changes were proposed in this pull request?

This PR proposes to simply ignore the results in examples that are timezone-dependent in `unix_timestamp` and `from_unixtime`.

```
Failed example:
    time_df.select(unix_timestamp('dt', 'yyyy-MM-dd').alias('unix_time')).collect()
Expected:
    [Row(unix_time=1428476400)]
Got:unix_timestamp
    [Row(unix_time=1428418800)]
```

```
Failed example:
    time_df.select(from_unixtime('unix_time').alias('ts')).collect()
Expected:
    [Row(ts=u'2015-04-08 00:00:00')]
Got:
    [Row(ts=u'2015-04-08 16:00:00')]
```

## How was this patch tested?

Manually tested and `./run-tests --modules pyspark-sql`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18597 from HyukjinKwon/SPARK-20456.
2017-07-11 15:23:03 +09:00
jinxing 97a1aa2c70 [SPARK-21315][SQL] Skip some spill files when generateIterator(startIndex) in ExternalAppendOnlyUnsafeRowArray.
## What changes were proposed in this pull request?

In current code, it is expensive to use `UnboundedFollowingWindowFunctionFrame`, because it is iterating from the start to lower bound every time calling `write` method. When traverse the iterator, it's possible to skip some spilled files thus to save some time.

## How was this patch tested?

Added unit test

Did a small test for benchmark:

Put 2000200 rows into `UnsafeExternalSorter`-- 2 spill files(each contains 1000000 rows) and inMemSorter contains 200 rows.
Move the iterator forward to index=2000001.

*With this change*:
`getIterator(2000001)`, it will cost almost 0ms~1ms;
*Without this change*:
`for(int i=0; i<2000001; i++)geIterator().loadNext()`, it will cost 300ms.

Author: jinxing <jinxing6042@126.com>

Closes #18541 from jinxing64/SPARK-21315.
2017-07-11 11:47:47 +08:00
Shixiong Zhu 833eab2c9b [SPARK-21369][CORE] Don't use Scala Tuple2 in common/network-*
## What changes were proposed in this pull request?

Remove all usages of Scala Tuple2 from common/network-* projects. Otherwise, Yarn users cannot use `spark.reducer.maxReqSizeShuffleToMem`.

## How was this patch tested?

Jenkins.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18593 from zsxwing/SPARK-21369.
2017-07-11 11:26:17 +08:00
gatorsmile 1471ee7af5 [SPARK-21350][SQL] Fix the error message when the number of arguments is wrong when invoking a UDF
### What changes were proposed in this pull request?
Users get a very confusing error when users specify a wrong number of parameters.
```Scala
    val df = spark.emptyDataFrame
    spark.udf.register("foo", (_: String).length)
    df.selectExpr("foo(2, 3, 4)")
```
```
org.apache.spark.sql.UDFSuite$$anonfun$9$$anonfun$apply$mcV$sp$12 cannot be cast to scala.Function3
java.lang.ClassCastException: org.apache.spark.sql.UDFSuite$$anonfun$9$$anonfun$apply$mcV$sp$12 cannot be cast to scala.Function3
	at org.apache.spark.sql.catalyst.expressions.ScalaUDF.<init>(ScalaUDF.scala:109)
```

This PR is to capture the exception and issue an error message that is consistent with what we did for built-in functions. After the fix, the error message is improved to
```
Invalid number of arguments for function foo; line 1 pos 0
org.apache.spark.sql.AnalysisException: Invalid number of arguments for function foo; line 1 pos 0
	at org.apache.spark.sql.catalyst.analysis.SimpleFunctionRegistry.lookupFunction(FunctionRegistry.scala:119)
```

### How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18574 from gatorsmile/statsCheck.
2017-07-11 11:19:59 +08:00
Takeshi Yamamuro a2bec6c92a [SPARK-21043][SQL] Add unionByName in Dataset
## What changes were proposed in this pull request?
This pr added `unionByName` in `DataSet`.
Here is how to use:
```
val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2")
val df2 = Seq((4, 5, 6)).toDF("col1", "col2", "col0")
df1.unionByName(df2).show

// output:
// +----+----+----+
// |col0|col1|col2|
// +----+----+----+
// |   1|   2|   3|
// |   6|   4|   5|
// +----+----+----+
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18300 from maropu/SPARK-21043-2.
2017-07-10 20:16:29 -07:00
chie8842 c3713fde86 [SPARK-21358][EXAMPLES] Argument of repartitionandsortwithinpartitions at pyspark
## What changes were proposed in this pull request?
At example of repartitionAndSortWithinPartitions at rdd.py, third argument should be True or False.
I proposed fix of example code.

## How was this patch tested?
* I rename test_repartitionAndSortWithinPartitions to test_repartitionAndSortWIthinPartitions_asc to specify boolean argument.
* I added test_repartitionAndSortWithinPartitions_desc to test False pattern at third argument.

(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 http://spark.apache.org/contributing.html before opening a pull request.

Author: chie8842 <chie8842@gmail.com>

Closes #18586 from chie8842/SPARK-21358.
2017-07-10 18:56:54 -07:00
Bryan Cutler d03aebbe65 [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  Data types except complex, date, timestamp, and decimal  are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
2017-07-10 15:21:03 -07:00
hyukjinkwon 2bfd5accdc [SPARK-21266][R][PYTHON] Support schema a DDL-formatted string in dapply/gapply/from_json
## What changes were proposed in this pull request?

This PR supports schema in a DDL formatted string for `from_json` in R/Python and `dapply` and `gapply` in R, which are commonly used and/or consistent with Scala APIs.

Additionally, this PR exposes `structType` in R to allow working around in other possible corner cases.

**Python**

`from_json`

```python
from pyspark.sql.functions import from_json

data = [(1, '''{"a": 1}''')]
df = spark.createDataFrame(data, ("key", "value"))
df.select(from_json(df.value, "a INT").alias("json")).show()
```

**R**

`from_json`

```R
df <- sql("SELECT named_struct('name', 'Bob') as people")
df <- mutate(df, people_json = to_json(df$people))
head(select(df, from_json(df$people_json, "name STRING")))
```

`structType.character`

```R
structType("a STRING, b INT")
```

`dapply`

```R
dapply(createDataFrame(list(list(1.0)), "a"), function(x) {x}, "a DOUBLE")
```

`gapply`

```R
gapply(createDataFrame(list(list(1.0)), "a"), "a", function(key, x) { x }, "a DOUBLE")
```

## How was this patch tested?

Doc tests for `from_json` in Python and unit tests `test_sparkSQL.R` in R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18498 from HyukjinKwon/SPARK-21266.
2017-07-10 10:40:03 -07:00
Juliusz Sompolski 18b3b00ecf [SPARK-21272] SortMergeJoin LeftAnti does not update numOutputRows
## What changes were proposed in this pull request?

Updating numOutputRows metric was missing from one return path of LeftAnti SortMergeJoin.

## How was this patch tested?

Non-zero output rows manually seen in metrics.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #18494 from juliuszsompolski/SPARK-21272.
2017-07-10 09:26:42 -07:00
jinxing 6a06c4b03c [SPARK-21342] Fix DownloadCallback to work well with RetryingBlockFetcher.
## What changes were proposed in this pull request?

When `RetryingBlockFetcher` retries fetching blocks. There could be two `DownloadCallback`s download the same content to the same target file. It could cause `ShuffleBlockFetcherIterator` reading a partial result.

This pr proposes to create and delete the tmp files in `OneForOneBlockFetcher`

Author: jinxing <jinxing6042@126.com>
Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #18565 from jinxing64/SPARK-21342.
2017-07-10 21:06:58 +08:00
Takeshi Yamamuro 647963a26a [SPARK-20460][SQL] Make it more consistent to handle column name duplication
## What changes were proposed in this pull request?
This pr made it more consistent to handle column name duplication. In the current master, error handling is different when hitting column name duplication:
```
// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#12, a#13.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Duplicate column(s) : "a" found, cannot save to JSON format;
  at org.apache.spark.sql.execution.datasources.json.JsonDataSource.checkConstraints(JsonDataSource.scala:81)
  at org.apache.spark.sql.execution.datasources.json.JsonDataSource.inferSchema(JsonDataSource.scala:63)
  at org.apache.spark.sql.execution.datasources.json.JsonFileFormat.inferSchema(JsonFileFormat.scala:57)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)

// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#41, a#42.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)

// If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896)
scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
|  1|  1|
+---+---+

// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#110, a#111.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```
When this patch applied, the results change to;
```

// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:156)

// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)

scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
|  1|  1|
+---+---+

// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
```

## How was this patch tested?
Added tests in `DataFrameReaderWriterSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17758 from maropu/SPARK-20460.
2017-07-10 15:58:34 +08:00
Dongjoon Hyun c444d10868 [MINOR][DOC] Remove obsolete ec2-scripts.md
## What changes were proposed in this pull request?

Since this document became obsolete, we had better remove this for Apache Spark 2.3.0. The original document is removed via SPARK-12735 on January 2016, and currently it's just redirection page. The only reference in Apache Spark website will go directly to the destination in https://github.com/apache/spark-website/pull/54.

## How was this patch tested?

N/A. This is a removal of documentation.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18578 from dongjoon-hyun/SPARK-REMOVE-EC2.
2017-07-10 07:46:47 +01:00
Eric Vandenberg 96d58f285b [SPARK-21219][CORE] Task retry occurs on same executor due to race condition with blacklisting
## What changes were proposed in this pull request?

There's a race condition in the current TaskSetManager where a failed task is added for retry (addPendingTask), and can asynchronously be assigned to an executor *prior* to the blacklist state (updateBlacklistForFailedTask), the result is the task might re-execute on the same executor.  This is particularly problematic if the executor is shutting down since the retry task immediately becomes a lost task (ExecutorLostFailure).  Another side effect is that the actual failure reason gets obscured by the retry task which never actually executed.  There are sample logs showing the issue in the https://issues.apache.org/jira/browse/SPARK-21219

The fix is to change the ordering of the addPendingTask and updatingBlackListForFailedTask calls in TaskSetManager.handleFailedTask

## How was this patch tested?

Implemented a unit test that verifies the task is black listed before it is added to the pending task.  Ran the unit test without the fix and it fails.  Ran the unit test with the fix and it passes.

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

Author: Eric Vandenberg <ericvandenberg@fb.com>

Closes #18427 from ericvandenbergfb/blacklistFix.
2017-07-10 14:40:20 +08:00
Wenchen Fan 0e80ecae30 [SPARK-21100][SQL][FOLLOWUP] cleanup code and add more comments for Dataset.summary
## What changes were proposed in this pull request?

Some code cleanup and adding comments to make the code more readable. Changed the way to generate result rows, to be more clear.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18570 from cloud-fan/summary.
2017-07-09 22:53:27 -07:00
jerryshao 457dc9ccbf [MINOR][DOC] Improve the docs about how to correctly set configurations
## What changes were proposed in this pull request?

Spark provides several ways to set configurations, either from configuration file, or from `spark-submit` command line options, or programmatically through `SparkConf` class. It may confuses beginners why some configurations set through `SparkConf` cannot take affect. So here add some docs to address this problems and let beginners know how to correctly set configurations.

## How was this patch tested?

N/A

Author: jerryshao <sshao@hortonworks.com>

Closes #18552 from jerryshao/improve-doc.
2017-07-10 11:22:28 +08:00
Wenchen Fan 680b33f166 [SPARK-18016][SQL][FOLLOWUP] merge declareAddedFunctions, initNestedClasses and declareNestedClasses
## What changes were proposed in this pull request?

These 3 methods have to be used together, so it makes more sense to merge them into one method and then the caller side only need to call one method.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18579 from cloud-fan/minor.
2017-07-09 16:30:35 -07:00
hyukjinkwon 08e0d033b4 [SPARK-21093][R] Terminate R's worker processes in the parent of R's daemon to prevent a leak
## What changes were proposed in this pull request?

This is a retry for #18320. This PR was reverted due to unexpected test failures with -10 error code.

I was unable to reproduce in MacOS, CentOS and Ubuntu but only in Jenkins. So, the tests proceeded to verify this and revert the past try here - https://github.com/apache/spark/pull/18456

This new approach was tested in https://github.com/apache/spark/pull/18463.

**Test results**:

- With the part of suspicious change in the past try (466325d3fd)

  Tests ran 4 times and 2 times passed and 2 time failed.

- Without the part of suspicious change in the past try (466325d3fd)

  Tests ran 5 times and they all passed.

- With this new approach (0a7589c09f)

  Tests ran 5 times and they all passed.

It looks the cause is as below (see 466325d3fd):

```diff
+ exitCode <- 1
...
+   data <- parallel:::readChild(child)
+   if (is.raw(data)) {
+     if (unserialize(data) == exitCode) {
      ...
+     }
+   }

...

- parallel:::mcexit(0L)
+ parallel:::mcexit(0L, send = exitCode)
```

Two possibilities I think

 - `parallel:::mcexit(.. , send = exitCode)`

   https://stat.ethz.ch/R-manual/R-devel/library/parallel/html/mcfork.html

   > It sends send to the master (unless NULL) and then shuts down the child process.

   However, it looks possible that the parent attemps to terminate the child right after getting our custom exit code. So, the child gets terminated between "send" and "shuts down", failing to exit properly.

 - A bug between `parallel:::mcexit(..., send = ...)` and `parallel:::readChild`.

**Proposal**:

To resolve this, I simply decided to avoid both possibilities with this new approach here (9ff89a7859). To support this idea, I explained with some quotation of the documentation as below:

https://stat.ethz.ch/R-manual/R-devel/library/parallel/html/mcfork.html

> `readChild` and `readChildren` return a raw vector with a "pid" attribute if data were available, an integer vector of length one with the process ID if a child terminated or `NULL` if the child no longer exists (no children at all for `readChildren`).

`readChild` returns "an integer vector of length one with the process ID if a child terminated" so we can check if it is `integer` and the same selected "process ID". I believe this makes sure that the children are exited.

In case that children happen to send any data manually to parent (which is why we introduced the suspicious part of the change (466325d3fd)), this should be raw bytes and will be discarded (and then will try to read the next and check if it is `integer` in the next loop).

## How was this patch tested?

Manual tests and Jenkins tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18465 from HyukjinKwon/SPARK-21093-retry-1.
2017-07-08 14:24:37 -07:00
Xiao Li c3712b77a9 [SPARK-21307][REVERT][SQL] Remove SQLConf parameters from the parser-related classes
## What changes were proposed in this pull request?
Since we do not set active sessions when parsing the plan, we are unable to correctly use SQLConf.get to find the correct active session. Since https://github.com/apache/spark/pull/18531 breaks the build, I plan to revert it at first.

## How was this patch tested?
The existing test cases

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18568 from gatorsmile/revert18531.
2017-07-08 11:56:19 -07:00
jinxing 062c336d06 [SPARK-21343] Refine the document for spark.reducer.maxReqSizeShuffleToMem.
## What changes were proposed in this pull request?

In current code, reducer can break the old shuffle service when `spark.reducer.maxReqSizeShuffleToMem` is enabled. Let's refine document.

Author: jinxing <jinxing6042@126.com>

Closes #18566 from jinxing64/SPARK-21343.
2017-07-09 00:27:58 +08:00
Marcelo Vanzin 9131bdb7e1 [SPARK-20342][CORE] Update task accumulators before sending task end event.
This makes sures that listeners get updated task information; otherwise it's
possible to write incomplete task information into event logs, for example,
making the information in a replayed UI inconsistent with the original
application.

Added a new unit test to try to detect the problem, but it's not guaranteed
to fail since it's a race; but it fails pretty reliably for me without the
scheduler changes.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18393 from vanzin/SPARK-20342.try2.
2017-07-09 00:24:54 +08:00
Zhenhua Wang 9fccc3627f [SPARK-21083][SQL] Store zero size and row count when analyzing empty table
## What changes were proposed in this pull request?

We should be able to store zero size and row count after analyzing empty table.

This pr also enhances the test cases for re-analyzing tables.

## How was this patch tested?

Added a new test case and enhanced some test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #18292 from wzhfy/analyzeNewColumn.
2017-07-08 20:44:12 +08:00
Dongjoon Hyun 0b8dd2d084 [SPARK-21345][SQL][TEST][TEST-MAVEN] SparkSessionBuilderSuite should clean up stopped sessions.
## What changes were proposed in this pull request?

`SparkSessionBuilderSuite` should clean up stopped sessions. Otherwise, it leaves behind some stopped `SparkContext`s interfereing with other test suites using `ShardSQLContext`.

Recently, master branch fails consequtively.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/

## How was this patch tested?

Pass the Jenkins with a updated suite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18567 from dongjoon-hyun/SPARK-SESSION.
2017-07-08 20:16:47 +08:00
caoxuewen 330bf5c998 [SPARK-20609][MLLIB][TEST] manually cleared 'spark.local.dir' before/after a test in ALSCleanerSuite
## What changes were proposed in this pull request?

This PR is similar to #17869.
Once` 'spark.local.dir'` is set. Unless this is manually cleared before/after a test. it could return the same directory even if this property is configured.
and add before/after for each likewise in ALSCleanerSuite.

## How was this patch tested?
existing test.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #18537 from heary-cao/ALSCleanerSuite.
2017-07-08 08:34:51 +01:00
Joachim Hereth 01f183e849 Mesos doc fixes
## What changes were proposed in this pull request?

Some link fixes for the documentation [Running Spark on Mesos](https://spark.apache.org/docs/latest/running-on-mesos.html):

* Updated Link to Mesos Frameworks (Projects built on top of Mesos)
* Update Link to Mesos binaries from Mesosphere (former link was redirected to dcos install page)

## How was this patch tested?

Documentation was built and changed page manually/visually inspected.

No code was changed, hence no dev tests.

Since these changes are rather trivial I did not open a new JIRA ticket.

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

Author: Joachim Hereth <joachim.hereth@numberfour.eu>

Closes #18564 from daten-kieker/mesos_doc_fixes.
2017-07-08 08:32:45 +01:00
Michael Patterson f5f02d213d [SPARK-20456][DOCS] Add examples for functions collection for pyspark
## What changes were proposed in this pull request?

This adds documentation to many functions in pyspark.sql.functions.py:
`upper`, `lower`, `reverse`, `unix_timestamp`, `from_unixtime`, `rand`, `randn`, `collect_list`, `collect_set`, `lit`
Add units to the trigonometry functions.
Renames columns in datetime examples to be more informative.
Adds links between some functions.

## How was this patch tested?

`./dev/lint-python`
`python python/pyspark/sql/functions.py`
`./python/run-tests.py --module pyspark-sql`

Author: Michael Patterson <map222@gmail.com>

Closes #17865 from map222/spark-20456.
2017-07-07 23:59:34 -07:00
wangmiao1981 a7b46c627b [SPARK-20307][SPARKR] SparkR: pass on setHandleInvalid to spark.mllib functions that use StringIndexer
## What changes were proposed in this pull request?

For randomForest classifier, if test data contains unseen labels, it will throw an error. The StringIndexer already has the handleInvalid logic. The patch add a new method to set the underlying StringIndexer handleInvalid logic.

This patch should also apply to other classifiers. This PR focuses on the main logic and randomForest classifier. I will do follow-up PR for other classifiers.

## How was this patch tested?

Add a new unit test based on the error case in the JIRA.

Author: wangmiao1981 <wm624@hotmail.com>

Closes #18496 from wangmiao1981/handle.
2017-07-07 23:51:32 -07:00
Prashant Sharma d0bfc67335 [SPARK-21069][SS][DOCS] Add rate source to programming guide.
## What changes were proposed in this pull request?

SPARK-20979 added a new structured streaming source: Rate source. This patch adds the corresponding documentation to programming guide.

## How was this patch tested?

Tested by running jekyll locally.

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

Closes #18562 from ScrapCodes/spark-21069/rate-source-docs.
2017-07-07 23:33:12 -07:00
Marcelo Vanzin 9760c15acb [SPARK-20379][CORE] Allow SSL config to reference env variables.
This change exposes the internal code path in SparkConf that allows
configs to be read with variable substitution applied, and uses that
new method in SSLOptions so that SSL configs can reference other
variables, and more importantly, environment variables, providing
a secure way to provide passwords to Spark when using SSL.

The approach is a little bit hacky, but is the smallest change possible.
Otherwise, the concept of "namespaced configs" would have to be added
to the config system, which would create a lot of noise for not much
gain at this point.

Tested with added unit tests, and on a real cluster with SSL enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18394 from vanzin/SPARK-20379.try2.
2017-07-08 14:20:09 +08:00
Takeshi Yamamuro 7896e7b99d [SPARK-21281][SQL] Use string types by default if array and map have no argument
## What changes were proposed in this pull request?
This pr modified code to use string types by default if `array` and `map` in functions have no argument. This behaviour is the same with Hive one;
```
hive> CREATE TEMPORARY TABLE t1 AS SELECT map();
hive> DESCRIBE t1;
_c0   map<string,string>

hive> CREATE TEMPORARY TABLE t2 AS SELECT array();
hive> DESCRIBE t2;
_c0   array<string>
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18516 from maropu/SPARK-21281.
2017-07-07 23:05:38 -07:00
Andrew Ray e1a172c201 [SPARK-21100][SQL] Add summary method as alternative to describe that gives quartiles similar to Pandas
## What changes were proposed in this pull request?

Adds method `summary`  that allows user to specify which statistics and percentiles to calculate. By default it include the existing statistics from `describe` and quartiles (25th, 50th, and 75th percentiles) similar to Pandas. Also changes the implementation of `describe` to delegate to `summary`.

## How was this patch tested?

additional unit test

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

Closes #18307 from aray/SPARK-21100.
2017-07-08 13:47:41 +08:00
Wang Gengliang a0fe32a219 [SPARK-21336] Revise rand comparison in BatchEvalPythonExecSuite
## What changes were proposed in this pull request?

Revise rand comparison in BatchEvalPythonExecSuite

In BatchEvalPythonExecSuite, there are two cases using the case "rand() > 3"
Rand() generates a random value in [0, 1), it is wired to be compared with 3, use 0.3 instead

## How was this patch tested?

unit test

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18560 from gengliangwang/revise_BatchEvalPythonExecSuite.
2017-07-07 15:39:29 -07:00
CodingCat fbbe37ed41 [SPARK-19358][CORE] LiveListenerBus shall log the event name when dropping them due to a fully filled queue
## What changes were proposed in this pull request?

Some dropped event will make the whole application behaves unexpectedly, e.g. some UI problem...we shall log the dropped event name to facilitate the debugging

## How was this patch tested?

Existing tests

Author: CodingCat <zhunansjtu@gmail.com>

Closes #16697 from CodingCat/SPARK-19358.
2017-07-07 20:10:24 +08:00
Wenchen Fan fef081309f [SPARK-21335][SQL] support un-aliased subquery
## What changes were proposed in this pull request?

un-aliased subquery is supported by Spark SQL for a long time. Its semantic was not well defined and had confusing behaviors, and it's not a standard SQL syntax, so we disallowed it in https://issues.apache.org/jira/browse/SPARK-20690 .

However, this is a breaking change, and we do have existing queries using un-aliased subquery. We should add the support back and fix its semantic.

This PR fixes the un-aliased subquery by assigning a default alias name.

After this PR, there is no syntax change from branch 2.2 to master, but we invalid a weird use case:
`SELECT v.i from (SELECT i FROM v)`. Now this query will throw analysis exception because users should not be able to use the qualifier inside a subquery.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18559 from cloud-fan/sub-query.
2017-07-07 20:04:30 +08:00
Yan Facai (颜发才) 56536e9992 [SPARK-21285][ML] VectorAssembler reports the column name of unsupported data type
## What changes were proposed in this pull request?
add the column name in the exception which is raised by unsupported data type.

## How was this patch tested?
+ [x] pass all tests.

Author: Yan Facai (颜发才) <facai.yan@gmail.com>

Closes #18523 from facaiy/ENH/vectorassembler_add_col.
2017-07-07 18:32:01 +08:00
Jacek Laskowski 7fcbb9b57f [SPARK-21313][SS] ConsoleSink's string representation
## What changes were proposed in this pull request?

Add `toString` with options for `ConsoleSink` so it shows nicely in query progress.

**BEFORE**

```
  "sink" : {
    "description" : "org.apache.spark.sql.execution.streaming.ConsoleSink4b340441"
  }
```

**AFTER**

```
  "sink" : {
    "description" : "ConsoleSink[numRows=10, truncate=false]"
  }
```

/cc zsxwing tdas

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18539 from jaceklaskowski/SPARK-21313-ConsoleSink-toString.
2017-07-07 08:31:30 +01:00
Liang-Chi Hsieh 5df99bd364 [SPARK-20703][SQL][FOLLOW-UP] Associate metrics with data writes onto DataFrameWriter operations
## What changes were proposed in this pull request?

Remove time metrics since it seems no way to measure it in non per-row tracking.

## How was this patch tested?

Existing tests.

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

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

Closes #18558 from viirya/SPARK-20703-followup.
2017-07-07 13:12:20 +08:00
Kazuaki Ishizaki c09b31eb8f [SPARK-21217][SQL] Support ColumnVector.Array.to<type>Array()
## What changes were proposed in this pull request?

This PR implements bulk-copy for `ColumnVector.Array.to<type>Array()` methods (e.g. `toIntArray()`) in `ColumnVector.Array` by using `System.arrayCopy()` or `Platform.copyMemory()`.

Before this PR, when one of these method is called, the generic method in `ArrayData` is called. It is not fast since element-wise copy is performed.

This PR can improve performance of a benchmark program by 1.9x and 3.2x.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_131-8u131-b11-0ubuntu1.16.04.2-b11 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz

Int Array                                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
------------------------------------------------------------------------------------------------
ON_HEAP                                        586 /  628         14.3          69.9
OFF_HEAP                                       893 /  902          9.4         106.5
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_131-8u131-b11-0ubuntu1.16.04.2-b11 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz

Int Array                                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
------------------------------------------------------------------------------------------------
ON_HEAP                                        306 /  331         27.4          36.4
OFF_HEAP                                       282 /  287         29.8          33.6
```

Source program
```
    (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => {
      val len = 8 * 1024 * 1024
      val column = ColumnVector.allocate(len * 2, new ArrayType(IntegerType, false), memMode)

      val data = column.arrayData
      var i = 0
      while (i < len) {
        data.putInt(i, i)
        i += 1
      }
      column.putArray(0, 0, len)

      val benchmark = new Benchmark("Int Array", len, minNumIters = 20)
      benchmark.addCase(s"$memMode") { iter =>
        var i = 0
        while (i < 50) {
          column.getArray(0).toIntArray
          i += 1
        }
      }
      benchmark.run
    }}
```

## How was this patch tested?

Added test suite

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #18425 from kiszk/SPARK-21217.
2017-07-07 13:09:32 +08:00
Takuya UESHIN 53c2eb59b2 [SPARK-21327][SQL][PYSPARK] ArrayConstructor should handle an array of typecode 'l' as long rather than int in Python 2.
## What changes were proposed in this pull request?

Currently `ArrayConstructor` handles an array of typecode `'l'` as `int` when converting Python object in Python 2 into Java object, so if the value is larger than `Integer.MAX_VALUE` or smaller than `Integer.MIN_VALUE` then the overflow occurs.

```python
import array
data = [Row(longarray=array.array('l', [-9223372036854775808, 0, 9223372036854775807]))]
df = spark.createDataFrame(data)
df.show(truncate=False)
```

```
+----------+
|longarray |
+----------+
|[0, 0, -1]|
+----------+
```

This should be:

```
+----------------------------------------------+
|longarray                                     |
+----------------------------------------------+
|[-9223372036854775808, 0, 9223372036854775807]|
+----------------------------------------------+
```

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18553 from ueshin/issues/SPARK-21327.
2017-07-07 14:05:22 +09:00
hyukjinkwon d451b7f43d [SPARK-21326][SPARK-21066][ML] Use TextFileFormat in LibSVMFileFormat and allow multiple input paths for determining numFeatures
## What changes were proposed in this pull request?

This is related with [SPARK-19918](https://issues.apache.org/jira/browse/SPARK-19918) and [SPARK-18362](https://issues.apache.org/jira/browse/SPARK-18362).

This PR proposes to use `TextFileFormat` and allow multiple input paths (but with a warning) when determining the number of features in LibSVM data source via an extra scan.

There are three points here:

- The main advantage of this change should be to remove file-listing bottlenecks in driver side.

- Another advantage is ones from using `FileScanRDD`. For example, I guess we can use `spark.sql.files.ignoreCorruptFiles` option when determining the number of features.

- We can unify the schema inference code path in text based data sources. This is also a preparation for [SPARK-21289](https://issues.apache.org/jira/browse/SPARK-21289).

## How was this patch tested?

Unit tests in `LibSVMRelationSuite`.

Closes #18288

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18556 from HyukjinKwon/libsvm-schema.
2017-07-07 12:24:03 +08:00
Jacek Laskowski e5bb26174d [SPARK-21329][SS] Make EventTimeWatermarkExec explicitly UnaryExecNode
## What changes were proposed in this pull request?

Making EventTimeWatermarkExec explicitly UnaryExecNode

/cc tdas zsxwing

## How was this patch tested?

Local build.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18509 from jaceklaskowski/EventTimeWatermarkExec-UnaryExecNode.
2017-07-06 18:11:41 -07:00
Wenchen Fan 40c7add3a4 [SPARK-20946][SQL] Do not update conf for existing SparkContext in SparkSession.getOrCreate
## What changes were proposed in this pull request?

SparkContext is shared by all sessions, we should not update its conf for only one session.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18536 from cloud-fan/config.
2017-07-07 08:44:31 +08:00