Commit graph

22096 commits

Author SHA1 Message Date
rimolive 13092d7337 [SPARK-24534][K8S] Bypass non spark-on-k8s commands
## What changes were proposed in this pull request?
This PR changes the entrypoint.sh to provide an option to run non spark-on-k8s commands (init, driver, executor) in order to let the user keep with the normal workflow without hacking the image to bypass the entrypoint

## How was this patch tested?
This patch was built manually in my local machine and I ran some tests with a combination of ```docker run``` commands.

Author: rimolive <ricardo.martinelli.oliveira@gmail.com>

Closes #21572 from rimolive/rimolive-spark-24534.
2018-06-19 13:25:00 -07:00
yucai 9dbe53eb6b [SPARK-24556][SQL] Always rewrite output partitioning in ReusedExchangeExec and InMemoryTableScanExec
## What changes were proposed in this pull request?

Currently, ReusedExchange and InMemoryTableScanExec only rewrite output partitioning if child's partitioning is HashPartitioning and do nothing for other partitioning, e.g., RangePartitioning. We should always rewrite it, otherwise, unnecessary shuffle could be introduced like https://issues.apache.org/jira/browse/SPARK-24556.

## How was this patch tested?

Add new tests.

Author: yucai <yyu1@ebay.com>

Closes #21564 from yucai/SPARK-24556.
2018-06-19 10:52:51 -07:00
Li Jin a78a904641 [SPARK-24521][SQL][TEST] Fix ineffective test in CachedTableSuite
## What changes were proposed in this pull request?

test("withColumn doesn't invalidate cached dataframe") in CachedTableSuite doesn't not work because:

The UDF is executed and test count incremented when "df.cache()" is called and the subsequent "df.collect()" has no effect on the test result.

This PR fixed this test and add another test for caching UDF.

## How was this patch tested?

Add new tests.

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21531 from icexelloss/fix-cache-test.
2018-06-19 10:42:08 -07:00
Xiao Li 9a75c18290 [SPARK-24542][SQL] UDF series UDFXPathXXXX allow users to pass carefully crafted XML to access arbitrary files
## What changes were proposed in this pull request?

UDF series UDFXPathXXXX allow users to pass carefully crafted XML to access arbitrary files. Spark does not have built-in access control. When users use the external access control library, users might bypass them and access the file contents.

This PR basically patches the Hive fix to Apache Spark. https://issues.apache.org/jira/browse/HIVE-18879

## How was this patch tested?

A unit test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21549 from gatorsmile/xpathSecurity.
2018-06-18 20:17:04 -07:00
Wenchen Fan 1737d45e08 [SPARK-24478][SQL][FOLLOWUP] Move projection and filter push down to physical conversion
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/21503, to completely move operator pushdown to the planner rule.

The code are mostly from https://github.com/apache/spark/pull/21319

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21574 from cloud-fan/followup.
2018-06-18 20:15:01 -07:00
Liang-Chi Hsieh 8f225e055c [SPARK-24548][SQL] Fix incorrect schema of Dataset with tuple encoders
## What changes were proposed in this pull request?

When creating tuple expression encoders, we should give the serializer expressions of tuple items correct names, so we can have correct output schema when we use such tuple encoders.

## How was this patch tested?

Added test.

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

Closes #21576 from viirya/SPARK-24548.
2018-06-18 11:01:17 -07:00
trystanleftwich bce1775525 [SPARK-24526][BUILD][TEST-MAVEN] Spaces in the build dir causes failures in the build/mvn script
## What changes were proposed in this pull request?

Fix the call to ${MVN_BIN} to be wrapped in quotes so it will handle having spaces in the path.

## How was this patch tested?

Ran the following to confirm using the build/mvn tool with a space in the build dir now works without error

```
mkdir /tmp/test\ spaces
cd /tmp/test\ spaces
git clone https://github.com/apache/spark.git
cd spark
# Remove all mvn references in PATH so the script will download mvn to the local dir
./build/mvn -DskipTests clean package
```

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

Author: trystanleftwich <trystan@atscale.com>

Closes #21534 from trystanleftwich/SPARK-24526.
2018-06-19 00:34:24 +08:00
Takeshi Yamamuro e219e692ef [SPARK-23772][SQL] Provide an option to ignore column of all null values or empty array during JSON schema inference
## What changes were proposed in this pull request?
This pr added a new JSON option `dropFieldIfAllNull ` to ignore column of all null values or empty array/struct during JSON schema inference.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Xiangrui Meng <meng@databricks.com>

Closes #20929 from maropu/SPARK-23772.
2018-06-19 00:24:54 +08:00
hyukjinkwon b0a9352559 [SPARK-24573][INFRA] Runs SBT checkstyle after the build to work around a side-effect
## What changes were proposed in this pull request?

