Commit graph

19185 commits

Author SHA1 Message Date
Michael Gummelt 2e30c0b9bc [SPARK-19702][MESOS] Increase default refuse_seconds timeout in the Mesos Spark Dispatcher
## What changes were proposed in this pull request?

Increase default refuse_seconds timeout, and make it configurable.  See JIRA for details on how this reduces the risk of starvation.

## How was this patch tested?

Unit tests, Manual testing, and Mesos/Spark integration test suite

cc susanxhuynh skonto jmlvanre

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes #17031 from mgummelt/SPARK-19702-suppress-revive.
2017-03-07 21:29:08 +00:00
Jason White 6f4684622a [SPARK-19561] [PYTHON] cast TimestampType.toInternal output to long
## What changes were proposed in this pull request?

Cast the output of `TimestampType.toInternal` to long to allow for proper Timestamp creation in DataFrames near the epoch.

## How was this patch tested?

Added a new test that fails without the change.

dongjoon-hyun davies Mind taking a look?

The contribution is my original work and I license the work to the project under the project’s open source license.

Author: Jason White <jason.white@shopify.com>

Closes #16896 from JasonMWhite/SPARK-19561.
2017-03-07 13:14:37 -08:00
uncleGen 49570ed05d [SPARK-19803][TEST] flaky BlockManagerReplicationSuite test failure
## What changes were proposed in this pull request?

200ms may be too short. Give more time for replication to happen and new block be reported to master

## How was this patch tested?

test manully

Author: uncleGen <hustyugm@gmail.com>
Author: dylon <hustyugm@gmail.com>

Closes #17144 from uncleGen/SPARK-19803.
2017-03-07 12:24:53 -08:00
Wenchen Fan d69aeeaff4 [SPARK-19516][DOC] update public doc to use SparkSession instead of SparkContext
## What changes were proposed in this pull request?

After Spark 2.0, `SparkSession` becomes the new entry point for Spark applications. We should update the public documents to reflect this.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16856 from cloud-fan/doc.
2017-03-07 11:32:36 -08:00
VinceShieh 4a9034b173 [SPARK-17498][ML] StringIndexer enhancement for handling unseen labels
## What changes were proposed in this pull request?
This PR is an enhancement to ML StringIndexer.
Before this PR, String Indexer only supports "skip"/"error" options to deal with unseen records.
But those unseen records might still be useful and user would like to keep the unseen labels in
certain use cases, This PR enables StringIndexer to support keeping unseen labels as
indices [numLabels].

'''Before
StringIndexer().setHandleInvalid("skip")
StringIndexer().setHandleInvalid("error")
'''After
support the third option "keep"
StringIndexer().setHandleInvalid("keep")

## How was this patch tested?
Test added in StringIndexerSuite

Signed-off-by: VinceShieh <vincent.xieintel.com>
(Please fill in changes proposed in this fix)

Author: VinceShieh <vincent.xie@intel.com>

Closes #16883 from VinceShieh/spark-17498.
2017-03-07 11:24:20 -08:00
Wenchen Fan c05baabf10 [SPARK-19765][SPARK-18549][SQL] UNCACHE TABLE should un-cache all cached plans that refer to this table
## What changes were proposed in this pull request?

When un-cache a table, we should not only remove the cache entry for this table, but also un-cache any other cached plans that refer to this table.

This PR also includes some refactors:

1. use `java.util.LinkedList` to store the cache entries, so that it's safer to remove elements while iterating
2. rename `invalidateCache` to `recacheByPlan`, which is more obvious about what it does.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17097 from cloud-fan/cache.
2017-03-07 09:21:58 -08:00
Takeshi Yamamuro 030acdd1f0 [SPARK-19637][SQL] Add to_json in FunctionRegistry
## What changes were proposed in this pull request?
This pr added entries  in `FunctionRegistry` and supported `to_json` in SQL.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #16981 from maropu/SPARK-19637.
2017-03-07 09:00:14 -08:00
wangzhenhua 932196d9e3 [SPARK-17075][SQL][FOLLOWUP] fix filter estimation issues
## What changes were proposed in this pull request?

1. support boolean type in binary expression estimation.
2. deal with compound Not conditions.
3. avoid convert BigInt/BigDecimal directly to double unless it's within range (0, 1).
4. reorganize test code.

## How was this patch tested?

modify related test cases.

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

Closes #17148 from wzhfy/fixFilter.
2017-03-06 23:53:53 -08:00
windpiger e52499ea9c [SPARK-19832][SQL] DynamicPartitionWriteTask get partitionPath should escape the partition name
## What changes were proposed in this pull request?

Currently in DynamicPartitionWriteTask, when we get the paritionPath of a parition, we just escape the partition value, not escape the partition name.

this will cause some problems for some  special partition name situation, for example :
1) if the partition name contains '%' etc,  there will be two partition path created in the filesytem, one is for escaped path like '/path/a%25b=1', another is for unescaped path like '/path/a%b=1'.
and the data inserted stored in unescaped path, while the show partitions table will return 'a%25b=1' which the partition name is escaped. So here it is not consist. And I think the data should be stored in the escaped path in filesystem, which Hive2.0.0 also have the same action.

2) if the partition name contains ':', there will throw exception that new Path("/path","a:b"), this is illegal which has a colon in the relative path.

