Commit graph

23558 commits

Author SHA1 Message Date
Devaraj K 1b75f3bcff [SPARK-17928][MESOS] No driver.memoryOverhead setting for mesos cluster mode
## What changes were proposed in this pull request?

Added a new configuration 'spark.mesos.driver.memoryOverhead' for providing the driver memory overhead in mesos cluster mode.

## How was this patch tested?
Verified it manually, Resource Scheduler allocates (drivermemory+ driver memoryOverhead) for driver in mesos cluster mode.

Closes #17726 from devaraj-kavali/SPARK-17928.

Authored-by: Devaraj K <devaraj@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-15 15:45:20 -06:00
Marcelo Vanzin 8a54492149 [SPARK-25857][CORE] Add developer documentation regarding delegation tokens.
Closes #23348 from vanzin/SPARK-25857.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-15 11:23:38 -08:00
Pralabh Kumar 7296999c47 [SPARK-26462][CORE] Use ConfigEntry for hardcoded configs for execution categories
## What changes were proposed in this pull request?

Make the following hardcoded configs to use ConfigEntry.
spark.memory
spark.storage
spark.io
spark.buffer
spark.rdd
spark.locality
spark.broadcast
spark.reducer

## How was this patch tested?

Existing tests.

Closes #23447 from pralabhkumar/execution_categories.

Authored-by: Pralabh Kumar <pkumar2@linkedin.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-15 12:50:07 -06:00
Gabor Somogyi 5ca45e8a3d [SPARK-26592][SS] Throw exception when kafka delegation token tried to obtain with proxy user
## What changes were proposed in this pull request?

Kafka is not yet support to obtain delegation token with proxy user. It has to be turned off until https://issues.apache.org/jira/browse/KAFKA-6945 implemented.

In this PR an exception will be thrown when this situation happens.

## How was this patch tested?

Additional unit test.

Closes #23511 from gaborgsomogyi/SPARK-26592.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-15 10:00:01 -08:00
Anton Okolnychyi b45ff02e77
[SPARK-26203][SQL][TEST] Benchmark performance of In and InSet expressions
## What changes were proposed in this pull request?

This PR contains benchmarks for `In` and `InSet` expressions. They cover literals of different data types and will help us to decide where to integrate the switch-based logic for bytes/shorts/ints.