Seems checkstyle affects the build in the PR builder in Jenkins. I can't reproduce in my local and seems it can only be reproduced in the PR builder.

I was checking the places it goes through and this is just a speculation that checkstyle's compilation in SBT has a side effect to the assembly build.

This PR proposes to run the SBT checkstyle after the build.

## How was this patch tested?

Jenkins tests.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21579 from HyukjinKwon/investigate-javastyle.
2018-06-18 15:32:34 +08:00
James Yu c7c0b086a0 add one supported type missing from the javadoc
## What changes were proposed in this pull request?

The supported java.math.BigInteger type is not mentioned in the javadoc of Encoders.bean()

## How was this patch tested?

only Javadoc fix

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

Author: James Yu <james@ispot.tv>

Closes #21544 from yuj/master.
2018-06-15 21:04:04 -07:00
Mukul Murthy e4fee395ec [SPARK-24525][SS] Provide an option to limit number of rows in a MemorySink
## What changes were proposed in this pull request?

Provide an option to limit number of rows in a MemorySink. Currently, MemorySink and MemorySinkV2 have unbounded size, meaning that if they're used on big data, they can OOM the stream. This change adds a maxMemorySinkRows option to limit how many rows MemorySink and MemorySinkV2 can hold. By default, they are still unbounded.

## How was this patch tested?

Added new unit tests.

Author: Mukul Murthy <mukul.murthy@databricks.com>

Closes #21559 from mukulmurthy/SPARK-24525.
2018-06-15 13:56:48 -07:00
Kazuaki Ishizaki 90da7dc241 [SPARK-24452][SQL][CORE] Avoid possible overflow in int add or multiple
## What changes were proposed in this pull request?

This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)

