Commit graph

23992 commits

Author SHA1 Message Date
wangguangxin.cn 46f9f44918 [SPARK-27202][MINOR][SQL] Update comments to keep according with code
## What changes were proposed in this pull request?

Update comments in `InMemoryFileIndex.listLeafFiles` to keep according with code.

## How was this patch tested?

existing test cases

Closes #24146 from WangGuangxin/SPARK-27202.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-20 17:54:28 -05:00
Rob Vesse 61d99462a0 [SPARK-26729][K8S] Make image names under test configurable
## What changes were proposed in this pull request?

Allow specifying system properties to customise the image names for the images used in the integration testing.  Useful if your CI/CD pipeline or policy requires using a different naming format.

This is one part of addressing SPARK-26729, I plan to have a follow up patch that will also make the names configurable when using `docker-image-tool.sh`

## How was this patch tested?

Ran integration tests against custom images generated by our CI/CD pipeline that do not follow Spark's existing hardcoded naming conventions using the new system properties to override the image names appropriately:

```
mvn clean integration-test -pl :spark-kubernetes-integration-tests_${SCALA_VERSION} \
            -Pkubernetes -Pkubernetes-integration-tests \
            -P${SPARK_HADOOP_PROFILE} -Dhadoop.version=${HADOOP_VERSION} \
            -Dspark.kubernetes.test.sparkTgz=${TARBALL} \
            -Dspark.kubernetes.test.imageTag=${TAG} \
            -Dspark.kubernetes.test.imageRepo=${REPO} \
            -Dspark.kubernetes.test.namespace=${K8S_NAMESPACE} \
            -Dspark.kubernetes.test.kubeConfigContext=${K8S_CONTEXT} \
            -Dspark.kubernetes.test.deployMode=${K8S_TEST_DEPLOY_MODE} \
            -Dspark.kubernetes.test.jvmImage=apache-spark \
            -Dspark.kubernetes.test.pythonImage=apache-spark-py \
            -Dspark.kubernetes.test.rImage=apache-spark-r \
            -Dtest.include.tags=k8s
...
[INFO] --- scalatest-maven-plugin:1.0:test (integration-test)  spark-kubernetes-integration-tests_2.12 ---
Discovery starting.
Discovery completed in 230 milliseconds.
Run starting. Expected test count is: 15
KubernetesSuite:
- Run SparkPi with no resources
- Run SparkPi with a very long application name.
- Use SparkLauncher.NO_RESOURCE
- Run SparkPi with a master URL without a scheme.
- Run SparkPi with an argument.
- Run SparkPi with custom labels, annotations, and environment variables.
- Run extraJVMOptions check on driver
- Run SparkRemoteFileTest using a remote data file
- Run SparkPi with env and mount secrets.
- Run PySpark on simple pi.py example
- Run PySpark with Python2 to test a pyfiles example
- Run PySpark with Python3 to test a pyfiles example
- Run PySpark with memory customization
- Run in client mode.
- Start pod creation from template
Run completed in 8 minutes, 33 seconds.
Total number of tests run: 15
Suites: completed 2, aborted 0
Tests: succeeded 15, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #23846 from rvesse/SPARK-26729.

Authored-by: Rob Vesse <rvesse@dotnetrdf.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-20 14:28:27 -07:00
Lantao Jin 93c6d2a198 [SPARK-27215][CORE] Correct the kryo configurations
## What changes were proposed in this pull request?

```scala
val KRYO_USE_UNSAFE = ConfigBuilder("spark.kyro.unsafe")
    .booleanConf
    .createWithDefault(false)

  val KRYO_USE_POOL = ConfigBuilder("spark.kyro.pool")
    .booleanConf
    .createWithDefault(true)