As discussed in [PR-23171](https://github.com/apache/spark/pull/23171), one potential approach is to convert `In` to `InSet` if all elements are literals independently of data types and the number of elements. According to the results of this PR, we might want to keep the threshold for the number of elements. The if-else approach approach might be faster for some data types on a small number of elements (structs? arrays? small decimals?).

### byte / short / int / long

Unless the number of items is really big, `InSet` is slower than `In` because of autoboxing .

Interestingly, `In` scales worse on bytes/shorts than on ints/longs. For example, `InSet` starts to match the performance on around 50 bytes/shorts while this does not happen on the same number of ints/longs. This is a bit strange as shorts/bytes (e.g., `(byte) 1`, `(short) 2`) are represented as ints in the bytecode.

### float / double

Use cases on floats/doubles also suffer from autoboxing. Therefore, `In` outperforms `InSet` on 10 elements.

Similarly to shorts/bytes, `In` scales worse on floats/doubles than on ints/longs because the equality condition is more complicated (e.g., `java.lang.Float.isNaN(filter_valueArg_0) && java.lang.Float.isNaN(9.0F)) || filter_valueArg_0 == 9.0F`).

### decimal

The reason why we have separate benchmarks for small and large decimals is that Spark might use longs to represent decimals in some cases.

If this optimization happens, then `equals` will be nothing else as comparing longs. If this does not happen, Spark will create an instance of `scala.BigDecimal` and use it for comparisons. The latter is more expensive.

`Decimal$hashCode` will always use `scala.BigDecimal$hashCode` even if the number is small enough to fit into a long variable. As a consequence, we see that use cases on small decimals are faster with `In` as they are using long comparisons under the hood. Large decimal values are always faster with `InSet`.

### string

`UTF8String$equals` is not cheap. Therefore, `In` does not really outperform `InSet` as in previous use cases.

### timestamp / date

Under the hood, timestamp/date values will be represented as long/int values. So, `In` allows us to avoid autoboxing.

### array

Arrays are working as expected. `In` is faster on 5 elements while `InSet` is faster on 15 elements. The benchmarks are using `UnsafeArrayData`.

### struct

`InSet` is always faster than `In` for structs. These benchmarks use `GenericInternalRow`.

Closes #23291 from aokolnychyi/spark-26203.

Lead-authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-15 07:25:50 -07:00
SongYadong a77505d4d3 [CORE][MINOR] Fix some typos about MemoryMode
## What changes were proposed in this pull request?

Fix typos in comments by replacing "in-heap" with "on-heap".

## How was this patch tested?

Existing Tests.

Closes #23533 from SongYadong/typos_inheap_to_onheap.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-15 14:40:00 +08:00
Maxim Gekk 33b5039cd3 [SPARK-25935][SQL] Allow null rows for bad records from JSON/CSV parsers
## What changes were proposed in this pull request?

This PR reverts  #22938 per discussion in #23325

Closes #23325

Closes #23543 from MaxGekk/return-nulls-from-json-parser.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-15 13:02:55 +08:00
Takeshi Yamamuro abc937b247 [MINOR][BUILD] Remove binary license/notice files in a source release for branch-2.4+ only
## What changes were proposed in this pull request?
To skip some steps to remove binary license/notice files in a source release for branch2.3 (these files only exist in master/branch-2.4 now), this pr checked a Spark release version in `dev/create-release/release-build.sh`.

## How was this patch tested?
Manually checked.

Closes #23538 from maropu/FixReleaseScript.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-14 19:17:39 -06:00
Shixiong Zhu bafc7ac025
[SPARK-26350][SS] Allow to override group id of the Kafka consumer
## What changes were proposed in this pull request?

This PR allows the user to override `kafka.group.id` for better monitoring or security. The user needs to make sure there are not multiple queries or sources using the same group id.

It also fixes a bug that the `groupIdPrefix` option cannot be retrieved.

## How was this patch tested?

The new added unit tests.

Closes #23301 from zsxwing/SPARK-26350.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-01-14 13:37:24 -08:00
Maxim Gekk 115fecfd84 [SPARK-26456][SQL] Cast date/timestamp to string by Date/TimestampFormatter
## What changes were proposed in this pull request?

In the PR, I propose to switch on `TimestampFormatter`/`DateFormatter` in casting dates/timestamps to strings. The changes should make the date/timestamp casting consistent to JSON/CSV datasources and time-related functions like `to_date`, `to_unix_timestamp`/`from_unixtime`.

Local formatters are moved out from `DateTimeUtils` to where they are actually used. It allows to avoid re-creation of new formatter instance per-each call. Another reason is to have separate parser for `PartitioningUtils` because default parsing pattern cannot be used (expected optional section `[.S]`).

## How was this patch tested?

It was tested by `DateTimeUtilsSuite`, `CastSuite` and `JDBC*Suite`.

Closes #23391 from MaxGekk/thread-local-date-format.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-14 21:59:25 +08:00
John Zhuge 3f8007102a [SPARK-26576][SQL] Broadcast hint not applied to partitioned table
## What changes were proposed in this pull request?

Make sure broadcast hint is applied to partitioned tables.

## How was this patch tested?

- A new unit test in PruneFileSourcePartitionsSuite
- Unit test suites touched by SPARK-14581: JoinOptimizationSuite, FilterPushdownSuite, ColumnPruningSuite, and PruneFiltersSuite

Closes #23507 from jzhuge/SPARK-26576.

Closes #23530 from jzhuge/SPARK-26576-master.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-13 15:36:40 -08:00
maryannxue 985f966b9c [SPARK-26065][FOLLOW-UP][SQL] Revert hint behavior in join reordering
## What changes were proposed in this pull request?

This is to fix a bug in #23036 that would cause a join hint to be applied on node it is not supposed to after join reordering. For example,
```
  val join = df.join(df, "id")
  val broadcasted = join.hint("broadcast")
  val join2 = join.join(broadcasted, "id").join(broadcasted, "id")
```
There should only be 2 broadcast hints on `join2`, but after join reordering there would be 4. It is because the hint application in join reordering compares the attribute set for testing relation equivalency.
Moreover, it could still be problematic even if the child relations were used in testing relation equivalency, due to the potential exprId conflict in nested self-join.

As a result, this PR simply reverts the join reorder hint behavior change introduced in #23036, which means if a join hint is present, the join node itself will not participate in the join reordering, while the sub-joins within its children still can.

## How was this patch tested?

Added new tests

Closes #23524 from maryannxue/query-hint-followup-2.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-13 15:30:45 -08:00
Bruce Robbins 09b05487b7 [SPARK-26450][SQL] Avoid rebuilding map of schema for every column in projection
## What changes were proposed in this pull request?

When creating some unsafe projections, Spark rebuilds the map of schema attributes once for each expression in the projection. Some file format readers create one unsafe projection per input file, others create one per task. ProjectExec also creates one unsafe projection per task. As a result, for wide queries on wide tables, Spark might build the map of schema attributes hundreds of thousands of times.

This PR changes two functions to reuse the same AttributeSeq instance when creating BoundReference objects for each expression in the projection. This avoids the repeated rebuilding of the map of schema attributes.

### Benchmarks

The time saved by this PR depends on size of the schema, size of the projection, number of input files (or number of file splits), number of tasks, and file format. I chose a couple of example cases.

In the following tests, I ran the query
```sql
select * from table where id1 = 1
```

Matching rows are about 0.2% of the table.

#### Orc table 6000 columns, 500K rows, 34 input files

baseline | pr | improvement
----|----|----
1.772306 min | 1.487267 min | 16.082943%

#### Orc table 6000 columns, 500K rows, *17* input files

baseline | pr | improvement
----|----|----
 1.656400 min | 1.423550 min | 14.057595%

#### Orc table 60 columns, 50M rows, 34 input files

baseline | pr | improvement
----|----|----
0.299878 min | 0.290339 min | 3.180926%

#### Parquet table 6000 columns, 500K rows, 34 input files

baseline | pr | improvement
----|----|----
1.478306 min | 1.373728 min | 7.074165%

Note: The parquet reader does not create an unsafe projection. However, the filter operation in the query causes the planner to add a ProjectExec, which does create an unsafe projection for each task. So these results have nothing to do with Parquet itself.

#### Parquet table 60 columns, 50M rows, 34 input files

baseline | pr | improvement
----|----|----
0.245006 min | 0.242200 min | 1.145099%

#### CSV table 6000 columns, 500K rows, 34 input files

baseline | pr | improvement
----|----|----
2.390117 min | 2.182778 min | 8.674844%

#### CSV table 60 columns, 50M rows, 34 input files

baseline | pr | improvement
----|----|----
1.520911 min | 1.510211 min | 0.703526%

## How was this patch tested?

SQL unit tests
Python core and SQL test

Closes #23392 from bersprockets/norebuild.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-13 23:54:19 +01:00
Petar Petrov c01152dd22 [SPARK-23182][CORE] Allow enabling TCP keep alive on the RPC connections
## What changes were proposed in this pull request?

Make it possible for the master to enable TCP keep alive on the RPC connections with clients.

## How was this patch tested?

Manually tested.

Added the following:
```
spark.rpc.io.enableTcpKeepAlive  true
```
to spark-defaults.conf.

Observed the following on the Spark master:
```
$ netstat -town | grep 7077
tcp6       0      0 10.240.3.134:7077       10.240.1.25:42851       ESTABLISHED keepalive (6736.50/0/0)
tcp6       0      0 10.240.3.134:44911      10.240.3.134:7077       ESTABLISHED keepalive (4098.68/0/0)
tcp6       0      0 10.240.3.134:7077       10.240.3.134:44911      ESTABLISHED keepalive (4098.68/0/0)
```

Which proves that the keep alive setting is taking effect.

It's currently possible to enable TCP keep alive on the worker / executor, but is not possible to configure on other RPC connections. It's unclear to me why this could be the case. Keep alive is more important for the master to protect it against suddenly departing workers / executors, thus I think it's very important to have it. Particularly this makes the master resilient in case of using preemptible worker VMs in GCE. GCE has the concept of shutdown scripts, which it doesn't guarantee to execute. So workers often don't get shutdown gracefully and the TCP connections on the master linger as there's nothing to close them. Thus the need of enabling keep alive.

This enables keep-alive on connections besides the master's connections, but that shouldn't cause harm.

Closes #20512 from peshopetrov/master.

Authored-by: Petar Petrov <petar.petrov@leanplum.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-13 13:39:12 -06:00
Maxim Gekk 4ff2b94a7c [SPARK-26503][CORE][DOC][FOLLOWUP] Get rid of spark.sql.legacy.timeParser.enabled
## What changes were proposed in this pull request?

The SQL config `spark.sql.legacy.timeParser.enabled` was removed by https://github.com/apache/spark/pull/23495. The PR cleans up the SQL migration guide and the comment for `UnixTimestamp`.

Closes #23529 from MaxGekk/get-rid-off-legacy-parser-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-13 11:20:22 +08:00
Kengo Seki 3bd77aa9f6 [SPARK-26564] Fix wrong assertions and error messages for parameter checking
## What changes were proposed in this pull request?

If users set equivalent values to spark.network.timeout and spark.executor.heartbeatInterval, they get the following message:

```
java.lang.IllegalArgumentException: requirement failed: The value of spark.network.timeout=120s must be no less than the value of spark.executor.heartbeatInterval=120s.
```

But it's misleading since it can be read as they could be equal. So this PR replaces "no less than" with "greater than". Also, it fixes similar inconsistencies found in MLlib and SQL components.

## How was this patch tested?

Ran Spark with equivalent values for them manually and confirmed that the revised message was displayed.

Closes #23488 from sekikn/SPARK-26564.

Authored-by: Kengo Seki <sekikn@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-12 14:53:33 -06:00
Oleksii Shkarupin 5b37092311
[SPARK-26538][SQL] Set default precision and scale for elements of postgres numeric array
## What changes were proposed in this pull request?

When determining CatalystType for postgres columns with type `numeric[]` set the type of array element to `DecimalType(38, 18)` instead of `DecimalType(0,0)`.

## How was this patch tested?

Tested with modified `org.apache.spark.sql.jdbc.JDBCSuite`.
Ran the `PostgresIntegrationSuite` manually.

Closes #23456 from a-shkarupin/postgres_numeric_array.

Lead-authored-by: Oleksii Shkarupin <a.shkarupin@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-12 11:06:39 -08:00
Dongjoon Hyun 3587a9a227
[SPARK-26607][SQL][TEST] Remove Spark 2.2.x testing from HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

The vote of final release of `branch-2.2` passed and the branch goes EOL. This PR removes Spark 2.2.x from the testing coverage.

## How was this patch tested?

Pass the Jenkins.

Closes #23526 from dongjoon-hyun/SPARK-26607.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-11 22:53:58 -08:00
Dongjoon Hyun e00ebd5c72
[SPARK-26482][K8S][TEST][FOLLOWUP] Fix compile failure
## What changes were proposed in this pull request?

This fixes K8S integration test compilation failure introduced by #23423 .
```scala
$ build/sbt -Pkubernetes-integration-tests test:package
...
[error] /Users/dongjoon/APACHE/spark/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala:71: type mismatch;
[error]  found   : org.apache.spark.internal.config.OptionalConfigEntry[Boolean]
[error]  required: String
[error]       .set(IS_TESTING, false)
[error]            ^
[error] /Users/dongjoon/APACHE/spark/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala:71: type mismatch;
[error]  found   : Boolean(false)
[error]  required: String
[error]       .set(IS_TESTING, false)
[error]                        ^
[error] two errors found
```

## How was this patch tested?

Pass the K8S integration test.

Closes #23527 from dongjoon-hyun/SPARK-26482.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-11 21:58:06 -08:00
Dongjoon Hyun 19e17acf2d
[SPARK-25692][TEST] Increase timeout in fetchBothChunks test
## What changes were proposed in this pull request?

`ChunkFetchIntegrationSuite.fetchBothChunks` fails frequently due to timeout in Apache Spark Jenkins environments.

```scala
org.apache.spark.network.ChunkFetchIntegrationSuite
[ERROR] fetchBothChunks(org.apache.spark.network.ChunkFetchIntegrationSuite)
Time elapsed: 5.015 s  <<< FAILURE!
java.lang.AssertionError: Timeout getting response from the server
	at org.apache.spark.network.ChunkFetchIntegrationSuite.fetchChunks(ChunkFetchIntegrationSuite.java:176)
	at org.apache.spark.network.ChunkFetchIntegrationSuite.fetchBothChunks(ChunkFetchIntegrationSuite.java:210)
```

The followings are the recent failures on `amp-jenkins-worker-05`. The timeout seems to be too sensitive in low-end machines. This PR increases the timeout from 5 seconds to 60 seconds in order to be more robust.

- [master 5856](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5856/)
- [master 5837](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5837/testReport)
- [master 5835](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5835/testReport)
- [master 5829](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5829/testReport)
- [master 5828](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5828/testReport)
- [master 5822](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5822/testReport)
- [master 5814](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5814/testReport)

- [SparkPullRequestBuilder 100784](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100784/consoleFull)

- [SparkPullRequestBuilder 100785](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100785/consoleFull)

- [SparkPullRequestBuilder 100787](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100787/consoleFull)

- [SparkPullRequestBuilder 100788](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100788/consoleFull)

## How was this patch tested?

N/A (Monitor the Jenkins on `amp-jenkins-worker-05` machine)

Closes #23522 from dongjoon-hyun/SPARK-25692.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-11 17:22:05 -08:00
Mukul Murthy ae382c94dd
[SPARK-26586][SS] Fix race condition that causes streams to run with unexpected confs
## What changes were proposed in this pull request?

Fix race condition where streams can have unexpected conf values.

New streaming queries should run with isolated SparkSessions so that they aren't affected by conf updates after they are started. In StreamExecution, the parent SparkSession is cloned and used to run each batch, but this cloning happens in a separate thread and may happen after DataStreamWriter.start() returns. If a stream is started and a conf key is set immediately after, the stream is likely to have the new value.

## How was this patch tested?

New unit test that fails prior to the production change and passes with it.

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

Closes #23513 from mukulmurthy/26586.

Authored-by: Mukul Murthy <mukul.murthy@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-01-11 11:46:14 -08:00
Liang-Chi Hsieh 50ebf3a43b
[SPARK-26551][SQL] Fix schema pruning error when selecting one complex field and having is not null predicate on another one
## What changes were proposed in this pull request?

Schema pruning has errors when selecting one complex field and having is not null predicate on another one:

```scala
val query = sql("select * from contacts")
  .where("name.middle is not null")
  .select(
    "id",
    "name.first",
    "name.middle",
    "name.last"
  )
  .where("last = 'Jones'")
  .select(count("id"))
```

```
java.lang.IllegalArgumentException: middle does not exist. Available: last
[info]   at org.apache.spark.sql.types.StructType.$anonfun$fieldIndex$1(StructType.scala:303)
[info]   at scala.collection.immutable.Map$Map1.getOrElse(Map.scala:119)
[info]   at org.apache.spark.sql.types.StructType.fieldIndex(StructType.scala:302)
[info]   at org.apache.spark.sql.execution.ProjectionOverSchema.$anonfun$getProjection$6(ProjectionOverSchema.scala:58)
[info]   at scala.Option.map(Option.scala:163)
[info]   at org.apache.spark.sql.execution.ProjectionOverSchema.getProjection(ProjectionOverSchema.scala:56)
[info]   at org.apache.spark.sql.execution.ProjectionOverSchema.unapply(ProjectionOverSchema.scala:32)
[info]   at org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaPruning$$anonfun$$nestedInanonfun$buildNewProjection$1$1.applyOrElse(Parque
tSchemaPruning.scala:153)
```

## How was this patch tested?

Added tests.

Closes #23474 from viirya/SPARK-26551.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-01-11 19:23:32 +00:00
Jungtaek Lim (HeartSaVioR) d9e4cf67c0 [SPARK-26482][CORE] Use ConfigEntry for hardcoded configs for ui categories
## What changes were proposed in this pull request?

The PR makes hardcoded configs below to use `ConfigEntry`.

* spark.ui
* spark.ssl
* spark.authenticate
* spark.master.rest
* spark.master.ui
* spark.metrics
* spark.admin
* spark.modify.acl

This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).

