Commit graph

22843 commits

Author SHA1 Message Date
Dongjoon Hyun e99ba8d7c8 [SPARK-25262][DOC][FOLLOWUP] Fix missing markup tag
## What changes were proposed in this pull request?

This adds a missing end markup tag. This should go `master` branch only.

## How was this patch tested?

This is a doc-only change. Manual via `SKIP_API=1 jekyll build`.

Closes #22584 from dongjoon-hyun/SPARK-25262.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-29 11:23:37 +08:00
DB Tsai 5d726b8659
[SPARK-25559][SQL] Remove the unsupported predicates in Parquet when possible
## What changes were proposed in this pull request?

Currently, in `ParquetFilters`, if one of the children predicates is not supported by Parquet, the entire predicates will be thrown away. In fact, if the unsupported predicate is in the top level `And` condition or in the child before hitting `Not` or `Or` condition, it can be safely removed.

## How was this patch tested?

Tests are added.

Closes #22574 from dbtsai/removeUnsupportedPredicatesInParquet.

Lead-authored-by: DB Tsai <d_tsai@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: DB Tsai <dbtsai@dbtsai.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-28 17:46:11 -07:00
Mukul Murthy 9362c5cc27
[SPARK-25449][CORE] Heartbeat shouldn't include accumulators for zero metrics
## What changes were proposed in this pull request?

Heartbeat shouldn't include accumulators for zero metrics.

Heartbeats sent from executors to the driver every 10 seconds contain metrics and are generally on the order of a few KBs. However, for large jobs with lots of tasks, heartbeats can be on the order of tens of MBs, causing tasks to die with heartbeat failures. We can mitigate this by not sending zero metrics to the driver.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Closes #22473 from mukulmurthy/25449-heartbeat.

Authored-by: Mukul Murthy <mukul.murthy@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-09-28 16:34:17 -07:00
Yuming Wang a281465686 [SPARK-25429][SQL] Use Set instead of Array to improve lookup performance
## What changes were proposed in this pull request?

Use `Set` instead of `Array` to improve `accumulatorIds.contains(acc.id)` performance.

This PR close https://github.com/apache/spark/pull/22420

## How was this patch tested?

manual tests.
Benchmark code:
```scala
def benchmark(func: () => Unit): Long = {
  val start = System.currentTimeMillis()
  func()
  val end = System.currentTimeMillis()
  end - start
}

val range = Range(1, 1000000)
val set = range.toSet
val array = range.toArray

for (i <- 0 until 5) {
  val setExecutionTime =
    benchmark(() => for (i <- 0 until 500) { set.contains(scala.util.Random.nextInt()) })
  val arrayExecutionTime =
    benchmark(() => for (i <- 0 until 500) { array.contains(scala.util.Random.nextInt()) })
  println(s"set execution time: $setExecutionTime, array execution time: $arrayExecutionTime")
}
```

Benchmark result:
```
set execution time: 4, array execution time: 2760
set execution time: 1, array execution time: 1911
set execution time: 3, array execution time: 2043
set execution time: 12, array execution time: 2214
set execution time: 6, array execution time: 1770
```

Closes #22579 from wangyum/SPARK-25429.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-28 15:08:15 -07:00
Dilip Biswal 7deef7a49b [SPARK-25458][SQL] Support FOR ALL COLUMNS in ANALYZE TABLE
## What changes were proposed in this pull request?
**Description from the JIRA :**
Currently, to collect the statistics of all the columns, users need to specify the names of all the columns when calling the command "ANALYZE TABLE ... FOR COLUMNS...". This is not user friendly. Instead, we can introduce the following SQL command to achieve it without specifying the column names.

```
   ANALYZE TABLE [db_name.]tablename COMPUTE STATISTICS FOR ALL COLUMNS;
```

## How was this patch tested?
Added new tests in SparkSqlParserSuite and StatisticsSuite

Closes #22566 from dilipbiswal/SPARK-25458.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-28 15:03:06 -07:00
Liang-Chi Hsieh b7d80349b0
[SPARK-25542][CORE][TEST] Move flaky test in OpenHashMapSuite to OpenHashSetSuite and make it against OpenHashSet
## What changes were proposed in this pull request?

The specified test in OpenHashMapSuite to test large items is somehow flaky to throw OOM.
By considering the original work #6763 that added this test, the test can be against OpenHashSetSuite. And by doing this should be to save memory because OpenHashMap allocates two more arrays when growing the map/set.

## How was this patch tested?

Existing tests.

