Commit graph

19254 commits

Author SHA1 Message Date
Juan Rodriguez Hortala 362ee93296 logging improvements
## What changes were proposed in this pull request?
Adding additional information to existing logging messages:
  - YarnAllocator: log the executor ID together with the container id when a container for an executor is launched.
  - NettyRpcEnv: log the receiver address when there is a timeout waiting for an answer to a remote call.
  - ExecutorAllocationManager: fix a typo in the logging message for the list of executors to be removed.

## How was this patch tested?
Build spark and submit the word count example to a YARN cluster using cluster mode

Author: Juan Rodriguez Hortala <hortala@amazon.com>

Closes #17411 from juanrh/logging-improvements.
2017-03-26 10:39:05 +01:00
Kazuaki Ishizaki 93bb0b911b [SPARK-20046][SQL] Facilitate loop optimizations in a JIT compiler regarding sqlContext.read.parquet()
## What changes were proposed in this pull request?

This PR improves performance of operations with `sqlContext.read.parquet()` by changing Java code generated by Catalyst. This PR is inspired by [the blog article](https://databricks.com/blog/2017/02/16/processing-trillion-rows-per-second-single-machine-can-nested-loop-joins-fast.html) and [this stackoverflow entry](http://stackoverflow.com/questions/40629435/fast-parquet-row-count-in-spark).

This PR changes generated code in the following two points.
1. Replace a while-loop with long instance variables a for-loop with int local variables
2. Suppress generation of `shouldStop()` method if this method is unnecessary (e.g. `append()` is not generated).

These points facilitates compiler optimizations in a JIT compiler by feeding the simplified Java code into the JIT compiler. The performance of `sqlContext.read.parquet().count` is improved by 1.09x.

Benchmark program:
```java
val dir = "/dev/shm/parquet"
val N = 1000 * 1000 * 40
val iters = 20
val benchmark = new Benchmark("Parquet", N * iters, minNumIters = 5, warmupTime = 30.seconds)
sparkSession.range(n).write.mode("overwrite").parquet(dir)

benchmark.addCase("count") { i: Int =>
  var n = 0
  var len = 0L
  while (n < iters) {
    len += sparkSession.read.parquet(dir).count
    n += 1
  }
}
benchmark.run
```

Performance result without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Parquet:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
w/o this PR                                   1152 / 1211        694.7           1.4       1.0X
```

Performance result with this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-47-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Parquet:                                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
with this PR                                  1053 / 1121        760.0           1.3       1.0X
```

Here is a comparison between generated code w/o and with this PR. Only the method ```agg_doAggregateWithoutKey``` is changed.

Generated code without this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private boolean agg_initAgg;
/* 009 */   private boolean agg_bufIsNull;
/* 010 */   private long agg_bufValue;
/* 011 */   private scala.collection.Iterator scan_input;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_numOutputRows;
/* 013 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_scanTime;
/* 014 */   private long scan_scanTime1;
/* 015 */   private org.apache.spark.sql.execution.vectorized.ColumnarBatch scan_batch;
/* 016 */   private int scan_batchIdx;
/* 017 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_numOutputRows;
/* 018 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_aggTime;
/* 019 */   private UnsafeRow agg_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 022 */
/* 023 */   public GeneratedIterator(Object[] references) {
/* 024 */     this.references = references;
/* 025 */   }
/* 026 */
/* 027 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 028 */     partitionIndex = index;
/* 029 */     this.inputs = inputs;
/* 030 */     agg_initAgg = false;
/* 031 */
/* 032 */     scan_input = inputs[0];
/* 033 */     this.scan_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 034 */     this.scan_scanTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 035 */     scan_scanTime1 = 0;
/* 036 */     scan_batch = null;
/* 037 */     scan_batchIdx = 0;
/* 038 */     this.agg_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 039 */     this.agg_aggTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[3];
/* 040 */     agg_result = new UnsafeRow(1);
/* 041 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 042 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 043 */
/* 044 */   }
/* 045 */
/* 046 */   private void agg_doAggregateWithoutKey() throws java.io.IOException {
/* 047 */     // initialize aggregation buffer
/* 048 */     agg_bufIsNull = false;
/* 049 */     agg_bufValue = 0L;
/* 050 */
/* 051 */     if (scan_batch == null) {
/* 052 */       scan_nextBatch();
/* 053 */     }
/* 054 */     while (scan_batch != null) {
/* 055 */       int numRows = scan_batch.numRows();
/* 056 */       while (scan_batchIdx < numRows) {
/* 057 */         int scan_rowIdx = scan_batchIdx++;
/* 058 */         // do aggregate
/* 059 */         // common sub-expressions
/* 060 */
/* 061 */         // evaluate aggregate function
/* 062 */         boolean agg_isNull1 = false;
/* 063 */
/* 064 */         long agg_value1 = -1L;
/* 065 */         agg_value1 = agg_bufValue + 1L;
/* 066 */         // update aggregation buffer
/* 067 */         agg_bufIsNull = false;
/* 068 */         agg_bufValue = agg_value1;
/* 069 */         if (shouldStop()) return;
/* 070 */       }
/* 071 */       scan_batch = null;
/* 072 */       scan_nextBatch();
/* 073 */     }
/* 074 */     scan_scanTime.add(scan_scanTime1 / (1000 * 1000));
/* 075 */     scan_scanTime1 = 0;
/* 076 */
/* 077 */   }
/* 078 */
/* 079 */   private void scan_nextBatch() throws java.io.IOException {
/* 080 */     long getBatchStart = System.nanoTime();
/* 081 */     if (scan_input.hasNext()) {
/* 082 */       scan_batch = (org.apache.spark.sql.execution.vectorized.ColumnarBatch)scan_input.next();
/* 083 */       scan_numOutputRows.add(scan_batch.numRows());
/* 084 */       scan_batchIdx = 0;
/* 085 */
/* 086 */     }
/* 087 */     scan_scanTime1 += System.nanoTime() - getBatchStart;
/* 088 */   }
/* 089 */
/* 090 */   protected void processNext() throws java.io.IOException {
/* 091 */     while (!agg_initAgg) {
/* 092 */       agg_initAgg = true;
/* 093 */       long agg_beforeAgg = System.nanoTime();
/* 094 */       agg_doAggregateWithoutKey();
/* 095 */       agg_aggTime.add((System.nanoTime() - agg_beforeAgg) / 1000000);
/* 096 */
/* 097 */       // output the result
/* 098 */
/* 099 */       agg_numOutputRows.add(1);
/* 100 */       agg_rowWriter.zeroOutNullBytes();
/* 101 */
/* 102 */       if (agg_bufIsNull) {
/* 103 */         agg_rowWriter.setNullAt(0);
/* 104 */       } else {
/* 105 */         agg_rowWriter.write(0, agg_bufValue);
/* 106 */       }
/* 107 */       append(agg_result);
/* 108 */     }
/* 109 */   }
/* 110 */ }
```

Generated code with this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private boolean agg_initAgg;
/* 009 */   private boolean agg_bufIsNull;
/* 010 */   private long agg_bufValue;
/* 011 */   private scala.collection.Iterator scan_input;
/* 012 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_numOutputRows;
/* 013 */   private org.apache.spark.sql.execution.metric.SQLMetric scan_scanTime;
/* 014 */   private long scan_scanTime1;
/* 015 */   private org.apache.spark.sql.execution.vectorized.ColumnarBatch scan_batch;
/* 016 */   private int scan_batchIdx;
/* 017 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_numOutputRows;
/* 018 */   private org.apache.spark.sql.execution.metric.SQLMetric agg_aggTime;
/* 019 */   private UnsafeRow agg_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 022 */
/* 023 */   public GeneratedIterator(Object[] references) {
/* 024 */     this.references = references;
/* 025 */   }
/* 026 */
/* 027 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 028 */     partitionIndex = index;
/* 029 */     this.inputs = inputs;
/* 030 */     agg_initAgg = false;
/* 031 */
/* 032 */     scan_input = inputs[0];
/* 033 */     this.scan_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 034 */     this.scan_scanTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 035 */     scan_scanTime1 = 0;
/* 036 */     scan_batch = null;
/* 037 */     scan_batchIdx = 0;
/* 038 */     this.agg_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 039 */     this.agg_aggTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[3];
/* 040 */     agg_result = new UnsafeRow(1);
/* 041 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 042 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 043 */
/* 044 */   }
/* 045 */
/* 046 */   private void agg_doAggregateWithoutKey() throws java.io.IOException {
/* 047 */     // initialize aggregation buffer
/* 048 */     agg_bufIsNull = false;
/* 049 */     agg_bufValue = 0L;
/* 050 */
/* 051 */     if (scan_batch == null) {
/* 052 */       scan_nextBatch();
/* 053 */     }
/* 054 */     while (scan_batch != null) {
/* 055 */       int numRows = scan_batch.numRows();
/* 056 */       int scan_localEnd = numRows - scan_batchIdx;
/* 057 */       for (int scan_localIdx = 0; scan_localIdx < scan_localEnd; scan_localIdx++) {
/* 058 */         int scan_rowIdx = scan_batchIdx + scan_localIdx;
/* 059 */         // do aggregate
/* 060 */         // common sub-expressions
/* 061 */
/* 062 */         // evaluate aggregate function
/* 063 */         boolean agg_isNull1 = false;
/* 064 */
/* 065 */         long agg_value1 = -1L;
/* 066 */         agg_value1 = agg_bufValue + 1L;
/* 067 */         // update aggregation buffer
/* 068 */         agg_bufIsNull = false;
/* 069 */         agg_bufValue = agg_value1;
/* 070 */         // shouldStop check is eliminated
/* 071 */       }
/* 072 */       scan_batchIdx = numRows;
/* 073 */       scan_batch = null;
/* 074 */       scan_nextBatch();
/* 075 */     }
/* 079 */   }
/* 080 */
/* 081 */   private void scan_nextBatch() throws java.io.IOException {
/* 082 */     long getBatchStart = System.nanoTime();
/* 083 */     if (scan_input.hasNext()) {
/* 084 */       scan_batch = (org.apache.spark.sql.execution.vectorized.ColumnarBatch)scan_input.next();
/* 085 */       scan_numOutputRows.add(scan_batch.numRows());
/* 086 */       scan_batchIdx = 0;
/* 087 */
/* 088 */     }
/* 089 */     scan_scanTime1 += System.nanoTime() - getBatchStart;
/* 090 */   }
/* 091 */
/* 092 */   protected void processNext() throws java.io.IOException {
/* 093 */     while (!agg_initAgg) {
/* 094 */       agg_initAgg = true;
/* 095 */       long agg_beforeAgg = System.nanoTime();
/* 096 */       agg_doAggregateWithoutKey();
/* 097 */       agg_aggTime.add((System.nanoTime() - agg_beforeAgg) / 1000000);
/* 098 */
/* 099 */       // output the result
/* 100 */
/* 101 */       agg_numOutputRows.add(1);
/* 102 */       agg_rowWriter.zeroOutNullBytes();
/* 103 */
/* 104 */       if (agg_bufIsNull) {
/* 105 */         agg_rowWriter.setNullAt(0);
/* 106 */       } else {
/* 107 */         agg_rowWriter.write(0, agg_bufValue);
/* 108 */       }
/* 109 */       append(agg_result);
/* 110 */     }
/* 111 */   }
/* 112 */ }
```