## How was this patch tested?

Existing tests.

Closes #23423 from HeartSaVioR/SPARK-26466.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-11 10:18:07 -08:00
Sean Owen 51a6ba0181 [SPARK-26503][CORE] Get rid of spark.sql.legacy.timeParser.enabled
## What changes were proposed in this pull request?

Per discussion in #23391 (comment) this proposes to just remove the old pre-Spark-3 time parsing behavior.

This is a rebase of https://github.com/apache/spark/pull/23411

## How was this patch tested?

Existing tests.

Closes #23495 from srowen/SPARK-26503.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-11 08:53:12 -06:00
Wenchen Fan 1f1d98c6fa [SPARK-26580][SQL] remove Scala 2.11 hack for Scala UDF
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/22732 , we tried our best to keep the behavior of Scala UDF unchanged in Spark 2.4.

However, since Spark 3.0, Scala 2.12 is the default. The trick that was used to keep the behavior unchanged doesn't work with Scala 2.12.

This PR proposes to remove the Scala 2.11 hack, as it's not useful.

## How was this patch tested?

existing tests.

Closes #23498 from cloud-fan/udf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-11 14:52:13 +08:00
Yuanjian Li 98e831d321 [SPARK-25921][FOLLOW UP][PYSPARK] Fix barrier task run without BarrierTaskContext while python worker reuse
## What changes were proposed in this pull request?