Closes #22569 from viirya/SPARK-25542.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-28 14:29:56 -07:00
Dongjoon Hyun 0b33f08683
[SPARK-23285][DOC][FOLLOWUP] Fix missing markup tag
## What changes were proposed in this pull request?

This adds a missing markup tag. This should go to `master/branch-2.4`.

## How was this patch tested?

Manual via `SKIP_API=1 jekyll build`.

Closes #22585 from dongjoon-hyun/SPARK-23285.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-28 14:10:24 -07:00
maryannxue e120a38c0c [SPARK-25505][SQL] The output order of grouping columns in Pivot is different from the input order
## What changes were proposed in this pull request?

The grouping columns from a Pivot query are inferred as "input columns - pivot columns - pivot aggregate columns", where input columns are the output of the child relation of Pivot. The grouping columns will be the leading columns in the pivot output and they should preserve the same order as specified by the input. For example,
```
SELECT * FROM (
  SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w
  FROM courseSales
)
PIVOT (
  sum(earnings)
  FOR course IN ('dotNET', 'Java')
)
```
The output columns should be "a, z, b, y, c, x, d, w, ..." but now it is "a, b, c, d, w, x, y, z, ..."

The fix is to use the child plan's `output` instead of `outputSet` so that the order can be preserved.

## How was this patch tested?

Added UT.

Closes #22519 from maryannxue/spark-25505.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-28 00:09:06 -07:00
Chris Zhao 3b7395fe02
[SPARK-25459][SQL] Add viewOriginalText back to CatalogTable
## What changes were proposed in this pull request?

The `show create table` will show a lot of generated attributes for views that created by older Spark version. This PR will basically revert https://issues.apache.org/jira/browse/SPARK-19272 back, so when you `DESC [FORMATTED|EXTENDED] view` will show the original view DDL text.

## How was this patch tested?
Unit test.

Closes #22458 from zheyuan28/testbranch.

Lead-authored-by: Chris Zhao <chris.zhao@databricks.com>
Co-authored-by: Christopher Zhao <chris.zhao@databricks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-27 17:55:08 -07:00
Marcelo Vanzin 5fd22d0536
[SPARK-25546][CORE] Don't cache value of EVENT_LOG_CALLSITE_LONG_FORM.
Caching the value of that config means different instances of SparkEnv
will always use whatever was the first value to be read. It also breaks
tests that use RDDInfo outside of the scope of a SparkContext.

Since this is not a performance sensitive area, there's no advantage
in caching the config value.

Closes #22558 from vanzin/SPARK-25546.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-27 09:26:50 -07:00
Wenchen Fan a1adde5408 [SPARK-24341][SQL][FOLLOWUP] remove duplicated error checking
## What changes were proposed in this pull request?

There are 2 places we check for problematic `InSubquery`: the rule `ResolveSubquery` and `InSubquery.checkInputDataTypes`. We should unify them.

## How was this patch tested?

existing tests

Closes #22563 from cloud-fan/followup.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 21:19:25 +08:00
Holden Karau f856fe4839 [SPARK-21436][CORE] Take advantage of known partitioner for distinct on RDDs to avoid a shuffle
## What changes were proposed in this pull request?

Special case the situation where we know the partioner and the number of requested partions output is the same as the current partioner to avoid a shuffle and instead compute distinct inside of each partion.

## How was this patch tested?

New unit test that verifies partitioner does not change if the partitioner is known and distinct is called with the same target # of partition.

Closes #22010 from holdenk/SPARK-21436-take-advantage-of-known-partioner-for-distinct-on-rdds.

Authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 20:57:56 +08:00
Gengliang Wang dd8f6b1ce8 [SPARK-25541][SQL][FOLLOWUP] Remove overriding filterKeys in CaseInsensitiveMap
## What changes were proposed in this pull request?

As per the discussion in https://github.com/apache/spark/pull/22553#pullrequestreview-159192221,
override `filterKeys` violates the documented semantics.

This PR is to remove it and add documentation.

Also fix one potential non-serializable map in `FileStreamOptions`.

The only one call of `CaseInsensitiveMap`'s `filterKeys` left is
c3c45cbd76/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala (L88-L90)
But this one is OK.

## How was this patch tested?

Existing unit tests.

Closes #22562 from gengliangwang/SPARK-25541-FOLLOWUP.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 19:53:13 +08:00
Marco Gaido 86a2450e09 [SPARK-25551][SQL] Remove unused InSubquery expression
## What changes were proposed in this pull request?

The PR removes the `InSubquery` expression which was introduced a long time ago and its only usage was removed in 4ce970d714. Hence it is not used anymore.