```
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: a:b
  at org.apache.hadoop.fs.Path.initialize(Path.java:205)
  at org.apache.hadoop.fs.Path.<init>(Path.java:171)
  at org.apache.hadoop.fs.Path.<init>(Path.java:88)
  ... 48 elided
Caused by: java.net.URISyntaxException: Relative path in absolute URI: a:b
  at java.net.URI.checkPath(URI.java:1823)
  at java.net.URI.<init>(URI.java:745)
  at org.apache.hadoop.fs.Path.initialize(Path.java:202)
  ... 50 more
```
## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #17173 from windpiger/fixDatasourceSpecialCharPartitionName.
2017-03-06 22:36:43 -08:00
actuaryzhang 1f6c090c15 [SPARK-19818][SPARKR] rbind should check for name consistency of input data frames
## What changes were proposed in this pull request?
Added checks for name consistency of input data frames in union.

## How was this patch tested?
new test.

Author: actuaryzhang <actuaryzhang10@gmail.com>

Closes #17159 from actuaryzhang/sparkRUnion.
2017-03-06 21:55:11 -08:00
wangzhenhua 9909f6d361 [SPARK-19350][SQL] Cardinality estimation of Limit and Sample
## What changes were proposed in this pull request?

Before this pr, LocalLimit/GlobalLimit/Sample propagates the same row count and column stats from its child, which is incorrect.
We can get the correct rowCount in Statistics for GlobalLimit/Sample whether cbo is enabled or not.
We don't know the rowCount for LocalLimit because we don't know the partition number at that time. Column stats should not be propagated because we don't know the distribution of columns after Limit or Sample.

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16696 from wzhfy/limitEstimation.
2017-03-06 21:45:36 -08:00
Tyson Condie b0a5cd8909 [SPARK-19719][SS] Kafka writer for both structured streaming and batch queires
## What changes were proposed in this pull request?

Add a new Kafka Sink and Kafka Relation for writing streaming and batch queries, respectively, to Apache Kafka.
### Streaming Kafka Sink
- When addBatch is called
-- If batchId is great than the last written batch
--- Write batch to Kafka
---- Topic will be taken from the record, if present, or from a topic option, which overrides topic in record.
-- Else ignore

### Batch Kafka Sink
- KafkaSourceProvider will implement CreatableRelationProvider
- CreatableRelationProvider#createRelation will write the passed in Dataframe to a Kafka
- Topic will be taken from the record, if present, or from topic option, which overrides topic in record.
- Save modes Append and ErrorIfExist supported under identical semantics. Other save modes result in an AnalysisException

tdas zsxwing

## How was this patch tested?

### The following unit tests will be included
- write to stream with topic field: valid stream write with data that includes an existing topic in the schema
- write structured streaming aggregation w/o topic field, with default topic: valid stream write with data that does not include a topic field, but the configuration includes a default topic
- write data with bad schema: various cases of writing data that does not conform to a proper schema e.g., 1. no topic field or default topic, and 2. no value field
- write data with valid schema but wrong types: data with a complete schema but wrong types e.g., key and value types are integers.
- write to non-existing topic: write a stream to a topic that does not exist in Kafka, which has been configured to not auto-create topics.
- write batch to kafka: simple write batch to Kafka, which goes through the same code path as streaming scenario, so validity checks will not be redone here.

### Examples
```scala
// Structured Streaming
val writer = inputStringStream.map(s => s.get(0).toString.getBytes()).toDF("value")
 .selectExpr("value as key", "value as value")
 .writeStream
 .format("kafka")
 .option("checkpointLocation", checkpointDir)
 .outputMode(OutputMode.Append)
 .option("kafka.bootstrap.servers", brokerAddress)
 .option("topic", topic)
 .queryName("kafkaStream")
 .start()

// Batch
val df = spark
 .sparkContext
 .parallelize(Seq("1", "2", "3", "4", "5"))
 .map(v => (topic, v))
 .toDF("topic", "value")

df.write
 .format("kafka")
 .option("kafka.bootstrap.servers",brokerAddress)
 .option("topic", topic)
 .save()
```
Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Tyson Condie <tcondie@gmail.com>

Closes #17043 from tcondie/kafka-writer.
2017-03-06 16:39:05 -08:00
Wojtek Szymanski f6471dc0d5 [SPARK-19709][SQL] Read empty file with CSV data source
## What changes were proposed in this pull request?

Bugfix for reading empty file with CSV data source. Instead of throwing `NoSuchElementException`, an empty data frame is returned.

## How was this patch tested?

Added new unit test in `org.apache.spark.sql.execution.datasources.csv.CSVSuite`

Author: Wojtek Szymanski <wk.szymanski@gmail.com>

Closes #17068 from wojtek-szymanski/SPARK-19709.
2017-03-06 13:19:36 -08:00
wm624@hotmail.com 926543664f [SPARK-19382][ML] Test sparse vectors in LinearSVCSuite
## What changes were proposed in this pull request?

Add unit tests for testing SparseVector.