## How was this patch tested?

Tested existing test suites

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

Closes #17378 from kiszk/SPARK-20046.
2017-03-26 09:20:22 +02:00
hyukjinkwon 2422c86f2c [SPARK-20092][R][PROJECT INFRA] Add the detection for Scala codes dedicated for R in AppVeyor tests
## What changes were proposed in this pull request?

We are currently detecting the changes in `R/` directory only and then trigger AppVeyor tests.

It seems we need to tests when there are Scala codes dedicated for R in `core/src/main/scala/org/apache/spark/api/r/`, `sql/core/src/main/scala/org/apache/spark/sql/api/r/` and `mllib/src/main/scala/org/apache/spark/ml/r/` too.

This will enables the tests, for example, for SPARK-20088.

## How was this patch tested?

Tests with manually created PRs.

- Changes in `sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala` https://github.com/spark-test/spark/pull/13
- Changes in `core/src/main/scala/org/apache/spark/api/r/SerDe.scala` https://github.com/spark-test/spark/pull/12
- Changes in `README.md` https://github.com/spark-test/spark/pull/14

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17427 from HyukjinKwon/SPARK-20092.
2017-03-25 23:29:02 -07:00
Wenchen Fan 0b903caef3 [SPARK-19949][SQL][FOLLOW-UP] move FailureSafeParser from catalyst to sql core
## What changes were proposed in this pull request?

