Commit graph

28060 commits

Author SHA1 Message Date
HyukjinKwon b46c7302db [SPARK-32704][SQL][TESTS][FOLLOW-UP] Check any physical rule instead of a specific rule in the test
### What changes were proposed in this pull request?

This PR only checks if there's any physical rule runs instead of a specific rule. This is rather just a trivial fix to make the tests more robust.

In fact, I faced a test failure from a in-house fork that applies a different physical rule that makes `CollapseCodegenStages` ineffective.

### Why are the changes needed?

To make the test more robust by unrelated changes.

### Does this PR introduce _any_ user-facing change?

No, test-only

### How was this patch tested?

Manually tested. Jenkins tests should pass.

Closes #29766 from HyukjinKwon/SPARK-32704.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-16 12:06:57 +09:00
HyukjinKwon 108c4c8fdc [SPARK-32481][SQL][TESTS][FOLLOW-UP] Skip the test if trash directory cannot be created
### What changes were proposed in this pull request?

This PR skips the test if trash directory cannot be created. It is possible that the trash directory cannot be created, for example, by permission. And the test fails below:

```
- SPARK-32481 Move data to trash on truncate table if enabled *** FAILED *** (154 milliseconds)
  fs.exists(trashPath) was false (DDLSuite.scala:3184)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
  at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
```

### Why are the changes needed?

To make the tests pass independently.

### Does this PR introduce _any_ user-facing change?

No, test-only.

### How was this patch tested?

Manually tested.

Closes #29759 from HyukjinKwon/SPARK-32481.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-16 08:11:43 +09:00
ulysses 888b343587 [SPARK-32827][SQL] Add spark.sql.maxMetadataStringLength config
### What changes were proposed in this pull request?

Add a new config `spark.sql.maxMetadataStringLength`. This config aims to limit metadata value length, e.g. file location.

### Why are the changes needed?

Some metadata have been abbreviated by `...` when I tried to add some test in `SQLQueryTestSuite`. We need to replace such value to `notIncludedMsg`. That caused we can't replace that like location value by `className` since the `className` has been abbreviated.

Here is a case:
```
CREATE table  explain_temp1 (key int, val int) USING PARQUET;

EXPLAIN EXTENDED SELECT sum(distinct val) FROM explain_temp1;

-- ignore parsed,analyzed,optimized
-- The output like
== Physical Plan ==
*HashAggregate(keys=[], functions=[sum(distinct cast(val#x as bigint)#xL)], output=[sum(DISTINCT val)#xL])
+- Exchange SinglePartition, true, [id=#x]
   +- *HashAggregate(keys=[], functions=[partial_sum(distinct cast(val#x as bigint)#xL)], output=[sum#xL])
      +- *HashAggregate(keys=[cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL])
         +- Exchange hashpartitioning(cast(val#x as bigint)#xL, 4), true, [id=#x]
            +- *HashAggregate(keys=[cast(val#x as bigint) AS cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL])
               +- *ColumnarToRow
                  +- FileScan parquet default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/home/runner/work/spark/spark/sql/core/spark-warehouse/org.apache.spark.sq...], PartitionFilters: ...
```

### Does this PR introduce _any_ user-facing change?

No, a new config.

### How was this patch tested?

new test.

Closes #29688 from ulysses-you/SPARK-32827.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 14:11:30 +00:00
Abhishek Dixit 6f36db1fa5 [SPARK-31448][PYTHON] Fix storage level used in persist() in dataframe.py
### What changes were proposed in this pull request?
Since the data is serialized on the Python side, we should make cache() in PySpark dataframes use StorageLevel.MEMORY_AND_DISK mode which has deserialized=false. This change was done to `pyspark/rdd.py` as part of SPARK-2014 but was missed from `pyspark/dataframe.py`

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Using existing tests

Closes #29242 from abhishekd0907/SPARK-31448.

Authored-by: Abhishek Dixit <abhishekdixit0907@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-15 08:41:22 -05:00
Kent Yao 316242b768 [SPARK-32874][SQL][TEST] Enhance result set meta data check for execute statement operation with thrift server
### What changes were proposed in this pull request?

This PR adds test cases for the result set metadata checking for Spark's `ExecuteStatementOperation` to make the JDBC API more future-proofing because any server-side change may affect the client compatibility.

### Why are the changes needed?

add test to prevent potential silent behavior change for JDBC users.

### Does this PR introduce _any_ user-facing change?

NO, test only
### How was this patch tested?

add new test

Closes #29746 from yaooqinn/SPARK-32874.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 13:07:03 +00:00
Zhenhua Wang 99384d1e83 [SPARK-32738][CORE] Should reduce the number of active threads if fatal error happens in Inbox.process
### What changes were proposed in this pull request?

Processing for `ThreadSafeRpcEndpoint` is controlled by  `numActiveThreads` in `Inbox`. Now if any fatal error happens during `Inbox.process`, `numActiveThreads` is not reduced. Then other threads can not process messages in that inbox, which causes the endpoint to "hang". For other type of endpoints, we also should keep  `numActiveThreads` correct.

This problem is more serious in previous Spark 2.x versions since the driver, executor and block manager endpoints are all thread safe endpoints.

To fix this, we should reduce the number of active threads if fatal error happens in `Inbox.process`.

### Why are the changes needed?

`numActiveThreads` is not correct when fatal error happens and will cause the described problem.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Add a new test.

Closes #29580 from wzhfy/deal_with_fatal_error.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 06:46:17 +00:00
herman c8baab1a1f [SPARK-32879][SQL] Refactor SparkSession initial options
### What changes were proposed in this pull request?
This PR refactors the way we propagate the options from the `SparkSession.Builder` to the` SessionState`. This currently done via a mutable map inside the SparkSession. These setting settings are then applied **after** the Session. This is a bit confusing when you expect something to be set when constructing the `SessionState`. This PR passes the options as a constructor parameter to the `SessionStateBuilder` and this will set the options when the configuration is created.

### Why are the changes needed?
It makes it easier to reason about the configurations set in a SessionState than before. We recently had an incident where someone was using `SparkSessionExtensions` to create a planner rule that relied on a conf to be set. While this is in itself probably incorrect usage, it still illustrated this somewhat funky behavior.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing tests.

Closes #29752 from hvanhovell/SPARK-32879.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 06:24:54 +00:00
Dongjoon Hyun d8a0d85692 [SPARK-32884][TESTS] Mark TPCDSQuery*Suite as ExtendedSQLTest
### What changes were proposed in this pull request?

This PR aims to mark the following suite as `ExtendedSQLTest` to reduce GitHub Action test time.
- TPCDSQuerySuite
- TPCDSQueryANSISuite
- TPCDSQueryWithStatsSuite

### Why are the changes needed?

Currently, the longest GitHub Action task is `Build and test / Build modules: sql - other tests` with `1h 57m 10s` while `Build and test / Build modules: sql - slow tests` takes `42m 20s`. With this PR, we can move the workload from `other tests` to `slow tests` task and reduce the total waiting time about 7 ~ 8 minutes.