## How was this patch tested?

existing UTs

Closes #22556 from mgaido91/minor_insubq.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 19:34:05 +08:00
Yuanjian Li 2a8cbfddba [SPARK-25314][SQL] Fix Python UDF accessing attributes from both side of join in join conditions
## What changes were proposed in this pull request?

Thanks for bahchis reporting this. It is more like a follow up work for #16581, this PR fix the scenario of Python UDF accessing attributes from both side of join in join condition.

## How was this patch tested?

Add  regression tests in PySpark and `BatchEvalPythonExecSuite`.

Closes #22326 from xuanyuanking/SPARK-25314.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 15:13:18 +08:00
Dilip Biswal d03e0af80d [SPARK-25522][SQL] Improve type promotion for input arguments of elementAt function
## What changes were proposed in this pull request?
In ElementAt, when first argument is MapType, we should coerce the key type and the second argument based on findTightestCommonType. This is not happening currently. We may produce wrong output as we will incorrectly downcast the right hand side double expression to int.

```SQL
spark-sql> select element_at(map(1,"one", 2, "two"), 2.2);

two
```

Also, when the first argument is ArrayType, the second argument should be an integer type or a smaller integral type that can be safely casted to an integer type. Currently we may do an unsafe cast. In the following case, we should fail with an error as 2.2 is not a integer index. But instead we down cast it to int currently and return a result instead.

```SQL
spark-sql> select element_at(array(1,2), 1.24D);

1
```
This PR also supports implicit cast between two MapTypes. I have followed similar logic that exists today to do implicit casts between two array types.
## How was this patch tested?
Added new tests in DataFrameFunctionSuite, TypeCoercionSuite.

Closes #22544 from dilipbiswal/SPARK-25522.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 15:04:59 +08:00
Wenchen Fan ff876137fa [SPARK-23715][SQL][DOC] improve document for from/to_utc_timestamp
## What changes were proposed in this pull request?

We have an agreement that the behavior of `from/to_utc_timestamp` is corrected, although the function itself doesn't make much sense in Spark: https://issues.apache.org/jira/browse/SPARK-23715

This PR improves the document.

## How was this patch tested?

N/A

Closes #22543 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 15:02:20 +08:00
yucai f309b28bd9
[SPARK-25485][SQL][TEST] Refactor UnsafeProjectionBenchmark to use main method
## What changes were proposed in this pull request?

Refactor `UnsafeProjectionBenchmark` to use main method.
Generate benchmark result:

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "catalyst/test:runMain org.apache.spark.sql.UnsafeProjectionBenchmark"
```

## How was this patch tested?

manual test

Closes #22493 from yucai/SPARK-25485.

Lead-authored-by: yucai <yyu1@ebay.com>
Co-authored-by: Yucai Yu <yucai.yu@foxmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-26 23:27:45 -07:00
王小刚 8b727994ed [SPARK-25468][WEBUI] Highlight current page index in the spark UI
## What changes were proposed in this pull request?

This PR is highlight current page index in the spark UI and history server UI,
https://issues.apache.org/jira/browse/SPARK-25468
I have add the following code in webui.css
```
.paginate_button.active>a {
    color: #999999;
    text-decoration: underline;
}
```
## How was this patch tested?
Manual tests for Chrome, Firefox and Safari
Before modifying:
![image](https://user-images.githubusercontent.com/10048468/45914897-01ca6c00-be7e-11e8-8e31-47d45db0c3bf.png)

After modifying:
![image](https://user-images.githubusercontent.com/10048468/45913987-7e564e00-be70-11e8-9c16-de17e2c63308.png)

Closes #22516 from Adamyuanyuan/spark-adam-25468.

Lead-authored-by: 王小刚 <wangxiaogang@chinatelecom.cn>
Co-authored-by: Adam Wang <Adamyuanyuan@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-27 00:02:05 -05:00
Takuya UESHIN ee214ef3a0 [SPARK-25525][SQL][PYSPARK] Do not update conf for existing SparkContext in SparkSession.getOrCreate.
## What changes were proposed in this pull request?

In [SPARK-20946](https://issues.apache.org/jira/browse/SPARK-20946), we modified `SparkSession.getOrCreate` to not update conf for existing `SparkContext` because `SparkContext` is shared by all sessions.
We should not update it in PySpark side as well.

## How was this patch tested?

Added tests.

Closes #22545 from ueshin/issues/SPARK-25525/not_update_existing_conf.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-27 12:37:03 +08:00
Shahid 5def10e61e
[SPARK-25536][CORE] metric value for METRIC_OUTPUT_RECORDS_WRITTEN is incorrect
## What changes were proposed in this pull request?
changed metric value of METRIC_OUTPUT_RECORDS_WRITTEN from 'task.metrics.inputMetrics.recordsRead' to 'task.metrics.outputMetrics.recordsWritten'.
This bug was introduced in SPARK-22190. https://github.com/apache/spark/pull/19426

## How was this patch tested?
Existing tests

Closes #22555 from shahidki31/SPARK-25536.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-26 21:10:39 -07:00
yucai 9063b17f3d
[SPARK-25481][SQL][TEST] Refactor ColumnarBatchBenchmark to use main method
## What changes were proposed in this pull request?

Refactor `ColumnarBatchBenchmark` to use main method.
Generate benchmark result:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.vectorized.ColumnarBatchBenchmark"
```