The `FailureSafeParser` is only used in sql core, it doesn't make sense to put it in catalyst module.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17408 from cloud-fan/minor.
2017-03-25 11:46:54 -07:00
sethah be85245a98
[SPARK-17137][ML][WIP] Compress logistic regression coefficients
## What changes were proposed in this pull request?

Use the new `compressed` method on matrices to store the logistic regression coefficients as sparse or dense - whichever is requires less memory.

Marked as WIP so we can add some performance test results. Basically, we should see if prediction is slower because of using a sparse matrix over a dense one. This can happen since sparse matrices do not use native BLAS operations when computing the margins.

## How was this patch tested?

Unit tests added.

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

Closes #17426 from sethah/SPARK-17137.
2017-03-25 17:41:59 +00:00
Kalvin Chau e8ddb91c7e [SPARK-20078][MESOS] Mesos executor configurability for task name and labels
## What changes were proposed in this pull request?

Adding configurable mesos executor names and labels using `spark.mesos.task.name` and `spark.mesos.task.labels`.

Labels were defined as `k1:v1,k2:v2`.

mgummelt

## How was this patch tested?

Added unit tests to verify labels were added correctly, with incorrect labels being ignored and added a test to test the name of the executor.

Tested with: `./build/sbt -Pmesos mesos/test`

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

Author: Kalvin Chau <kalvin.chau@viasat.com>

Closes #17404 from kalvinnchau/mesos-config.
2017-03-25 10:42:15 +00:00
Xiao Li a2ce0a2e30 [HOTFIX][SQL] Fix the failed test cases in GeneratorFunctionSuite
### What changes were proposed in this pull request?
Multiple tests failed. Revert the changes on `supportCodegen` of `GenerateExec`. For example,

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

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17425 from gatorsmile/turnOnCodeGenGenerateExec.
2017-03-24 23:27:42 -07:00
Herman van Hovell 0a6c50711b [SPARK-20070][SQL] Fix 2.10 build
## What changes were proposed in this pull request?
Commit 91fa80fe8a broke the build for scala 2.10. The commit uses `Regex.regex` field which is not available in Scala 2.10. This PR fixes this.

## How was this patch tested?
Existing tests.

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

Closes #17420 from hvanhovell/SPARK-20070-2.0.
2017-03-25 01:07:50 +01:00
Roxanne Moslehi f88f56b835 [DOCS] Clarify round mode for format_number & round functions
## What changes were proposed in this pull request?

Updated the description for the `format_number` description to indicate that it uses `HALF_EVEN` rounding. Updated the description for the `round` description to indicate that it uses `HALF_UP` rounding.

## How was this patch tested?

Just changing the two function comments so no testing involved.

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

Author: Roxanne Moslehi <rmoslehi@palantir.com>
Author: roxannemoslehi <rmoslehi@berkeley.edu>

Closes #17399 from roxannemoslehi/patch-1.
2017-03-25 00:10:30 +01:00
Liang-Chi Hsieh e011004bed [SPARK-19846][SQL] Add a flag to disable constraint propagation
## What changes were proposed in this pull request?

Constraint propagation can be computation expensive and block the driver execution for long time. For example, the below benchmark needs 30mins.

Compared with previous PRs #16998, #16785, this is a much simpler option: add a flag to disable constraint propagation.

### Benchmark

Run the following codes locally.

    import org.apache.spark.ml.{Pipeline, PipelineStage}
    import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer, VectorAssembler}
    import org.apache.spark.sql.internal.SQLConf

    spark.conf.set(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key, false)

    val df = (1 to 40).foldLeft(Seq((1, "foo"), (2, "bar"), (3, "baz")).toDF("id", "x0"))((df, i) => df.withColumn(s"x$i", $"x0"))

    val indexers = df.columns.tail.map(c => new StringIndexer()
      .setInputCol(c)
      .setOutputCol(s"${c}_indexed")
      .setHandleInvalid("skip"))

    val encoders = indexers.map(indexer => new OneHotEncoder()
      .setInputCol(indexer.getOutputCol)
      .setOutputCol(s"${indexer.getOutputCol}_encoded")
      .setDropLast(true))

    val stages: Array[PipelineStage] = indexers ++ encoders
    val pipeline = new Pipeline().setStages(stages)

    val startTime = System.nanoTime
    pipeline.fit(df).transform(df).show
    val runningTime = System.nanoTime - startTime

Before this patch: 1786001 ms ~= 30 mins
After this patch: 26392 ms = less than half of a minute

Related PRs: #16998, #16785.

## 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 #17186 from viirya/add-flag-disable-constraint-propagation.
2017-03-25 00:04:51 +01:00
Reynold Xin b5c5bd98ea Disable generate codegen since it fails my workload. 2017-03-24 23:57:29 +01:00
Herman van Hovell 91fa80fe8a [SPARK-20070][SQL] Redact DataSourceScanExec treeString
## What changes were proposed in this pull request?
The explain output of `DataSourceScanExec` can contain sensitive information (like Amazon keys). Such information should not end up in logs, or be exposed to non privileged users.

This PR addresses this by adding a redaction facility for the `DataSourceScanExec.treeString`. A user can enable this by setting a regex in the `spark.redaction.string.regex` configuration.

## How was this patch tested?
Added a unit test to check the output of DataSourceScanExec.

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

