Commit graph

19962 commits

Author SHA1 Message Date
Herman van Hovell e2f32ee45a [SPARK-21258][SQL] Fix WindowExec complex object aggregation with spilling
## What changes were proposed in this pull request?
`WindowExec` currently improperly stores complex objects (UnsafeRow, UnsafeArrayData, UnsafeMapData, UTF8String) during aggregation by keeping a reference in the buffer used by `GeneratedMutableProjections` to the actual input data. Things go wrong when the input object (or the backing bytes) are reused for other things. This could happen in window functions when it starts spilling to disk. When reading the back the spill files the `UnsafeSorterSpillReader` reuses the buffer to which the `UnsafeRow` points, leading to weird corruption scenario's. Note that this only happens for aggregate functions that preserve (parts of) their input, for example `FIRST`, `LAST`, `MIN` & `MAX`.

This was not seen before, because the spilling logic was not doing actual spills as much and actually used an in-memory page. This page was not cleaned up during window processing and made sure unsafe objects point to their own dedicated memory location. This was changed by https://github.com/apache/spark/pull/16909, after this PR Spark spills more eagerly.

This PR provides a surgical fix because we are close to releasing Spark 2.2. This change just makes sure that there cannot be any object reuse at the expensive of a little bit of performance. We will follow-up with a more subtle solution at a later point.

## How was this patch tested?
Added a regression test to `DataFrameWindowFunctionsSuite`.

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

Closes #18470 from hvanhovell/SPARK-21258.
2017-06-30 12:34:09 +08:00
Shixiong Zhu cfc696f4a4 [SPARK-21253][CORE][HOTFIX] Fix Scala 2.10 build
## What changes were proposed in this pull request?

A follow up PR to fix Scala 2.10 build for #18472

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18478 from zsxwing/SPARK-21253-2.
2017-06-29 20:56:37 -07:00
IngoSchuster 88a536babf [SPARK-21176][WEB UI] Limit number of selector threads for admin ui proxy servlets to 8
## What changes were proposed in this pull request?
Please see also https://issues.apache.org/jira/browse/SPARK-21176

This change limits the number of selector threads that jetty creates to maximum 8 per proxy servlet (Jetty default is number of processors / 2).
The newHttpClient for Jettys ProxyServlet class is overwritten to avoid the Jetty defaults (which are designed for high-performance http servers).
Once https://github.com/eclipse/jetty.project/issues/1643 is available, the code could be cleaned up to avoid the method override.

I really need this on v2.1.1 - what is the best way for a backport automatic merge works fine)? Shall I create another PR?

## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
The patch was tested manually on a Spark cluster with a head node that has 88 processors using JMX to verify that the number of selector threads is now limited to 8 per proxy.

gurvindersingh zsxwing can you please review the change?

Author: IngoSchuster <ingo.schuster@de.ibm.com>
Author: Ingo Schuster <ingo.schuster@de.ibm.com>

Closes #18437 from IngoSchuster/master.
2017-06-30 11:16:09 +08:00
Shixiong Zhu 80f7ac3a60 [SPARK-21253][CORE] Disable spark.reducer.maxReqSizeShuffleToMem
## What changes were proposed in this pull request?

Disable spark.reducer.maxReqSizeShuffleToMem because it breaks the old shuffle service.

Credits to wangyum

Closes #18466

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Yuming Wang <wgyumg@gmail.com>

Closes #18467 from zsxwing/SPARK-21253.
2017-06-30 11:02:22 +08:00
Shixiong Zhu 4996c53949 [SPARK-21253][CORE] Fix a bug that StreamCallback may not be notified if network errors happen
## What changes were proposed in this pull request?

If a network error happens before processing StreamResponse/StreamFailure events, StreamCallback.onFailure won't be called.

This PR fixes `failOutstandingRequests` to also notify outstanding StreamCallbacks.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18472 from zsxwing/fix-stream-2.
2017-06-30 10:56:48 +08:00
Feng Liu f9151bebca [SPARK-21188][CORE] releaseAllLocksForTask should synchronize the whole method
## What changes were proposed in this pull request?

Since the objects `readLocksByTask`, `writeLocksByTask` and `info`s are coupled and supposed to be modified by other threads concurrently, all the read and writes of them in the method `releaseAllLocksForTask` should be protected by a single synchronized block like other similar methods.

## How was this patch tested?

existing tests

Author: Feng Liu <fengliu@databricks.com>

Closes #18400 from liufengdb/synchronize.
2017-06-29 16:03:15 -07:00
Liang-Chi Hsieh 18066f2e61 [SPARK-21052][SQL] Add hash map metrics to join
## What changes were proposed in this pull request?