### Does this PR introduce _any_ user-facing change?

No. This is a test-only change.

### How was this patch tested?

Pass the GitHub Action with the reduced running time.

Closes #29755 from dongjoon-hyun/SPARK-SLOWTEST.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-15 14:38:01 +09:00
yi.wu 0811666ab1 [SPARK-32878][CORE] Avoid scheduling TaskSetManager which has no pending tasks
### What changes were proposed in this pull request?

This PR proposes to avoid scheduling the (non-zombie) TaskSetManager which has no pending tasks.

### Why are the changes needed?

Currently, Spark always tries to schedule a (non-zombie) TaskSetManager even if it has no pending tasks. This causes notable problems for the barrier TaskSetManager: 1. `calculateAvailableSlots` can be called for multiple times for a launched barrier TaskSetManager; 2. user would see "Skip current round of resource offers for barrier stage" log message for
a launched barrier TaskSetManager all the time until the barrier TaskSetManager finishes, which is quite confused.

Besides, scheduling a TaskSetManager always involves many function invocations even if there're no pending tasks.

Therefore, I think we can skip those un-schedulable TasksetManagers to avoid the potential overhead.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass existing tests.

Closes #29750 from Ngone51/filter-out-unschedulable-stage.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 21:15:06 -07:00
LantaoJin 7a9b066c66 [SPARK-32715][CORE] Fix memory leak when failed to store pieces of broadcast
### What changes were proposed in this pull request?
In TorrentBroadcast.scala
```scala
L133: if (!blockManager.putSingle(broadcastId, value, MEMORY_AND_DISK, tellMaster = false))
L137: TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec)
L147: if (!blockManager.putBytes(pieceId, bytes, MEMORY_AND_DISK_SER, tellMaster = true))
```
After the original value is saved successfully(TorrentBroadcast.scala: L133), but the following `blockifyObject()`(L137) or store piece(L147) steps are failed. There is no opportunity to release broadcast from memory.

This patch is to remove all pieces of the broadcast when failed to blockify or failed to store some pieces of a broadcast.

### Why are the changes needed?
We use Spark thrift-server as a long-running service. A bad query submitted a heavy BroadcastNestLoopJoin operation and made driver full GC. We killed the bad query but we found the driver's memory usage was still high and full GCs were still frequent. By investigating with GC dump and log, we found the broadcast may memory leak.