Closes #17397 from hvanhovell/SPARK-20070.
2017-03-24 15:52:48 -07:00
sethah e8810b73c4
[SPARK-17471][ML] Add compressed method to ML matrices
## What changes were proposed in this pull request?

This patch adds a `compressed` method to ML `Matrix` class, which returns the minimal storage representation of the matrix - either sparse or dense. Because the space occupied by a sparse matrix is dependent upon its layout (i.e. column major or row major), this method must consider both cases. It may also be useful to force the layout to be column or row major beforehand, so an overload is added which takes in a `columnMajor: Boolean` parameter.

The compressed implementation relies upon two new abstract methods `toDense(columnMajor: Boolean)` and `toSparse(columnMajor: Boolean)`, similar to the compressed method implemented in the `Vector` class. These methods also allow the layout of the resulting matrix to be specified via the `columnMajor` parameter. More detail on the new methods is given below.
## How was this patch tested?

Added many new unit tests
## New methods (summary, not exhaustive list)

**Matrix trait**
- `private[ml] def toDenseMatrix(columnMajor: Boolean): DenseMatrix` (abstract) - converts the matrix (either sparse or dense) to dense format
- `private[ml] def toSparseMatrix(columnMajor: Boolean): SparseMatrix` (abstract) -  converts the matrix (either sparse or dense) to sparse format
- `def toDense: DenseMatrix = toDense(true)`  - converts the matrix (either sparse or dense) to dense format in column major layout
- `def toSparse: SparseMatrix = toSparse(true)` -  converts the matrix (either sparse or dense) to sparse format in column major layout
- `def compressed: Matrix` - finds the minimum space representation of this matrix, considering both column and row major layouts, and converts it
- `def compressed(columnMajor: Boolean): Matrix` - finds the minimum space representation of this matrix considering only column OR row major, and converts it

**DenseMatrix class**
- `private[ml] def toDenseMatrix(columnMajor: Boolean): DenseMatrix` - converts the dense matrix to a dense matrix, optionally changing the layout (data is NOT duplicated if the layouts are the same)
- `private[ml] def toSparseMatrix(columnMajor: Boolean): SparseMatrix` - converts the dense matrix to sparse matrix, using the specified layout

**SparseMatrix class**
- `private[ml] def toDenseMatrix(columnMajor: Boolean): DenseMatrix` - converts the sparse matrix to a dense matrix, using the specified layout
- `private[ml] def toSparseMatrix(columnMajors: Boolean): SparseMatrix` - converts the sparse matrix to sparse matrix. If the sparse matrix contains any explicit zeros, they are removed. If the layout requested does not match the current layout, data is copied to a new representation. If the layouts match and no explicit zeros exist, the current matrix is returned.

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

Closes #15628 from sethah/matrix_compress.
2017-03-24 20:32:42 +00:00
Adam Budde 707e501832 [SPARK-19911][STREAMING] Add builder interface for Kinesis DStreams
## What changes were proposed in this pull request?

- Add new KinesisDStream.scala containing KinesisDStream.Builder class
- Add KinesisDStreamBuilderSuite test suite
- Make KinesisInputDStream ctor args package private for testing
- Add JavaKinesisDStreamBuilderSuite test suite
- Add args to KinesisInputDStream and KinesisReceiver for optional
  service-specific auth (Kinesis, DynamoDB and CloudWatch)
## How was this patch tested?

Added ```KinesisDStreamBuilderSuite``` to verify builder class works as expected

Author: Adam Budde <budde@amazon.com>

Closes #17250 from budde/KinesisStreamBuilder.
2017-03-24 12:40:29 -07:00
Jacek Laskowski 9299d071f9 [SQL][MINOR] Fix for typo in Analyzer
## What changes were proposed in this pull request?

Fix for typo in Analyzer

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17409 from jaceklaskowski/analyzer-typo.
2017-03-24 09:56:05 -07:00
Nick Pentreath d9f4ce6943 [SPARK-15040][ML][PYSPARK] Add Imputer to PySpark
Add Python wrapper for `Imputer` feature transformer.

## How was this patch tested?

New doc tests and tweak to PySpark ML `tests.py`

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

Closes #17316 from MLnick/SPARK-15040-pyspark-imputer.
2017-03-24 08:01:15 -07:00
Xiao Li 344f38b04b [SPARK-19970][SQL][FOLLOW-UP] Table owner should be USER instead of PRINCIPAL in kerberized clusters #17311
### What changes were proposed in this pull request?
This is a follow-up for the PR: https://github.com/apache/spark/pull/17311

- For safety, use `sessionState` to get the user name, instead of calling `SessionState.get()` in the function `toHiveTable`.
- Passing `user names` instead of `conf` when calling `toHiveTable`.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17405 from gatorsmile/user.
2017-03-24 14:42:33 +08:00
Eric Liang 8e558041aa [SPARK-19820][CORE] Add interface to kill tasks w/ a reason
This commit adds a killTaskAttempt method to SparkContext, to allow users to
kill tasks so that they can be re-scheduled elsewhere.

This also refactors the task kill path to allow specifying a reason for the task kill. The reason is propagated opaquely through events, and will show up in the UI automatically as `(N killed: $reason)` and `TaskKilled: $reason`. Without this change, there is no way to provide the user feedback through the UI.

Currently used reasons are "stage cancelled", "another attempt succeeded", and "killed via SparkContext.killTask". The user can also specify a custom reason through `SparkContext.killTask`.

cc rxin