This adds the average hash map probe metrics to join operator such as `BroadcastHashJoin` and `ShuffledHashJoin`.

This PR adds the API to `HashedRelation` to get average hash map probe.

## How was this patch tested?

Related test cases are added.

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

Closes #18301 from viirya/SPARK-21052.
2017-06-29 21:28:48 +08:00
杨治国10192065 29bd251dd5 [SPARK-21225][CORE] Considering CPUS_PER_TASK when allocating task slots for each WorkerOffer
JIRA Issue:https://issues.apache.org/jira/browse/SPARK-21225
    In the function "resourceOffers", It declare a variable "tasks" for storage the tasks which have allocated a executor. It declared like this:
`val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores))`
    But, I think this code only conside a situation for that one task per core. If the user set "spark.task.cpus" as 2 or 3, It really don't need so much Mem. I think It can motify as follow:
val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK))
 to instead.
    Motify like this the other earning is that it's more easy to understand the way how the tasks allocate offers.

Author: 杨治国10192065 <yang.zhiguo@zte.com.cn>

Closes #18435 from JackYangzg/motifyTaskCoreDisp.
2017-06-29 20:53:48 +08:00
fjh100456 d7da2b94d6 [SPARK-21135][WEB UI] On history server page,duration of incompleted applications should be hidden instead of showing up as 0
## What changes were proposed in this pull request?

Hide duration of incompleted applications.

## How was this patch tested?

manual tests

Author: fjh100456 <fu.jinhua6@zte.com.cn>

Closes #18351 from fjh100456/master.
2017-06-29 10:01:12 +01:00
jinxing d106a74c53 [SPARK-21240] Fix code style for constructing and stopping a SparkContext in UT.
## What changes were proposed in this pull request?

Same with SPARK-20985.
Fix code style for constructing and stopping a `SparkContext`. Assure the context is stopped to avoid other tests complain that there's only one `SparkContext` can exist.

Author: jinxing <jinxing6042@126.com>

Closes #18454 from jinxing64/SPARK-21240.
2017-06-29 09:59:36 +01:00
Nick Pentreath 70085e83d1 [SPARK-21210][DOC][ML] Javadoc 8 fixes for ML shared param traits
PR #15999 included fixes for doc strings in the ML shared param traits (occurrences of `>` and `>=`).

This PR simply uses the HTML-escaped version of the param doc to embed into the Scaladoc, to ensure that when `SharedParamsCodeGen` is run, the generated javadoc will be compliant for Java 8.

## How was this patch tested?
Existing tests

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

Closes #18420 from MLnick/shared-params-javadoc8.
2017-06-29 09:51:12 +01:00
actuaryzhang a2d5623548 [SPARK-20889][SPARKR] Grouped documentation for NONAGGREGATE column methods
## What changes were proposed in this pull request?

Grouped documentation for nonaggregate column methods.

Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>

Closes #18422 from actuaryzhang/sparkRDocNonAgg.
2017-06-29 01:23:13 -07:00
Wenchen Fan 9f6b3e65cc [SPARK-21238][SQL] allow nested SQL execution
## What changes were proposed in this pull request?

This is kind of another follow-up for https://github.com/apache/spark/pull/18064 .

In #18064 , we wrap every SQL command with SQL execution, which makes nested SQL execution very likely to happen. #18419 trid to improve it a little bit, by introduing `SQLExecition.ignoreNestedExecutionId`. However, this is not friendly to data source developers, they may need to update their code to use this `ignoreNestedExecutionId` API.

This PR proposes a new solution, to just allow nested execution. The downside is that, we may have multiple executions for one query. We can improve this by updating the data organization in SQLListener, to have 1-n mapping from query to execution, instead of 1-1 mapping. This can be done in a follow-up.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18450 from cloud-fan/execution-id.
2017-06-29 14:37:42 +08:00
Sital Kedia a946be35ac [SPARK-3577] Report Spill size on disk for UnsafeExternalSorter
## What changes were proposed in this pull request?

Report Spill size on disk for UnsafeExternalSorter

## How was this patch tested?

Tested by running a job on cluster and verify the spill size on disk.

Author: Sital Kedia <skedia@fb.com>

Closes #17471 from sitalkedia/fix_disk_spill_size.
2017-06-29 14:25:51 +08:00
wangzhenhua 82e24912d6 [SPARK-21237][SQL] Invalidate stats once table data is changed
## What changes were proposed in this pull request?

Invalidate spark's stats after data changing commands:

- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #18449 from wzhfy/removeStats.
2017-06-29 11:32:29 +08:00
Wenchen Fan 25c2edf6f9 [SPARK-21229][SQL] remove QueryPlan.preCanonicalized
## What changes were proposed in this pull request?

`QueryPlan.preCanonicalized` is only overridden in a few places, and it does introduce an extra concept to `QueryPlan` which may confuse people.

This PR removes it and override `canonicalized` in these places

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18440 from cloud-fan/minor.
2017-06-29 11:21:50 +08:00
Felix Cheung fc92d25f2a Revert "[SPARK-21094][R] Terminate R's worker processes in the parent of R's daemon to prevent a leak"
This reverts commit 6b3d02285e.
2017-06-28 20:06:29 -07:00
hyukjinkwon db44f5f3e8 [SPARK-21224][R] Specify a schema by using a DDL-formatted string when reading in R
## What changes were proposed in this pull request?

This PR proposes to support a DDL-formetted string as schema as below:

```r
mockLines <- c("{\"name\":\"Michael\"}",
               "{\"name\":\"Andy\", \"age\":30}",
               "{\"name\":\"Justin\", \"age\":19}")
jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp")
writeLines(mockLines, jsonPath)
df <- read.df(jsonPath, "json", "name STRING, age DOUBLE")
collect(df)
```

## How was this patch tested?

Tests added in `test_streaming.R` and `test_sparkSQL.R` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18431 from HyukjinKwon/r-ddl-schema.
2017-06-28 19:36:00 -07:00
Yanbo Liang 0c8444cf6d [SPARK-14657][SPARKR][ML] RFormula w/o intercept should output reference category when encoding string terms
## What changes were proposed in this pull request?

Please see [SPARK-14657](https://issues.apache.org/jira/browse/SPARK-14657) for detail of this bug.
I searched online and test some other cases, found when we fit R glm model(or other models powered by R formula) w/o intercept on a dataset including string/category features, one of the categories in the first category feature is being used as reference category, we will not drop any category for that feature.
I think we should keep consistent semantics between Spark RFormula and R formula.
## How was this patch tested?

Add standard unit tests.

cc mengxr

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #12414 from yanboliang/spark-14657.
2017-06-29 10:32:32 +08:00
actuaryzhang 376d90d556 [SPARK-20889][SPARKR] Grouped documentation for STRING column methods
## What changes were proposed in this pull request?

Grouped documentation for string column methods.

Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>

Closes #18366 from actuaryzhang/sparkRDocString.
2017-06-28 19:31:54 -07:00
Wang Gengliang b72b8521d9 [SPARK-21222] Move elimination of Distinct clause from analyzer to optimizer
## What changes were proposed in this pull request?

Move elimination of Distinct clause from analyzer to optimizer

Distinct clause is useless after MAX/MIN clause. For example,
"Select MAX(distinct a) FROM src from"
is equivalent of
"Select MAX(a) FROM src from"
However, this optimization is implemented in analyzer. It should be in optimizer.

## How was this patch tested?

Unit test

gatorsmile cloud-fan

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18429 from gengliangwang/distinct_opt.
2017-06-29 08:47:31 +08:00
Burak Yavuz e68aed70fb [SPARK-21216][SS] Hive strategies missed in Structured Streaming IncrementalExecution
## What changes were proposed in this pull request?

If someone creates a HiveSession, the planner in `IncrementalExecution` doesn't take into account the Hive scan strategies. This causes joins of Streaming DataFrame's with Hive tables to fail.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #18426 from brkyvz/hive-join.
2017-06-28 10:45:45 -07:00
Wenchen Fan 838effb98a Revert "[SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas"
This reverts commit e44697606f.
2017-06-28 14:28:40 +08:00
actuaryzhang e793bf248b [SPARK-20889][SPARKR] Grouped documentation for MATH column methods
## What changes were proposed in this pull request?

Grouped documentation for math column methods.

Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>

Closes #18371 from actuaryzhang/sparkRDocMath.
2017-06-27 23:15:45 -07:00
Eric Vandenberg 2d686a19e3 [SPARK-21155][WEBUI] Add (? running tasks) into Spark UI progress
## What changes were proposed in this pull request?

Add metric on number of running tasks to status bar on Jobs / Active Jobs.

## How was this patch tested?

Run a long running (1 minute) query in spark-shell and use localhost:4040 web UI to observe progress.  See jira for screen snapshot.

Author: Eric Vandenberg <ericvandenberg@fb.com>

Closes #18369 from ericvandenbergfb/runningTasks.
2017-06-28 09:26:33 +08:00
Liang-Chi Hsieh fd8c931a30 [SPARK-19104][SQL] Lambda variables in ExternalMapToCatalyst should be global
## What changes were proposed in this pull request?

The issue happens in `ExternalMapToCatalyst`. For example, the following codes create `ExternalMapToCatalyst` to convert Scala Map to catalyst map format.

    val data = Seq.tabulate(10)(i => NestedData(1, Map("key" -> InnerData("name", i + 100))))
    val ds = spark.createDataset(data)

The `valueConverter` in `ExternalMapToCatalyst` looks like:

    if (isnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true))) null else named_struct(name, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).name, true), value, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).value)