The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```

To avoid this problem, this PR performs cast from int to long in right hand side.

## How was this patch tested?

Existing UTs.

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

Closes #21481 from kiszk/SPARK-24452.
2018-06-15 13:47:48 -07:00
Tathagata Das b5ccf0d395 [SPARK-24396][SS][PYSPARK] Add Structured Streaming ForeachWriter for python
## What changes were proposed in this pull request?

This PR adds `foreach` for streaming queries in Python. Users will be able to specify their processing logic in two different ways.
- As a function that takes a row as input.
- As an object that has methods `open`, `process`, and `close` methods.

See the python docs in this PR for more details.

## How was this patch tested?
Added java and python unit tests

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

Closes #21477 from tdas/SPARK-24396.
2018-06-15 12:56:39 -07:00
Jacek Laskowski 495d8cf09a [SPARK-24490][WEBUI] Use WebUI.addStaticHandler in web UIs
`WebUI` defines `addStaticHandler` that web UIs don't use (and simply introduce duplication). Let's clean them up and remove duplications.

Local build and waiting for Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #21510 from jaceklaskowski/SPARK-24490-Use-WebUI.addStaticHandler.
2018-06-15 09:59:02 -07:00
Ruben Berenguel Montoro 6567fc43ac [PYTHON] Fix typo in serializer exception
## What changes were proposed in this pull request?

Fix typo in exception raised in Python serializer

## How was this patch tested?

No code changes

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

Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>

Closes #21566 from rberenguel/fix_typo_pyspark_serializers.
2018-06-15 16:59:00 +08:00
Ryan Blue 22daeba59b [SPARK-24478][SQL] Move projection and filter push down to physical conversion
## What changes were proposed in this pull request?

This removes the v2 optimizer rule for push-down and instead pushes filters and required columns when converting to a physical plan, as suggested by marmbrus. This makes the v2 relation cleaner because the output and filters do not change in the logical plan.

A side-effect of this change is that the stats from the logical (optimized) plan no longer reflect pushed filters and projection. This is a temporary state, until the planner gathers stats from the physical plan instead. An alternative to this approach is 9d3a11e68b.

The first commit was proposed in #21262. This PR replaces #21262.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #21503 from rdblue/SPARK-24478-move-push-down-to-physical-conversion.
2018-06-14 20:59:42 -07:00
mcheah 270a9a3cac [SPARK-24248][K8S] Use level triggering and state reconciliation in scheduling and lifecycle
## What changes were proposed in this pull request?

Previously, the scheduler backend was maintaining state in many places, not only for reading state but also writing to it. For example, state had to be managed in both the watch and in the executor allocator runnable. Furthermore, one had to keep track of multiple hash tables.

We can do better here by:

1. Consolidating the places where we manage state. Here, we take inspiration from traditional Kubernetes controllers. These controllers tend to follow a level-triggered mechanism. This means that the controller will continuously monitor the API server via watches and polling, and on periodic passes, the controller will reconcile the current state of the cluster with the desired state. We implement this by introducing the concept of a pod snapshot, which is a given state of the executors in the Kubernetes cluster. We operate periodically on snapshots. To prevent overloading the API server with polling requests to get the state of the cluster (particularly for executor allocation where we want to be checking frequently to get executors to launch without unbearably bad latency), we use watches to populate snapshots by applying observed events to a previous snapshot to get a new snapshot. Whenever we do poll the cluster, the polled state replaces any existing snapshot - this ensures eventual consistency and mirroring of the cluster, as is desired in a level triggered architecture.

2. Storing less specialized in-memory state in general. Previously we were creating hash tables to represent the state of executors. Instead, it's easier to represent state solely by the snapshots.

## How was this patch tested?

Integration tests should test there's no regressions end to end. Unit tests to be updated, in particular focusing on different orderings of events, particularly accounting for when events come in unexpected ordering.

Author: mcheah <mcheah@palantir.com>

Closes #21366 from mccheah/event-queue-driven-scheduling.
2018-06-14 15:56:21 -07:00
Gabor Somogyi 18cb0c0798 [SPARK-24319][SPARK SUBMIT] Fix spark-submit execution where no main class is required.
## What changes were proposed in this pull request?

With [PR 20925](https://github.com/apache/spark/pull/20925) now it's not possible to execute the following commands:
* run-example
* run-example --help
* run-example --version
* run-example --usage-error
* run-example --status ...
* run-example --kill ...

In this PR the execution will be allowed for the mentioned commands.

## How was this patch tested?

Existing unit tests extended + additional written.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #21450 from gaborgsomogyi/SPARK-24319.
2018-06-14 14:54:46 -07:00
Maxim Gekk b8f27ae3b3 [SPARK-24543][SQL] Support any type as DDL string for from_json's schema
## What changes were proposed in this pull request?

In the PR, I propose to support any DataType represented as DDL string for the from_json function. After the changes, it will be possible to specify `MapType` in SQL like:
```sql
select from_json('{"a":1, "b":2}', 'map<string, int>')
```
and in Scala (similar in other languages)
```scala
val in = Seq("""{"a": {"b": 1}}""").toDS()
val schema = "map<string, map<string, int>>"
val out = in.select(from_json($"value", schema, Map.empty[String, String]))
```

## How was this patch tested?

Added a couple sql tests and modified existing tests for Python and Scala. The former tests were modified because it is not imported for them in which format schema for `from_json` is provided.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21550 from MaxGekk/from_json-ddl-schema.
2018-06-14 13:27:27 -07:00
Li Jin d3eed8fd6d [SPARK-24563][PYTHON] Catch TypeError when testing existence of HiveConf when creating pysp…
…ark shell

## What changes were proposed in this pull request?

This PR catches TypeError when testing existence of HiveConf when creating pyspark shell

## How was this patch tested?

Manually tested. Here are the manual test cases:

Build with hive:
```
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$ bin/pyspark
Python 3.6.5 | packaged by conda-forge | (default, Apr  6 2018, 13:44:09)
[GCC 4.2.1 Compatible Apple LLVM 6.1.0 (clang-602.0.53)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
18/06/14 14:55:41 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /__ / .__/\_,_/_/ /_/\_\   version 2.4.0-SNAPSHOT
      /_/

Using Python version 3.6.5 (default, Apr  6 2018 13:44:09)
SparkSession available as 'spark'.
>>> spark.conf.get('spark.sql.catalogImplementation')
'hive'
```

Build without hive:
```
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$ bin/pyspark
Python 3.6.5 | packaged by conda-forge | (default, Apr  6 2018, 13:44:09)
[GCC 4.2.1 Compatible Apple LLVM 6.1.0 (clang-602.0.53)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
18/06/14 15:04:52 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /__ / .__/\_,_/_/ /_/\_\   version 2.4.0-SNAPSHOT
      /_/

Using Python version 3.6.5 (default, Apr  6 2018 13:44:09)
SparkSession available as 'spark'.
>>> spark.conf.get('spark.sql.catalogImplementation')
'in-memory'
```

Failed to start shell:
```
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$ bin/pyspark
Python 3.6.5 | packaged by conda-forge | (default, Apr  6 2018, 13:44:09)
[GCC 4.2.1 Compatible Apple LLVM 6.1.0 (clang-602.0.53)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
18/06/14 15:07:53 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
/Users/icexelloss/workspace/spark/python/pyspark/shell.py:45: UserWarning: Failed to initialize Spark session.
  warnings.warn("Failed to initialize Spark session.")
Traceback (most recent call last):
  File "/Users/icexelloss/workspace/spark/python/pyspark/shell.py", line 41, in <module>
    spark = SparkSession._create_shell_session()
  File "/Users/icexelloss/workspace/spark/python/pyspark/sql/session.py", line 581, in _create_shell_session
    return SparkSession.builder.getOrCreate()
  File "/Users/icexelloss/workspace/spark/python/pyspark/sql/session.py", line 168, in getOrCreate
    raise py4j.protocol.Py4JError("Fake Py4JError")
py4j.protocol.Py4JError: Fake Py4JError
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$
```

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21569 from icexelloss/SPARK-24563-fix-pyspark-shell-without-hive.
2018-06-14 13:16:20 -07:00
Marco Gaido fdadc4be08 [SPARK-24495][SQL] EnsureRequirement returns wrong plan when reordering equal keys
## What changes were proposed in this pull request?

`EnsureRequirement` in its `reorder` method currently assumes that the same key appears only once in the join condition. This of course might not be the case, and when it is not satisfied, it returns a wrong plan which produces a wrong result of the query.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21529 from mgaido91/SPARK-24495.
2018-06-14 09:20:41 -07:00
Xingbo Jiang 534065efeb [MINOR][CORE][TEST] Remove unnecessary sort in UnsafeInMemorySorterSuite
## What changes were proposed in this pull request?

We don't require specific ordering of the input data, the sort action is not necessary and misleading.

## How was this patch tested?

Existing test suite.

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

Closes #21536 from jiangxb1987/sorterSuite.
2018-06-14 14:20:48 +08:00
Marco Gaido 3bf76918fb [SPARK-24531][TESTS] Replace 2.3.0 version with 2.3.1
## What changes were proposed in this pull request?

The PR updates the 2.3 version tested to the new release 2.3.1.

## How was this patch tested?

existing UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21543 from mgaido91/patch-1.
2018-06-13 15:18:19 -07:00
Jose Torres 1b46f41c55 [SPARK-24235][SS] Implement continuous shuffle writer for single reader partition.
## What changes were proposed in this pull request?

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit

Implement continuous shuffle write RDD for a single reader partition. (I don't believe any implementation changes are actually required for multiple reader partitions, but this PR is already very large, so I want to exclude those for now to keep the size down.)

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21428 from jose-torres/writerTask.
2018-06-13 13:13:01 -07:00
Herman van Hovell 299d297e25 [SPARK-24500][SQL] Make sure streams are materialized during Tree transforms.
## What changes were proposed in this pull request?
If you construct catalyst trees using `scala.collection.immutable.Stream` you can run into situations where valid transformations do not seem to have any effect. There are two causes for this behavior:
- `Stream` is evaluated lazily. Note that default implementation will generally only evaluate a function for the first element (this makes testing a bit tricky).
- `TreeNode` and `QueryPlan` use side effects to detect if a tree has changed. Mapping over a stream is lazy and does not need to trigger this side effect. If this happens the node will invalidly assume that it did not change and return itself instead if the newly created node (this is for GC reasons).

This PR fixes this issue by forcing materialization on streams in `TreeNode` and `QueryPlan`.

## How was this patch tested?
Unit tests were added to `TreeNodeSuite` and `LogicalPlanSuite`. An integration test was added to the `PlannerSuite`

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

Closes #21539 from hvanhovell/SPARK-24500.
2018-06-13 07:09:48 -07:00
Arun Mahadevan 7703b46d28 [SPARK-24479][SS] Added config for registering streamingQueryListeners
## What changes were proposed in this pull request?

Currently a "StreamingQueryListener" can only be registered programatically. We could have a new config "spark.sql.streamingQueryListeners" similar to  "spark.sql.queryExecutionListeners" and "spark.extraListeners" for users to register custom streaming listeners.

## How was this patch tested?

New unit test and running example programs.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21504 from arunmahadevan/SPARK-24480.
2018-06-13 20:43:16 +08:00
Jungtaek Lim 4c388bccf1 [SPARK-24485][SS] Measure and log elapsed time for filesystem operations in HDFSBackedStateStoreProvider
## What changes were proposed in this pull request?

This patch measures and logs elapsed time for each operation which communicate with file system (mostly remote HDFS in production) in HDFSBackedStateStoreProvider to help investigating any latency issue.

## How was this patch tested?

Manually tested.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21506 from HeartSaVioR/SPARK-24485.
2018-06-13 12:36:20 +08:00
Jungtaek Lim 3352d6fe9a [SPARK-24466][SS] Fix TextSocketMicroBatchReader to be compatible with netcat again
## What changes were proposed in this pull request?

TextSocketMicroBatchReader was no longer be compatible with netcat due to launching temporary reader for reading schema, and closing reader, and re-opening reader. While reliable socket server should be able to handle this without any issue, nc command normally can't handle multiple connections and simply exits when closing temporary reader.

This patch fixes TextSocketMicroBatchReader to be compatible with netcat again, via deferring opening socket to the first call of planInputPartitions() instead of constructor.

## How was this patch tested?

Added unit test which fails on current and succeeds with the patch. And also manually tested.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21497 from HeartSaVioR/SPARK-24466.
2018-06-13 12:34:46 +08:00
Li Jin 9786ce66c5 [SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.

```
       >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
       >>> from pyspark.sql import Window
       >>> df = spark.createDataFrame(
       ...     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
       ...     ("id", "v"))
       >>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
       ... def mean_udf(v):
       ...     return v.mean()
       >>> w = Window.partitionBy('id')
       >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
       +---+----+------+
       | id|   v|mean_v|
       +---+----+------+
       |  1| 1.0|   1.5|
       |  1| 2.0|   1.5|
       |  2| 3.0|   6.0|
       |  2| 5.0|   6.0|
       |  2|10.0|   6.0|
       +---+----+------+
```

The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)

Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.

## How was this patch tested?

WindowPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-13 09:10:52 +08:00
Marco Gaido f53818d35b [SPARK-24506][UI] Add UI filters to tabs added after binding
## What changes were proposed in this pull request?

Currently, `spark.ui.filters` are not applied to the handlers added after binding the server. This means that every page which is added after starting the UI will not have the filters configured on it. This can allow unauthorized access to the pages.

The PR adds the filters also to the handlers added after the UI starts.

## How was this patch tested?

manual tests (without the patch, starting the thriftserver with `--conf spark.ui.filters=org.apache.hadoop.security.authentication.server.AuthenticationFilter --conf spark.org.apache.hadoop.security.authentication.server.AuthenticationFilter.params="type=simple"` you can access `http://localhost:4040/sqlserver`; with the patch, 401 is the response as for the other pages).

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21523 from mgaido91/SPARK-24506.
2018-06-12 16:42:44 -07:00
Xingbo Jiang 0d3714d221 [SPARK-23010][BUILD][FOLLOWUP] Fix java checkstyle failure of kubernetes-integration-tests
## What changes were proposed in this pull request?

Fix java checkstyle failure of kubernetes-integration-tests

## How was this patch tested?

Checked manually on my local environment.

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

Closes #21545 from jiangxb1987/k8s-checkstyle.
2018-06-12 15:57:43 -07:00
Kazuaki Ishizaki ada28f2595 [SPARK-23933][SQL] Add map_from_arrays function
## What changes were proposed in this pull request?

The PR adds the SQL function `map_from_arrays`. The behavior of the function is based on Presto's `map`. Since SparkSQL already had a `map` function, we prepared the different name for this behavior.

This function returns returns a map from a pair of arrays for keys and values.

## How was this patch tested?

Added UTs

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

Closes #21258 from kiszk/SPARK-23933.
2018-06-12 12:31:22 -07:00
Fangshi Li cc88d7fad1 [SPARK-24216][SQL] Spark TypedAggregateExpression uses getSimpleName that is not safe in scala
## What changes were proposed in this pull request?

When user create a aggregator object in scala and pass the aggregator to Spark Dataset's agg() method, Spark's will initialize TypedAggregateExpression with the nodeName field as aggregator.getClass.getSimpleName. However, getSimpleName is not safe in scala environment, depending on how user creates the aggregator object. For example, if the aggregator class full qualified name is "com.my.company.MyUtils$myAgg$2$", the getSimpleName will throw java.lang.InternalError "Malformed class name". This has been reported in scalatest https://github.com/scalatest/scalatest/pull/1044 and discussed in many scala upstream jiras such as SI-8110, SI-5425.

To fix this issue, we follow the solution in https://github.com/scalatest/scalatest/pull/1044 to add safer version of getSimpleName as a util method, and TypedAggregateExpression will invoke this util method rather than getClass.getSimpleName.

## How was this patch tested?
added unit test

Author: Fangshi Li <fli@linkedin.com>

Closes #21276 from fangshil/SPARK-24216.
2018-06-12 12:10:08 -07:00
DylanGuedes f0ef1b311d [SPARK-23931][SQL] Adds arrays_zip function to sparksql
Signed-off-by: DylanGuedes <djmgguedesgmail.com>

## What changes were proposed in this pull request?

Addition of arrays_zip function to spark sql functions.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Unit tests that checks if the results are correct.

Author: DylanGuedes <djmgguedes@gmail.com>

Closes #21045 from DylanGuedes/SPARK-23931.
2018-06-12 11:57:25 -07:00
Sanket Chintapalli 3af1d3e6d9 [SPARK-24416] Fix configuration specification for killBlacklisted executors
## What changes were proposed in this pull request?

spark.blacklist.killBlacklistedExecutors is defined as

(Experimental) If set to "true", allow Spark to automatically kill, and attempt to re-create, executors when they are blacklisted. Note that, when an entire node is added to the blacklist, all of the executors on that node will be killed.

I presume the killing of blacklisted executors only happens after the stage completes successfully and all tasks have completed or on fetch failures (updateBlacklistForFetchFailure/updateBlacklistForSuccessfulTaskSet). It is confusing because the definition states that the executor will be attempted to be recreated as soon as it is blacklisted. This is not true while the stage is in progress and an executor is blacklisted, it will not attempt to cleanup until the stage finishes.

Author: Sanket Chintapalli <schintap@yahoo-inc.com>

Closes #21475 from redsanket/SPARK-24416.
2018-06-12 13:55:08 -05:00
Marco Gaido 2824f1436b [SPARK-24531][TESTS] Remove version 2.2.0 from testing versions in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

Removing version 2.2.0 from testing versions in HiveExternalCatalogVersionsSuite as it is not present anymore in the mirrors and this is blocking all the open PRs.

## How was this patch tested?

running UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21540 from mgaido91/SPARK-24531.
2018-06-12 09:56:35 -07:00
Lee Dongjin 5d6a53d983 [SPARK-15064][ML] Locale support in StopWordsRemover
## What changes were proposed in this pull request?

Add locale support for `StopWordsRemover`.

## How was this patch tested?

[Scala|Python] unit tests.

Author: Lee Dongjin <dongjin@apache.org>

Closes #21501 from dongjinleekr/feature/SPARK-15064.
2018-06-12 08:16:37 -07:00
Tom Saleeba 1d7db65e96 docs: fix typo
no => no[t]

## What changes were proposed in this pull request?

Fixing a typo.

## How was this patch tested?

Visual check of the docs.

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

Author: Tom Saleeba <tom.saleeba@gmail.com>

Closes #21496 from tomsaleeba/patch-1.
2018-06-12 09:22:52 -05:00
Wenchen Fan 01452ea9c7 [SPARK-24502][SQL] flaky test: UnsafeRowSerializerSuite
## What changes were proposed in this pull request?

`UnsafeRowSerializerSuite` calls `UnsafeProjection.create` which accesses `SQLConf.get`, while the current active SparkSession may already be stopped, and we may hit exception like this

```
sbt.ForkMain$ForkError: java.lang.IllegalStateException: LiveListenerBus is stopped.
	at org.apache.spark.scheduler.LiveListenerBus.addToQueue(LiveListenerBus.scala:97)
	at org.apache.spark.scheduler.LiveListenerBus.addToStatusQueue(LiveListenerBus.scala:80)
	at org.apache.spark.sql.internal.SharedState.<init>(SharedState.scala:93)
	at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:120)
	at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:120)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.sql.SparkSession.sharedState$lzycompute(SparkSession.scala:120)
	at org.apache.spark.sql.SparkSession.sharedState(SparkSession.scala:119)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:286)
	at org.apache.spark.sql.test.TestSparkSession.sessionState$lzycompute(TestSQLContext.scala:42)
	at org.apache.spark.sql.test.TestSparkSession.sessionState(TestSQLContext.scala:41)
	at org.apache.spark.sql.SparkSession$$anonfun$1$$anonfun$apply$1.apply(SparkSession.scala:95)
	at org.apache.spark.sql.SparkSession$$anonfun$1$$anonfun$apply$1.apply(SparkSession.scala:95)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.sql.SparkSession$$anonfun$1.apply(SparkSession.scala:95)
	at org.apache.spark.sql.SparkSession$$anonfun$1.apply(SparkSession.scala:94)
	at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:126)
	at org.apache.spark.sql.catalyst.expressions.CodeGeneratorWithInterpretedFallback.createObject(CodeGeneratorWithInterpretedFallback.scala:54)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:157)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:150)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite.org$apache$spark$sql$execution$UnsafeRowSerializerSuite$$unsafeRowConverter(UnsafeRowSerializerSuite.scala:54)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite.org$apache$spark$sql$execution$UnsafeRowSerializerSuite$$toUnsafeRow(UnsafeRowSerializerSuite.scala:49)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite$$anonfun$2.apply(UnsafeRowSerializerSuite.scala:63)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite$$anonfun$2.apply(UnsafeRowSerializerSuite.scala:60)
...
```

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21518 from cloud-fan/test.
2018-06-11 22:08:44 -07:00
Marcelo Vanzin dc22465f3e [SPARK-23732][DOCS] Fix source links in generated scaladoc.
Apply the suggestion on the bug to fix source links. Tested with
the 2.3.1 release docs.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21521 from vanzin/SPARK-23732.
2018-06-12 09:32:14 +08:00
liutang123 048197749e [SPARK-22144][SQL] ExchangeCoordinator combine the partitions of an 0 sized pre-shuffle to 0
## What changes were proposed in this pull request?
when the length of pre-shuffle's partitions is 0, the length of post-shuffle's partitions should be 0 instead of spark.sql.shuffle.partitions.

## How was this patch tested?
ExchangeCoordinator converted a  pre-shuffle that partitions is 0 to a post-shuffle that partitions is 0 instead of one that partitions is spark.sql.shuffle.partitions.

Author: liutang123 <liutang123@yeah.net>

Closes #19364 from liutang123/SPARK-22144.
2018-06-11 17:48:07 -07:00
Xiaodong f5af86ea75 [SPARK-24134][DOCS] A missing full-stop in doc "Tuning Spark".
## What changes were proposed in this pull request?

In the document [Tuning Spark -> Determining Memory Consumption](https://spark.apache.org/docs/latest/tuning.html#determining-memory-consumption), a full stop was missing in the second paragraph.

It's `...use SizeEstimator’s estimate method This is useful for experimenting...`, while there is supposed to be a full stop before `This`.

Screenshot showing before change is attached below.
<img width="1033" alt="screen shot 2018-05-01 at 5 22 32 pm" src="https://user-images.githubusercontent.com/11539188/39468206-778e3d8a-4d64-11e8-8a92-38464952b54b.png">

## How was this patch tested?

This is a simple change in doc. Only one full stop was added in plain text.

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

Author: Xiaodong <11539188+XD-DENG@users.noreply.github.com>

Closes #21205 from XD-DENG/patch-1.
2018-06-11 17:13:11 -05:00
Fokko Driesprong 2dc047a318 [SPARK-24520] Double braces in documentations
There are double braces in the markdown, which break the link.

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## 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.

Author: Fokko Driesprong <fokkodriesprong@godatadriven.com>

Closes #21528 from Fokko/patch-1.
2018-06-11 17:12:33 -05:00
Jonathan Kelly 9b6f24202f [MINOR][CORE] Log committer class used by HadoopMapRedCommitProtocol
## What changes were proposed in this pull request?

When HadoopMapRedCommitProtocol is used (e.g., when using saveAsTextFile() or
saveAsHadoopFile() with RDDs), it's not easy to determine which output committer
class was used, so this PR simply logs the class that was used, similarly to what
is done in SQLHadoopMapReduceCommitProtocol.

## How was this patch tested?

Built Spark then manually inspected logging when calling saveAsTextFile():

```scala
scala> sc.setLogLevel("INFO")
scala> sc.textFile("README.md").saveAsTextFile("/tmp/out")
...
18/05/29 10:06:20 INFO HadoopMapRedCommitProtocol: Using output committer class org.apache.hadoop.mapred.FileOutputCommitter
```

Author: Jonathan Kelly <jonathak@amazon.com>

Closes #21452 from ejono/master.
2018-06-11 16:41:15 -05:00
Huaxin Gao a99d284c16 [SPARK-19826][ML][PYTHON] add spark.ml Python API for PIC
## What changes were proposed in this pull request?

add spark.ml Python API for PIC

## How was this patch tested?

add doctest

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #21513 from huaxingao/spark--19826.
2018-06-11 12:15:14 -07:00
edorigatti 3e5b4ae63a [SPARK-23754][PYTHON][FOLLOWUP] Move UDF stop iteration wrapping from driver to executor
## What changes were proposed in this pull request?
SPARK-23754 was fixed in #21383 by changing the UDF code to wrap the user function, but this required a hack to save its argspec. This PR reverts this change and fixes the `StopIteration` bug in the worker

## How does this work?

The root of the problem is that when an user-supplied function raises a `StopIteration`, pyspark might stop processing data, if this function is used in a for-loop. The solution is to catch `StopIteration`s exceptions and re-raise them as `RuntimeError`s, so that the execution fails and the error is reported to the user. This is done using the `fail_on_stopiteration` wrapper, in different ways depending on where the function is used:
 - In RDDs, the user function is wrapped in the driver, because this function is also called in the driver itself.
 - In SQL UDFs, the function is wrapped in the worker, since all processing happens there. Moreover, the worker needs the signature of the user function, which is lost when wrapping it, but passing this signature to the worker requires a not so nice hack.

## How was this patch tested?

Same tests, plus tests for pandas UDFs

Author: edorigatti <emilio.dorigatti@gmail.com>

Closes #21467 from e-dorigatti/fix_udf_hack.
2018-06-11 10:15:42 +08:00
Marco Gaido f07c5064a3 [SPARK-24468][SQL] Handle negative scale when adjusting precision for decimal operations
## What changes were proposed in this pull request?

In SPARK-22036 we introduced the possibility to allow precision loss in arithmetic operations (according to the SQL standard). The implementation was drawn from Hive's one, where Decimals with a negative scale are not allowed in the operations.

The PR handles the case when the scale is negative, removing the assertion that it is not.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21499 from mgaido91/SPARK-24468.
2018-06-08 18:51:56 -07:00
Thiruvasakan Paramasivan 36a3409134
[SPARK-24412][SQL] Adding docs about automagical type casting in isin and isInCollection APIs
## What changes were proposed in this pull request?
Update documentation for `isInCollection` API to clealy explain the "auto-casting" of elements if their types are different.

## How was this patch tested?
No-Op

Author: Thiruvasakan Paramasivan <thiru@apple.com>

Closes #21519 from trvskn/sql-doc-update.
2018-06-08 17:17:43 -07:00
Sean Suchter f433ef7867 [SPARK-23010][K8S] Initial checkin of k8s integration tests.
These tests were developed in the https://github.com/apache-spark-on-k8s/spark-integration repo
by several contributors. This is a copy of the current state into the main apache spark repo.
The only changes from the current spark-integration repo state are:
* Move the files from the repo root into resource-managers/kubernetes/integration-tests
* Add a reference to these tests in the root README.md
* Fix a path reference in dev/dev-run-integration-tests.sh
* Add a TODO in include/util.sh

## What changes were proposed in this pull request?

Incorporation of Kubernetes integration tests.

## How was this patch tested?

This code has its own unit tests, but the main purpose is to provide the integration tests.
I tested this on my laptop by running dev/dev-run-integration-tests.sh --spark-tgz ~/spark-2.4.0-SNAPSHOT-bin--.tgz

The spark-integration tests have already been running for months in AMPLab, here is an example:
https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-scheduled-spark-integration-master/

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

Author: Sean Suchter <sean-github@suchter.com>
Author: Sean Suchter <ssuchter@pepperdata.com>

Closes #20697 from ssuchter/ssuchter-k8s-integration-tests.
2018-06-08 15:15:24 -07:00
hyukjinkwon b070ded284 [SPARK-17756][PYTHON][STREAMING] Workaround to avoid return type mismatch in PythonTransformFunction
## What changes were proposed in this pull request?

This PR proposes to wrap the transformed rdd within `TransformFunction`. `PythonTransformFunction` looks requiring to return `JavaRDD` in `_jrdd`.

39e2bad6a8/python/pyspark/streaming/util.py (L67)

6ee28423ad/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala (L43)

However, this could be `JavaPairRDD` by some APIs, for example, `zip` in PySpark's RDD API.
`_jrdd` could be checked as below:

```python
>>> rdd.zip(rdd)._jrdd.getClass().toString()
u'class org.apache.spark.api.java.JavaPairRDD'
```

So, here, I wrapped it with `map` so that it ensures returning `JavaRDD`.

```python
>>> rdd.zip(rdd).map(lambda x: x)._jrdd.getClass().toString()
u'class org.apache.spark.api.java.JavaRDD'
```

I tried to elaborate some failure cases as below:

```python
from pyspark.streaming import StreamingContext
ssc = StreamingContext(spark.sparkContext, 10)
ssc.queueStream([sc.range(10)]) \
    .transform(lambda rdd: rdd.cartesian(rdd)) \
    .pprint()
ssc.start()
```

```python
from pyspark.streaming import StreamingContext
ssc = StreamingContext(spark.sparkContext, 10)
ssc.queueStream([sc.range(10)]).foreachRDD(lambda rdd: rdd.cartesian(rdd))
ssc.start()
```

```python
from pyspark.streaming import StreamingContext
ssc = StreamingContext(spark.sparkContext, 10)
ssc.queueStream([sc.range(10)]).foreachRDD(lambda rdd: rdd.zip(rdd))
ssc.start()
```

```python
from pyspark.streaming import StreamingContext
ssc = StreamingContext(spark.sparkContext, 10)
ssc.queueStream([sc.range(10)]).foreachRDD(lambda rdd: rdd.zip(rdd).union(rdd.zip(rdd)))
ssc.start()
```

```python
from pyspark.streaming import StreamingContext
ssc = StreamingContext(spark.sparkContext, 10)
ssc.queueStream([sc.range(10)]).foreachRDD(lambda rdd: rdd.zip(rdd).coalesce(1))
ssc.start()
```

## How was this patch tested?

Unit tests were added in `python/pyspark/streaming/tests.py` and manually tested.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19498 from HyukjinKwon/SPARK-17756.
2018-06-09 01:27:51 +07:00