It's the follow-up PR for #22962, contains the following works:
- Remove `__init__` in TaskContext and BarrierTaskContext.
- Add more comments to explain the fix.
- Rewrite UT in a new class.

## How was this patch tested?

New UT in test_taskcontext.py

Closes #23435 from xuanyuanking/SPARK-25921-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-11 14:28:37 +08:00
Dongjoon Hyun 270916f8cd
[SPARK-26584][SQL] Remove spark.sql.orc.copyBatchToSpark internal conf
## What changes were proposed in this pull request?

This PR aims to remove internal ORC configuration to simplify the code path for Spark 3.0.0. This removes the configuration `spark.sql.orc.copyBatchToSpark` and related ORC codes including tests and benchmarks.

## How was this patch tested?

Pass the Jenkins with the reduced test coverage.

Closes #23503 from dongjoon-hyun/SPARK-26584.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-10 08:42:23 -08:00
Sean Owen 2f8a938805 [SPARK-26539][CORE] Remove spark.memory.useLegacyMode and StaticMemoryManager
## What changes were proposed in this pull request?

Remove spark.memory.useLegacyMode and StaticMemoryManager. Update tests that used the StaticMemoryManager to equivalent use of UnifiedMemoryManager.

## How was this patch tested?

Existing tests, with modifications to make them work with a different mem manager.