## How was this patch tested?

manual tests

Closes #22490 from yucai/SPARK-25481.

Lead-authored-by: yucai <yyu1@ebay.com>
Co-authored-by: Yucai Yu <yucai.yu@foxmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-26 20:40:10 -07:00
Takuya UESHIN c3c45cbd76 [SPARK-25540][SQL][PYSPARK] Make HiveContext in PySpark behave as the same as Scala.
## What changes were proposed in this pull request?

In Scala, `HiveContext` sets a config `spark.sql.catalogImplementation` of the given `SparkContext` and then passes to `SparkSession.builder`.
The `HiveContext` in PySpark should behave as the same as Scala.

## How was this patch tested?

Existing tests.

Closes #22552 from ueshin/issues/SPARK-25540/hive_context.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 09:51:20 +08:00
Wenchen Fan d0990e3dfe [SPARK-25454][SQL] add a new config for picking minimum precision for integral literals
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/20023 proposed to allow precision lose during decimal operations, to reduce the possibilities of overflow. This is a behavior change and is protected by the DECIMAL_OPERATIONS_ALLOW_PREC_LOSS config. However, that PR introduced another behavior change: pick a minimum precision for integral literals, which is not protected by a config. This PR add a new config for it: `spark.sql.literal.pickMinimumPrecision`.

This can allow users to work around issue in SPARK-25454, which is caused by a long-standing bug of negative scale.

## How was this patch tested?

a new test

Closes #22494 from cloud-fan/decimal.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-26 17:47:05 -07:00
Ilan Filonenko 51540c2fa6 [SPARK-25372][YARN][K8S] Deprecate and generalize keytab / principal config
## What changes were proposed in this pull request?

SparkSubmit already logs in the user if a keytab is provided, the only issue is that it uses the existing configs which have "yarn" in their name. As such, the configs were changed to:

`spark.kerberos.keytab` and `spark.kerberos.principal`.

## How was this patch tested?

Will be tested with K8S tests, but needs to be tested with Yarn

- [x] K8S Secure HDFS tests
- [x] Yarn Secure HDFS tests vanzin

Closes #22362 from ifilonenko/SPARK-25372.

Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-26 17:24:52 -07:00
Shahid 5ee2166183 [SPARK-25533][CORE][WEBUI] AppSummary should hold the information about succeeded Jobs and completed stages only
## What changes were proposed in this pull request?
Currently, In the spark UI, when there are failed jobs or failed stages, display message for the completed jobs and completed stages are not consistent with the previous versions of spark.
Reason is because, AppSummary holds the information about all the jobs and stages. But, In the below code, it checks against the completedJobs and completedStages. So, AppSummary should hold only successful jobs and stages.

66d29870c0/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala (L306)
 66d29870c0/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala (L119)
So, we should  keep only completed jobs and stage information in the AppSummary, to make it consistent with Spark2.2

## How was this patch tested?
Test steps:
 bin/spark-shell
```
sc.parallelize(1 to 5, 5).collect()
sc.parallelize(1 to 5, 2).map{ x => throw new RuntimeException("Fail")}.collect()
```
**Before fix:**