In the stage overview UI the reasons are summarized:
![1](https://cloud.githubusercontent.com/assets/14922/23929209/a83b2862-08e1-11e7-8b3e-ae1967bbe2e5.png)

Within the stage UI you can see individual task kill reasons:
![2](https://cloud.githubusercontent.com/assets/14922/23929200/9a798692-08e1-11e7-8697-72b27ad8a287.png)

Existing tests, tried killing some stages in the UI and verified the messages are as expected.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekl@google.com>

Closes #17166 from ericl/kill-reason.
2017-03-23 23:30:44 -07:00
jinxing 19596c28b6 [SPARK-16929] Improve performance when check speculatable tasks.
## What changes were proposed in this pull request?
1. Use a MedianHeap to record durations of successful tasks.  When check speculatable tasks, we can get the median duration with O(1) time complexity.

2. `checkSpeculatableTasks` will synchronize `TaskSchedulerImpl`. If `checkSpeculatableTasks` doesn't finish with 100ms, then the possibility exists for that thread to release and then immediately re-acquire the lock. Change `scheduleAtFixedRate` to be `scheduleWithFixedDelay` when call method of `checkSpeculatableTasks`.
## How was this patch tested?
Added MedianHeapSuite.

Author: jinxing <jinxing6042@126.com>

Closes #16867 from jinxing64/SPARK-16929.
2017-03-23 23:25:56 -07:00
Kazuaki Ishizaki bb823ca4b4 [SPARK-19959][SQL] Fix to throw NullPointerException in df[java.lang.Long].collect
## What changes were proposed in this pull request?

This PR fixes `NullPointerException` in the generated code by Catalyst. When we run the following code, we get the following `NullPointerException`. This is because there is no null checks for `inputadapter_value`  while `java.lang.Long inputadapter_value` at Line 30 may have `null`.

This happen when a type of DataFrame is nullable primitive type such as `java.lang.Long` and the wholestage codegen is used. While the physical plan keeps `nullable=true` in `input[0, java.lang.Long, true].longValue`, `BoundReference.doGenCode` ignores `nullable=true`. Thus, nullcheck code will not be generated and `NullPointerException` will occur.

This PR checks the nullability and correctly generates nullcheck if needed.
```java
sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF.collect
```

```java
Caused by: java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(generated.java:37)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:393)
...
```

Generated code without this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       java.lang.Long inputadapter_value = (java.lang.Long)inputadapter_row.get(0, null);
/* 031 */
/* 032 */       boolean serializefromobject_isNull = true;
/* 033 */       long serializefromobject_value = -1L;
/* 034 */       if (!false) {
/* 035 */         serializefromobject_isNull = false;
/* 036 */         if (!serializefromobject_isNull) {
/* 037 */           serializefromobject_value = inputadapter_value.longValue();
/* 038 */         }
/* 039 */
/* 040 */       }
/* 041 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 042 */
/* 043 */       if (serializefromobject_isNull) {
/* 044 */         serializefromobject_rowWriter.setNullAt(0);
/* 045 */       } else {
/* 046 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 047 */       }
/* 048 */       append(serializefromobject_result);
/* 049 */       if (shouldStop()) return;
/* 050 */     }
/* 051 */   }
/* 052 */ }
```

Generated code with this PR

```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 031 */       java.lang.Long inputadapter_value = inputadapter_isNull ? null : ((java.lang.Long)inputadapter_row.get(0, null));
/* 032 */
/* 033 */       boolean serializefromobject_isNull = true;
/* 034 */       long serializefromobject_value = -1L;
/* 035 */       if (!inputadapter_isNull) {
/* 036 */         serializefromobject_isNull = false;
/* 037 */         if (!serializefromobject_isNull) {
/* 038 */           serializefromobject_value = inputadapter_value.longValue();
/* 039 */         }
/* 040 */
/* 041 */       }
/* 042 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 043 */
/* 044 */       if (serializefromobject_isNull) {
/* 045 */         serializefromobject_rowWriter.setNullAt(0);
/* 046 */       } else {
/* 047 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 048 */       }
/* 049 */       append(serializefromobject_result);
/* 050 */       if (shouldStop()) return;
/* 051 */     }
/* 052 */   }
/* 053 */ }
```

## How was this patch tested?

Added new test suites in `DataFrameSuites`

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

Closes #17302 from kiszk/SPARK-19959.
2017-03-24 12:57:56 +08:00
Timothy Hunter d27daa54bd [SPARK-19636][ML] Feature parity for correlation statistics in MLlib
## What changes were proposed in this pull request?

This patch adds the Dataframes-based support for the correlation statistics found in the `org.apache.spark.mllib.stat.correlation.Statistics`, following the design doc discussed in the JIRA ticket.

The current implementation is a simple wrapper around the `spark.mllib` implementation. Future optimizations can be implemented at a later stage.

## How was this patch tested?

```
build/sbt "testOnly org.apache.spark.ml.stat.StatisticsSuite"
```

Author: Timothy Hunter <timhunter@databricks.com>

Closes #17108 from thunterdb/19636.
2017-03-23 18:42:13 -07:00
Burak Yavuz 93581fbc18 Fix compilation of the Scala 2.10 master branch
## What changes were proposed in this pull request?

Fixes break caused by: 746a558de2

## How was this patch tested?

Compiled with `build/sbt -Dscala2.10 sql/compile` locally

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17403 from brkyvz/onceTrigger2.10.
2017-03-23 17:57:31 -07:00
sureshthalamati c791180705 [SPARK-10849][SQL] Adds option to the JDBC data source write for user to specify database column type for the create table
## What changes were proposed in this pull request?
Currently JDBC data source creates tables in the target database using the default type mapping, and the JDBC dialect mechanism.  If users want to specify different database data type for only some of columns, there is no option available. In scenarios where default mapping does not work, users are forced to create tables on the target database before writing. This workaround is probably not acceptable from a usability point of view. This PR is to provide a user-defined type mapping for specific columns.

The solution is to allow users to specify database column data type for the create table  as JDBC datasource option(createTableColumnTypes) on write. Data type information can be specified in the same format as table schema DDL format (e.g: `name CHAR(64), comments VARCHAR(1024)`).

All supported target database types can not be specified ,  the data types has to be valid spark sql data types also.  For example user can not specify target database  CLOB data type. This will be supported in the follow-up PR.

Example:
```Scala
df.write
.option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)")
.jdbc(url, "TEST.DBCOLTYPETEST", properties)
```
## How was this patch tested?
Added new test cases to the JDBCWriteSuite

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

Closes #16209 from sureshthalamati/jdbc_custom_dbtype_option_json-spark-10849.
2017-03-23 17:39:33 -07:00
erenavsarogullari b7be05a203 [SPARK-19567][CORE][SCHEDULER] Support some Schedulable variables immutability and access
## What changes were proposed in this pull request?
Some `Schedulable` Entities(`Pool` and `TaskSetManager`) variables need refactoring for _immutability_ and _access modifiers_ levels as follows:
- From `var` to `val` (if there is no requirement): This is important to support immutability as much as possible.
  - Sample => `Pool`: `weight`, `minShare`, `priority`, `name` and `taskSetSchedulingAlgorithm`.
- Access modifiers: Specially, `var`s access needs to be restricted from other parts of codebase to prevent potential side effects.
  - `TaskSetManager`: `tasksSuccessful`, `totalResultSize`, `calculatedTasks` etc...

This PR is related with #15604 and has been created seperatedly to keep patch content as isolated and to help the reviewers.

## How was this patch tested?
Added new UTs and existing UT coverage.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #16905 from erenavsarogullari/SPARK-19567.
2017-03-23 17:20:52 -07:00
Tyson Condie 746a558de2 [SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?

An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.

In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.

## How was this patch tested?

A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.

In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.

marmbrus tdas zsxwing

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

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

Closes #17219 from tcondie/stream-commit.
2017-03-23 14:32:05 -07:00
Ye Yin b0ae6a38a3 Typo fixup in comment
## What changes were proposed in this pull request?

Fixup typo in comment.

## How was this patch tested?

Don't need.

Author: Ye Yin <eyniy@qq.com>

Closes #17396 from hustcat/fix.
2017-03-23 13:30:50 +01:00
Sean Owen b70c03a420 [INFRA] Close stale PRs
Closes #16819
Closes #13467
Closes #16083
Closes #17135
Closes #8785
Closes #16278
Closes #16997
Closes #17073
Closes #17220

Added:
Closes #12059
Closes #12524
Closes #12888
Closes #16061

Author: Sean Owen <sowen@cloudera.com>

Closes #17386 from srowen/StalePRs.
2017-03-23 08:42:42 +00:00
hyukjinkwon aefe798905 [MINOR][BUILD] Fix javadoc8 break
## What changes were proposed in this pull request?

Several javadoc8 breaks have been introduced. This PR proposes fix those instances so that we can build Scala/Java API docs.

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:6: error: reference not found
[error]  * <code>flatMapGroupsWithState</code> operations on {link KeyValueGroupedDataset}.
[error]                                                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:10: error: reference not found
[error]  * Both, <code>mapGroupsWithState</code> and <code>flatMapGroupsWithState</code> in {link KeyValueGroupedDataset}
[error]                                                                                            ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:51: error: reference not found
[error]  *    {link GroupStateTimeout.ProcessingTimeTimeout}) or event time (i.e.
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:52: error: reference not found
[error]  *    {link GroupStateTimeout.EventTimeTimeout}).
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:158: error: reference not found
[error]  *           Spark SQL types (see {link Encoder} for more details).
[error]                                          ^
[error] .../spark/mllib/target/java/org/apache/spark/ml/fpm/FPGrowthParams.java:26: error: bad use of '>'
[error]    * Number of partitions (>=1) used by parallel FP-growth. By default the param is not set, and
[error]                            ^
[error] .../spark/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java:30: error: reference not found
[error]  * {link org.apache.spark.sql.KeyValueGroupedDataset#flatMapGroupsWithState(
[error]           ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:211: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:232: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:254: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:277: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found
[error]  * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe.
[error]           ^
[error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found
[error]  * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe.
[error]                                     ^
[info] 13 errors
```