Closes #23457 from srowen/SPARK-26539.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-10 08:57:44 -06:00
Wenchen Fan 6955638eae [SPARK-26459][SQL] replace UpdateNullabilityInAttributeReferences with FixNullability
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/18576

The newly added rule `UpdateNullabilityInAttributeReferences` does the same thing the `FixNullability` does, we only need to keep one of them.

This PR removes `UpdateNullabilityInAttributeReferences`, and use `FixNullability` to replace it. Also rename it to `UpdateAttributeNullability`

## How was this patch tested?

existing tests

Closes #23390 from cloud-fan/nullable.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-01-10 20:15:25 +09:00
Dongjoon Hyun c7daa95d7f
[SPARK-22128][CORE][BUILD] Add paranamer dependency to core module
## What changes were proposed in this pull request?

With Scala-2.12 profile, Spark application fails while Spark is okay. For example, our documented `SimpleApp` Java example succeeds to compile but it fails at runtime because it doesn't use `paranamer 2.8` and hits [SPARK-22128](https://issues.apache.org/jira/browse/SPARK-22128). This PR aims to declare it explicitly for the Spark applications. Note that this doesn't introduce new dependency to Spark itself.

https://dist.apache.org/repos/dist/dev/spark/3.0.0-SNAPSHOT-2019_01_09_13_59-e853afb-docs/_site/quick-start.html

The following is the dependency tree from the Spark application.

**BEFORE**
```
$ mvn dependency:tree -Dincludes=com.thoughtworks.paranamer
[INFO] --- maven-dependency-plugin:2.8:tree (default-cli)  simple ---
[INFO] my.test:simple:jar:1.0-SNAPSHOT
[INFO] \- org.apache.spark:spark-sql_2.12🫙3.0.0-SNAPSHOT:compile
[INFO]    \- org.apache.spark:spark-core_2.12🫙3.0.0-SNAPSHOT:compile
[INFO]       \- org.apache.avro:avro:jar:1.8.2:compile
[INFO]          \- com.thoughtworks.paranamer:paranamer:jar:2.7:compile
```