There is a `CreateNamedStruct` expression (`named_struct`) to create a row of `InnerData.name` and `InnerData.value` that are referred by `ExternalMapToCatalyst_value52`.

Because `ExternalMapToCatalyst_value52` are local variable, when `CreateNamedStruct` splits expressions to individual functions, the local variable can't be accessed anymore.

## How was this patch tested?

Jenkins tests.

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

Closes #18418 from viirya/SPARK-19104.
2017-06-28 00:57:05 +08:00
hyukjinkwon b32bd005e4 [INFRA] Close stale PRs
## What changes were proposed in this pull request?

This PR proposes to close stale PRs, mostly the same instances with https://github.com/apache/spark/pull/18017

I believe the author in #14807 removed his account.

Closes #7075
Closes #8927
Closes #9202
Closes #9366
Closes #10861
Closes #11420
Closes #12356
Closes #13028
Closes #13506
Closes #14191
Closes #14198
Closes #14330
Closes #14807
Closes #15839
Closes #16225
Closes #16685
Closes #16692
Closes #16995
Closes #17181
Closes #17211
Closes #17235
Closes #17237
Closes #17248
Closes #17341
Closes #17708
Closes #17716
Closes #17721
Closes #17937

Added:
Closes #14739
Closes #17139
Closes #17445
Closes #18042
Closes #18359

Added:
Closes #16450
Closes #16525
Closes #17738

Added:
Closes #16458
Closes #16508
Closes #17714

Added:
Closes #17830
Closes #14742

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18417 from HyukjinKwon/close-stale-pr.
2017-06-27 13:14:12 +01:00
Wang Gengliang 3cb3ccce12 [SPARK-21196] Split codegen info of query plan into sequence
codegen info of query plan can be very long.
In debugging console / web page, it would be more readable if the subtrees and corresponding codegen are split into sequence.

Example:

```java
codegenStringSeq(sql("select 1").queryExecution.executedPlan)
```
The example will return Seq[(String, String)] of length 1, containing the subtree as string and the corresponding generated code.

The subtree as string:

> (*Project [1 AS 1#0]
> +- Scan OneRowRelation[]

The generated code:
```java
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 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 project_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_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 */     project_result = new UnsafeRow(1);
/* 022 */     project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 0);
/* 023 */     project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_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 */       project_rowWriter.write(0, 1);
/* 031 */       append(project_result);
/* 032 */       if (shouldStop()) return;
/* 033 */     }
/* 034 */   }
/* 035 */
/* 036 */ }
```
## What changes were proposed in this pull request?
add method codegenToSeq: split codegen info of query plan into sequence

## How was this patch tested?
unit test

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

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18409 from gengliangwang/codegen.
2017-06-27 17:24:46 +08:00
Wenchen Fan c22810004f [SPARK-20213][SQL][FOLLOW-UP] introduce SQLExecution.ignoreNestedExecutionId
## What changes were proposed in this pull request?

in https://github.com/apache/spark/pull/18064, to work around the nested sql execution id issue, we introduced several internal methods in `Dataset`, like `collectInternal`, `countInternal`, `showInternal`, etc., to avoid nested execution id.

However, this approach has poor expansibility. When we hit other nested execution id cases, we may need to add more internal methods in `Dataset`.

Our goal is to ignore the nested execution id in some cases, and we can have a better approach to achieve this goal, by introducing `SQLExecution.ignoreNestedExecutionId`. Whenever we find a place which needs to ignore the nested execution, we can just wrap the action with `SQLExecution.ignoreNestedExecutionId`, and this is more expansible than the previous approach.

The idea comes from https://github.com/apache/spark/pull/17540/files#diff-ab49028253e599e6e74cc4f4dcb2e3a8R57 by rdblue

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18419 from cloud-fan/follow.
2017-06-27 02:35:51 +08:00
jerryshao 9e50a1d37a [SPARK-13669][SPARK-20898][CORE] Improve the blacklist mechanism to handle external shuffle service unavailable situation
## What changes were proposed in this pull request?

Currently we are running into an issue with Yarn work preserving enabled + external shuffle service.
In the work preserving enabled scenario, the failure of NM will not lead to the exit of executors, so executors can still accept and run the tasks. The problem here is when NM is failed, external shuffle service is actually inaccessible, so reduce tasks will always complain about the “Fetch failure”, and the failure of reduce stage will make the parent stage (map stage) rerun. The tricky thing here is Spark scheduler is not aware of the unavailability of external shuffle service, and will reschedule the map tasks on the executor where NM is failed, and again reduce stage will be failed with “Fetch failure”, and after 4 retries, the job is failed. This could also apply to other cluster manager with external shuffle service.

So here the main problem is that we should avoid assigning tasks to those bad executors (where shuffle service is unavailable). Current Spark's blacklist mechanism could blacklist executors/nodes by failure tasks, but it doesn't handle this specific fetch failure scenario. So here propose to improve the current application blacklist mechanism to handle fetch failure issue (especially with external shuffle service unavailable issue), to blacklist the executors/nodes where shuffle fetch is unavailable.

## How was this patch tested?

Unit test and small cluster verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #17113 from jerryshao/SPARK-13669.
2017-06-26 11:14:03 -05:00
Burak Yavuz 5282bae040 [SPARK-21153] Use project instead of expand in tumbling windows
## What changes were proposed in this pull request?

Time windowing in Spark currently performs an Expand + Filter, because there is no way to guarantee the amount of windows a timestamp will fall in, in the general case. However, for tumbling windows, a record is guaranteed to fall into a single bucket. In this case, doubling the number of records with Expand is wasteful, and can be improved by using a simple Projection instead.

Benchmarks show that we get an order of magnitude performance improvement after this patch.

## How was this patch tested?

Existing unit tests. Benchmarked using the following code:

```scala
import org.apache.spark.sql.functions._

spark.time {
  spark.range(numRecords)
    .select(from_unixtime((current_timestamp().cast("long") * 1000 + 'id / 1000) / 1000) as 'time)
    .select(window('time, "10 seconds"))
    .count()
}
```

Setup:
 - 1 c3.2xlarge worker (8 cores)

![image](https://user-images.githubusercontent.com/5243515/27348748-ed991b84-55a9-11e7-8f8b-6e7abc524417.png)

1 B rows ran in 287 seconds after this optimization. I didn't wait for it to finish without the optimization. Shows about 5x improvement for large number of records.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #18364 from brkyvz/opt-tumble.
2017-06-26 01:26:32 -07:00
hyukjinkwon 6b3d02285e [SPARK-21093][R] Terminate R's worker processes in the parent of R's daemon to prevent a leak
## What changes were proposed in this pull request?

`mcfork` in R looks opening a pipe ahead but the existing logic does not properly close it when it is executed hot. This leads to the failure of more forking due to the limit for number of files open.

This hot execution looks particularly for `gapply`/`gapplyCollect`. For unknown reason, this happens more easily in CentOS and could be reproduced in Mac too.

All the details are described in https://issues.apache.org/jira/browse/SPARK-21093

This PR proposes simply to terminate R's worker processes in the parent of R's daemon to prevent a leak.

## How was this patch tested?

I ran the codes below on both CentOS and Mac with that configuration disabled/enabled.

```r
df <- createDataFrame(list(list(1L, 1, "1", 0.1)), c("a", "b", "c", "d"))
collect(gapply(df, "a", function(key, x) { x }, schema(df)))
collect(gapply(df, "a", function(key, x) { x }, schema(df)))
...  # 30 times
```

Also, now it passes R tests on CentOS as below:

```
SparkSQL functions: Spark package found in SPARK_HOME: .../spark
..............................................................................................................................................................
..............................................................................................................................................................
..............................................................................................................................................................
..............................................................................................................................................................
..............................................................................................................................................................
....................................................................................................................................
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18320 from HyukjinKwon/SPARK-21093.
2017-06-25 11:05:57 -07:00
Zhenhua Wang 884347e1f7 [HOT FIX] fix stats functions in the recent patch
## What changes were proposed in this pull request?

Builds failed due to the recent [merge](b449a1d6aa). This is because [PR#18309](https://github.com/apache/spark/pull/18309) needed update after [this patch](b803b66a81) was merged.

## How was this patch tested?
N/A

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18415 from wzhfy/hotfixStats.
2017-06-25 15:06:29 +01:00
Masha Basmanova b449a1d6aa [SPARK-21079][SQL] Calculate total size of a partition table as a sum of individual partitions
## What changes were proposed in this pull request?

Storage URI of a partitioned table may or may not point to a directory under which individual partitions are stored. In fact, individual partitions may be located in totally unrelated directories. Before this change, ANALYZE TABLE table COMPUTE STATISTICS command calculated total size of a table by adding up sizes of files found under table's storage URI. This calculation could produce 0 if partitions are stored elsewhere.

This change uses storage URIs of individual partitions to calculate the sizes of all partitions of a table and adds these up to produce the total size of a table.

CC: wzhfy

## How was this patch tested?

Added unit test.

Ran ANALYZE TABLE xxx COMPUTE STATISTICS on a partitioned Hive table and verified that sizeInBytes is calculated correctly. Before this change, the size would be zero.

Author: Masha Basmanova <mbasmanova@fb.com>

Closes #18309 from mbasmanova/mbasmanova-analyze-part-table.
2017-06-24 22:49:35 -07:00
gatorsmile 2e1586f60a [SPARK-21203][SQL] Fix wrong results of insertion of Array of Struct
### What changes were proposed in this pull request?
```SQL
CREATE TABLE `tab1`
(`custom_fields` ARRAY<STRUCT<`id`: BIGINT, `value`: STRING>>)
USING parquet

INSERT INTO `tab1`
SELECT ARRAY(named_struct('id', 1, 'value', 'a'), named_struct('id', 2, 'value', 'b'))

SELECT custom_fields.id, custom_fields.value FROM tab1
```

The above query always return the last struct of the array, because the rule `SimplifyCasts` incorrectly rewrites the query. The underlying cause is we always use the same `GenericInternalRow` object when doing the cast.

### How was this patch tested?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18412 from gatorsmile/castStruct.
2017-06-24 22:35:59 +08:00
hyukjinkwon 7c7bc8fc0f [SPARK-21189][INFRA] Handle unknown error codes in Jenkins rather then leaving incomplete comment in PRs
## What changes were proposed in this pull request?

Recently, Jenkins tests were unstable due to unknown reasons as below:

```
 /home/jenkins/workspace/SparkPullRequestBuilder/dev/lint-r ; process was terminated by signal 9
    test_result_code, test_result_note = run_tests(tests_timeout)
  File "./dev/run-tests-jenkins.py", line 140, in run_tests
    test_result_note = ' * This patch **fails %s**.' % failure_note_by_errcode[test_result_code]
KeyError: -9
```

```
Traceback (most recent call last):
  File "./dev/run-tests-jenkins.py", line 226, in <module>
    main()
  File "./dev/run-tests-jenkins.py", line 213, in main
    test_result_code, test_result_note = run_tests(tests_timeout)
  File "./dev/run-tests-jenkins.py", line 140, in run_tests
    test_result_note = ' * This patch **fails %s**.' % failure_note_by_errcode[test_result_code]
KeyError: -10
```

This exception looks causing failing to update the comments in the PR. For example:

![2017-06-23 4 19 41](https://user-images.githubusercontent.com/6477701/27470626-d035ecd8-582f-11e7-883e-0ae6941659b7.png)

![2017-06-23 4 19 50](https://user-images.githubusercontent.com/6477701/27470629-d11ba782-582f-11e7-97e0-64d28cbc19aa.png)

these comment just remain.

This always requires, for both reviewers and the author, a overhead to click and check the logs, which I believe are not really useful.

This PR proposes to leave the code in the PR comment messages and let update the comments.

## How was this patch tested?

Jenkins tests below, I manually gave the error code to test this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18399 from HyukjinKwon/jenkins-print-errors.
2017-06-24 10:14:31 +01:00
Marcelo Vanzin bfd73a7c48 [SPARK-21159][CORE] Don't try to connect to launcher in standalone cluster mode.
Monitoring for standalone cluster mode is not implemented (see SPARK-11033), but
the same scheduler implementation is used, and if it tries to connect to the
launcher it will fail. So fix the scheduler so it only tries that in client mode;
cluster mode applications will be correctly launched and will work, but monitoring
through the launcher handle will not be available.

Tested by running a cluster mode app with "SparkLauncher.startApplication".

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18397 from vanzin/SPARK-21159.
2017-06-24 13:23:43 +08:00
Gabor Feher b837bf9ae9 [SPARK-20555][SQL] Fix mapping of Oracle DECIMAL types to Spark types in read path
## What changes were proposed in this pull request?

This PR is to revert some code changes in the read path of https://github.com/apache/spark/pull/14377. The original fix is https://github.com/apache/spark/pull/17830

When merging this PR, please give the credit to gaborfeher

## How was this patch tested?

Added a test case to OracleIntegrationSuite.scala

Author: Gabor Feher <gabor.feher@lynxanalytics.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #18408 from gatorsmile/OracleType.
2017-06-23 21:53:38 -07:00
hyukjinkwon 7525ce98b4 [SPARK-20431][SS][FOLLOWUP] Specify a schema by using a DDL-formatted string in DataStreamReader
## What changes were proposed in this pull request?

This pr supported a DDL-formatted string in `DataStreamReader.schema`.
This fix could make users easily define a schema without importing the type classes.

For example,

```scala
scala> spark.readStream.schema("col0 INT, col1 DOUBLE").load("/tmp/abc").printSchema()
root
 |-- col0: integer (nullable = true)
 |-- col1: double (nullable = true)
```

## How was this patch tested?

Added tests in `DataStreamReaderWriterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18373 from HyukjinKwon/SPARK-20431.
2017-06-24 11:39:41 +08:00
Xiao Li 03eb6117af [SPARK-21164][SQL] Remove isTableSample from Sample and isGenerated from Alias and AttributeReference
## What changes were proposed in this pull request?
`isTableSample` and `isGenerated ` were introduced for SQL Generation respectively by https://github.com/apache/spark/pull/11148 and https://github.com/apache/spark/pull/11050

Since SQL Generation is removed, we do not need to keep `isTableSample`.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18379 from gatorsmile/CleanSample.
2017-06-23 14:48:33 -07:00
Dilip Biswal 13c2a4f2f8 [SPARK-20417][SQL] Move subquery error handling to checkAnalysis from Analyzer
## What changes were proposed in this pull request?
Currently we do a lot of validations for subquery in the Analyzer. We should move them to CheckAnalysis which is the framework to catch and report Analysis errors. This was mentioned as a review comment in SPARK-18874.

## How was this patch tested?
Exists tests + A few tests added to SQLQueryTestSuite.

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

Closes #17713 from dilipbiswal/subquery_checkanalysis.
2017-06-23 11:02:54 -07:00
Ong Ming Yang 4cc62951a2 [MINOR][DOCS] Docs in DataFrameNaFunctions.scala use wrong method
## What changes were proposed in this pull request?

* Following the first few examples in this file, the remaining methods should also be methods of `df.na` not `df`.
* Filled in some missing parentheses

## How was this patch tested?

N/A

Author: Ong Ming Yang <me@ongmingyang.com>

Closes #18398 from ongmingyang/master.
2017-06-23 10:56:59 -07:00
Tathagata Das 2ebd0838d1 [SPARK-21192][SS] Preserve State Store provider class configuration across StreamingQuery restarts
## What changes were proposed in this pull request?

If the SQL conf for StateStore provider class is changed between restarts (i.e. query started with providerClass1 and attempted to restart using providerClass2), then the query will fail in a unpredictable way as files saved by one provider class cannot be used by the newer one.

Ideally, the provider class used to start the query should be used to restart the query, and the configuration in the session where it is being restarted should be ignored.

This PR saves the provider class config to OffsetSeqLog, in the same way # shuffle partitions is saved and recovered.

## How was this patch tested?
new unit tests

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

Closes #18402 from tdas/SPARK-21192.
2017-06-23 10:55:02 -07:00
Dhruve Ashar 1ebe7ffe07 [SPARK-21181] Release byteBuffers to suppress netty error messages
## What changes were proposed in this pull request?
We are explicitly calling release on the byteBuf's used to encode the string to Base64 to suppress the memory leak error message reported by netty. This is to make it less confusing for the user.

### Changes proposed in this fix
By explicitly invoking release on the byteBuf's we are decrement the internal reference counts for the wrappedByteBuf's. Now, when the GC kicks in, these would be reclaimed as before, just that netty wouldn't report any memory leak error messages as the internal ref. counts are now 0.

## How was this patch tested?
Ran a few spark-applications and examined the logs. The error message no longer appears.

Original PR was opened against branch-2.1 => https://github.com/apache/spark/pull/18392

Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #18407 from dhruve/master.
2017-06-23 10:36:29 -07:00
wangzhenhua b803b66a81 [SPARK-21180][SQL] Remove conf from stats functions since now we have conf in LogicalPlan
## What changes were proposed in this pull request?

After wiring `SQLConf` in logical plan ([PR 18299](https://github.com/apache/spark/pull/18299)), we can remove the need of passing `conf` into `def stats` and `def computeStats`.

## How was this patch tested?

Covered by existing tests, plus some modified existing tests.

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

Closes #18391 from wzhfy/removeConf.
2017-06-23 10:33:53 -07:00
hyukjinkwon 07479b3cfb [SPARK-21149][R] Add job description API for R
## What changes were proposed in this pull request?

Extend `setJobDescription` to SparkR API.

## How was this patch tested?

It looks difficult to add a test. Manually tested as below:

```r
df <- createDataFrame(iris)
count(df)
setJobDescription("This is an example job.")
count(df)
```

prints ...

![2017-06-22 12 05 49](https://user-images.githubusercontent.com/6477701/27415670-2a649936-5743-11e7-8e95-312f1cd103af.png)

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18382 from HyukjinKwon/SPARK-21149.
2017-06-23 09:59:24 -07:00
Takeshi Yamamuro f3dea60793 [SPARK-21144][SQL] Print a warning if the data schema and partition schema have the duplicate columns
## What changes were proposed in this pull request?
The current master outputs unexpected results when the data schema and partition schema have the duplicate columns:
```
withTempPath { dir =>
  val basePath = dir.getCanonicalPath
  spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=1").toString)
  spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=a").toString)
  spark.read.parquet(basePath).show()
}

+---+
|foo|
+---+
|  1|
|  1|
|  a|
|  a|
|  1|
|  a|
+---+
```
This patch added code to print a warning when the duplication found.

## How was this patch tested?
Manually checked.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18375 from maropu/SPARK-21144-3.
2017-06-23 09:28:02 -07:00
hyukjinkwon 5dca10b8fd [SPARK-21193][PYTHON] Specify Pandas version in setup.py
## What changes were proposed in this pull request?

It looks we missed specifying the Pandas version. This PR proposes to fix it. For the current state, it should be Pandas 0.13.0 given my test. This PR propose to fix it as 0.13.0.

Running the codes below:

```python
from pyspark.sql.types import *

schema = StructType().add("a", IntegerType()).add("b", StringType())\
                     .add("c", BooleanType()).add("d", FloatType())
data = [
    (1, "foo", True, 3.0,), (2, "foo", True, 5.0),
    (3, "bar", False, -1.0), (4, "bar", False, 6.0),
]
spark.createDataFrame(data, schema).toPandas().dtypes
```

prints ...

**With Pandas 0.13.0** - released, 2014-01

```
a      int32
b     object
c       bool
d    float32
dtype: object
```

**With Pandas 0.12.0** -  - released, 2013-06

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/dataframe.py", line 1734, in toPandas
    pdf[f] = pdf[f].astype(t, copy=False)
TypeError: astype() got an unexpected keyword argument 'copy'
```

without `copy`

```
a      int32
b     object
c       bool
d    float32
dtype: object
```

**With Pandas 0.11.0** - released, 2013-03

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/dataframe.py", line 1734, in toPandas
    pdf[f] = pdf[f].astype(t, copy=False)
TypeError: astype() got an unexpected keyword argument 'copy'
```

without `copy`

```
a      int32
b     object
c       bool
d    float32
dtype: object
```

**With Pandas 0.10.0** -  released, 2012-12

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/dataframe.py", line 1734, in toPandas
    pdf[f] = pdf[f].astype(t, copy=False)
TypeError: astype() got an unexpected keyword argument 'copy'
```

without `copy`

```
a      int64  # <- this should be 'int32'
b     object
c       bool
d    float64  # <- this should be 'float32'
```

## How was this patch tested?

Manually tested with Pandas from 0.10.0 to 0.13.0.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18403 from HyukjinKwon/SPARK-21193.
2017-06-23 21:51:55 +08:00
10129659 acd208ee50 [SPARK-21115][CORE] If the cores left is less than the coresPerExecutor,the cores left will not be allocated, so it should not to check in every schedule
## What changes were proposed in this pull request?
If we start an app with the param --total-executor-cores=4 and spark.executor.cores=3, the cores left is always 1, so it will try to allocate executors in the function org.apache.spark.deploy.master.startExecutorsOnWorkers in every schedule.
Another question is, is it will be better to allocate another executor with 1 core for the cores left.

## How was this patch tested?
unit test

Author: 10129659 <chen.yanshan@zte.com.cn>

Closes #18322 from eatoncys/leftcores.
2017-06-23 20:53:26 +08:00
jinxing 153dd49b74 [SPARK-21047] Add test suites for complicated cases in ColumnarBatchSuite
## What changes were proposed in this pull request?
Current ColumnarBatchSuite has very simple test cases for `Array` and `Struct`. This pr wants to add  some test suites for complicated cases in ColumnVector.

Author: jinxing <jinxing6042@126.com>

Closes #18327 from jinxing64/SPARK-21047.
2017-06-23 20:41:17 +08:00