```
jekyll 3.3.1 | Error:  Unidoc generation failed
```

## How was this patch tested?

Manually via `jekyll build`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17389 from HyukjinKwon/minor-javadoc8-fix.
2017-03-23 08:41:30 +00:00
hyukjinkwon 07c12c09a7 [SPARK-18579][SQL] Use ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace options in CSV writing
## What changes were proposed in this pull request?

This PR proposes to support _not_ trimming the white spaces when writing out. These are `false` by default in CSV reading path but these are `true` by default in CSV writing in univocity parser.

Both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` options are not being used for writing and therefore, we are always trimming the white spaces.

It seems we should provide a way to keep this white spaces easily.

WIth the data below:

```scala
val df = spark.read.csv(Seq("a , b  , c").toDS)
df.show()
```

```
+---+----+---+
|_c0| _c1|_c2|
+---+----+---+
| a | b  |  c|
+---+----+---+
```

**Before**

```scala
df.write.csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+-----+
|value|
+-----+
|a,b,c|
+-----+
```

It seems this can't be worked around via `quoteAll` too.

```scala
df.write.option("quoteAll", true).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```
```
+-----------+
|      value|
+-----------+
|"a","b","c"|
+-----------+
```

**After**

```scala
df.write.option("ignoreLeadingWhiteSpace", false).option("ignoreTrailingWhiteSpace", false).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+----------+
|     value|
+----------+
|a , b  , c|
+----------+
```

Note that this case is possible in R

```r
> system("cat text.csv")
f1,f2,f3
a , b  , c
> df <- read.csv(file="text.csv")
> df
  f1   f2 f3