**AFTER**
```
[INFO] --- maven-dependency-plugin:2.8:tree (default-cli)  simple ---
[INFO] my.test:simple:jar:1.0-SNAPSHOT
[INFO] \- org.apache.spark:spark-sql_2.12🫙3.0.0-SNAPSHOT:compile
[INFO]    \- org.apache.spark:spark-core_2.12🫙3.0.0-SNAPSHOT:compile
[INFO]       \- com.thoughtworks.paranamer:paranamer:jar:2.8:compile
```

## How was this patch tested?

Pass the Jenkins. And manually test with the sample app is running.

Closes #23502 from dongjoon-hyun/SPARK-26583.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-10 00:40:21 -08:00
Dongjoon Hyun b316ebf0c2
[SPARK-26491][K8S][FOLLOWUP] Fix compile failure
## What changes were proposed in this pull request?

This fixes the compilation error.

```
$ cd resource-managers/kubernetes/integration-tests
$ mvn test-compile
[ERROR] /Users/dongjoon/APACHE/spark/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala:71: type mismatch;
 found   : org.apache.spark.internal.config.OptionalConfigEntry[Boolean]
 required: String
[ERROR]       .set(IS_TESTING, false)
[ERROR]            ^
```

## How was this patch tested?

Pass the Jenkins K8S Integration test or Manual.

Closes #23505 from dongjoon-hyun/SPARK-26491.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-09 23:39:42 -08:00
Maxim Gekk 73c7b126c6 [SPARK-26546][SQL] Caching of java.time.format.DateTimeFormatter
## What changes were proposed in this pull request?

Added a cache for  java.time.format.DateTimeFormatter instances with keys consist of pattern and locale. This should allow to avoid parsing of timestamp/date patterns each time when new instance of `TimestampFormatter`/`DateFormatter` is created.

## How was this patch tested?

By existing test suites `TimestampFormatterSuite`/`DateFormatterSuite` and `JsonFunctionsSuite`/`JsonSuite`.

Closes #23462 from MaxGekk/time-formatter-caching.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-10 10:32:20 +08:00
Jamison Bennett 1a47233f99 [SPARK-26493][SQL] Allow multiple spark.sql.extensions
## What changes were proposed in this pull request?

Allow multiple spark.sql.extensions to be specified in the
configuration.

## How was this patch tested?

New tests are added.

Closes #23398 from jamisonbennett/SPARK-26493.

Authored-by: Jamison Bennett <jamison.bennett@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-10 10:23:03 +08:00
maryannxue 2d01bccbd4 [SPARK-26065][FOLLOW-UP][SQL] Fix the Failure when having two Consecutive Hints
## What changes were proposed in this pull request?

This is to fix a bug in https://github.com/apache/spark/pull/23036, which would lead to an exception in case of two consecutive hints.

## How was this patch tested?

Added a new test.

Closes #23501 from maryannxue/query-hint-followup.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-09 14:31:26 -08:00
Wenchen Fan e853afb416 [SPARK-26448][SQL] retain the difference between 0.0 and -0.0
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/23043 , we introduced a behavior change: Spark users are not able to distinguish 0.0 and -0.0 anymore.

This PR proposes an alternative fix to the original bug, to retain the difference between 0.0 and -0.0 inside Spark.

The idea is, we can rewrite the window partition key, join key and grouping key during logical phase, to normalize the special floating numbers. Thus only operators care about special floating numbers need to pay the perf overhead, and end users can distinguish -0.0.

## How was this patch tested?

existing test

Closes #23388 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-09 13:50:32 -08:00
Peter Toth 49c062b2e0
[SPARK-25484][SQL][TEST] Refactor ExternalAppendOnlyUnsafeRowArrayBenchmark
## What changes were proposed in this pull request?

Refactor ExternalAppendOnlyUnsafeRowArrayBenchmark to use main method.

## How was this patch tested?

Manually tested and regenerated results.
Please note that `spark.memory.debugFill` setting has a huge impact on this benchmark. Since it is set to true by default when running the benchmark from SBT, we need to disable it:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt ";project sql;set javaOptions in Test += \"-Dspark.memory.debugFill=false\";test:runMain org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark"
```

Closes #22617 from peter-toth/SPARK-25484.

Lead-authored-by: Peter Toth <peter.toth@gmail.com>
Co-authored-by: Peter Toth <ptoth@hortonworks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-09 09:54:21 -08:00
Yuanjian Li dbbba80b3c [SPARK-26549][PYSPARK] Fix for python worker reuse take no effect for parallelize lazy iterable range
## What changes were proposed in this pull request?

During the follow-up work(#23435) for PySpark worker reuse scenario, we found that the worker reuse takes no effect for `sc.parallelize(xrange(...))`. It happened because of the specialize rdd.parallelize logic for xrange(introduced in #3264) generated data by lazy iterable range, which don't need to use the passed-in iterator. But this will break the end of stream checking in python worker and finally cause worker reuse takes no effect. See more details in [SPARK-26549](https://issues.apache.org/jira/browse/SPARK-26549) description.

We fix this by force using the passed-in iterator.

## How was this patch tested?
New UT in test_worker.py.

Closes #23470 from xuanyuanking/SPARK-26549.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-09 11:55:12 +08:00
Liupengcheng eb42bb493b [SPARK-26529] Add debug logs for confArchive when preparing local resource
## What changes were proposed in this pull request?

Currently, `Client#createConfArchive` do not handle IOException, and some detail info is not provided in logs. Sometimes, this may delay the time of locating the root cause of io error.
This PR will add debug logs for confArchive when preparing local resource.