```
**kyro should be kryo**

## How was this patch tested?

no need

Closes #24156 from LantaoJin/SPARK-27215.

Authored-by: Lantao Jin <jinlantao@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-20 14:27:05 -07:00
Marcelo Vanzin ec5e34205a [SPARK-27094][YARN] Work around RackResolver swallowing thread interrupt.
To avoid the case where the YARN libraries would swallow the exception and
prevent YarnAllocator from shutting down, call the offending code in a
separate thread, so that the parent thread can respond appropriately to
the shut down.

As a safeguard, also explicitly stop the executor launch thread pool when
shutting down the application, to prevent new executors from coming up
after the application started its shutdown.

Tested with unit tests + some internal tests on real cluster.

Closes #24017 from vanzin/SPARK-27094.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-20 11:48:06 -07:00
Sean Owen c65f9b2bc3 [SPARK-26839][SQL] Work around classloader changes in Java 9 for Hive isolation
Note, this doesn't really resolve the JIRA, but makes the changes we can make so far that would be required to solve it.

## What changes were proposed in this pull request?

Java 9+ changed how ClassLoaders work. The two most salient points:
- The boot classloader no longer 'sees' the platform classes. A new 'platform classloader' does and should be the parent of new ClassLoaders
- The system classloader is no longer a URLClassLoader, so we can't get the URLs of JARs in its classpath

## How was this patch tested?

We'll see whether Java 8 tests still pass here. Java 11 tests do not fully pass at this point; more notes below. This does make progress on the failures though.

(NB: to test with Java 11, you need to build with Java 8 first, setting JAVA_HOME and java's executable correctly, then switch both to Java 11 for testing.)

Closes #24057 from srowen/SPARK-26839.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-20 09:12:52 -05:00
Gengliang Wang ef2d63bfb1 [SPARK-27201][WEBUI] Toggle full job description on click
## What changes were proposed in this pull request?

Previously, in https://github.com/apache/spark/pull/6646 there was an improvement to show full job description after double clicks.
I think this is a bit hard to be noticed by some users. I suggest changing the event to one click.
Also, after the full description is shown, another click should be able to hide the overflow text again.

Before click:
![short](https://user-images.githubusercontent.com/1097932/54608784-79bfca80-4a8c-11e9-912b-30799be0d6cb.png)

After click:
![full](https://user-images.githubusercontent.com/1097932/54608790-7b898e00-4a8c-11e9-9251-86061158db68.png)

Click again:
![short](https://user-images.githubusercontent.com/1097932/54608784-79bfca80-4a8c-11e9-912b-30799be0d6cb.png)

## How was this patch tested?

Manually check.

Closes #24145 from gengliangwang/showDescriptionDetail.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 21:29:13 +09:00
Maxim Gekk 1882912cca [SPARK-27199][SQL] Replace TimeZone by ZoneId in TimestampFormatter API
## What changes were proposed in this pull request?

In the PR, I propose to use `ZoneId` instead of `TimeZone` in:
- the `apply` and `getFractionFormatter ` methods of the `TimestampFormatter` object,
- and in implementations of the `TimestampFormatter` trait like `FractionTimestampFormatter`.

The reason of the changes is to avoid unnecessary conversion from `TimeZone` to `ZoneId` because `ZoneId` is used in `TimestampFormatter` implementations internally, and the conversion is performed via `String` which is not for free. Also taking into account that `TimeZone` instances are converted from `String` in some cases, the worse case looks like `String` -> `TimeZone` -> `String` -> `ZoneId`. The PR eliminates the unneeded conversions.

## How was this patch tested?

It was tested by `DateExpressionsSuite`, `DateTimeUtilsSuite` and `TimestampFormatterSuite`.

Closes #24141 from MaxGekk/zone-id.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 21:28:11 +09:00
Ajith 1f692e522c [SPARK-27200][WEBUI][HISTORYSERVER] History Environment tab must sort Configurations/Properties by default
Environment Page in SparkUI have all the configuration sorted by key. But this is not the case in History server case, to keep UX same, we can have it sorted in history server too

## What changes were proposed in this pull request?

On render of Env page the properties are sorted before creating page

## How was this patch tested?

Manually tested in UI

Closes #24143 from ajithme/historyenv.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 20:16:17 +09:00
Huon Wilson b67d369572 [SPARK-27099][SQL] Add 'xxhash64' for hashing arbitrary columns to Long
## What changes were proposed in this pull request?

This introduces a new SQL function 'xxhash64' for getting a 64-bit hash of an arbitrary number of columns.

This is designed to exactly mimic the 32-bit `hash`, which uses
MurmurHash3. The name is designed to be more future-proof than the
'hash', by indicating the exact algorithm used, similar to md5 and the
sha hashes.

## How was this patch tested?

The tests for the existing `hash` function were duplicated to run with `xxhash64`.

Closes #24019 from huonw/hash64.

Authored-by: Huon Wilson <Huon.Wilson@data61.csiro.au>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-20 16:34:34 +08:00
Darcy Shen 9a43852f17 [SPARK-27160][SQL] Fix DecimalType when building orc filters
## What changes were proposed in this pull request?
DecimalType Literal should not be casted to Long.

eg. For `df.filter("x < 3.14")`, assuming df (x in DecimalType) reads from a ORC table and uses the native ORC reader with predicate push down enabled, we will push down the `x < 3.14` predicate to the ORC reader via a SearchArgument.

OrcFilters will construct the SearchArgument, but not handle the DecimalType correctly.

The previous impl will construct `x < 3` from `x < 3.14`.

## How was this patch tested?
```
$ sbt
> sql/testOnly *OrcFilterSuite
> sql/testOnly *OrcQuerySuite -- -z "27160"
```

Closes #24092 from sadhen/spark27160.

Authored-by: Darcy Shen <sadhen@zoho.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-19 20:28:46 -07:00
Dongjoon Hyun 257391497b [SPARK-26975][SQL] Support nested-column pruning over limit/sample/repartition
## What changes were proposed in this pull request?

As [SPARK-26958](https://github.com/apache/spark/pull/23862/files) benchmark shows, nested-column pruning has limitations. This PR aims to remove the limitations on `limit/repartition/sample`. Here, repartition means `Repartition`, not `RepartitionByExpression`.

**PREPARATION**
```scala
scala> spark.range(100).map(x => (x, (x, s"$x" * 100))).toDF("col1", "col2").write.mode("overwrite").save("/tmp/p")
scala> sql("set spark.sql.optimizer.nestedSchemaPruning.enabled=true")
scala> spark.read.parquet("/tmp/p").createOrReplaceTempView("t")
```

**BEFORE**
```scala
scala> sql("SELECT col2._1 FROM (SELECT col2 FROM t LIMIT 1000000)").explain
== Physical Plan ==
CollectLimit 1000000
+- *(1) Project [col2#22._1 AS _1#28L]
   +- *(1) FileScan parquet [col2#22] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint>>

scala> sql("SELECT col2._1 FROM (SELECT /*+ REPARTITION(1) */ col2 FROM t)").explain
== Physical Plan ==
*(2) Project [col2#22._1 AS _1#33L]
+- Exchange RoundRobinPartitioning(1)
   +- *(1) Project [col2#22]
      +- *(1) FileScan parquet [col2#22] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint,_2:string>>
```

**AFTER**
```scala
scala> sql("SELECT col2._1 FROM (SELECT /*+ REPARTITION(1) */ col2 FROM t)").explain
== Physical Plan ==
Exchange RoundRobinPartitioning(1)
+- *(1) Project [col2#5._1 AS _1#11L]
   +- *(1) FileScan parquet [col2#5] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint>>
```

This supercedes https://github.com/apache/spark/pull/23542 and https://github.com/apache/spark/pull/23873 .

## How was this patch tested?

Pass the Jenkins with a newly added test suite.

Closes #23964 from dongjoon-hyun/SPARK-26975-ALIAS.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: DB Tsai <d_tsai@apple.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-19 20:24:22 -07:00
Dongjoon Hyun 4d5247778a [SPARK-27197][SQL][TEST] Add ReadNestedSchemaTest for file-based data sources
## What changes were proposed in this pull request?

The reader schema is said to be evolved (or projected) when it changed after the data is written by writers. Apache Spark file-based data sources have a test coverage for that; e.g. [ReadSchemaSuite.scala](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala). This PR aims to add a test coverage for nested columns by adding and hiding nested columns.

## How was this patch tested?

Pass the Jenkins with newly added tests.

Closes #24139 from dongjoon-hyun/SPARK-27197.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-20 00:22:05 +00:00
Hyukjin Kwon c99463d4cf [SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below:

**Before:**

```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/functions.py", line 51, in _
    jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
    return f(*a, **kw)
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace:
py4j.Py4JException: Method ascii([class java.lang.String]) does not exist
	at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
	at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339)
	at py4j.Gateway.invoke(Gateway.java:276)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
```

```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/functions.py", line 78, in _
    jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1),
ValueError: could not convert string to float: id
```

**After:**

```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
DataFrame[ascii(value): int]
```

```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
DataFrame[ATAN2(id, id): double]
```

Note that,

- This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names.

- I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions.

- There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity.
  ```python
  >>> spark.range(1).select(when(lit(True), col("id"))).show()
  ```

  ```
  +--------------------------+
  |CASE WHEN true THEN id END|
  +--------------------------+
  |                         0|
  +--------------------------+
  ```

  ```python
  >>> spark.range(1).select(when(lit(True), "id")).show()
  ```

  ```
  +--------------------------+
  |CASE WHEN true THEN id END|
  +--------------------------+
  |                        id|
  +--------------------------+
  ```

This PR also fixes as below:

https://github.com/apache/spark/pull/23882 fixed it to:

- Rename `_create_function` to `_create_name_function`
- Define new `_create_function` to take strings as column names.

This PR, I proposes to:

- Revert `_create_name_function` name to `_create_function`.
- Define new `_create_function_over_column` to take strings as column names.

## How was this patch tested?

Some unit tests were added for binary math / string functions.

Closes #24121 from HyukjinKwon/SPARK-26979.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 08:06:10 +09:00
weixiuli 8b0aa59218 [SPARK-26288][CORE] add initRegisteredExecutorsDB
## What changes were proposed in this pull request?

As we all know that spark on Yarn uses DB https://github.com/apache/spark/pull/7943 to record RegisteredExecutors information which can be reloaded and used again when the ExternalShuffleService is restarted .

The RegisteredExecutors information can't be recorded both in the mode of spark's standalone and spark on k8s , which will cause the RegisteredExecutors information to be lost ,when the ExternalShuffleService is restarted.

To solve the problem above, a method is proposed and is committed .

## How was this patch tested?
new  unit tests

Closes #23393 from weixiuli/SPARK-26288.

Authored-by: weixiuli <weixiuli@jd.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-19 16:16:43 -05:00
Yuming Wang 6783831f68 [SPARK-27179][BUILD] Exclude javax.ws.rs:jsr311-api from hadoop-client
## What changes were proposed in this pull request?
Since [YARN-7113](https://issues.apache.org/jira/browse/YARN-7113)(Hadoop-3.1.0), `hadoop-client` add `javax.ws.rs:jsr311-api` to its dependency. This conflict with [javax.ws.rs-api-2.0.1.jar](f26a1f3d37/dev/deps/spark-deps-hadoop-3.1 (L105)).
```shell
build/sbt  "core/testOnly *.UISeleniumSuite *.HistoryServerSuite" -Phadoop-3.2
...
[info] <pre>    Server Error</pre></p><h3>Caused by:</h3><pre>java.lang.NoSuchMethodError: javax.ws.rs.core.Application.getProperties()Ljava/util/Map;
...
```

This pr exclude `javax.ws.rs:jsr311-api` from hadoop-client.

## How was this patch tested?

manual tests:
```shell
build/sbt  "core/testOnly *.UISeleniumSuite *.HistoryServerSuite" -Phadoop-3.2
```

Closes #24114 from wangyum/SPARK-27179.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-19 13:31:40 -04:00
Zhu, Lipeng 99c427b1d3 [SPARK-27168][SQL][TEST] Add docker integration test for MsSql server
## What changes were proposed in this pull request?

This PR aims to add a JDBC integration test for MsSql server.

## How was this patch tested?

```
./build/mvn clean install -DskipTests
./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 \
-Dtest=none -DwildcardSuites=org.apache.spark.sql.jdbc.MsSqlServerIntegrationSuite
```

Closes #24099 from lipzhu/SPARK-27168.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-19 08:43:23 -07:00
s71955 e402de5fd0 [SPARK-26176][SQL] Verify column names for CTAS with STORED AS
## What changes were proposed in this pull request?
Currently, users meet job abortions while creating a table using the Hive serde "STORED AS" with invalid column names. We had better prevent this by raising **AnalysisException** with a guide to use aliases instead like Paquet data source tables.
thus making compatible with error message shown while creating Parquet/ORC native table.

**BEFORE**
```scala
scala> sql("set spark.sql.hive.convertMetastoreParquet=false")
scala> sql("CREATE TABLE a STORED AS PARQUET AS SELECT 1 AS `COUNT(ID)`")
Caused by: java.lang.IllegalArgumentException: No enum constant parquet.schema.OriginalType.col1
```

**AFTER**
```scala
scala> sql("CREATE TABLE a STORED AS PARQUET AS SELECT 1 AS `COUNT(ID)`")
 Please use alias to rename it.;eption: Attribute name "count(ID)" contains invalid character(s) among " ,;{}()\n\t=".
```

## How was this patch tested?
Pass the Jenkins with the newly added test case.

Closes #24075 from sujith71955/master_serde.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 20:29:47 +08:00
Takeshi Yamamuro 901c7408a4 [SPARK-27161][SQL][FOLLOWUP] Drops non-keywords from docs/sql-keywords.md
## What changes were proposed in this pull request?
This pr is a follow-up of #24093 and includes fixes below;
 - Lists up all the keywords of Spark only (that is, drops non-keywords there); I listed up all the keywords of ANSI SQL-2011 in the previous commit (SPARK-26215).
 - Sorts the keywords in `SqlBase.g4` in a alphabetical order

## How was this patch tested?
Pass Jenkins.

Closes #24125 from maropu/SPARK-27161-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 20:18:40 +08:00
mwlon d5c08fcaab [SPARK-26555][SQL] make ScalaReflection subtype checking thread safe
## What changes were proposed in this pull request?

Make ScalaReflection subtype checking thread safe by adding a lock. There is a thread safety bug in the <:< operator in all versions of scala (https://github.com/scala/bug/issues/10766).

## How was this patch tested?

Existing tests and a new one for the new subtype checking function.

Closes #24085 from mwlon/SPARK-26555.

Authored-by: mwlon <mloncaric@hmc.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 18:22:01 +08:00
wuyi a8af23d7ab [SPARK-27193][SQL] CodeFormatter should format multiple comment lines correctly
## What changes were proposed in this pull request?

when enable `spark.sql.codegen.comments`,  there will be multiple comment lines. However, CodeFormatter can not handle multiple comment lines currently:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ /**
 * Codegend pipeline for stage (id=1)
 * *(1) Project [(id#0L + 1) AS (id + 1)#3L]
 * +- *(1) Filter (id#0L = 1)
 *    +- *(1) Range (0, 10, step=1, splits=4)
 */
/* 006 */ // codegenStageId=1
/* 007 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
```

After applying this pr:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ /**
/* 006 */  * Codegend pipeline for stage (id=1)
/* 007 */  * *(1) Project [(id#0L + 1) AS (id + 1)#4L]
/* 008 */  * +- *(1) Filter (id#0L = 1)
/* 009 */  *    +- *(1) Range (0, 10, step=1, splits=2)
/* 010 */  */
/* 011 */ // codegenStageId=1
/* 012 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
```

## How was this patch tested?

Tested Manually.

Closes #24133 from Ngone51/fix-codeformatter-for-multi-comment-lines.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 14:47:51 +08:00
Gengliang Wang 28d35c8578 [SPARK-27162][SQL] Add new method asCaseSensitiveMap in CaseInsensitiveStringMap
## What changes were proposed in this pull request?

Currently, DataFrameReader/DataFrameReader supports setting Hadoop configurations via method `.option()`.
E.g, the following test case should be passed in both ORC V1 and V2
```
  class TestFileFilter extends PathFilter {
    override def accept(path: Path): Boolean = path.getParent.getName != "p=2"
  }

  withTempPath { dir =>
      val path = dir.getCanonicalPath

      val df = spark.range(2)
      df.write.orc(path + "/p=1")
      df.write.orc(path + "/p=2")
      val extraOptions = Map(
        "mapred.input.pathFilter.class" -> classOf[TestFileFilter].getName,
        "mapreduce.input.pathFilter.class" -> classOf[TestFileFilter].getName
      )
      assert(spark.read.options(extraOptions).orc(path).count() === 2)
    }
  }
```
While Hadoop Configurations are case sensitive, the current data source V2 APIs are using `CaseInsensitiveStringMap` in the top level entry `TableProvider`.
To create Hadoop configurations correctly, I suggest
1. adding a new method `asCaseSensitiveMap` in `CaseInsensitiveStringMap`.
2. Make `CaseInsensitiveStringMap` read-only to ambiguous conversion in `asCaseSensitiveMap`

## How was this patch tested?

Unit test

Closes #24094 from gengliangwang/originalMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 13:35:47 +08:00
Dongjoon Hyun 26e9849cb4 [SPARK-27195][SQL][TEST] Add AvroReadSchemaSuite
## What changes were proposed in this pull request?

The reader schema is said to be evolved (or projected) when it changed after the data is written by writers. Apache Spark file-based data sources have a test coverage for that, [ReadSchemaSuite.scala](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala). This PR aims to add `AvroReadSchemaSuite` to ensure the minimal consistency among file-based data sources and prevent a future regression in Avro data source.

## How was this patch tested?

Pass the Jenkins with the newly added test suite.

Closes #24135 from dongjoon-hyun/SPARK-27195.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-18 20:10:30 -07:00
shane knapp 5564fe5151 [SPARK-27178][K8S] add nss to the spark/k8s Dockerfile
## What changes were proposed in this pull request?

while performing some tests on our existing minikube and k8s infrastructure, i noticed that the integration tests were failing. i dug in and discovered the following message buried at the end of the stacktrace:

```
  Caused by: java.io.FileNotFoundException: /usr/lib/libnss3.so
  	at sun.security.pkcs11.Secmod.initialize(Secmod.java:193)
  	at sun.security.pkcs11.SunPKCS11.<init>(SunPKCS11.java:218)
  	... 81 more
```
after i added the `nss` package to `resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile`, everything worked.

this is also impacting current builds.  see:  https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/8959/console

## How was this patch tested?

i tested locally before pushing, and the build system will test the rest.

Closes #24111 from shaneknapp/add-nss-package-to-dockerfile.

Authored-by: shane knapp <incomplete@gmail.com>
Signed-off-by: shane knapp <incomplete@gmail.com>
2019-03-18 16:38:42 -07:00
pgandhi 7043aee1ba [SPARK-27112][CORE] : Create a resource ordering between threads to resolve the deadlocks encountered …
…when trying to kill executors either due to dynamic allocation or blacklisting

## What changes were proposed in this pull request?

There are two deadlocks as a result of the interplay between three different threads:

**task-result-getter thread**

**spark-dynamic-executor-allocation thread**

**dispatcher-event-loop thread(makeOffers())**

The fix ensures ordering synchronization constraint by acquiring lock on `TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in `makeOffers()` as well as killExecutors() method. This ensures resource ordering between the threads and thus, fixes the deadlocks.

## How was this patch tested?

Manual Tests

Closes #24072 from pgandhi999/SPARK-27112-2.

Authored-by: pgandhi <pgandhi@verizonmedia.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-18 10:33:51 -05:00
Ryan Blue e348f14259 [SPARK-26811][SQL] Add capabilities to v2.Table
## What changes were proposed in this pull request?

This adds a new method, `capabilities` to `v2.Table` that returns a set of `TableCapability`. Capabilities are used to fail queries during analysis checks, `V2WriteSupportCheck`, when the table does not support operations, like truncation.

## How was this patch tested?

Existing tests for regressions, added new analysis suite, `V2WriteSupportCheckSuite`, for new capability checks.

Closes #24012 from rdblue/SPARK-26811-add-capabilities.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-18 18:25:11 +08:00
Wenchen Fan dbcb4792f2 [SPARK-27161][SQL] improve the document of SQL keywords
## What changes were proposed in this pull request?

Make it more clear about how Spark categories keywords regarding to the config `spark.sql.parser.ansi.enabled`

## How was this patch tested?

existing tests

Closes #24093 from cloud-fan/parser.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-18 15:19:52 +09:00
André Sá de Mello f9180f8752 [SPARK-26979][PYTHON] Add missing string column name support for some SQL functions
## What changes were proposed in this pull request?

Most SQL functions defined in `spark.sql.functions` have two calling patterns, one with a Column object as input, and another with a string representing a column name, which is then converted into a Column object internally.

There are, however, a few notable exceptions:

- lower()
- upper()
- abs()
- bitwiseNOT()
- ltrim()
- rtrim()
- trim()
- ascii()
- base64()
- unbase64()

While this doesn't break anything, as you can easily create a Column object yourself prior to passing it to one of these functions, it has two undesirable consequences:

1. It is surprising - it breaks coder's expectations when they are first starting with Spark. Every API should be as consistent as possible, so as to make the learning curve smoother and to reduce causes for human error;

2. It gets in the way of stylistic conventions. Most of the time it makes Python code more readable to use literal names, and the API provides ample support for that, but these few exceptions prevent this pattern from being universally applicable.

This patch is meant to fix the aforementioned problem.

### Effect

This patch **enables** support for passing column names as input to those functions mentioned above.

### Side effects

This PR also **fixes** an issue with some functions being defined multiple times by using `_create_function()`.

### How it works

`_create_function()` was redefined to always convert the argument to a Column object. The old implementation has been kept under `_create_name_function()`, and is still being used to generate the following special functions:

- lit()
- col()
- column()
- asc()
- desc()
- asc_nulls_first()
- asc_nulls_last()
- desc_nulls_first()
- desc_nulls_last()

This is because these functions can only take a column name as their argument. This is not a problem, as their semantics require so.

## How was this patch tested?

Ran ./dev/run-tests and tested it manually.

Closes #23882 from asmello/col-name-support-pyspark.

Authored-by: André Sá de Mello <amello@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 12:58:16 -05:00
Ajith fc88d3df5c [SPARK-27164][CORE] RDD.countApprox on empty RDDs schedules jobs which never complete
## What changes were proposed in this pull request?

When Result stage has zero tasks, the Job End event is never fired, hence the Job is always running in UI. Example: sc.emptyRDD[Int].countApprox(1000) never finishes even it has no tasks to launch

## How was this patch tested?

Added UT

Closes #24100 from ajithme/emptyRDD.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 12:56:41 -05:00
fitermay 1bc481b779 [SPARK-27070] Improve performance of DefaultPartitionCoalescer
This time tested against Scala 2.11 as well

Closes #24116 from fitermay/master.

Authored-by: fitermay <fiterman@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 11:47:14 -05:00
Ajith c324e1da9d [SPARK-27122][CORE] Jetty classes must not be return via getters in org.apache.spark.ui.WebUI
## What changes were proposed in this pull request?

When we run YarnSchedulerBackendSuite, the class path seems to be made from the classes folder(resource-managers/yarn/target/scala-2.12/classes) instead of jar (resource-managers/yarn/target/spark-yarn_2.12-3.0.0-SNAPSHOT.jar) . ui.getHandlers is in spark-core and its loaded from spark-core.jar which is shaded and hence refers to org.spark_project.jetty.servlet.ServletContextHandler

Here in  org.apache.spark.scheduler.cluster.YarnSchedulerBackend, as its not shaded, it expects org.eclipse.jetty.servlet.ServletContextHandler
Refer discussion  https://issues.apache.org/jira/browse/SPARK-27122?focusedCommentId=16792318&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16792318

Hence as a fix, org.apache.spark.ui.WebUI must only return a wrapper class instance or references so that Jetty classes can be avoided in getters which are accessed outside spark-core

## How was this patch tested?

Existing UT can pass

Closes #24088 from ajithme/shadebug.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 06:44:02 -05:00
lichaoqun 4132c989db [MINOR][CORE] spark.diskStore.subDirectories <= 0 should throw Exception
## What changes were proposed in this pull request?
this pr add check this spark.diskStore.subDirectories > 0.This value need to be checked before it can be used.

## How was this patch tested?
N/A

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

Closes #24024 from lcqzte10192193/wid-lcq-190308.

Authored-by: lichaoqun <li.chaoqun@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 06:43:14 -05:00
hehuiyuan d6a3cbea5d [MINOR][DOC] Add "completedStages" metircs for namespace=appStatus
## What changes were proposed in this pull request?

Add completedStages metircs for  namespace=appStatus for monitoring.md.

Closes #24109 from hehuiyuan/hehuiyuan-patch-5.

Authored-by: hehuiyuan <hehuiyuan@ZBMAC-C02WD3K5H.local>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 06:42:00 -05:00
Liupengcheng cad475dcc9 [SPARK-26941][YARN] Fix incorrect computation of maxNumExecutorFailures in ApplicationMaster for streaming
## What changes were proposed in this pull request?

Currently, when enabled streaming dynamic allocation for streaming applications, the maxNumExecutorFailures in ApplicationMaster is still computed with `spark.dynamicAllocation.maxExecutors`.

Actually, we should consider `spark.streaming.dynamicAllocation.maxExecutors` instead.

Related codes:
f87153a3ac/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (L101)

## How was this patch tested?

NA

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

Closes #23845 from liupc/Fix-incorrect-maxNumExecutorFailures-for-streaming.

Lead-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-16 19:45:05 -05:00
Yuming Wang 9c0af746e5 [SPARK-27175][BUILD] Upgrade hadoop-3 to 3.2.0
## What changes were proposed in this pull request?

This PR upgrade `hadoop-3` to `3.2.0`  to workaround [HADOOP-16086](https://issues.apache.org/jira/browse/HADOOP-16086). Otherwise some test case will throw IllegalArgumentException:
```java
02:44:34.707 ERROR org.apache.hadoop.hive.ql.exec.Task: Job Submission failed with exception 'java.io.IOException(Cannot initialize Cluster. Please check your configuration for mapreduce.framework.name and the correspond server addresses.)'
java.io.IOException: Cannot initialize Cluster. Please check your configuration for mapreduce.framework.name and the correspond server addresses.
	at org.apache.hadoop.mapreduce.Cluster.initialize(Cluster.java:116)
	at org.apache.hadoop.mapreduce.Cluster.<init>(Cluster.java:109)
	at org.apache.hadoop.mapreduce.Cluster.<init>(Cluster.java:102)
	at org.apache.hadoop.mapred.JobClient.init(JobClient.java:475)
	at org.apache.hadoop.mapred.JobClient.<init>(JobClient.java:454)
	at org.apache.hadoop.hive.ql.exec.mr.ExecDriver.execute(ExecDriver.java:369)
	at org.apache.hadoop.hive.ql.exec.mr.MapRedTask.execute(MapRedTask.java:151)
	at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:199)
	at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:100)
	at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:2183)
	at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1839)
	at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1526)
	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1237)
	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1227)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$runHive$1(HiveClientImpl.scala:730)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:283)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:221)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:220)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:266)
	at org.apache.spark.sql.hive.client.HiveClientImpl.runHive(HiveClientImpl.scala:719)
	at org.apache.spark.sql.hive.client.HiveClientImpl.runSqlHive(HiveClientImpl.scala:709)
	at org.apache.spark.sql.hive.StatisticsSuite.createNonPartitionedTable(StatisticsSuite.scala:719)
	at org.apache.spark.sql.hive.StatisticsSuite.$anonfun$testAlterTableProperties$2(StatisticsSuite.scala:822)
```

## How was this patch tested?

manual tests

Closes #24106 from wangyum/SPARK-27175.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-16 19:42:05 -05:00
Jungtaek Lim (HeartSaVioR) 4adbcdc424 [SPARK-22000][SQL][FOLLOW-UP] Fix bad test to ensure it can test properly
## What changes were proposed in this pull request?

There was some mistake on test code: it has wrong assertion. The patch proposes fixing it, as well as fixing other stuff to make test really pass.

## How was this patch tested?

Fixed unit test.

Closes #24112 from HeartSaVioR/SPARK-22000-hotfix.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-17 08:25:40 +09:00
Lantao Jin 6a6075ac96 [SPARK-27157][DOCS] Add Executor level metrics to monitoring docs
## What changes were proposed in this pull request?

A sub-task of [SPARK-23206](https://issues.apache.org/jira/browse/SPARK-23206)
Add Executor level metrics to monitoring docs

## How was this patch tested?

jekyll

Closes #24090 from LantaoJin/SPARK-27157.

Authored-by: Lantao Jin <jinlantao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-16 14:52:19 -05:00
Dilip Biswal aea9a574c4 [SPARK-27134][SQL] array_distinct function does not work correctly with columns containing array of array
## What changes were proposed in this pull request?
Correct the logic to compute the distinct.

Below is a small repro snippet.

```
scala> val df = Seq(Seq(Seq(1, 2), Seq(1, 2), Seq(1, 2), Seq(3, 4), Seq(4, 5))).toDF("array_col")
df: org.apache.spark.sql.DataFrame = [array_col: array<array<int>>]

scala> val distinctDF = df.select(array_distinct(col("array_col")))
distinctDF: org.apache.spark.sql.DataFrame = [array_distinct(array_col): array<array<int>>]

scala> df.show(false)
+----------------------------------------+
|array_col                               |
+----------------------------------------+
|[[1, 2], [1, 2], [1, 2], [3, 4], [4, 5]]|
+----------------------------------------+
```
Error
```
scala> distinctDF.show(false)
+-------------------------+
|array_distinct(array_col)|
+-------------------------+
|[[1, 2], [1, 2], [1, 2]] |
+-------------------------+
```
Expected result
```
scala> distinctDF.show(false)
+-------------------------+
|array_distinct(array_col)|
+-------------------------+
|[[1, 2], [3, 4], [4, 5]] |
+-------------------------+
```
## How was this patch tested?
Added an additional test.

Closes #24073 from dilipbiswal/SPARK-27134.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-16 14:30:42 -05:00
Dilip Biswal 7a136f8670 [SPARK-27096][SQL][FOLLOWUP] Do the correct validation of join types in R side and fix join docs for scala, python and r
## What changes were proposed in this pull request?
This is a minor follow-up PR for SPARK-27096. The original PR reconciled the join types supported between dataset and sql interface. In case of R, we do the join type validation in the R side. In this PR we do the correct validation and adds tests in R to test all the join types along with the error condition. Along with this, i made the necessary doc correction.

## How was this patch tested?
Add R tests.

Closes #24087 from dilipbiswal/joinfix_followup.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-16 13:04:54 +09:00
SongYadong ec11790580 [CORE][MINOR] Correct the comment to show heartbeat interval is configurable
## What changes were proposed in this pull request?

Executor heartbeat interval is configurable by `"spark.executor.heartbeatInterval"`. But in a comment, heartbeat interval is presented as a constant `10s`. This pr tries to correct the description.

## How was this patch tested?

Existing unit tests.

Closes #24101 from SongYadong/heartbeat_interval_comment.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-15 20:30:36 -05:00
Zhu, Lipeng 8ee09f26d5 [SPARK-27159][SQL] update mssql server dialect to support binary type
## What changes were proposed in this pull request?

Change the binary type mapping from default blob to varbinary(max) for mssql server.
https://docs.microsoft.com/en-us/sql/t-sql/data-types/binary-and-varbinary-transact-sql?view=sql-server-2017
![image](https://user-images.githubusercontent.com/698621/54351715-0e8c8780-468b-11e9-8931-7ecb85c5ad6b.png)

## How was this patch tested?

Unit test.

Closes #24091 from lipzhu/SPARK-27159.

Authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-15 20:21:59 -05:00
Dongjoon Hyun 4bab69b22a Revert "[SPARK-27070] Fix performance bug in DefaultPartitionCoalescer"
This reverts commit 21db4336b0.
2019-03-15 14:56:08 -07:00
Gengliang Wang 2a37d6ed93 [SPARK-27132][SQL] Improve file source V2 framework
## What changes were proposed in this pull request?

During the migration of CSV V2(https://github.com/apache/spark/pull/24005), I find that we can improve the file source v2 framework by:
1. check duplicated column names in both read and write
2. Not all the file sources support filter push down. So remove `SupportsPushDownFilters` from FileScanBuilder
3. The method `isSplitable` might require data source options. Add a new member `options` to FileScan.
4. Make `FileTable.schema` a lazy value instead of a method.

## How was this patch tested?

Unit test

Closes #24066 from gengliangwang/reviseFileSourceV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-15 11:58:03 +08:00
Dongjoon Hyun 74d2f04183 [SPARK-27166][SQL] Improve printSchema to print up to the given level
## What changes were proposed in this pull request?

This PR aims to improve `printSchema` to be able to print up to the given level of the schema.

```scala
scala> val df = Seq((1,(2,(3,4)))).toDF
df: org.apache.spark.sql.DataFrame = [_1: int, _2: struct<_1: int, _2: struct<_1: int, _2: int>>]

scala> df.printSchema
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)
| | |-- _1: integer (nullable = false)
| | |-- _2: integer (nullable = false)

scala> df.printSchema(1)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)

scala> df.printSchema(2)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)

scala> df.printSchema(3)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)
| | |-- _1: integer (nullable = false)
| | |-- _2: integer (nullable = false)
```

## How was this patch tested?

Pass the Jenkins with the newly added test case.

Closes #24098 from dongjoon-hyun/SPARK-27166.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 20:27:55 -07:00
Dongjoon Hyun f26a1f3d37 [SPARK-27165][SPARK-27107][BUILD][SQL] Upgrade Apache ORC to 1.5.5
## What changes were proposed in this pull request?

This PR aims to update Apache ORC dependency to fix [SPARK-27107](https://issues.apache.org/jira/browse/SPARK-27107) .
```
[ORC-452] Support converting MAP column from JSON to ORC Improvement
[ORC-447] Change the docker scripts to keep a persistent m2 cache
[ORC-463] Add `version` command
[ORC-475] ORC reader should lazily get filesystem
[ORC-476] Make SearchAgument kryo buffer size configurable
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #24096 from dongjoon-hyun/SPARK-27165.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 20:14:31 -07:00
Holden Karau ce89d09bdf [SPARK-26343][K8S] Try to speed up running local k8s integration tests
Speed up running k8s integration tests locally by allowing folks to skip the tgz dist build and extraction

Run tests locally without a distribution of Spark, just a local build

Closes #23380 from holdenk/SPARK-26343-Speed-up-running-the-kubernetes-integration-tests-locally.

Authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-14 19:39:48 -07:00
Gengliang Wang 6d22ee3969 [SPARK-27136][SQL] Remove data source option check_files_exist
## What changes were proposed in this pull request?

The data source option check_files_exist is introduced in In #23383 when the file source V2 framework is implemented. In the PR, FileIndex was created as a member of FileTable, so that we could implement partition pruning like 0f9fcab in the future. At that time `FileIndex`es will always be created for file writes, so we needed the option to decide whether to check file existence.

After https://github.com/apache/spark/pull/23774, the option is not needed anymore, since Dataframe writes won't create unnecessary FileIndex. This PR is to remove the option.

## How was this patch tested?

Unit test.

Closes #24069 from gengliangwang/removeOptionCheckFilesExist.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-15 10:19:26 +08:00
Dave DeCaprio 8819eaba4d [SPARK-26917][SQL] Further reduce locks in CacheManager
## What changes were proposed in this pull request?

Further load increases in our production environment have shown that even the read locks can cause some contention, since they contain a mechanism that turns a read lock into an exclusive lock if a writer has been starved out.  This PR reduces the potential for lock contention even further than https://github.com/apache/spark/pull/23833.  Additionally, it uses more idiomatic scala than the previous implementation.

cloud-fan & gatorsmile This is a relatively minor improvement to the previous CacheManager changes.  At this point, I think we finally are doing the minimum possible amount of locking.

## How was this patch tested?

Has been tested on a live system where the blocking was causing major issues and it is working well.
CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.

Closes #24028 from DaveDeCaprio/read-locks-master.

Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-15 10:13:34 +08:00
fitermay 21db4336b0 [SPARK-27070] Fix performance bug in DefaultPartitionCoalescer
When trying to coalesce a UnionRDD of two large FileScanRDDs
(each with a few million partitions) into around 8k partitions
the driver can stall for over an hour.

Profiler shows that over 90% of the time is spent in TimSort
which is invoked by `pickBin`.  This patch replaces sorting with a more
efficient `min` for the purpose of finding the least occupied
PartitionGroup

Closes #23986 from fitermay/SPARK-27070.

Authored-by: fitermay <fiterman@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-14 20:13:18 -05:00
Yuming Wang f0b6245ea4 [SPARK-27158][BUILD] dev/mima and dev/scalastyle support dynamic profiles
## What changes were proposed in this pull request?

`dev/mima` and `dev/scalastyle` support dynamic reading profiles from `modules.py`.

## How was this patch tested?

manual tests

Closes #24089 from wangyum/SPARK-27158.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-15 08:20:42 +09:00
Shahid 8b5224097b [SPARK-27145][MINOR] Close store in the SQLAppStatusListenerSuite after test
## What changes were proposed in this pull request?
We create many stores in the SQLAppStatusListenerSuite, but we need to the close store after test.

## How was this patch tested?
Existing tests

Closes #24079 from shahidki31/SPARK-27145.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-14 13:08:41 -07:00