We can't add mixed DenseVector and SparseVector test case, as discussed in JIRA 19382.

 def merge(other: MultivariateOnlineSummarizer): this.type = {
if (this.totalWeightSum != 0.0 && other.totalWeightSum != 0.0) {
require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " +
s"Expecting $n but got $
{other.n}

.")

## How was this patch tested?

Unit tests

Author: wm624@hotmail.com <wm624@hotmail.com>
Author: Miao Wang <wangmiao1981@users.noreply.github.com>

Closes #16784 from wangmiao1981/bk.
2017-03-06 13:08:59 -08:00
jiangxingbo 9991c2dad6 [SPARK-19211][SQL] Explicitly prevent Insert into View or Create View As Insert
## What changes were proposed in this pull request?

Currently we don't explicitly forbid the following behaviors:
1. The statement CREATE VIEW AS INSERT INTO throws the following exception:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.hadoop.hive.ql.metadata.HiveException: at least one column must be specified for the table;
 scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: The number of columns produced by the SELECT clause (num: `0`) does not match the number of column names specified by CREATE VIEW (num: `2`).;
```

2. The statement INSERT INTO view VALUES throws the following exception from checkAnalysis:
```
scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: Inserting into an RDD-based table is not allowed.;;
'InsertIntoTable View (`default`.`testView`, [a#16,b#17]), false, false
+- LocalRelation [col1#14, col2#15]
```

After this PR, the behavior changes to:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;

scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;

scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: `default`.`testView` is a view, inserting into a view is not allowed;
```

## How was this patch tested?

Add a new test case in `SparkSqlParserSuite`;
Update the corresponding test case in `SQLViewSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17125 from jiangxb1987/insert-with-view.
2017-03-06 12:35:03 -08:00
Imran Rashid 12bf832407 [SPARK-19796][CORE] Fix serialization of long property values in TaskDescription
## What changes were proposed in this pull request?

The properties that are serialized with a TaskDescription can have very long values (eg. "spark.job.description" which is set to the full sql statement with the thrift-server).  DataOutputStream.writeUTF() does not work well for long strings, so this changes the way those values are serialized to handle longer strings.

## How was this patch tested?

Updated existing unit test to reproduce the issue.  All unit tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #17140 from squito/SPARK-19796.
2017-03-06 14:06:11 -06:00
windpiger 096df6d933 [SPARK-19257][SQL] location for table/partition/database should be java.net.URI
## What changes were proposed in this pull request?

Currently we treat the location of table/partition/database as URI string.

It will be safer if we can make the type of location as java.net.URI.

In this PR, there are following classes changes:
**1. CatalogDatabase**
```
case class CatalogDatabase(
    name: String,
    description: String,
    locationUri: String,
    properties: Map[String, String])
--->
case class CatalogDatabase(
    name: String,
    description: String,
    locationUri: URI,
    properties: Map[String, String])
```
**2. CatalogStorageFormat**
```
case class CatalogStorageFormat(
    locationUri: Option[String],
    inputFormat: Option[String],
    outputFormat: Option[String],
    serde: Option[String],
    compressed: Boolean,
    properties: Map[String, String])
---->
case class CatalogStorageFormat(
    locationUri: Option[URI],
    inputFormat: Option[String],
    outputFormat: Option[String],
    serde: Option[String],
    compressed: Boolean,
    properties: Map[String, String])
```

Before and After this PR, it is transparent for user, there is no change that the user should concern. The `String` to `URI` just happened in SparkSQL internally.

Here list some operation related location:
**1. whitespace in the location**
   e.g.  `/a/b c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...) LOCATION '/a/b c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b c/d`,
   and the real path in the FileSystem also show `/a/b c/d`

**2. colon(:) in the location**
   e.g.  `/a/b:c/d`
   For both table location and partition location,
   when `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION '/a/b:c/d'` ,

  **In linux file system**
   `DESC EXTENDED t ` show the location is `/a/b:c/d`,
   and the real path in the FileSystem also show `/a/b:c/d`

  **in HDFS** throw exception:
  `java.lang.IllegalArgumentException: Pathname /a/b:c/d from hdfs://iZbp1151s8hbnnwriekxdeZ:9000/a/b:c/d is not a valid DFS filename.`

  **while** After `INSERT INTO TABLE t PARTITION(a="a:b") SELECT 1`
   then `DESC EXTENDED t ` show the location is `/xxx/a=a%3Ab`,
   and the real path in the FileSystem also show `/xxx/a=a%3Ab`

**3. percent sign(%) in the location**
   e.g.  `/a/b%c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...) LOCATION '/a/b%c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b%c/d`,
   and the real path in the FileSystem also show `/a/b%c/d`

**4. encoded(%25) in the location**
   e.g.  `/a/b%25c/d`
   For both table location and partition location,
   After `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION '/a/b%25c/d'` ,
   then `DESC EXTENDED t ` show the location is `/a/b%25c/d`,
   and the real path in the FileSystem also show `/a/b%25c/d`

   **while** After `INSERT INTO TABLE t PARTITION(a="%25") SELECT 1`
   then `DESC EXTENDED t ` show the location is `/xxx/a=%2525`,
   and the real path in the FileSystem also show `/xxx/a=%2525`

**Additionally**, except the location, there are two other factors will affect the location of the table/partition. one is the table name which does not allowed to have special characters, and the  other is `partition name` which have the same actions with `partition value`, and `partition name` with special character situation has add some testcase and resolve a bug in [PR](https://github.com/apache/spark/pull/17173)

### Summary:
After `CREATE TABLE  t... (PARTITIONED BY ...)  LOCATION path`,
the path which we get from `DESC TABLE` and `real path in FileSystem` are all the same with the `CREATE TABLE` command(different filesystem has different action that allow what kind of special character to create the path, e.g. HDFS does not allow colon, but linux filesystem allow it ).

`DataBase` also have the same logic with `CREATE TABLE`

while if the `partition value` has some special character like `%` `:` `#` etc, then we will get the path with encoded `partition value` like `/xxx/a=A%25B` from `DESC TABLE` and `real path in FileSystem`

In this PR, the core change code is using `new Path(str).toUri` and `new Path(uri).toString`
which transfrom `str to uri `or `uri to str`.
for example:
```
val str = '/a/b c/d'
val uri = new Path(str).toUri  --> '/a/b%20c/d'
val strFromUri = new Path(uri).toString -> '/a/b c/d'
```

when we restore table/partition from metastore, or get the location from `CREATE TABLE` command, we can use it as above to change string to uri `new Path(str).toUri `

## How was this patch tested?
unit test added.
The `current master branch` also `passed all the test cases` added in this PR by a litter change.
https://github.com/apache/spark/pull/17149/files#diff-b7094baa12601424a5d19cb930e3402fR1764
here `toURI` -> `toString` when test in master branch.

This can show that this PR  is transparent for user.

Author: windpiger <songjun@outlook.com>

Closes #17149 from windpiger/changeStringToURI.
2017-03-06 10:44:26 -08:00
Gaurav 46a64d1e0a [SPARK-19304][STREAMING][KINESIS] fix kinesis slow checkpoint recovery
## What changes were proposed in this pull request?
added a limit to getRecords api call call in KinesisBackedBlockRdd. This helps reduce the amount of data returned by kinesis api call making the recovery considerably faster

As we are storing the `fromSeqNum` & `toSeqNum` in checkpoint metadata, we can also store the number of records. Which can later be used for api call.

## How was this patch tested?
The patch was manually tested

Apologies for any silly mistakes, opening first pull request

Author: Gaurav <gaurav@techtinium.com>

Closes #16842 from Gauravshah/kinesis_checkpoint_recovery_fix_2_1_0.
2017-03-06 10:41:49 -08:00
Cheng Lian 339b53a131 [SPARK-19737][SQL] New analysis rule for reporting unregistered functions without relying on relation resolution
## What changes were proposed in this pull request?

This PR adds a new `Once` analysis rule batch consists of a single analysis rule `LookupFunctions` that performs simple existence check over `UnresolvedFunctions` without actually resolving them.

The benefit of this rule is that it doesn't require function arguments to be resolved first and therefore doesn't rely on relation resolution, which may incur potentially expensive partition/schema discovery cost.

Please refer to [SPARK-19737][1] for more details about the motivation.

## How was this patch tested?

New test case added in `AnalysisErrorSuite`.

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

Author: Cheng Lian <lian@databricks.com>

Closes #17168 from liancheng/spark-19737-lookup-functions.
2017-03-06 10:36:50 -08:00
Tejas Patil 2a0bc867a4 [SPARK-17495][SQL] Support Decimal type in Hive-hash
## What changes were proposed in this pull request?

Hive hash to support Decimal datatype. [Hive internally normalises decimals](4ba713ccd8/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java (L307)) and I have ported that logic as-is to HiveHash.

## How was this patch tested?

Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17056 from tejasapatil/SPARK-17495_decimal.
2017-03-06 10:16:20 -08:00
uncleGen 207067ead6 [SPARK-19822][TEST] CheckpointSuite.testCheckpointedOperation: should not filter checkpointFilesOfLatestTime with the PATH string.
## What changes were proposed in this pull request?

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73800/testReport/

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException: The code
passed to eventually never returned normally. Attempted 617 times over 10.003740484 seconds.
Last failure message: 8 did not equal 2.
	at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
	at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:336)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	at org.apache.spark.streaming.DStreamCheckpointTester$class.generateOutput(CheckpointSuite
.scala:172)
	at org.apache.spark.streaming.CheckpointSuite.generateOutput(CheckpointSuite.scala:211)
```

the check condition is:

```
val checkpointFilesOfLatestTime = Checkpoint.getCheckpointFiles(checkpointDir).filter {
     _.toString.contains(clock.getTimeMillis.toString)
}
// Checkpoint files are written twice for every batch interval. So assert that both
// are written to make sure that both of them have been written.
assert(checkpointFilesOfLatestTime.size === 2)
```

the path string may contain the `clock.getTimeMillis.toString`, like `3500` :

```
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500.bk
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500
                                                       ▲▲▲▲
```

so we should only check the filename, but not the whole path.

## How was this patch tested?

Jenkins.

Author: uncleGen <hustyugm@gmail.com>

Closes #17167 from uncleGen/flaky-CheckpointSuite.
2017-03-05 18:17:30 -08:00
hyukjinkwon 224e0e785b [SPARK-19701][SQL][PYTHON] Throws a correct exception for 'in' operator against column
## What changes were proposed in this pull request?

This PR proposes to remove incorrect implementation that has been not executed so far (at least from Spark 1.5.2) for `in` operator and throw a correct exception rather than saying it is a bool. I tested the codes above in 1.5.2, 1.6.3, 2.1.0 and in the master branch as below:

**1.5.2**

```python
>>> df = sqlContext.createDataFrame([[1]])
>>> 1 in df._1
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark-1.5.2-bin-hadoop2.6/python/pyspark/sql/column.py", line 418, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**1.6.3**

```python
>>> 1 in sqlContext.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark-1.6.3-bin-hadoop2.6/python/pyspark/sql/column.py", line 447, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**2.1.0**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark-2.1.0-bin-hadoop2.7/python/pyspark/sql/column.py", line 426, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**Current Master**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/column.py", line 452, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**After**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/column.py", line 184, in __contains__
    raise ValueError("Cannot apply 'in' operator against a column: please use 'contains' "
ValueError: Cannot apply 'in' operator against a column: please use 'contains' in a string column or 'array_contains' function for an array column.
```

In more details,

It seems the implementation intended to support this

```python
1 in df.column
```

However, currently, it throws an exception as below:

```python
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/column.py", line 426, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

What happens here is as below:

```python
class Column(object):
    def __contains__(self, item):
        print "I am contains"
        return Column()
    def __nonzero__(self):
        raise Exception("I am nonzero.")

>>> 1 in Column()
I am contains
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "<stdin>", line 6, in __nonzero__
Exception: I am nonzero.
```

It seems it calls `__contains__` first and then `__nonzero__` or `__bool__` is being called against `Column()` to make this a bool (or int to be specific).

It seems `__nonzero__` (for Python 2), `__bool__` (for Python 3) and `__contains__` forcing the the return into a bool unlike other operators. There are few references about this as below:

https://bugs.python.org/issue16011
http://stackoverflow.com/questions/12244074/python-source-code-for-built-in-in-operator/12244378#12244378
http://stackoverflow.com/questions/38542543/functionality-of-python-in-vs-contains/38542777

It seems we can't overwrite `__nonzero__` or `__bool__` as a workaround to make this working because these force the return type as a bool as below:

```python
class Column(object):
    def __contains__(self, item):
        print "I am contains"
        return Column()
    def __nonzero__(self):
        return "a"

>>> 1 in Column()
I am contains
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
TypeError: __nonzero__ should return bool or int, returned str
```

## How was this patch tested?

Added unit tests in `tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17160 from HyukjinKwon/SPARK-19701.
2017-03-05 18:04:52 -08:00
Sue Ann Hong 70f9d7f71c [SPARK-19535][ML] RecommendForAllUsers RecommendForAllItems for ALS on Dataframe
## What changes were proposed in this pull request?

This is a simple implementation of RecommendForAllUsers & RecommendForAllItems for the Dataframe version of ALS. It uses Dataframe operations (not a wrapper on the RDD implementation). Haven't benchmarked against a wrapper, but unit test examples do work.

## How was this patch tested?

Unit tests
```
$ build/sbt
> mllib/testOnly *ALSSuite -- -z "recommendFor"
> mllib/testOnly
```

Author: Your Name <you@example.com>
Author: sueann <sueann@databricks.com>

Closes #17090 from sueann/SPARK-19535.
2017-03-05 16:49:31 -08:00
hyukjinkwon 369a148e59 [SPARK-19595][SQL] Support json array in from_json
## What changes were proposed in this pull request?

This PR proposes to both,

**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**

Currently, it only reads the single row when the input is a json array. So, the codes below:

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                 [1]|
+--------------------+
```

This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                null|
+--------------------+
```

**Support json arrays in `from_json` with `ArrayType` as the schema.**

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```

prints

```
+-------------------+
|jsontostruct(array)|
+-------------------+
|         [[1], [2]]|
+-------------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16929 from HyukjinKwon/disallow-array.
2017-03-05 14:35:06 -08:00
Felix Cheung 80d5338b32 [SPARK-19795][SPARKR] add column functions to_json, from_json
## What changes were proposed in this pull request?

Add column functions: to_json, from_json, and tests covering error cases.

## How was this patch tested?

unit tests, manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17134 from felixcheung/rtojson.
2017-03-05 12:37:02 -08:00
Takeshi Yamamuro 14bb398fae [SPARK-19254][SQL] Support Seq, Map, and Struct in functions.lit
## What changes were proposed in this pull request?
This pr is to support Seq, Map, and Struct in functions.lit; it adds a new IF named `lit2` with `TypeTag` for avoiding type erasure.

## How was this patch tested?
Added tests in `LiteralExpressionSuite`

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #16610 from maropu/SPARK-19254.
2017-03-05 03:53:19 -08:00
uncleGen f48461ab2b [SPARK-19805][TEST] Log the row type when query result dose not match
## What changes were proposed in this pull request?

improve the log message when query result does not match.

before pr:

```
== Results ==
!== Correct Answer - 3 ==   == Spark Answer - 3 ==
 [1]                        [1]
 [2]                        [2]
 [3]                        [3]

```

after pr:

~~== Results ==
!== Correct Answer - 3 ==   == Spark Answer - 3 ==
!RowType[string]            RowType[integer]
 [1]                        [1]
 [2]                        [2]
 [3]                        [3]~~

```
== Results ==
!== Correct Answer - 3 ==   == Spark Answer - 3 ==
!struct<value:string>       struct<value:int>
 [1]                        [1]
 [2]                        [2]
 [3]                        [3]
```

## How was this patch tested?

Jenkins

Author: uncleGen <hustyugm@gmail.com>

Closes #17145 from uncleGen/improve-test-result.
2017-03-05 03:35:42 -08:00
liuxian 42c4cd9e2a [SPARK-19792][WEBUI] In the Master Page,the column named “Memory per Node” ,I think it is not all right
Signed-off-by: liuxian <liu.xian3zte.com.cn>

## What changes were proposed in this pull request?

Open the spark web page,in the Master Page ,have two tables:Running Applications table and Completed Applications table, to the column named “Memory per Node” ,I think it is not all right ,because a node may be not have only one executor.So I think that should be named as “Memory per Executor”.Otherwise easy to let the user misunderstanding

## How was this patch tested?

N/A

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17132 from 10110346/wid-lx-0302.
2017-03-05 10:23:50 +00:00
Yuming Wang 6b0cfd9fa5 [SPARK-19550][SPARKR][DOCS] Update R document to use JDK8
## What changes were proposed in this pull request?

Update R document to use JDK8.

## How was this patch tested?

manual tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17162 from wangyum/SPARK-19550.
2017-03-04 16:43:31 +00:00
Shixiong Zhu fbc4058037 [SPARK-19816][SQL][TESTS] Fix an issue that DataFrameCallbackSuite doesn't recover the log level
## What changes were proposed in this pull request?

"DataFrameCallbackSuite.execute callback functions when a DataFrame action failed" sets the log level to "fatal" but doesn't recover it. Hence, tests running after it won't output any logs except fatal logs.

This PR uses `testQuietly` instead to avoid changing the log level.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17156 from zsxwing/SPARK-19816.
2017-03-03 19:00:35 -08:00
Marcelo Vanzin 9e5b4ce727 [SPARK-19084][SQL] Ensure context class loader is set when initializing Hive.
A change in Hive 2.2 (most probably HIVE-13149) causes this code path to fail,
since the call to "state.getConf.setClassLoader" does not actually change the
context's class loader. Spark doesn't yet officially support Hive 2.2, but some
distribution-specific metastore client libraries may have that change (as certain
versions of CDH already do), and this also makes it easier to support 2.2 when it
comes out.

Tested with existing unit tests; we've also used this patch extensively with Hive
metastore client jars containing the offending patch.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17154 from vanzin/SPARK-19804.
2017-03-03 18:44:31 -08:00
Shixiong Zhu a6a7a95e2f [SPARK-19718][SS] Handle more interrupt cases properly for Hadoop
## What changes were proposed in this pull request?

[SPARK-19617](https://issues.apache.org/jira/browse/SPARK-19617) changed `HDFSMetadataLog` to enable interrupts when using the local file system. However, now we hit [HADOOP-12074](https://issues.apache.org/jira/browse/HADOOP-12074): `Shell.runCommand` converts `InterruptedException` to `new IOException(ie.toString())` before Hadoop 2.8. This is the Hadoop patch to fix HADOOP-1207: 95c73d49b1

This PR adds new logic to handle the following cases related to `InterruptedException`.
- Check if the message of IOException starts with `java.lang.InterruptedException`. If so, treat it as `InterruptedException`. This is for pre-Hadoop 2.8.
- Treat `InterruptedIOException` as `InterruptedException`. This is for Hadoop 2.8+ and other places that may throw `InterruptedIOException` when the thread is interrupted.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17044 from zsxwing/SPARK-19718.
2017-03-03 17:10:11 -08:00
Xiao Li f5fdbe0436 [SPARK-13446][SQL] Support reading data from Hive 2.0.1 metastore
### What changes were proposed in this pull request?
This PR is to make Spark work with Hive 2.0's metastores. Compared with Hive 1.2, Hive 2.0's metastore has an API update due to removal of `HOLD_DDLTIME` in https://issues.apache.org/jira/browse/HIVE-12224. Based on the following Hive JIRA description, `HOLD_DDLTIME` should be removed from our internal API too. (https://github.com/apache/spark/pull/17063 was submitted for it):
> This arcane feature was introduced long ago via HIVE-1394 It was broken as soon as it landed, HIVE-1442 and is thus useless. Fact that no one has fixed it since informs that its not really used by anyone. Better is to remove it so no one hits the bug of HIVE-1442

In the next PR, we will support 2.1.0 metastore, whose APIs were changed due to https://issues.apache.org/jira/browse/HIVE-12730. However, before that, we need a code cleanup for stats collection and setting.

### How was this patch tested?
Added test cases to VersionsSuite.scala

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17061 from gatorsmile/Hive2.
2017-03-03 16:59:52 -08:00
Bryan Cutler 44281ca81d [SPARK-19348][PYTHON] PySpark keyword_only decorator is not thread-safe
## What changes were proposed in this pull request?
The `keyword_only` decorator in PySpark is not thread-safe.  It writes kwargs to a static class variable in the decorator, which is then retrieved later in the class method as `_input_kwargs`.  If multiple threads are constructing the same class with different kwargs, it becomes a race condition to read from the static class variable before it's overwritten.  See [SPARK-19348](https://issues.apache.org/jira/browse/SPARK-19348) for reproduction code.

This change will write the kwargs to a member variable so that multiple threads can operate on separate instances without the race condition.  It does not protect against multiple threads operating on a single instance, but that is better left to the user to synchronize.

## How was this patch tested?
Added new unit tests for using the keyword_only decorator and a regression test that verifies `_input_kwargs` can be overwritten from different class instances.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #16782 from BryanCutler/pyspark-keyword_only-threadsafe-SPARK-19348.
2017-03-03 16:43:45 -08:00
Takuya UESHIN 2a7921a813 [SPARK-18939][SQL] Timezone support in partition values.
## What changes were proposed in this pull request?

This is a follow-up pr of #16308 and #16750.

This pr enables timezone support in partition values.

We should use `timeZone` option introduced at #16750 to parse/format partition values of the `TimestampType`.

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT` which will be used for partition values, the values written by the default timezone option, which is `"GMT"` because the session local timezone is `"GMT"` here, are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")

scala> val df = Seq((1, new java.sql.Timestamp(1451606400000L))).toDF("i", "ts")
df: org.apache.spark.sql.DataFrame = [i: int, ts: timestamp]

scala> df.show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2016-01-01 00:00:00|
+---+-------------------+

scala> df.write.partitionBy("ts").save("/path/to/gmtpartition")
```

```sh
$ ls /path/to/gmtpartition/
_SUCCESS			ts=2016-01-01 00%3A00%3A00
```

whereas setting the option to `"PST"`, they are:

```scala
scala> df.write.option("timeZone", "PST").partitionBy("ts").save("/path/to/pstpartition")
```

```sh
$ ls /path/to/pstpartition/
_SUCCESS			ts=2015-12-31 16%3A00%3A00
```

We can properly read the partition values if the session local timezone and the timezone of the partition values are the same:

```scala
scala> spark.read.load("/path/to/gmtpartition").show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2016-01-01 00:00:00|
+---+-------------------+
```

And even if the timezones are different, we can properly read the values with setting corrent timezone option:

```scala
// wrong result
scala> spark.read.load("/path/to/pstpartition").show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2015-12-31 16:00:00|
+---+-------------------+

// correct result
scala> spark.read.option("timeZone", "PST").load("/path/to/pstpartition").show()
+---+-------------------+
|  i|                 ts|
+---+-------------------+
|  1|2016-01-01 00:00:00|
+---+-------------------+
```

## How was this patch tested?

Existing tests and added some tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #17053 from ueshin/issues/SPARK-18939.
2017-03-03 16:35:54 -08:00
jerryshao ba186a841f [MINOR][DOC] Fix doc for web UI https configuration
## What changes were proposed in this pull request?

Doc about enabling web UI https is not correct, "spark.ui.https.enabled" is not existed, actually enabling SSL is enough for https.

## How was this patch tested?

N/A

Author: jerryshao <sshao@hortonworks.com>

Closes #17147 from jerryshao/fix-doc-ssl.
2017-03-03 14:23:31 -08:00
Burak Yavuz 9314c08377 [SPARK-19774] StreamExecution should call stop() on sources when a stream fails
## What changes were proposed in this pull request?

We call stop() on a Structured Streaming Source only when the stream is shutdown when a user calls streamingQuery.stop(). We should actually stop all sources when the stream fails as well, otherwise we may leak resources, e.g. connections to Kafka.

## How was this patch tested?

Unit tests in `StreamingQuerySuite`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17107 from brkyvz/close-source.
2017-03-03 10:35:15 -08:00
Pete Robbins 37a1c0e461 [SPARK-19710][SQL][TESTS] Fix ordering of rows in query results
## What changes were proposed in this pull request?
Changes to SQLQueryTests to make the order of the results constant.
Where possible ORDER BY has been added to match the existing expected output

## How was this patch tested?
Test runs on x86, zLinux (big endian), ppc (big endian)

Author: Pete Robbins <robbinspg@gmail.com>

Closes #17039 from robbinspg/SPARK-19710.
2017-03-03 07:53:46 -08:00
Liang-Chi Hsieh 98bcc188f9 [SPARK-19758][SQL] Resolving timezone aware expressions with time zone when resolving inline table
## What changes were proposed in this pull request?

When we resolve inline tables in analyzer, we will evaluate the expressions of inline tables.

When it evaluates a `TimeZoneAwareExpression` expression, an error will happen because the `TimeZoneAwareExpression` is not associated with timezone yet.

So we need to resolve these `TimeZoneAwareExpression`s with time zone when resolving inline tables.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17114 from viirya/resolve-timeawareexpr-inline-table.
2017-03-03 07:14:37 -08:00
Dongjoon Hyun 776fac3988 [SPARK-19801][BUILD] Remove JDK7 from Travis CI
## What changes were proposed in this pull request?

Since Spark 2.1.0, Travis CI was supported by SPARK-15207 for automated PR verification (JDK7/JDK8 maven compilation and Java Linter) and contributors can see the additional result via their Travis CI dashboard (or PC).

This PR aims to make `.travis.yml` up-to-date by removing JDK7 which was removed via SPARK-19550.

## How was this patch tested?

See the result via Travis CI.

- https://travis-ci.org/dongjoon-hyun/spark/builds/207111713

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17143 from dongjoon-hyun/SPARK-19801.
2017-03-03 12:00:54 +01:00
Zhe Sun 0bac3e4cde [SPARK-19797][DOC] ML pipeline document correction
## What changes were proposed in this pull request?
Description about pipeline in this paragraph is incorrect https://spark.apache.org/docs/latest/ml-pipeline.html#how-it-works

> If the Pipeline had more **stages**, it would call the LogisticRegressionModel’s transform() method on the DataFrame before passing the DataFrame to the next stage.

Reason: Transformer could also be a stage. But only another Estimator will invoke an transform call and pass the data to next stage. The description in the document misleads ML pipeline users.

## How was this patch tested?
This is a tiny modification of **docs/ml-pipelines.md**. I jekyll build the modification and check the compiled document.

Author: Zhe Sun <ymwdalex@gmail.com>

Closes #17137 from ymwdalex/SPARK-19797-ML-pipeline-document-correction.
2017-03-03 11:55:57 +01:00
uncleGen fa50143cd3 [SPARK-19739][CORE] propagate S3 session token to cluser
## What changes were proposed in this pull request?

propagate S3 session token to cluser

## How was this patch tested?

existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #17080 from uncleGen/SPARK-19739.
2017-03-03 11:49:00 +01:00
hyukjinkwon d556b31703 [SPARK-18699][SQL][FOLLOWUP] Add explanation in CSV parser and minor cleanup
## What changes were proposed in this pull request?

This PR suggests adding some comments in `UnivocityParser` logics to explain what happens. Also, it proposes, IMHO, a little bit cleaner (at least easy for me to explain).

## How was this patch tested?

Unit tests in `CSVSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17142 from HyukjinKwon/SPARK-18699.
2017-03-03 00:50:58 -08:00
windpiger 982f3223b4 [SPARK-18726][SQL] resolveRelation for FileFormat DataSource don't need to listFiles twice
## What changes were proposed in this pull request?

Currently when we resolveRelation for a `FileFormat DataSource` without providing user schema, it will execute `listFiles`  twice in `InMemoryFileIndex` during `resolveRelation`.

This PR add a `FileStatusCache` for DataSource, this can avoid listFiles twice.

But there is a bug in `InMemoryFileIndex` see:
 [SPARK-19748](https://github.com/apache/spark/pull/17079)
 [SPARK-19761](https://github.com/apache/spark/pull/17093),
so this pr should be after SPARK-19748/ SPARK-19761.

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #17081 from windpiger/resolveDataSourceScanFilesTwice.
2017-03-02 23:54:01 -08:00
guifeng e24f21b5f8 [SPARK-19779][SS] Delete needless tmp file after restart structured streaming job
## What changes were proposed in this pull request?

[SPARK-19779](https://issues.apache.org/jira/browse/SPARK-19779)

The PR (https://github.com/apache/spark/pull/17012) can to fix restart a Structured Streaming application using hdfs as fileSystem, but also exist a problem that a tmp file of delta file is still reserved in hdfs. And Structured Streaming don't delete the tmp file generated when restart streaming job in future.

## How was this patch tested?
 unit tests

Author: guifeng <guifengleaf@gmail.com>

Closes #17124 from gf53520/SPARK-19779.
2017-03-02 21:19:29 -08:00
Sunitha Kambhampati f37bb14302 [SPARK-19602][SQL][TESTS] Add tests for qualified column names
## What changes were proposed in this pull request?
- Add tests covering different scenarios with qualified column names
- Please see Section 2 in the design doc for the various test scenarios [here](https://issues.apache.org/jira/secure/attachment/12854681/Design_ColResolution_JIRA19602.pdf)
- As part of SPARK-19602, changes are made to support three part column name. In order to aid in the review and to reduce the diff, the test scenarios are separated out into this PR.

## How was this patch tested?
- This is a **test only** change. The individual test suites were run successfully.

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #17067 from skambha/colResolutionTests.
2017-03-02 21:19:22 -08:00
sethah 93ae176e89 [SPARK-19745][ML] SVCAggregator captures coefficients in its closure
## What changes were proposed in this pull request?

JIRA: [SPARK-19745](https://issues.apache.org/jira/browse/SPARK-19745)

Reorganize SVCAggregator to avoid serializing coefficients. This patch also makes the gradient array a `lazy val` which will avoid materializing a large array on the driver before shipping the class to the executors. This improvement stems from https://github.com/apache/spark/pull/16037. Actually, probably all ML aggregators can benefit from this.

We can either: a.) separate the gradient improvement into another patch b.) keep what's here _plus_ add the lazy evaluation to all other aggregators in this patch or c.) keep it as is.

## How was this patch tested?

This is an interesting question! I don't know of a reasonable way to test this right now. Ideally, we could perform an optimization and look at the shuffle write data for each task, and we could compare the size to what it we know it should be: `numCoefficients * 8 bytes`. Not sure if there is a good way to do that right now? We could discuss this here or in another JIRA, but I suspect it would be a significant undertaking.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #17076 from sethah/svc_agg.
2017-03-02 19:38:25 -08:00
Imran Rashid 8417a7ae6c [SPARK-19276][CORE] Fetch Failure handling robust to user error handling
## What changes were proposed in this pull request?

Fault-tolerance in spark requires special handling of shuffle fetch
failures.  The Executor would catch FetchFailedException and send a
special msg back to the driver.

However, intervening user code could intercept that exception, and wrap
it with something else.  This even happens in SparkSQL.  So rather than
checking the thrown exception only, we'll store the fetch failure directly
in the TaskContext, where users can't touch it.

## How was this patch tested?

Added a test case which failed before the fix.  Full test suite via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #16639 from squito/SPARK-19276.
2017-03-02 16:46:01 -08:00
Patrick Woody 433d9eb615 [SPARK-19631][CORE] OutputCommitCoordinator should not allow commits for already failed tasks
## What changes were proposed in this pull request?

Previously it was possible for there to be a race between a task failure and committing the output of a task. For example, the driver may mark a task attempt as failed due to an executor heartbeat timeout (possibly due to GC), but the task attempt actually ends up coordinating with the OutputCommitCoordinator once the executor recovers and committing its result. This will lead to any retry attempt failing because the task result has already been committed despite the original attempt failing.

This ensures that any previously failed task attempts cannot enter the commit protocol.

## How was this patch tested?

Added a unit test

Author: Patrick Woody <pwoody@palantir.com>

Closes #16959 from pwoody/pw/recordFailuresForCommitter.
2017-03-02 15:55:32 -08:00
Mark Grover 5ae3516bfb [SPARK-19720][CORE] Redact sensitive information from SparkSubmit console
## What changes were proposed in this pull request?
This change redacts senstive information (based on `spark.redaction.regex` property)
from the Spark Submit console logs. Such sensitive information is already being
redacted from event logs and yarn logs, etc.

## How was this patch tested?
Testing was done manually to make sure that the console logs were not printing any
sensitive information.

Here's some output from the console:

```
Spark properties used, including those specified through
 --conf and those from the properties file /etc/spark2/conf/spark-defaults.conf:
  (spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted))
  (spark.authenticate,false)
  (spark.executorEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted))
```

```
System properties:
(spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted))
(spark.authenticate,false)
(spark.executorEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted))
```
There is a risk if new print statements were added to the console down the road, sensitive information may still get leaked, since there is no test that asserts on the console log output. I considered it out of the scope of this JIRA to write an integration test to make sure new leaks don't happen in the future.

Running unit tests to make sure nothing else is broken by this change.

Author: Mark Grover <mark@apache.org>

Closes #17047 from markgrover/master_redaction.
2017-03-02 10:33:56 -08:00