## How was this patch tested?

unittest

Closes #23444 from liupc/Add-logs-for-IOException-when-preparing-local-resource.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-09 10:39:25 +08:00
Gengliang Wang 311f32f37f [SPARK-26571][SQL] Update Hive Serde mapping with canonical name of Parquet and Orc FileFormat
## What changes were proposed in this pull request?

Currently Spark table maintains Hive catalog storage format, so that Hive client can read it.  In `HiveSerDe.scala`, Spark uses a mapping from its data source to HiveSerde. The mapping is old, we need to update with latest canonical name of Parquet and Orc FileFormat.

Otherwise the following queries will result in wrong Serde value in Hive table(default value `org.apache.hadoop.mapred.SequenceFileInputFormat`), and Hive client will fail to read the output table:
```
df.write.format("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat").saveAsTable(..)
```

```
df.write.format("org.apache.spark.sql.execution.datasources.orc.OrcFileFormat").saveAsTable(..)
```

This minor PR is to fix the mapping.

## How was this patch tested?

Unit test.

Closes #23491 from gengliangwang/fixHiveSerdeMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-09 10:18:33 +08:00
Imran Rashid 32515d205a [SPARK-26349][PYSPARK] Forbid insecure py4j gateways
Spark always creates secure py4j connections between java and python,
but it also allows users to pass in their own connection. This ensures
that even passed in connections are secure.

Added test cases verifying the failure with a (mocked) insecure gateway.

This is closely related to SPARK-26019, but this entirely forbids the
insecure connection, rather than creating the "escape-hatch".

Closes #23441 from squito/SPARK-26349.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-01-08 11:26:36 -08:00
“attilapiros” e103c4a5e7 [SPARK-24920][CORE] Allow sharing Netty's memory pool allocators
## What changes were proposed in this pull request?

Introducing shared polled ByteBuf allocators.
This feature can be enabled via the "spark.network.sharedByteBufAllocators.enabled" configuration.

When it is on then only two pooled ByteBuf allocators are created:
- one for transport servers where caching is allowed and
- one for transport clients where caching is disabled

This way the cache allowance remains as before.
Both shareable pools are created with numCores parameter set to 0 (which defaults to the available processors) as conf.serverThreads() and conf.clientThreads() are module dependant and the lazy creation of this allocators would lead to unpredicted behaviour.

When "spark.network.sharedByteBufAllocators.enabled" is false then a new allocator is created for every transport client and server separately as was before this PR.

## How was this patch tested?

Existing unit tests.

Closes #23278 from attilapiros/SPARK-24920.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-08 13:11:11 -06:00
Marcelo Vanzin 2783e4c45f [SPARK-24522][UI] Create filter to apply HTTP security checks consistently.
Currently there is code scattered in a bunch of places to do different
things related to HTTP security, such as access control, setting
security-related headers, and filtering out bad content. This makes it
really easy to miss these things when writing new UI code.

This change creates a new filter that does all of those things, and
makes sure that all servlet handlers that are attached to the UI get
the new filter and any user-defined filters consistently. The extent
of the actual features should be the same as before.

The new filter is added at the end of the filter chain, because authentication
is done by custom filters and thus needs to happen first. This means that
custom filters see unfiltered HTTP requests - which is actually the current
behavior anyway.

As a side-effect of some of the code refactoring, handlers added after
the initial set also get wrapped with a GzipHandler, which didn't happen
before.

Tested with added unit tests and in a history server with SPNEGO auth
configured.

Closes #23302 from vanzin/SPARK-24522.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-01-08 11:25:33 -06:00
“attilapiros” c101182b10 [SPARK-26002][SQL] Fix day of year calculation for Julian calendar days
## What changes were proposed in this pull request?

Fixing leap year calculations for date operators (year/month/dayOfYear) where the Julian calendars are used (before 1582-10-04). In a Julian calendar every years which are multiples of 4 are leap years (there is no extra exception for years multiples of 100).

## How was this patch tested?

With a unit test ("SPARK-26002: correct day of year calculations for Julian calendar years") which focuses to these corner cases.

Manually:

```
scala> sql("select year('1500-01-01')").show()

+------------------------------+
|year(CAST(1500-01-01 AS DATE))|
+------------------------------+
|                          1500|
+------------------------------+

scala> sql("select dayOfYear('1100-01-01')").show()

+-----------------------------------+
|dayofyear(CAST(1100-01-01 AS DATE))|
+-----------------------------------+
|                                  1|
+-----------------------------------+
```