1 a   b    c
> write.csv(df, file="text1.csv", quote=F, row.names=F)
> system("cat text1.csv")
f1,f2,f3
a , b  , c
```

## How was this patch tested?

Unit tests in `CSVSuite` and manual tests for Python.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17310 from HyukjinKwon/SPARK-18579.
2017-03-23 00:25:01 -07:00
Sameer Agarwal 12cd00706c [BUILD][MINOR] Fix 2.10 build
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/17385 breaks the 2.10 sbt/maven builds by hitting an empty-string interpolation bug (https://issues.scala-lang.org/browse/SI-7919).

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-sbt-scala-2.10/4072/
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-scala-2.10/3987/

## How was this patch tested?

Compiles

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17391 from sameeragarwal/build-fix.
2017-03-22 15:58:42 -07:00
Tathagata Das 82b598b963 [SPARK-20057][SS] Renamed KeyedState to GroupState in mapGroupsWithState
## What changes were proposed in this pull request?

Since the state is tied a "group" in the "mapGroupsWithState" operations, its better to call the state "GroupState" instead of a key. This would make it more general if you extends this operation to RelationGroupedDataset and python APIs.

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

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

Closes #17385 from tdas/SPARK-20057.
2017-03-22 12:30:36 -07:00
hyukjinkwon 80fd070389 [SPARK-20018][SQL] Pivot with timestamp and count should not print internal representation
## What changes were proposed in this pull request?

Currently, when we perform count with timestamp types, it prints the internal representation as the column name as below:

```scala
Seq(new java.sql.Timestamp(1)).toDF("a").groupBy("a").pivot("a").count().show()
```

```
+--------------------+----+
|                   a|1000|
+--------------------+----+
|1969-12-31 16:00:...|   1|
+--------------------+----+
```

This PR proposes to use external Scala value instead of the internal representation in the column names as below:

```
+--------------------+-----------------------+
|                   a|1969-12-31 16:00:00.001|
+--------------------+-----------------------+
|1969-12-31 16:00:...|                      1|
+--------------------+-----------------------+
```

## How was this patch tested?

Unit test in `DataFramePivotSuite` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17348 from HyukjinKwon/SPARK-20018.
2017-03-22 09:58:46 -07:00
hyukjinkwon 465818389a [SPARK-19949][SQL][FOLLOW-UP] Clean up parse modes and update related comments
## What changes were proposed in this pull request?

This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix.

Also, this PR modifies some tests related parse modes.

## How was this patch tested?

Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17377 from HyukjinKwon/SPARK-19949.
2017-03-22 09:52:37 -07:00
Prashant Sharma 0caade6340 [SPARK-20027][DOCS] Compilation fix in java docs.
## What changes were proposed in this pull request?

During build/sbt publish-local, build breaks due to javadocs errors. This patch fixes those errors.

## How was this patch tested?

Tested by running the sbt build.

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

Closes #17358 from ScrapCodes/docs-fix.
2017-03-22 13:52:03 +00:00
uncleGen facfd60886 [SPARK-20021][PYSPARK] Miss backslash in python code
## What changes were proposed in this pull request?

Add backslash for line continuation in python code.

## How was this patch tested?

Jenkins.

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

Closes #17352 from uncleGen/python-example-doc.
2017-03-22 11:10:08 +00:00
Xiao Li 7343a09401 [SPARK-20023][SQL] Output table comment for DESC FORMATTED
### What changes were proposed in this pull request?
Currently, `DESC FORMATTED` did not output the table comment, unlike what `DESC EXTENDED` does. This PR is to fix it.

Also correct the following displayed names in `DESC FORMATTED`, for being consistent with `DESC EXTENDED`
- `"Create Time:"` -> `"Created:"`
- `"Last Access Time:"` -> `"Last Access:"`

### How was this patch tested?
Added test cases in `describe.sql`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17381 from gatorsmile/descFormattedTableComment.
2017-03-22 19:08:28 +08:00
Yanbo Liang 478fbc866f [SPARK-19925][SPARKR] Fix SparkR spark.getSparkFiles fails when it was called on executors.
## What changes were proposed in this pull request?
SparkR ```spark.getSparkFiles``` fails when it was called on executors, see details at [SPARK-19925](https://issues.apache.org/jira/browse/SPARK-19925).

## How was this patch tested?
Add unit tests, and verify this fix at standalone and yarn cluster.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #17274 from yanboliang/spark-19925.
2017-03-21 21:50:54 -07:00
Tathagata Das c1e87e384d [SPARK-20030][SS] Event-time-based timeout for MapGroupsWithState
## What changes were proposed in this pull request?

Adding event time based timeout. The user sets the timeout timestamp directly using `KeyedState.setTimeoutTimestamp`. The keys times out when the watermark crosses the timeout timestamp.

## How was this patch tested?
Unit tests

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

Closes #17361 from tdas/SPARK-20030.
2017-03-21 21:27:08 -07:00
Kunal Khamar 2d73fcced0 [SPARK-20051][SS] Fix StreamSuite flaky test - recover from v2.1 checkpoint
## What changes were proposed in this pull request?

There is a race condition between calling stop on a streaming query and deleting directories in `withTempDir` that causes test to fail, fixing to do lazy deletion using delete on shutdown JVM hook.

## How was this patch tested?

- Unit test
  - repeated 300 runs with no failure

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17382 from kunalkhamar/partition-bugfix.
2017-03-21 18:56:14 -07:00
hyukjinkwon 9281a3d504 [SPARK-19919][SQL] Defer throwing the exception for empty paths in CSV datasource into DataSource
## What changes were proposed in this pull request?

This PR proposes to defer throwing the exception within `DataSource`.

Currently, if other datasources fail to infer the schema, it returns `None` and then this is being validated in `DataSource` as below:

```
scala> spark.read.json("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for JSON. It must be specified manually.;
```

```
scala> spark.read.orc("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for ORC. It must be specified manually.;
```

```
scala> spark.read.parquet("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for Parquet. It must be specified manually.;
```

However, CSV it checks it within the datasource implementation and throws another exception message as below:

```
scala> spark.read.csv("emptydir")
java.lang.IllegalArgumentException: requirement failed: Cannot infer schema from an empty set of files
```

We could remove this duplicated check and validate this in one place in the same way with the same message.

## How was this patch tested?

Unit test in `CSVSuite` and manual test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17256 from HyukjinKwon/SPARK-19919.
2017-03-22 08:41:46 +08:00
Will Manning a04dcde8cb clarify array_contains function description
## What changes were proposed in this pull request?

The description in the comment for array_contains is vague/incomplete (i.e., doesn't mention that it returns `null` if the array is `null`); this PR fixes that.

## How was this patch tested?

No testing, since it merely changes a comment.

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

Author: Will Manning <lwwmanning@gmail.com>

Closes #17380 from lwwmanning/patch-1.
2017-03-22 00:40:48 +01:00
Felix Cheung a8877bdbba [SPARK-19237][SPARKR][CORE] On Windows spark-submit should handle when java is not installed
## What changes were proposed in this pull request?

When SparkR is installed as a R package there might not be any java runtime.
If it is not there SparkR's `sparkR.session()` will block waiting for the connection timeout, hanging the R IDE/shell, without any notification or message.

## How was this patch tested?

manually

- [x] need to test on Windows

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #16596 from felixcheung/rcheckjava.
2017-03-21 14:24:41 -07:00
zhaorongsheng 7dbc162f12 [SPARK-20017][SQL] change the nullability of function 'StringToMap' from 'false' to 'true'
## What changes were proposed in this pull request?

Change the nullability of function `StringToMap` from `false` to `true`.

Author: zhaorongsheng <334362872@qq.com>

Closes #17350 from zhaorongsheng/bug-fix_strToMap_NPE.
2017-03-21 11:30:55 -07:00
Joseph K. Bradley ae4b91d1f5 [SPARK-20039][ML] rename ChiSquare to ChiSquareTest
## What changes were proposed in this pull request?

I realized that since ChiSquare is in the package stat, it's pretty unclear if it's the hypothesis test, distribution, or what. This PR renames it to ChiSquareTest to clarify this.

## How was this patch tested?

Existing unit tests

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #17368 from jkbradley/SPARK-20039.
2017-03-21 11:01:25 -07:00
Xin Wu 4c0ff5f585 [SPARK-19261][SQL] Alter add columns for Hive serde and some datasource tables
## What changes were proposed in this pull request?
Support` ALTER TABLE ADD COLUMNS (...) `syntax for Hive serde and some datasource tables.
In this PR, we consider a few aspects:

1. View is not supported for `ALTER ADD COLUMNS`

2. Since tables created in SparkSQL with Hive DDL syntax will populate table properties with schema information, we need make sure the consistency of the schema before and after ALTER operation in order for future use.

3. For embedded-schema type of format, such as `parquet`, we need to make sure that the predicate on the newly-added columns can be evaluated properly, or pushed down properly. In case of the data file does not have the columns for the newly-added columns, such predicates should return as if the column values are NULLs.

4. For datasource table, this feature does not support the following:
4.1 TEXT format, since there is only one default column `value` is inferred for text format data.
4.2 ORC format, since SparkSQL native ORC reader does not support the difference between user-specified-schema and inferred schema from ORC files.
4.3 Third party datasource types that implements RelationProvider, including the built-in JDBC format, since different implementations by the vendors may have different ways to dealing with schema.
4.4 Other datasource types, such as `parquet`, `json`, `csv`, `hive` are supported.

5. Column names being added can not be duplicate of any existing data column or partition column names. Case sensitivity is taken into consideration according to the sql configuration.

6. This feature also supports In-Memory catalog, while Hive support is turned off.
## How was this patch tested?
Add new test cases

Author: Xin Wu <xinwu@us.ibm.com>

Closes #16626 from xwu0226/alter_add_columns.
2017-03-21 08:49:54 -07:00
Zheng RuiFeng 63f077fbe5 [SPARK-20041][DOC] Update docs for NaN handling in approxQuantile
## What changes were proposed in this pull request?
Update docs for NaN handling in approxQuantile.

## How was this patch tested?
existing tests.

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #17369 from zhengruifeng/doc_quantiles_nan.
2017-03-21 08:45:59 -07:00
wangzhenhua 14865d7ff7 [SPARK-17080][SQL][FOLLOWUP] Improve documentation, change buildJoin method structure and add a debug log
## What changes were proposed in this pull request?

1. Improve documentation for class `Cost` and `JoinReorderDP` and method `buildJoin()`.
2. Change code structure of `buildJoin()` to make the logic clearer.
3. Add a debug-level log to record information for join reordering, including time cost, the number of items and the number of plans in memo.

## How was this patch tested?

Not related.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17353 from wzhfy/reorderFollow.
2017-03-21 08:44:09 -07:00
jianran.tfh 650d03cfc9 [SPARK-19998][BLOCK MANAGER] Change the exception log to add RDD id of the related the block
## What changes were proposed in this pull request?

"java.lang.Exception: Could not compute split, block $blockId not found" doesn't have the rdd id info, the "BlockManager: Removing RDD $id" has only the RDD id, so it couldn't find that the Exception's reason is the Removing; so it's better block not found Exception add RDD id info

## How was this patch tested?

Existing tests

Author: jianran.tfh <jianran.tfh@taobao.com>
Author: jianran <tanfanhua1984@163.com>

Closes #17334 from jianran/SPARK-19998.
2017-03-21 15:15:19 +00:00
christopher snow 7620aed828 [SPARK-20011][ML][DOCS] Clarify documentation for ALS 'rank' parameter
## What changes were proposed in this pull request?

API documentation and collaborative filtering documentation page changes to clarify inconsistent description of ALS rank parameter.

 - [DOCS] was previously: "rank is the number of latent factors in the model."
 - [API] was previously:  "rank - number of features to use"

This change describes rank in both places consistently as:

 - "Number of features to use (also referred to as the number of latent factors)"

Author: Chris Snow <chris.snowuk.ibm.com>

Author: christopher snow <chsnow123@gmail.com>

Closes #17345 from snowch/SPARK-20011.
2017-03-21 13:23:59 +00:00
Xiao Li d2dcd6792f [SPARK-20024][SQL][TEST-MAVEN] SessionCatalog reset need to set the current database of ExternalCatalog
### What changes were proposed in this pull request?
SessionCatalog API setCurrentDatabase does not set the current database of the underlying ExternalCatalog. Thus, weird errors could come in the test suites after we call reset. We need to fix it.

So far, have not found the direct impact in the other code paths because we expect all the SessionCatalog APIs should always use the current database value we managed, unless some of code paths skip it. Thus, we fix it in the test-only function reset().

### How was this patch tested?
Multiple test case failures are observed in mvn and add a test case in SessionCatalogSuite.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17354 from gatorsmile/useDB.
2017-03-20 22:52:45 -07:00