![screenshot from 2018-09-26 03-24-53](https://user-images.githubusercontent.com/23054875/46045669-f60bcd80-c13b-11e8-9aa6-a2e5a2038dba.png)

![screenshot from 2018-09-26 03-25-08](https://user-images.githubusercontent.com/23054875/46045699-0ae86100-c13c-11e8-94e5-ad35944c7615.png)

**After fix:**
![screenshot from 2018-09-26 03-16-14](https://user-images.githubusercontent.com/23054875/46045636-d83e6880-c13b-11e8-98df-f49d15c18958.png)
![screenshot from 2018-09-26 03-16-28](https://user-images.githubusercontent.com/23054875/46045645-e1c7d080-c13b-11e8-8c9c-d32e1f663356.png)

Closes #22549 from shahidki31/SPARK-25533.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-26 10:47:49 -07:00
Reynold Xin e702fb1d52 [SPARK-24519][CORE] Compute SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS only once
## What changes were proposed in this pull request?
Previously SPARK-24519 created a modifiable config SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS. However, the config is being parsed for every creation of MapStatus, which could be very expensive. Another problem with the previous approach is that it created the illusion that this can be changed dynamically at runtime, which was not true. This PR changes it so the config is computed only once.

## How was this patch tested?
Removed a test case that's no longer valid.

Closes #22521 from rxin/SPARK-24519.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-26 10:15:16 -07:00
Reza Safi bd2ae857d1 [SPARK-25318] Add exception handling when wrapping the input stream during the the fetch or stage retry in response to a corrupted block
SPARK-4105 provided a solution to block corruption issue by retrying the fetch or the stage. In that solution there is a step that wraps the input stream with compression and/or encryption. This step is prone to exceptions, but in the current code there is no exception handling for this step and this has caused confusion for the user. The  confusion was that after SPARK-4105 the user expects to see either a fetchFailed exception or a warning about a corrupted block. However an exception during wrapping can fail the job without any of those.  This change adds exception handling for the wrapping step and also adds a fetch retry if we experience a corruption during the wrapping step. The reason for adding the retry is that usually user won't experience the same failure after rerunning the job and so it seems reasonable try to fetch and wrap one more time instead of failing.

Closes #22325 from rezasafi/localcorruption.

Authored-by: Reza Safi <rezasafi@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-26 09:29:58 -07:00
Rong Tang a2ac5a72cc [SPARK-25509][CORE] Windows doesn't support POSIX permissions
## What changes were proposed in this pull request?

SHS V2 cannot enabled in Windows, because windows doesn't support POSIX permission.

## How was this patch tested?

test case fails in windows without this fix.
org.apache.spark.deploy.history.HistoryServerDiskManagerSuite  test("leasing space")

SHS V2  cannot run successfully in Windows without this fix.

java.lang.UnsupportedOperationException: 'posix:permissions' not supported as initial attribute
  at sun.nio.fs.WindowsSecurityDescriptor.fromAttribute(WindowsSecurityDescriptor.java:358)

Closes #22520 from jianjianjiao/FixWindowsPermssionsIssue.

Authored-by: Rong Tang <rotang@microsoft.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-26 10:37:17 -05:00
seancxmao cf5c9c4b55 [SPARK-20937][DOCS] Describe spark.sql.parquet.writeLegacyFormat property in Spark SQL, DataFrames and Datasets Guide
## What changes were proposed in this pull request?
Describe spark.sql.parquet.writeLegacyFormat property in Spark SQL, DataFrames and Datasets Guide.

## How was this patch tested?
N/A

Closes #22453 from seancxmao/SPARK-20937.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-26 22:14:14 +08:00
Marco Gaido 44a71741d5 [SPARK-25379][SQL] Improve AttributeSet and ColumnPruning performance
## What changes were proposed in this pull request?

This PR contains 3 optimizations:
 1)  it improves significantly the operation `--` on `AttributeSet`. As a benchmark for the `--` operation, the following code has been run
```
test("AttributeSet -- benchmark") {
    val attrSetA = AttributeSet((1 to 100).map { i => AttributeReference(s"c$i", IntegerType)() })
    val attrSetB = AttributeSet(attrSetA.take(80).toSeq)
    val attrSetC = AttributeSet((1 to 100).map { i => AttributeReference(s"c2_$i", IntegerType)() })
    val attrSetD = AttributeSet((attrSetA.take(50) ++ attrSetC.take(50)).toSeq)
    val attrSetE = AttributeSet((attrSetC.take(50) ++ attrSetA.take(50)).toSeq)
    val n_iter = 1000000
    val t0 = System.nanoTime()
    (1 to n_iter) foreach { _ =>
      val r1 = attrSetA -- attrSetB
      val r2 = attrSetA -- attrSetC
      val r3 = attrSetA -- attrSetD
      val r4 = attrSetA -- attrSetE
    }
    val t1 = System.nanoTime()
    val totalTime = t1 - t0
    println(s"Average time: ${totalTime / n_iter} us")
  }
```
The results are:
```
Before PR - Average time: 67674 us (100  %)
After PR -  Average time: 28827 us (42.6 %)
```
2) In `ColumnPruning`, it replaces the occurrences of `(attributeSet1 -- attributeSet2).nonEmpty` with `attributeSet1.subsetOf(attributeSet2)` which is order of magnitudes more efficient (especially where there are many attributes). Running the previous benchmark replacing `--` with `subsetOf` returns:
```
Average time: 67 us (0.1 %)
```

3) Provides a more efficient way of building `AttributeSet`s, which can greatly improve the performance of the methods `references` and `outputSet` of `Expression` and `QueryPlan`. This basically avoids unneeded operations (eg. creating many `AttributeEqual` wrapper classes which could be avoided)

The overall effect of those optimizations has been tested on `ColumnPruning` with the following benchmark:

```
test("ColumnPruning benchmark") {
    val attrSetA = (1 to 100).map { i => AttributeReference(s"c$i", IntegerType)() }
    val attrSetB = attrSetA.take(80)
    val attrSetC = attrSetA.take(20).map(a => Alias(Add(a, Literal(1)), s"${a.name}_1")())

    val input = LocalRelation(attrSetA)
    val query1 = Project(attrSetB, Project(attrSetA, input)).analyze
    val query2 = Project(attrSetC, Project(attrSetA, input)).analyze
    val query3 = Project(attrSetA, Project(attrSetA, input)).analyze
    val nIter = 100000
    val t0 = System.nanoTime()
    (1 to nIter).foreach { _ =>
      ColumnPruning(query1)
      ColumnPruning(query2)
      ColumnPruning(query3)
    }
    val t1 = System.nanoTime()
    val totalTime = t1 - t0
    println(s"Average time: ${totalTime / nIter} us")
}
```

The output of the test is:

```
Before PR - Average time: 733471 us (100  %)
After PR  - Average time: 362455 us (49.4 %)
```

The performance improvement has been evaluated also on the `SQLQueryTestSuite`'s queries:

```
(before) org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              518413198 / 1377707172                          2756 / 15717
(after)  org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              415432579 / 1121147950                          2756 / 15717
% Running time                                                                                                  80.1% / 81.3%
```

Also other rules benefit especially from (3), despite the impact is lower, eg:
```
(before) org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  307341442 / 623436806                           2154 / 16480
(after)  org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  290511312 / 560962495                           2154 / 16480
% Running time                                                                                                  94.5% / 90.0%
```

The reason why the impact on the `SQLQueryTestSuite`'s queries is lower compared to the other benchmark is that the optimizations are more significant when the number of attributes involved is higher. Since in the tests we often have very few attributes, the effect there is lower.

## How was this patch tested?

run benchmarks + existing UTs

Closes #22364 from mgaido91/SPARK-25379.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-26 21:34:18 +08:00
Gengliang Wang b39e228ce8 [SPARK-25541][SQL] CaseInsensitiveMap should be serializable after '-' or 'filterKeys'
## What changes were proposed in this pull request?

`CaseInsensitiveMap` is declared as Serializable. However, it is no serializable after `-` operator or `filterKeys` method.

This PR fix the issue by  overriding the operator `-` and method `filterKeys`. So the we can avoid potential `NotSerializableException` on using `CaseInsensitiveMap`.

## How was this patch tested?

New test suite.

Closes #22553 from gengliangwang/fixCaseInsensitiveMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-26 19:41:45 +08:00
Dongjoon Hyun 81cbcca600
[SPARK-25534][SQL] Make SQLHelper trait
## What changes were proposed in this pull request?

Currently, Spark has 7 `withTempPath` and 6 `withSQLConf` functions. This PR aims to remove duplicated and inconsistent code and reduce them to the following meaningful implementations.

**withTempPath**
- `SQLHelper.withTempPath`: The one which was used in `SQLTestUtils`.

**withSQLConf**
- `SQLHelper.withSQLConf`: The one which was used in `PlanTest`.
- `ExecutorSideSQLConfSuite.withSQLConf`: The one which doesn't throw `AnalysisException` on StaticConf changes.
- `SQLTestUtils.withSQLConf`: The one which overrides intentionally to change the active session.
```scala
  protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    SparkSession.setActiveSession(spark)
    super.withSQLConf(pairs: _*)(f)
  }
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #22548 from dongjoon-hyun/SPARK-25534.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-25 23:03:54 -07:00
Maxim Gekk 473d0d862d [SPARK-25514][SQL] Generating pretty JSON by to_json
## What changes were proposed in this pull request?

The PR introduces new JSON option `pretty` which allows to turn on `DefaultPrettyPrinter` of `Jackson`'s Json generator. New option is useful in exploring of deep nested columns and in converting of JSON columns in more readable representation (look at the added test).

## How was this patch tested?

Added rount trip test which convert an JSON string to pretty representation via `from_json()` and `to_json()`.

Closes #22534 from MaxGekk/pretty-json.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-26 09:52:15 +08:00
Huaxin Gao cb77a66891 [SPARK-21291][R] add R partitionBy API in DataFrame
## What changes were proposed in this pull request?

add R partitionBy API in write.df
I didn't add bucketBy in write.df. The last line of write.df is
```
write <- handledCallJMethod(write, "save")
```
save doesn't support bucketBy right now.
```
 assertNotBucketed("save")
```

## How was this patch tested?

Add unit test in test_sparkSQL.R

Closes #22537 from huaxingao/spark-21291.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-26 09:37:44 +08:00
gatorsmile 8c2edf46d0 [SPARK-24324][PYTHON][FOLLOW-UP] Rename the Conf to spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName
## What changes were proposed in this pull request?

Add the legacy prefix for spark.sql.execution.pandas.groupedMap.assignColumnsByPosition and rename it to spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName

## How was this patch tested?
The existing tests.

Closes #22540 from gatorsmile/renameAssignColumnsByPosition.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-26 09:32:51 +08:00
Imran Rashid 9bb3a0c67b [SPARK-25422][CORE] Don't memory map blocks streamed to disk.
After data has been streamed to disk, the buffers are inserted into the
memory store in some cases (eg., with broadcast blocks).  But broadcast
code also disposes of those buffers when the data has been read, to
ensure that we don't leave mapped buffers using up memory, which then
leads to garbage data in the memory store.

## How was this patch tested?

Ran the old failing test in a loop. Full tests on jenkins

Closes #22546 from squito/SPARK-25422-master.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-26 08:45:27 +08:00
Shixiong Zhu 66d29870c0
[SPARK-25495][SS] FetchedData.reset should reset all fields
## What changes were proposed in this pull request?

`FetchedData.reset` should reset `_nextOffsetInFetchedData` and `_offsetAfterPoll`. Otherwise it will cause inconsistent cached data and may make Kafka connector return wrong results.

## How was this patch tested?

The new unit test.

Closes #22507 from zsxwing/fix-kafka-reset.

Lead-authored-by: Shixiong Zhu <zsxwing@gmail.com>
Co-authored-by: Shixiong Zhu <shixiong@databricks.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-09-25 11:42:27 -07:00
yucai 04db035378
[SPARK-25486][TEST] Refactor SortBenchmark to use main method
## What changes were proposed in this pull request?

Refactor SortBenchmark to use main method.
Generate benchmark result:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.SortBenchmark"
```

## How was this patch tested?

manual tests

Closes #22495 from yucai/SPARK-25486.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-25 11:13:05 -07:00
Reynold Xin 9cbd001e24 [SPARK-23907][SQL] Revert regr_* functions entirely
## What changes were proposed in this pull request?
This patch reverts entirely all the regr_* functions added in SPARK-23907. These were added by mgaido91 (and proposed by gatorsmile) to improve compatibility with other database systems, without any actual use cases. However, they are very rarely used, and in Spark there are much better ways to compute these functions, due to Spark's flexibility in exposing real programming APIs.

I'm going through all the APIs added in Spark 2.4 and I think we should revert these. If there are strong enough demands and more use cases, we can add them back in the future pretty easily.

## How was this patch tested?
Reverted test cases also.

Closes #22541 from rxin/SPARK-23907.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-25 20:13:07 +08:00
Dilip Biswal 7d8f5b62c5 [SPARK-25519][SQL] ArrayRemove function may return incorrect result when right expression is implicitly downcasted.
## What changes were proposed in this pull request?
In ArrayRemove, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result.

Example :
```SQL
spark-sql> select array_remove(array(1,2,3), 1.23D);
       [2,3]
```
```SQL
spark-sql> select array_remove(array(1,2,3), 'foo');
        NULL
```
We should safely coerce both left and right hand side expressions.
## How was this patch tested?
Added tests in DataFrameFunctionsSuite

Closes #22542 from dilipbiswal/SPARK-25519.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-25 12:05:04 +08:00
Shahid 615792da42
[SPARK-25503][CORE][WEBUI] Total task message in stage page is ambiguous
## What changes were proposed in this pull request?
Test steps :
   1) bin/spark-shell --conf spark.ui.retainedTasks=10
   2) val rdd = sc.parallelize(1 to 1000, 1000)
  3)  rdd.count