Closes #23000 from attilapiros/julianOffByDays.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-09 01:24:47 +08:00
liuxian b7113822d5 [MINOR][WEBUI] Modify the name of the column named "shuffle spill" in the StagePage
## What changes were proposed in this pull request?

![default](https://user-images.githubusercontent.com/24688163/50752687-16463f00-128a-11e9-8ee3-4d156f7631f6.png)
For this DAG, it has no shuffle operation, only sorting, and sorting leads to spill.

![default](https://user-images.githubusercontent.com/24688163/50752974-0f6bfc00-128b-11e9-9362-a0f440e02359.png)
So I think the name of the column named "shuffle spill" is not all right  in the StagePage

## How was this patch tested?
Manual testing

Closes #23483 from 10110346/shufflespillwebui.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-08 10:45:23 -06:00
Wenchen Fan 72a572ffd6 [SPARK-26323][SQL] Scala UDF should still check input types even if some inputs are of type Any
## What changes were proposed in this pull request?

For Scala UDF, when checking input nullability, we will skip inputs with type `Any`, and only check the inputs that provide nullability info.

We should do the same for checking input types.

## How was this patch tested?

new tests

Closes #23275 from cloud-fan/udf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-08 22:44:33 +08:00
Yuming Wang 29a7d2da44 [SPARK-24196][SQL] Implement Spark's own GetSchemasOperation
## What changes were proposed in this pull request?

This PR fix SQL Client tools can't show DBs by implementing Spark's own `GetSchemasOperation`.

## How was this patch tested?
unit tests and manual tests
![image](https://user-images.githubusercontent.com/5399861/47782885-3dd5d400-dd3c-11e8-8586-59a8c15c7020.png)
![image](https://user-images.githubusercontent.com/5399861/47782899-4928ff80-dd3c-11e8-9d2d-ba9580ba4301.png)

Closes #22903 from wangyum/SPARK-24196.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-07 18:59:43 -08:00
Dongjoon Hyun 6f35ede31c
[SPARK-26554][BUILD][FOLLOWUP] Use GitHub instead of GitBox to check HEADER
## What changes were proposed in this pull request?

This PR uses GitHub repository instead of GitBox because GitHub repo returns HTTP header status correctly.

## How was this patch tested?

Manual.

```
$ ./do-release-docker.sh -d /tmp/test -n
Branch [branch-2.4]:
Current branch version is 2.4.1-SNAPSHOT.
Release [2.4.1]:
RC # [1]:
This is a dry run. Please confirm the ref that will be built for testing.
Ref [v2.4.1-rc1]:
```

Closes #23482 from dongjoon-hyun/SPARK-26554-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-07 17:54:05 -08:00
Adrian Tanase 5fb5a0292d [MINOR][K8S] add missing docs for podTemplateContainerName properties
## What changes were proposed in this pull request?

Adding docs for an enhancement that came in late in this PR: #22146
Currently the docs state that we're going to use the first container in a pod template, which was the implementation for some time, until it was improved with 2 new properties.

## How was this patch tested?

I tested that the properties work by combining pod templates with client-mode and a simple pod template.

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

Closes #23155 from aditanase/k8s-readme.

Authored-by: Adrian Tanase <atanase@adobe.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-07 19:03:38 -06:00
Hyukjin Kwon 5102ccc4ab [SPARK-26339][SQL][FOLLOW-UP] Issue warning instead of throwing an exception for underscore files
## What changes were proposed in this pull request?

The PR https://github.com/apache/spark/pull/23446 happened to introduce a behaviour change - empty dataframes can't be read anymore from underscore files. It looks controversial to allow or disallow this case so this PR targets to fix to issue warning instead of throwing an exception to be more conservative.

**Before**

```scala
scala> spark.read.schema("a int").parquet("_tmp*").show()
org.apache.spark.sql.AnalysisException: All paths were ignored:
file:/.../_tmp
  file:/.../_tmp1;
  at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360)
  at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:651)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:635)
  ... 49 elided

scala> spark.read.text("_tmp*").show()
org.apache.spark.sql.AnalysisException: All paths were ignored:
file:/.../_tmp
  file:/.../_tmp1;
  at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360)
  at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219)
  at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:723)
  at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:695)
  ... 49 elided
```

**After**

```scala
scala> spark.read.schema("a int").parquet("_tmp*").show()
19/01/07 15:14:43 WARN DataSource: All paths were ignored:
  file:/.../_tmp
  file:/.../_tmp1
+---+
|  a|
+---+
+---+

scala> spark.read.text("_tmp*").show()
19/01/07 15:14:51 WARN DataSource: All paths were ignored:
  file:/.../_tmp
  file:/.../_tmp1
+-----+
|value|
+-----+
+-----+
```

## How was this patch tested?

Manually tested as above.

Closes #23481 from HyukjinKwon/SPARK-26339.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-07 15:48:54 -08:00
Marco Gaido 1a641525e6 [SPARK-26491][CORE][TEST] Use ConfigEntry for hardcoded configs for test categories
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.test` and `spark.testing` configs to use `ConfigEntry` and put them in the config package.

## How was this patch tested?

existing UTs

Closes #23413 from mgaido91/SPARK-26491.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-07 15:35:33 -08:00