> 2020-08-19T18:54:02.824-0700: [Full GC (Allocation Failure)
2020-08-19T18:54:02.824-0700: [Class Histogram (before full gc):
116G->112G(170G), 184.9121920 secs]
[Eden: 32.0M(7616.0M)->0.0B(8704.0M) Survivors: 1088.0M->0.0B Heap: 116.4G(170.0G)->112.9G(170.0G)], [Metaspace: 177285K->177270K(182272K)]
1: 676531691 72035438432 [B
2: 676502528 32472121344 org.apache.spark.sql.catalyst.expressions.UnsafeRow
3: 99551 12018117568 [Ljava.lang.Object;
4: 26570 4349629040 [I
5: 6 3264536688 [Lorg.apache.spark.sql.catalyst.InternalRow;
6: 1708819 256299456 [C
7: 2338 179615208 [J
8: 1703669 54517408 java.lang.String
9: 103860 34896960 org.apache.spark.status.TaskDataWrapper
10: 177396 25545024 java.net.URI
...

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually test. This UT is hard to write and the patch is straightforward.

Closes #29558 from LantaoJin/SPARK-32715.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 18:24:52 -07:00
Kousuke Saruta 4fac6d501a [SPARK-32871][BUILD] Append toMap to Map#filterKeys if the result of filter is concatenated with another Map for Scala 2.13
### What changes were proposed in this pull request?

This PR appends `toMap` to `Map` instances with `filterKeys` if such maps is to be concatenated with another maps.

### Why are the changes needed?

As of Scala 2.13, Map#filterKeys return a MapView, not the original Map type.
This can cause compile error.
```
/sql/DataFrameReader.scala:279: type mismatch;
[error]  found   : Iterable[(String, String)]
[error]  required: java.util.Map[String,String]
[error] Error occurred in an application involving default arguments.
[error]       val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava)
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Compile passed with the following command.
`build/mvn -Pscala-2.13 -Phive -Phive-thriftserver -Pyarn -Pkubernetes -DskipTests test-compile`

Closes #29742 from sarutak/fix-filterKeys-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-15 09:27:47 +09:00
William Hyun d58a4a310a [SPARK-32882][K8S] Remove python2 installation in K8s python image
### What changes were proposed in this pull request?
This PR aims to remove python2 installation in K8s python image because spark 3.1 does not support python2.

### Why are the changes needed?

This will save disk space.

**BEFORE**
```
kubespark/spark-py ... 917MB
```

**AFTER**
```
kubespark/spark-py ... 823MB
```

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?

Pass the Jenkins with the K8s IT.

Closes #29751 from williamhyun/remove_py2.

Authored-by: William Hyun <williamhyun3@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 16:03:19 -07:00
Ankur Dave 72550c3be7 [SPARK-32872][CORE] Prevent BytesToBytesMap at MAX_CAPACITY from exceeding growth threshold
### What changes were proposed in this pull request?

When BytesToBytesMap is at `MAX_CAPACITY` and reaches its growth threshold, `numKeys >= growthThreshold` is true but `longArray.size() / 2 < MAX_CAPACITY` is false. This correctly prevents the map from growing, but `canGrowArray` incorrectly remains true. Therefore the map keeps accepting new keys and exceeds its growth threshold. If we attempt to spill the map in this state, the UnsafeKVExternalSorter will not be able to reuse the long array for sorting. By this point the task has typically consumed all available memory, so the allocation of the new pointer array is likely to fail.

This PR fixes the issue by setting `canGrowArray` to false in this case. This prevents the map from accepting new elements when it cannot grow to accommodate them.

### Why are the changes needed?

Without this change, hash aggregations will fail when the number of groups per task is greater than `MAX_CAPACITY / 2 = 2^28` (approximately 268 million), and when the grouping aggregation is the only memory-consuming operator in its stage.

For example, the final aggregation in `SELECT COUNT(DISTINCT id) FROM tbl` fails when `tbl` contains 1 billion distinct values and when `spark.sql.shuffle.partitions=1`.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Reproducing this issue requires building a very large BytesToBytesMap. Because this is infeasible to do in a unit test, this PR was tested manually by adding the following test to AbstractBytesToBytesMapSuite. Before this PR, the test fails in 8.5 minutes. With this PR, the test passes in 1.5 minutes.

```java
public abstract class AbstractBytesToBytesMapSuite {
  // ...
  Test
  public void respectGrowthThresholdAtMaxCapacity() {
    TestMemoryManager memoryManager2 =
        new TestMemoryManager(
            new SparkConf()
            .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true)
            .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 25600 * 1024 * 1024L)
            .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)
            .set(package$.MODULE$.SHUFFLE_COMPRESS(), false));
    TaskMemoryManager taskMemoryManager2 = new TaskMemoryManager(memoryManager2, 0);
    final long pageSizeBytes = 8000000 + 8; // 8 bytes for end-of-page marker
    final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager2, 1024, pageSizeBytes);

    try {
      // Insert keys into the map until it stops accepting new keys.
      for (long i = 0; i < BytesToBytesMap.MAX_CAPACITY; i++) {
        if (i % (1024 * 1024) == 0) System.out.println("Inserting element " + i);
        final long[] value = new long[]{i};
        BytesToBytesMap.Location loc = map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8);
        Assert.assertFalse(loc.isDefined());
        boolean success =
            loc.append(value, Platform.LONG_ARRAY_OFFSET, 8, value, Platform.LONG_ARRAY_OFFSET, 8);
        if (!success) break;
      }

      // The map should grow to its max capacity.
      long capacity = map.getArray().size() / 2;
      Assert.assertTrue(capacity == BytesToBytesMap.MAX_CAPACITY);

      // The map should stop accepting new keys once it has reached its growth
      // threshold, which is half the max capacity.
      Assert.assertTrue(map.numKeys() == BytesToBytesMap.MAX_CAPACITY / 2);

      map.free();
    } finally {
      map.free();
    }
  }
}
```

Closes #29744 from ankurdave/SPARK-32872.

Authored-by: Ankur Dave <ankurdave@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 13:58:15 -07:00
HyukjinKwon 0696f04672 [SPARK-32876][SQL] Change default fallback versions to 3.0.1 and 2.4.7 in HiveExternalCatalogVersionsSuite
### What changes were proposed in this pull request?

The Jenkins job fails to get the versions. This was fixed by adding temporary fallbacks at https://github.com/apache/spark/pull/28536.
This still doesn't work without the temporary fallbacks. See https://github.com/apache/spark/pull/29694

This PR adds new fallbacks since 2.3 is EOL and Spark 3.0.1 and 2.4.7 are released.

### Why are the changes needed?

To test correctly in Jenkins.

### Does this PR introduce _any_ user-facing change?

No, dev-only

### How was this patch tested?

Jenkins and GitHub Actions builds should test.

Closes #29748 from HyukjinKwon/SPARK-32876.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 13:54:21 -07:00
tanel.kiis@gmail.com 7a17158a4d [SPARK-32868][SQL] Add more order irrelevant aggregates to EliminateSorts
### What changes were proposed in this pull request?

Mark `BitAggregate` as order irrelevant in `EliminateSorts`.

### Why are the changes needed?

Performance improvements in some queries

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Generalized an existing UT

Closes #29740 from tanelk/SPARK-32868.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-14 22:52:33 +09:00
Yuanjian Li 5e825482d7 [SPARK-32844][SQL] Make DataFrameReader.table take the specified options for datasource v1
### What changes were proposed in this pull request?
Make `DataFrameReader.table` take the specified options for datasource v1.

### Why are the changes needed?
Keep the same behavior of v1/v2 datasource, the v2 fix has been done in SPARK-32592.

### Does this PR introduce _any_ user-facing change?
Yes. The DataFrameReader.table will take the specified options. Also, if there are the same key and value exists in specified options and table properties, an exception will be thrown.

### How was this patch tested?
New UT added.

Closes #29712 from xuanyuanking/SPARK-32844.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-14 09:20:24 +00:00
Cheng Su 978f531010 [SPARK-32854][SS] Minor code and doc improvement for stream-stream join
### What changes were proposed in this pull request?

Several minor code and documentation improvement for stream-stream join. Specifically:

* Remove extending from `SparkPlan`, as extending from `BinaryExecNode` is enough.
* Return `left/right.outputPartitioning` for `Left/RightOuter` in `outputPartitioning`, as the `PartitioningCollection` wrapper is unnecessary (similar to batch joins `ShuffledHashJoinExec`, `SortMergeJoinExec`).
*  Avoid per-row check for join type (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L486-L492), by creating the method before the loop of reading rows (`generateFilteredJoinedRow` in `storeAndJoinWithOtherSide`). Similar optimization (i.e. create auxiliary method/variable per different join type before the iterator of input rows) has been done in batch join world (`SortMergeJoinExec`, `ShuffledHashJoinExec`).
* Minor fix for comment/indentation for better readability.

### Why are the changes needed?

Minor optimization to avoid per-row unnecessary work (this probably can be optimized away by compiler, but we can do a better join to avoid it at the first place). And other comment/indentation fix to have better code readability for future developers.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests in `StreamingJoinSuite.scala` as no new logic is introduced.

Closes #29724 from c21/streaming.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-14 08:49:51 +00:00
Kousuke Saruta b121f0d459 [SPARK-32873][BUILD] Fix code which causes error when build with sbt and Scala 2.13
### What changes were proposed in this pull request?

This PR fix code which causes error when build with sbt and Scala 2.13 like as follows.
```
[error] [warn] /home/kou/work/oss/spark-scala-2.13/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala:251: method with a single empty parameter list overrides method without any parameter list
[error] [warn]   override def hasNext(): Boolean = requestOffset < part.untilOffset
[error] [warn]
[error] [warn] /home/kou/work/oss/spark-scala-2.13/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala:294: method with a single empty parameter list overrides method without any parameter list
[error] [warn]   override def hasNext(): Boolean = okNext
```

More specifically, what this PR fixes are

* Methods which has an empty parameter list and overrides an method which has no parameter list.
```
override def hasNext(): Boolean = okNext
```

* Methods which has no parameter list and overrides an method which has an empty parameter list.
```
      override def next: (Int, Double) = {
```

* Infix operator expression that the operator wraps.
```
    3L * math.min(k, numFeatures) * math.min(k, numFeatures)
    3L * math.min(k, numFeatures) * math.min(k, numFeatures) +
    + math.max(math.max(k, numFeatures), 4L * math.min(k, numFeatures)
      math.max(math.max(k, numFeatures), 4L * math.min(k, numFeatures) *
    * math.min(k, numFeatures) + 4L * math.min(k, numFeatures))
```

### Why are the changes needed?

For building Spark with sbt and Scala 2.13.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

After this change and #29742 applied, compile passed with the following command.
```
build/sbt -Pscala-2.13  -Phive -Phive-thriftserver -Pyarn -Pkubernetes compile test:compile
```

Closes #29745 from sarutak/fix-code-for-sbt-and-spark-2.13.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-14 15:34:58 +09:00
HyukjinKwon 742fcff350 [SPARK-32839][WINDOWS] Make Spark scripts working with the spaces in paths on Windows
### What changes were proposed in this pull request?

If you install Spark under the path that has whitespaces, it does not work on Windows, for example as below:

```
>>> SparkSession.builder.getOrCreate()
Presence of build for multiple Scala versions detected (C:\...\assembly\target\scala-2.13 and C:\...\assembly\target\scala-2.12).
Remove one of them or, set SPARK_SCALA_VERSION=2.13 in spark-env.cmd.
Visit https://spark.apache.org/docs/latest/configuration.html#environment-variables for more details about setting environment variables in spark-env.cmd.
Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd.
```

This PR fixes the whitespace handling to support any paths on Windows.

### Why are the changes needed?

To support Spark working with whitespaces in paths on Windows.

### Does this PR introduce _any_ user-facing change?

Yes, users will be able to install and run Spark under the paths with whitespaces.

### How was this patch tested?

Manually tested.

Closes #29706 from HyukjinKwon/window-space-path.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-14 13:15:14 +09:00
gengjiaan e558b8a0fd [SPARK-31847][CORE][TESTS] DAGSchedulerSuite: Rewrite the test framework to support apply specified spark configurations
### What changes were proposed in this pull request?
`DAGSchedulerSuite` exists some issue:
`afterEach` and `init` are called when the `SparkConf` of the default `SparkContext` has no configuration that the test case must set. This causes the `SparkContext` initialized in `beforeEach` to be discarded without being used, resulting in waste. On the other hand, the flexibility to add configurations to `SparkConf` should be addressed by the test framework.

Test suites inherits `LocalSparkContext` can be simplified.

### Why are the changes needed?
Reduce overhead about init `SparkContext`.
Rewrite the test framework to support apply specified spark configurations.

### Does this PR introduce _any_ user-facing change?
'No'.

### How was this patch tested?
Jenkins test.

Closes #29228 from beliefer/extend-test-frame-for-dag.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-14 11:57:29 +09:00
Kousuke Saruta fbb0f37685 [SPARK-32869][BUILD] Ignore deprecation warnings for build with Scala 2.13 and sbt
### What changes were proposed in this pull request?

This PR changes SparkBuild.scala to ignore deprecation warnings for build with Scala 2.13 and sbt.
Actually, deprecation warnings are already ignored for Scala 2.12 but string matching logic for deprecation warnings should be changed for Scala 2.13.
Currently, if a warning message contains `is deprecated`, it's ignored but some warnings contain "are deprecated` and `will be deprecated`.

```
[error] [warn] /home/kou/work/oss/spark-scala-2.13/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala:656: multiarg infix syntax looks\
 like a tuple and will be deprecated
[error] [warn]         if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) }
```
```
[error] [warn] /home/kou/work/oss/spark-scala-2.13/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala:35: view bounds are de\
precated; use an implicit parameter instead.
[error]   example: instead of `def f[A <% Int](a: A)` use `def f[A](a: A)(implicit ev: A => Int)`
[error] [warn] class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag](
```

### Why are the changes needed?

Enable to build Spark with Scala 2.13 and sbt.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Build with the following command and confirmed deprecation warnings are not treated as fatal ( Build itself doesn't pass due to another problem).
`build/sbt -Pscala-2.13  package`

Closes #29741 from sarutak/scala-2.13-deprecated-warning.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-14 10:00:15 +09:00
Chao Sun a6d6ea3efe [SPARK-32802][SQL] Avoid using SpecificInternalRow in RunLengthEncoding#Encoder
### What changes were proposed in this pull request?

Currently `RunLengthEncoding#Encoder` uses `SpecificInternalRow` as a holder for the current value when calculating compression stats and doing the actual compression. It calls `ColumnType.copyField` and `ColumnType.getField` on the internal row which incurs extra cost comparing to directly operating on the internal type. This proposes to replace the `SpecificInternalRow` with `T#InternalType` to avoid the extra cost.

### Why are the changes needed?

Operating on `SpecificInternalRow` carries certain cost and negatively impact performance when using `RunLengthEncoding` for compression.

With the change I see some improvements through `CompressionSchemeBenchmark`:

```diff
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 BOOLEAN Encode:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    1              1           0      51957.0           0.0       1.0X
-RunLengthEncoding(2.502)                            549            555           9        122.2           8.2       0.0X
-BooleanBitSet(0.125)                                296            301           3        226.6           4.4       0.0X
+PassThrough(1.000)                                    2              2           0      42985.4           0.0       1.0X
+RunLengthEncoding(2.517)                            487            500          10        137.7           7.3       0.0X
+BooleanBitSet(0.125)                                348            353           4        192.8           5.2       0.0X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 SHORT Encode (Lower Skew):                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    3              3           0      22779.9           0.0       1.0X
-RunLengthEncoding(1.520)                           1186           1192           9         56.6          17.7       0.0X
+PassThrough(1.000)                                    3              4           0      21216.6           0.0       1.0X
+RunLengthEncoding(1.493)                            882            931          50         76.1          13.1       0.0X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 SHORT Encode (Higher Skew):               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    3              4           0      21352.2           0.0       1.0X
-RunLengthEncoding(2.009)                           1173           1175           3         57.2          17.5       0.0X
+PassThrough(1.000)                                    3              3           0      22388.6           0.0       1.0X
+RunLengthEncoding(2.015)                            924            941          23         72.6          13.8       0.0X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 INT Encode (Lower Skew):                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    9             10           1       7410.1           0.1       1.0X
-RunLengthEncoding(1.000)                           1499           1502           4         44.8          22.3       0.0X
-DictionaryEncoding(0.500)                           621            630          11        108.0           9.3       0.0X
-IntDelta(0.250)                                     134            149          10        502.0           2.0       0.1X
+PassThrough(1.000)                                    9             10           1       7575.9           0.1       1.0X
+RunLengthEncoding(1.002)                            952            966          12         70.5          14.2       0.0X
+DictionaryEncoding(0.500)                           561            567           6        119.7           8.4       0.0X
+IntDelta(0.250)                                     129            134           3        521.9           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 INT Encode (Higher Skew):                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    9             10           1       7668.3           0.1       1.0X
-RunLengthEncoding(1.332)                           1561           1685         175         43.0          23.3       0.0X
-DictionaryEncoding(0.501)                           616            642          21        108.9           9.2       0.0X
-IntDelta(0.250)                                     126            131           2        533.4           1.9       0.1X
+PassThrough(1.000)                                    9             10           1       7494.1           0.1       1.0X
+RunLengthEncoding(1.336)                            974            987          13         68.9          14.5       0.0X
+DictionaryEncoding(0.501)                           709            719          10         94.6          10.6       0.0X
+IntDelta(0.250)                                     127            132           4        528.4           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 LONG Encode (Lower Skew):                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                   18             19           1       3803.0           0.3       1.0X
-RunLengthEncoding(0.754)                           1526           1540          20         44.0          22.7       0.0X
-DictionaryEncoding(0.250)                           735            759          33         91.3          11.0       0.0X
-LongDelta(0.125)                                    126            129           2        530.8           1.9       0.1X
+PassThrough(1.000)                                   19             21           1       3543.5           0.3       1.0X
+RunLengthEncoding(0.747)                           1049           1058          12         63.9          15.6       0.0X
+DictionaryEncoding(0.250)                           620            634          17        108.2           9.2       0.0X
+LongDelta(0.125)                                    129            132           2        520.1           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 LONG Encode (Higher Skew):                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                   18             20           1       3705.4           0.3       1.0X
-RunLengthEncoding(1.002)                           1665           1669           6         40.3          24.8       0.0X
-DictionaryEncoding(0.251)                           890            901          11         75.4          13.3       0.0X
-LongDelta(0.125)                                    125            130           3        537.2           1.9       0.1X
+PassThrough(1.000)                                   18             20           2       3726.8           0.3       1.0X
+RunLengthEncoding(0.999)                           1076           1077           2         62.4          16.0       0.0X
+DictionaryEncoding(0.251)                           904            919          19         74.3          13.5       0.0X
+LongDelta(0.125)                                    125            131           4        536.5           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 STRING Encode:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                   27             30           2       2497.1           0.4       1.0X
-RunLengthEncoding(0.892)                           3443           3587         204         19.5          51.3       0.0X
-DictionaryEncoding(0.167)                          2286           2290           6         29.4          34.1       0.0X
+PassThrough(1.000)                                   28             31           2       2430.2           0.4       1.0X
+RunLengthEncoding(0.889)                           1798           1800           3         37.3          26.8       0.0X
+DictionaryEncoding(0.167)                          1956           1959           4         34.3          29.1       0.0X
```

In the above diff, new results are with changes in this PR. It can be seen that encoding performance has improved quite a lot especially for string type.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Relies on existing unit tests.

Closes #29654 from sunchao/SPARK-32802.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 22:19:30 -07:00
bowen.li 0549c20c6f [SPARK-32865][DOC] python section in quickstart page doesn't display SPARK_VERSION correctly
### What changes were proposed in this pull request?

In https://github.com/apache/spark/blame/master/docs/quick-start.md#L402,it should be `{{site.SPARK_VERSION}}` rather than `{site.SPARK_VERSION}`

### Why are the changes needed?

SPARK_VERSION isn't displayed correctly, as shown below

![image](https://user-images.githubusercontent.com/1892692/93006726-d03c8680-f514-11ea-85e3-1d7cfb682ef2.png)

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

tested locally, as shown below

![image](https://user-images.githubusercontent.com/1892692/93006712-a6835f80-f514-11ea-8d78-6831c9d65265.png)

Closes #29738 from bowenli86/doc.

Authored-by: bowen.li <bowenli86@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 21:45:55 -07:00
Chao Sun 3d08084022 [SPARK-24994][SQL] Add UnwrapCastInBinaryComparison optimizer to simplify literal types
### What changes were proposed in this pull request?

Currently, in cases like the following:
```sql
SELECT * FROM t WHERE age < 40
```
where `age` is of short type, Spark won't be able to simplify this and can only generate filter `cast(age, int) < 40`. This won't get pushed down to datasources and therefore is not optimized.

This PR proposes a optimizer rule to improve this when the following constraints are satisfied:
 - input expression is binary comparisons when one side is a cast operation and another is a literal.
 - both the cast child expression and literal are of integral type (i.e., byte, short, int or long)

When this is true, it tries to do several optimizations to either simplify the expression or move the cast to the literal side, so
result filter for the above case becomes `age < cast(40 as smallint)`. This is better since the cast can be optimized away later and the filter can be pushed down to data sources.

This PR follows a similar effort in Presto (https://prestosql.io/blog/2019/05/21/optimizing-the-casts-away.html). Here we only handles integral types but plan to extend to other types as follow-ups.

### Why are the changes needed?

As mentioned in the previous section, when cast is not optimized, it cannot be pushed down to data sources which can lead
to unnecessary IO and therefore longer job time and waste of resources. This helps to improve that.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unit tests for both the optimizer rule and filter pushdown on datasource level for both Orc and Parquet.

Closes #29565 from sunchao/SPARK-24994.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 21:34:35 -07:00
Karol Chmist 3be552ccc8 [SPARK-30090][SHELL] Adapt Spark REPL to Scala 2.13
### What changes were proposed in this pull request?

This is an attempt to adapt Spark REPL to Scala 2.13.

It is based on a [scala-2.13 branch](https://github.com/smarter/spark/tree/scala-2.13) made by smarter.

I had to set Scala version to 2.13 in some places, and to adapt some other modules, before I could start working on the REPL itself. These are separate commits on the branch that probably would be fixed beforehand, and thus dropped before the merge of this PR.

I couldn't find a way to run the initialization code with existing REPL classes in Scala 2.13.2, so I [modified REPL in Scala](e9cc0dd547) to make it work. With this modification I managed to run Spark Shell, along with the units tests passing, which is good news.

The bad news is that it requires an upstream change in Scala, which must be accepted first. I'd be happy to change it if someone points a way to do it differently. If not, I'd propose a PR in Scala to introduce `ILoop.internalReplAutorunCode`.

### Why are the changes needed?

REPL in Scala changed quite a lot, so current version of Spark REPL needed to be adapted.

### Does this PR introduce _any_ user-facing change?

In the previous version of `SparkILoop`, a lot of Scala's `ILoop` code was [overridden and duplicated](2bc7b75537) to make the welcome message a bit more pleasant. In this PR, the message is in a bit different order, but it's still acceptable IMHO.

Before this PR:
```
20/05/15 15:32:39 WARN Utils: Your hostname, hermes resolves to a loopback address: 127.0.1.1; using 192.168.1.28 instead (on interface enp0s31f6)
20/05/15 15:32:39 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
20/05/15 15:32:39 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/05/15 15:32:45 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041.
Spark context Web UI available at http://192.168.1.28:4041
Spark context available as 'sc' (master = local[*], app id = local-1589549565502).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.1-SNAPSHOT
      /_/

Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_242)
Type in expressions to have them evaluated.
Type :help for more information.

scala>
```

With this PR:
```
20/05/15 15:32:15 WARN Utils: Your hostname, hermes resolves to a loopback address: 127.0.1.1; using 192.168.1.28 instead (on interface enp0s31f6)
20/05/15 15:32:15 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
20/05/15 15:32:15 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.13.2-20200422-211118-706ef1b (OpenJDK 64-Bit Server VM, Java 1.8.0_242)
Type in expressions to have them evaluated.
Type :help for more information.
Spark context Web UI available at http://192.168.1.28:4040
Spark context available as 'sc' (master = local[*], app id = local-1589549541259).
Spark session available as 'spark'.

scala>
```

It seems that currently the welcoming message is still an improvement from [the original ticket](https://issues.apache.org/jira/browse/SPARK-24785), albeit in a different order. As a bonus, some fragile code duplication was removed.

### How was this patch tested?

Existing tests pass in `repl`module. The REPL runs in a terminal and the following code executed correctly:

```
scala> spark.range(1000 * 1000 * 1000).count()
val res0: Long = 1000000000
```

Closes #28545 from karolchmist/scala-2.13-repl.

Authored-by: Karol Chmist <info+github@chmist.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-12 18:15:15 -05:00
KevinSmile bbbd907780 [SPARK-32804][LAUNCHER] Fix run-example command builder bug
### What changes were proposed in this pull request?

Bug fix in run-example command builder (as described in [SPARK-32804], run-example failed in standalone-cluster mode):
1. Missing primaryResource arg.
2. Wrong appResource arg.

which will affect `SparkSubmit` in Standalone-Cluster mode:
32d87c2b59/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L695-L696)

and get error at:
f55694638d/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala (L74-L89)

### Why are the changes needed?

Bug: run-example failed in standalone-cluster mode

### Does this PR introduce _any_ user-facing change?

Yes. User can run-example in standalone-cluster mode now.

### How was this patch tested?

New ut added.
Also it's a user-facing bug, so better re-check the real case in [SPARK-32804].

Closes #29653 from KevinSmile/bug-fix-master.

Authored-by: KevinSmile <kevinwang013@hotmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-12 16:12:37 -05:00
sandeep.katta 2009f95340 [SPARK-32779][SQL][FOLLOW-UP] Delete Unused code
### What changes were proposed in this pull request?
Follow-up PR as per the review comments in [29649](8d45542e91 (r487140171))

### Why are the changes needed?
Delete the un used code

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing UT

Closes #29736 from sandeep-katta/deadlockfollowup.

Authored-by: sandeep.katta <sandeep.katta2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 13:22:54 -07:00
Sean Owen ce566bed17 [SPARK-32180][FOLLOWUP] Fix .rst error in new Pyspark installation guide
This simply fixes an .rst generation error in https://github.com/apache/spark/pull/29640

Closes #29735 from srowen/SPARK-32180.2.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2020-09-11 20:08:22 -07:00
Takeshi Yamamuro 4269c2c252 [SPARK-32851][SQL][TEST] Tests should fail if errors happen when generating projection code
### What changes were proposed in this pull request?

This PR intends to set `CODEGEN_ONLY` at `CODEGEN_FACTORY_MODE` in test spark context so that tests can fail if errors happen when generating expr code.

### Why are the changes needed?

I noticed that the code generation of `SafeProjection` failed in the existing test (https://issues.apache.org/jira/browse/SPARK-32828) but it passed because `FALLBACK` was set at `CODEGEN_FACTORY_MODE` (by default) in `SharedSparkSession`. To get aware of these failures quickly, I think its worth setting `CODEGEN_ONLY` at `CODEGEN_FACTORY_MODE`.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #29721 from maropu/ExprCodegenTest.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-12 08:42:07 +09:00
Dongjoon Hyun b4be6a6d12 [SPARK-32845][SS][TESTS] Add sinkParameter to check sink options robustly in DataStreamReaderWriterSuite
### What changes were proposed in this pull request?

This PR aims to add `sinkParameter`  to check sink options robustly and independently in DataStreamReaderWriterSuite

### Why are the changes needed?

`LastOptions.parameters` is designed to catch three cases: `sourceSchema`, `createSource`, `createSink`. However, `StreamQuery.stop` invokes `queryExecutionThread.join`, `runStream`, `createSource` immediately and reset the stored options by `createSink`.

To catch `createSink` options, currently, the test suite is trying a workaround pattern. However, we observed a flakiness in this pattern sometimes. If we split `createSink` option separately, we don't need this workaround and can eliminate this flakiness.

```scala
val query = df.writeStream.
   ...
   .start()
assert(LastOptions.paramters(..))
query.stop()
```

### Does this PR introduce _any_ user-facing change?

No. This is a test-only change.

### How was this patch tested?

Pass the newly updated test case.

Closes #29730 from dongjoon-hyun/SPARK-32845.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-11 11:48:34 -07:00
Rohit.Mishra f6322d1cb1 [SPARK-32180][PYTHON][DOCS] Installation page of Getting Started in PySpark documentation
### What changes were proposed in this pull request?
This PR proposes to add getting started- installation to new PySpark docs.

### Why are the changes needed?
Better documentation.

### Does this PR introduce _any_ user-facing change?
No. Documentation only.

### How was this patch tested?
Generating documents locally.

Closes #29640 from rohitmishr1484/SPARK-32180-Getting-Started-Installation.

Authored-by: Rohit.Mishra <rohit.mishra@utopusinsights.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-11 10:38:01 -05:00
Peter Toth 94cac5978c [SPARK-32730][SQL][FOLLOW-UP] Improve LeftAnti SortMergeJoin right side buffering
### What changes were proposed in this pull request?

This is a follow-up to https://github.com/apache/spark/pull/29572.

LeftAnti SortMergeJoin should not buffer all matching right side rows when bound condition is empty, this is unnecessary and can lead to performance degradation especially when spilling happens.

### Why are the changes needed?

Performance improvement.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New UT.

Closes #29727 from peter-toth/SPARK-32730-improve-leftsemi-sortmergejoin-followup.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-11 13:42:33 +00:00
Wenchen Fan 9f4f49cbaa [SPARK-32853][SQL] Consecutive save/load calls in DataFrame/StreamReader/Writer should not fail
### What changes were proposed in this pull request?

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

In https://github.com/apache/spark/pull/29328 , we forbid the use case that path option and path parameter are both specified.  However,  it breaks some use cases:
```
val dfr =  spark.read.format(...).option(...)
dfr.load(path1).xxx
dfr.load(path2).xxx
```

The reason is that: `load` has side effects. It will set path option to the `DataFrameReader` instance. The next time you call `load`, Spark will fail because both path option and path parameter are specified.

This PR removes the side effect of `save`/`load`/`start`  to not set the path option.

### Why are the changes needed?

recover some use cases

### Does this PR introduce _any_ user-facing change?

Yes, some use cases fail before this PR, and can run successfully after this PR.

### How was this patch tested?

new tests

Closes #29723 from cloud-fan/df.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-11 06:15:58 -07:00
yangjiang fe2ab255d1 [MINOR][SQL] Fix a typo at 'spark.sql.sources.fileCompressionFactor' error message in SQLConf
### What changes were proposed in this pull request?

 fix typo in SQLConf

### Why are the changes needed?

typo fix to increase readability

### Does this PR introduce _any_ user-facing change?

### How was this patch tested?

no test

Closes #29668 from Ted-Jiang/fix_annotate.

Authored-by: yangjiang <yangjiang@ebay.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-11 08:05:34 -05:00
Wenchen Fan 328d81a2d1 [SPARK-32677][SQL][DOCS][MINOR] Improve code comment in CreateFunctionCommand
### What changes were proposed in this pull request?

We made a mistake in https://github.com/apache/spark/pull/29502, as there is no code comment to explain why we can't load the UDF class when creating functions. This PR improves the code comment.

### Why are the changes needed?

To avoid making the same mistake.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

N/A

Closes #29713 from cloud-fan/comment.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-11 09:22:56 +09:00
Kousuke Saruta 5f468cc21e [SPARK-32822][SQL] Change the number of partitions to zero when a range is empty with WholeStageCodegen disabled or falled back
### What changes were proposed in this pull request?

This PR changes the behavior of RangeExec with WholeStageCodegen disabled or falled back to change the number of partitions to zero when a range is empty.

In the current master, if WholeStageCodegen effects, the number of partitions of an empty range will be changed to zero.
```
spark.range(1, 1, 1, 1000).rdd.getNumPartitions
res0: Int = 0
```
But it doesn't if WholeStageCodegen is disabled or falled back.
```
spark.conf.set("spark.sql.codegen.wholeStage", false)
spark.range(1, 1, 1, 1000).rdd.getNumPartitions
res2: Int = 1000
```

### Why are the changes needed?

To archive better performance even though WholeStageCodegen disabled or falled back.

### Does this PR introduce _any_ user-facing change?

Yes. the number of partitions gotten with `getNumPartitions` for an empty range will be changed when WholeStageCodegen is disabled.

### How was this patch tested?

New test.

Closes #29681 from sarutak/zero-size-range.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-11 09:11:35 +09:00
gengjiaan a22871f50a [SPARK-32777][SQL] Aggregation support aggregate function with multiple foldable expressions
### What changes were proposed in this pull request?
Spark SQL exists a bug show below:

```
spark.sql(
  " SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2, 3)")
  .show()
+-----------------+--------------------+
|count(DISTINCT 2)|count(DISTINCT 2, 3)|
+-----------------+--------------------+
|                1|                   1|
+-----------------+--------------------+

spark.sql(
  " SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3, 2)")
  .show()
+-----------------+--------------------+
|count(DISTINCT 2)|count(DISTINCT 3, 2)|
+-----------------+--------------------+
|                1|                   0|
+-----------------+--------------------+
```
The first query is correct, but the second query is not.
The root reason is the second query rewrited by `RewriteDistinctAggregates` who expand the output but lost the 2.

### Why are the changes needed?
Fix a bug.
`SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3, 2)` should return `1, 1`

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
New UT

Closes #29626 from beliefer/support-multiple-foldable-distinct-expressions.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 11:25:32 +00:00
Kent Yao 5669b212ec [SPARK-32840][SQL] Invalid interval value can happen to be just adhesive with the unit
### What changes were proposed in this pull request?
In this PR, we add a checker for STRING form interval value ahead for parsing multiple units intervals and fail directly if the interval value contains alphabets to prevent correctness issues like `interval '1 day 2' day`=`3 days`.

### Why are the changes needed?

fix correctness issue

### Does this PR introduce _any_ user-facing change?

yes, in spark 3.0.0 `interval '1 day 2' day`=`3 days` but now we fail with ParseException
### How was this patch tested?

add a test.

Closes #29708 from yaooqinn/SPARK-32840.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 11:20:05 +00:00
Takeshi Yamamuro 7eb76d6988 [SPARK-32828][SQL] Cast from a derived user-defined type to a base type
### What changes were proposed in this pull request?

This PR intends to fix an existing bug below in `UserDefinedTypeSuite`;
```
[info] - SPARK-19311: UDFs disregard UDT type hierarchy (931 milliseconds)
16:22:35.936 WARN org.apache.spark.sql.catalyst.expressions.SafeProjection: Expr codegen error and falling back to interpreter mode
org.apache.spark.SparkException: Cannot cast org.apache.spark.sql.ExampleSubTypeUDT46b1771f to org.apache.spark.sql.ExampleBaseTypeUDT31e8d979.
	at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeCastFunction(Cast.scala:891)
	at org.apache.spark.sql.catalyst.expressions.CastBase.doGenCode(Cast.scala:852)
	at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:147)
    ...
```

### Why are the changes needed?

bugfix

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unit tests.

Closes #29691 from maropu/FixUdtBug.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-10 19:19:26 +09:00
Jungtaek Lim (HeartSaVioR) 8f61005723 [SPARK-32456][SS][FOLLOWUP] Update doc to note about using SQL statement with streaming Dataset
### What changes were proposed in this pull request?

This patch proposes to update the doc (both SS guide doc and Dataset dropDuplicates method doc) to leave a note to check on using SQL statements with streaming Dataset.

Once end users create a temp view based on streaming Dataset, they won't bother with thinking about "streaming" and do whatever they do with batch query. In many cases it works, but not just smoothly for the case when streaming aggregation is involved. They still need to concern about maintaining state store.

### Why are the changes needed?

Although SPARK-32456 fixed the weird error message, as a side effect some operations are enabled on streaming workload via SQL statement, which is error-prone if end users don't indicate what they're doing.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Only doc change.

Closes #29461 from HeartSaVioR/SPARK-32456-FOLLOWUP-DOC.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 08:10:32 +00:00
HyukjinKwon dbc41376de [SPARK-32841][BUILD] Use Apache Hadoop 3.2.0 for PyPI and CRAN
### What changes were proposed in this pull request?

PyPI and CRAN did not change because of the concern about selecting Hadoop and Hive versions.

For PyPI, now there is a PR open at https://github.com/apache/spark/pull/29703
For CRAN, we can already select Hadoop and Hive versions via `SparkR::install.spark`.

### Why are the changes needed?

To keep the default profiles consistent in distributions

### Does this PR introduce _any_ user-facing change?

Yes, the default distributions will use Hadoop 3.2.

### How was this patch tested?

Jenkins tests.

Closes #29704 from HyukjinKwon/SPARK-32058.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-10 00:11:07 -07:00
Dongjoon Hyun 2f85f9516c [SPARK-32832][SS] Use CaseInsensitiveMap for DataStreamReader/Writer options
### What changes were proposed in this pull request?

This PR aims to fix indeterministic behavior on DataStreamReader/Writer options like the following.
```scala
scala> spark.readStream.format("parquet").option("paTh", "1").option("PATH", "2").option("Path", "3").option("patH", "4").option("path", "5").load()
org.apache.spark.sql.AnalysisException: Path does not exist: 1;
```

### Why are the changes needed?

This will make the behavior deterministic.

### Does this PR introduce _any_ user-facing change?

Yes, but the previous behavior is indeterministic.

### How was this patch tested?

Pass the newly test cases.

Closes #29702 from dongjoon-hyun/SPARK-32832.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-09 23:41:32 -07:00
Jungtaek Lim (HeartSaVioR) db89b0e1b8 [SPARK-32831][SS] Refactor SupportsStreamingUpdate to represent actual meaning of the behavior
### What changes were proposed in this pull request?

This PR renames `SupportsStreamingUpdate` to `SupportsStreamingUpdateAsAppend` as the new interface name represents the actual behavior clearer. This PR also removes the `update()` method (so the interface is more likely a marker), as the implementations of `SupportsStreamingUpdateAsAppend` should support append mode by default, hence no need to trigger some flag on it.

### Why are the changes needed?

SupportsStreamingUpdate was intended to revive the functionality of Streaming update output mode for internal data sources, but despite the name, that interface isn't really used to do actual update on sink; all sinks are implementing this interface to do append, so strictly saying, it's just to support update as append. Renaming the interface would make it clear.

### Does this PR introduce _any_ user-facing change?

No, as the class is only for internal data sources.

### How was this patch tested?

Jenkins test will follow.

Closes #29693 from HeartSaVioR/SPARK-32831.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-09-10 15:33:18 +09:00
HyukjinKwon 4a096131ee Revert "[SPARK-32772][SQL][FOLLOWUP] Remove legacy silent support mode for spark-sql CLI"
This reverts commit f1f7ae420e.
2020-09-10 14:23:10 +09:00
Bryan Cutler e0538bd38c [SPARK-32312][SQL][PYTHON][TEST-JAVA11] Upgrade Apache Arrow to version 1.0.1
### What changes were proposed in this pull request?

Upgrade Apache Arrow to version 1.0.1 for the Java dependency and increase minimum version of PyArrow to 1.0.0.

This release marks a transition to binary stability of the columnar format (which was already informally backward-compatible going back to December 2017) and a transition to Semantic Versioning for the Arrow software libraries. Also note that the Java arrow-memory artifact has been split to separate dependence on netty-buffer and allow users to select an allocator. Spark will continue to use `arrow-memory-netty` to maintain performance benefits.

Version 1.0.0 - 1.0.0 include the following selected fixes/improvements relevant to Spark users:

ARROW-9300 - [Java] Separate Netty Memory to its own module
ARROW-9272 - [C++][Python] Reduce complexity in python to arrow conversion
ARROW-9016 - [Java] Remove direct references to Netty/Unsafe Allocators
ARROW-8664 - [Java] Add skip null check to all Vector types
ARROW-8485 - [Integration][Java] Implement extension types integration
ARROW-8434 - [C++] Ipc RecordBatchFileReader deserializes the Schema multiple times
ARROW-8314 - [Python] Provide a method to select a subset of columns of a Table
ARROW-8230 - [Java] Move Netty memory manager into a separate module
ARROW-8229 - [Java] Move ArrowBuf into the Arrow package
ARROW-7955 - [Java] Support large buffer for file/stream IPC
ARROW-7831 - [Java] unnecessary buffer allocation when calling splitAndTransferTo on variable width vectors
ARROW-6111 - [Java] Support LargeVarChar and LargeBinary types and add integration test with C++
ARROW-6110 - [Java] Support LargeList Type and add integration test with C++
ARROW-5760 - [C++] Optimize Take implementation
ARROW-300 - [Format] Add body buffer compression option to IPC message protocol using LZ4 or ZSTD
ARROW-9098 - RecordBatch::ToStructArray cannot handle record batches with 0 column
ARROW-9066 - [Python] Raise correct error in isnull()
ARROW-9223 - [Python] Fix to_pandas() export for timestamps within structs
ARROW-9195 - [Java] Wrong usage of Unsafe.get from bytearray in ByteFunctionsHelper class
ARROW-7610 - [Java] Finish support for 64 bit int allocations
ARROW-8115 - [Python] Conversion when mixing NaT and datetime objects not working
ARROW-8392 - [Java] Fix overflow related corner cases for vector value comparison
ARROW-8537 - [C++] Performance regression from ARROW-8523
ARROW-8803 - [Java] Row count should be set before loading buffers in VectorLoader
ARROW-8911 - [C++] Slicing a ChunkedArray with zero chunks segfaults

View release notes here:
https://arrow.apache.org/release/1.0.1.html
https://arrow.apache.org/release/1.0.0.html

### Why are the changes needed?

Upgrade brings fixes, improvements and stability guarantees.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Existing tests with pyarrow 1.0.0 and 1.0.1

Closes #29686 from BryanCutler/arrow-upgrade-100-SPARK-32312.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-10 14:16:19 +09:00
Kent Yao 9ab8a2c36d [SPARK-32826][SQL] Set the right column size for the null type in SparkGetColumnsOperation
### What changes were proposed in this pull request?

In Spark 3.0.0, the SparkGetColumnsOperation can not recognize NULL columns but now we can because the side effect of https://issues.apache.org/jira/browse/SPARK-32696 / f14f3742e0, but the test coverage for this change was not added.

In Spark, the column size for null fields should be 1, in this PR, we set the right column size for the null type.

### Why are the changes needed?

test coverage and fix the client-side information about the null type through jdbc

### Does this PR introduce _any_ user-facing change?

NO
### How was this patch tested?

added ut both for this pr and SPARK-32696

Closes #29687 from yaooqinn/SPARK-32826.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 04:53:38 +00:00
Liang-Chi Hsieh add267c4de [SPARK-32819][SQL] ignoreNullability parameter should be effective recursively
### What changes were proposed in this pull request?

This patch proposes to check `ignoreNullability` parameter recursively in `equalsStructurally` method.

### Why are the changes needed?

`equalsStructurally` is used to check type equality. We can optionally ask to ignore nullability check. But the parameter `ignoreNullability` is not passed recursively down to nested types. So it produces weird error like:

```
data type mismatch: argument 3 requires array<array<string>> type, however ... is of array<array<string>> type.
```

when running the query `select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array( x ) ) )`.

### Does this PR introduce _any_ user-facing change?

Yes, fixed a bug when running user query.

### How was this patch tested?

Unit tests.

Closes #29698 from viirya/SPARK-32819.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 02:53:22 +00:00
Dongjoon Hyun 06a994517f [SPARK-32836][SS][TESTS] Fix DataStreamReaderWriterSuite to check writer options correctly
### What changes were proposed in this pull request?

This PR aims to fix the test coverage at `DataStreamReaderWriterSuite`.

### Why are the changes needed?

Currently, the test case checks `DataStreamReader` options instead of `DataStreamWriter` options.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the revised test case.

Closes #29701 from dongjoon-hyun/SPARK-32836.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-09 19:46:55 -07:00
Terry Kim ab2fa881ed [SPARK-32516][SQL][FOLLOWUP] Remove unnecessary check if path string is empty for DataFrameWriter.save(), DataStreamReader.load() and DataStreamWriter.start()
### What changes were proposed in this pull request?

This PR is a follow up to https://github.com/apache/spark/pull/29543#discussion_r485409606, which correctly points out that the check for the empty string is not necessary.

### Why are the changes needed?

The unnecessary check actually could cause more confusion.

For example,
```scala
scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("")
java.lang.IllegalArgumentException: Can not create a Path from an empty string
  at org.apache.hadoop.fs.Path.checkPathArg(Path.java:168)
```
even when `path` option is available. This PR addresses to fix this confusion.

### Does this PR introduce _any_ user-facing change?

Yes, now the above example prints the consistent exception message whether the path parameter value is empty or not.
```scala
scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("")
org.apache.spark.sql.AnalysisException: There is a 'path' option set and save() is called with a path parameter. Either remove the path option, or call save() without the parameter. To ignore this check, set 'spark.sql.legacy.pathOptionBehavior.enabled' to 'true'.;
  at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:290)
  at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:856)
  ... 47 elided
```

### How was this patch tested?

Added unit tests.

Closes #29697 from imback82/SPARK-32516-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 01:48:51 +00:00
Wenchen Fan f7995c576a Revert "[SPARK-32677][SQL] Load function resource before create"
This reverts commit 05fcf26b79.
2020-09-09 18:15:22 +00:00