Stage page tab in the UI will display 10 tasks, but display message is wrong. It should reverse.

**Before fix :**
![webui_1](https://user-images.githubusercontent.com/23054875/45917921-8926d800-be9c-11e8-8da5-3998d07e3ccc.jpg)

**After fix**
![spark_web_ui2](https://user-images.githubusercontent.com/23054875/45917935-b4112c00-be9c-11e8-9d10-4fcc8e88568f.jpg)

## How was this patch tested?

Manually tested

Closes #22525 from shahidki31/SparkUI.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-24 20:03:52 -07:00
hyukjinkwon 2c9ffda1b5 [BUILD] Closes stale PR
Closes #22517
2018-09-25 07:38:40 +08:00
Shahid 3ce2e008ec [SPARK-25502][CORE][WEBUI] Empty Page when page number exceeds the reatinedTask size.
## What changes were proposed in this pull request?
Test steps :
1)  bin/spark-shell --conf spark.ui.retainedTasks=200
```
val rdd = sc.parallelize(1 to 1000, 1000)
rdd.count
```

Stage tab in the UI will display 10 pages with 100 tasks per page. But number of retained tasks is only 200. So, from the 3rd page onwards will display nothing.
 We have to calculate total pages based on the number of tasks need display in the UI.

**Before fix:**
![empty_4](https://user-images.githubusercontent.com/23054875/45918251-b1650580-bea1-11e8-90d3-7e0d491981a2.jpg)

**After fix:**
![empty_3](https://user-images.githubusercontent.com/23054875/45918257-c2ae1200-bea1-11e8-960f-dfbdb4a90ae7.jpg)

## How was this patch tested?

Manually tested

Closes #22526 from shahidki31/SPARK-25502.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-24 14:17:42 -07:00
Dilip Biswal bb49661e19 [SPARK-25416][SQL] ArrayPosition function may return incorrect result when right expression is implicitly down casted
## What changes were proposed in this pull request?
In ArrayPosition, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result.

Example :
```SQL
spark-sql> select array_position(array(1), 1.34);
1
```
```SQL
spark-sql> select array_position(array(1), 'foo');
null
```

We should safely coerce both left and right hand side expressions.
## How was this patch tested?
Added tests in DataFrameFunctionsSuite

Closes #22407 from dilipbiswal/SPARK-25416.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-24 21:37:51 +08:00
Stan Zhai 804515f821 [SPARK-21318][SQL] Improve exception message thrown by lookupFunction
## What changes were proposed in this pull request?

The function actually exists in current selected database, and it's failed to init during `lookupFunciton`, but the exception message is:
```
This function is neither a registered temporary function nor a permanent function registered in the database 'default'.
```

This is not conducive to positioning problems. This PR fix the problem.

## How was this patch tested?

new test case + manual tests

Closes #18544 from stanzhai/fix-udf-error-message.

Authored-by: Stan Zhai <mail@stanzhai.site>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-24 21:33:12 +08:00
hyukjinkwon c3b4a94a91 [SPARKR] Match pyspark features in SparkR communication protocol 2018-09-24 19:25:02 +08:00
Yuming Wang c79072aafa
[SPARK-25478][SQL][TEST] Refactor CompressionSchemeBenchmark to use main method
## What changes were proposed in this pull request?

Refactor `CompressionSchemeBenchmark` to use main method.
Generate benchmark result:
```sh
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.columnar.compression.CompressionSchemeBenchmark"
```

## How was this patch tested?

manual tests

Closes #22486 from wangyum/SPARK-25478.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-23 20:46:40 -07:00
Yuming Wang d522a563ad [SPARK-25415][SQL][FOLLOW-UP] Add Locale.ROOT when toUpperCase
## What changes were proposed in this pull request?

Add `Locale.ROOT` when `toUpperCase`.

## How was this patch tested?

manual tests

Closes #22531 from wangyum/SPARK-25415.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-24 09:30:07 +08:00
seancxmao 9bf04d8543
[SPARK-25489][ML][TEST] Refactor UDTSerializationBenchmark
## What changes were proposed in this pull request?
Refactor `UDTSerializationBenchmark` to use main method and print the output as a separate file.

Run blow command to generate benchmark results:

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "mllib/test:runMain org.apache.spark.mllib.linalg.UDTSerializationBenchmark"
```

## How was this patch tested?
Manual tests.

Closes #22499 from seancxmao/SPARK-25489.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-23 13:34:06 -07:00