Commit graph

27747 commits

Author SHA1 Message Date
Cheng Su ae82768c13 [SPARK-32421][SQL] Add code-gen for shuffled hash join
### What changes were proposed in this pull request?

Adding codegen for shuffled hash join. Shuffled hash join codegen is very similar to broadcast hash join codegen. So most of code change is to refactor existing codegen in `BroadcastHashJoinExec` to `HashJoin`.

Example codegen for query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153):

```
  def shuffleHashJoin(): Unit = {
    val N: Long = 4 << 20
    withSQLConf(
      SQLConf.SHUFFLE_PARTITIONS.key -> "2",
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000",
      SQLConf.PREFER_SORTMERGEJOIN.key -> "false") {
      codegenBenchmark("shuffle hash join", N) {
        val df1 = spark.range(N).selectExpr(s"id as k1")
        val df2 = spark.range(N / 3).selectExpr(s"id * 3 as k2")
        val df = df1.join(df2, col("k1") === col("k2"))
        assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined)
        df.noop()
      }
    }
  }
```

Shuffled hash join codegen:

```
== Subtree 3 / 3 (maxMethodCodeSize:113; maxConstantPoolSize:126(0.19% used); numInnerClasses:0) ==
*(3) ShuffledHashJoin [k1#2L], [k2#6L], Inner, BuildRight
:- *(1) Project [id#0L AS k1#2L]
:  +- *(1) Range (0, 4194304, step=1, splits=1)
+- *(2) Project [(id#4L * 3) AS k2#6L]
   +- *(2) Range (0, 1398101, step=1, splits=1)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage3(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=3
/* 006 */ final class GeneratedIteratorForCodegenStage3 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator inputadapter_input_0;
/* 010 */   private org.apache.spark.sql.execution.joins.HashedRelation shj_relation_0;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] shj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 012 */
/* 013 */   public GeneratedIteratorForCodegenStage3(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input_0 = inputs[0];
/* 021 */     shj_relation_0 = ((org.apache.spark.sql.execution.joins.ShuffledHashJoinExec) references[0] /* plan */).buildHashedRelation(inputs[1]);
/* 022 */     shj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 023 */
/* 024 */   }
/* 025 */
/* 026 */   private void shj_doConsume_0(InternalRow inputadapter_row_0, long shj_expr_0_0) throws java.io.IOException {
/* 027 */     // generate join key for stream side
/* 028 */
/* 029 */     // find matches from HashRelation
/* 030 */     scala.collection.Iterator shj_matches_0 = false ?
/* 031 */     null : (scala.collection.Iterator)shj_relation_0.get(shj_expr_0_0);
/* 032 */     if (shj_matches_0 != null) {
/* 033 */       while (shj_matches_0.hasNext()) {
/* 034 */         UnsafeRow shj_matched_0 = (UnsafeRow) shj_matches_0.next();
/* 035 */         {
/* 036 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[1] /* numOutputRows */).add(1);
/* 037 */
/* 038 */           long shj_value_1 = shj_matched_0.getLong(0);
/* 039 */           shj_mutableStateArray_0[0].reset();
/* 040 */
/* 041 */           shj_mutableStateArray_0[0].write(0, shj_expr_0_0);
/* 042 */
/* 043 */           shj_mutableStateArray_0[0].write(1, shj_value_1);
/* 044 */           append((shj_mutableStateArray_0[0].getRow()).copy());
/* 045 */
/* 046 */         }
/* 047 */       }
/* 048 */     }
/* 049 */
/* 050 */   }
/* 051 */
/* 052 */   protected void processNext() throws java.io.IOException {
/* 053 */     while ( inputadapter_input_0.hasNext()) {
/* 054 */       InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next();
/* 055 */
/* 056 */       long inputadapter_value_0 = inputadapter_row_0.getLong(0);
/* 057 */
/* 058 */       shj_doConsume_0(inputadapter_row_0, inputadapter_value_0);
/* 059 */       if (shouldStop()) return;
/* 060 */     }
/* 061 */   }
/* 062 */
/* 063 */ }
```

Broadcast hash join codegen for the same query (for reference here):

```
== Subtree 2 / 2 (maxMethodCodeSize:280; maxConstantPoolSize:218(0.33% used); numInnerClasses:0) ==
*(2) BroadcastHashJoin [k1#2L], [k2#6L], Inner, BuildRight, false
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4194304, step=1, splits=1)
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#22]
   +- *(1) Project [(id#4L * 3) AS k2#6L]
      +- *(1) Range (0, 1398101, step=1, splits=1)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private org.apache.spark.sql.execution.joins.LongHashedRelation bhj_relation_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */
/* 032 */     bhj_relation_0 = ((org.apache.spark.sql.execution.joins.LongHashedRelation) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcast */).value()).asReadOnlyCopy();
/* 033 */     incPeakExecutionMemory(bhj_relation_0.estimatedSize());
/* 034 */
/* 035 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 036 */
/* 037 */   }
/* 038 */
/* 039 */   private void initRange(int idx) {
/* 040 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 041 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L);
/* 042 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4194304L);
/* 043 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 044 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 045 */     long partitionEnd;
/* 046 */
/* 047 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 048 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 049 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 050 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 051 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 052 */     } else {
/* 053 */       range_nextIndex_0 = st.longValue();
/* 054 */     }
/* 055 */     range_batchEnd_0 = range_nextIndex_0;
/* 056 */
/* 057 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 058 */     .multiply(step).add(start);
/* 059 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 060 */       partitionEnd = Long.MAX_VALUE;
/* 061 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 062 */       partitionEnd = Long.MIN_VALUE;
/* 063 */     } else {
/* 064 */       partitionEnd = end.longValue();
/* 065 */     }
/* 066 */
/* 067 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 068 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 069 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 070 */     if (range_numElementsTodo_0 < 0) {
/* 071 */       range_numElementsTodo_0 = 0;
/* 072 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 073 */       range_numElementsTodo_0++;
/* 074 */     }
/* 075 */   }
/* 076 */
/* 077 */   private void bhj_doConsume_0(long bhj_expr_0_0) throws java.io.IOException {
/* 078 */     // generate join key for stream side
/* 079 */
/* 080 */     // find matches from HashedRelation
/* 081 */     UnsafeRow bhj_matched_0 = false ? null: (UnsafeRow)bhj_relation_0.getValue(bhj_expr_0_0);
/* 082 */     if (bhj_matched_0 != null) {
/* 083 */       {
/* 084 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 085 */
/* 086 */         long bhj_value_2 = bhj_matched_0.getLong(0);
/* 087 */         range_mutableStateArray_0[3].reset();
/* 088 */
/* 089 */         range_mutableStateArray_0[3].write(0, bhj_expr_0_0);
/* 090 */
/* 091 */         range_mutableStateArray_0[3].write(1, bhj_value_2);
/* 092 */         append((range_mutableStateArray_0[3].getRow()));
/* 093 */
/* 094 */       }
/* 095 */     }
/* 096 */
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     // initialize Range
/* 101 */     if (!range_initRange_0) {
/* 102 */       range_initRange_0 = true;
/* 103 */       initRange(partitionIndex);
/* 104 */     }
/* 105 */
/* 106 */     while (true) {
/* 107 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 108 */         long range_nextBatchTodo_0;
/* 109 */         if (range_numElementsTodo_0 > 1000L) {
/* 110 */           range_nextBatchTodo_0 = 1000L;
/* 111 */           range_numElementsTodo_0 -= 1000L;
/* 112 */         } else {
/* 113 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 114 */           range_numElementsTodo_0 = 0;
/* 115 */           if (range_nextBatchTodo_0 == 0) break;
/* 116 */         }
/* 117 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 118 */       }
/* 119 */
/* 120 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 121 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 122 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 123 */
/* 124 */         bhj_doConsume_0(range_value_0);
/* 125 */
/* 126 */         if (shouldStop()) {
/* 127 */           range_nextIndex_0 = range_value_0 + 1L;
/* 128 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 129 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 130 */           return;
/* 131 */         }
/* 132 */
/* 133 */       }
/* 134 */       range_nextIndex_0 = range_batchEnd_0;
/* 135 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 136 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 137 */       range_taskContext_0.killTaskIfInterrupted();
/* 138 */     }
/* 139 */   }
/* 140 */
/* 141 */ }
```

### Why are the changes needed?

Codegen shuffled hash join can help save CPU cost. We added shuffled hash join codegen internally in our fork, and seeing obvious improvement in benchmark compared to current non-codegen code path.

Test example query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153), seeing 30% wall clock time improvement compared to existing non-codegen code path:

Enable shuffled hash join code-gen:

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join wholestage off
  Stopped after 2 iterations, 1358 ms
  Running case: shuffle hash join wholestage on
  Stopped after 5 iterations, 2323 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off                    649            679          43          6.5         154.7       1.0X
shuffle hash join wholestage on                     436            465          45          9.6         103.9       1.5X
```

Disable shuffled hash join codegen:

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join wholestage off
  Stopped after 2 iterations, 1345 ms
  Running case: shuffle hash join wholestage on
  Stopped after 5 iterations, 2967 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off                    646            673          37          6.5         154.1       1.0X
shuffle hash join wholestage on                     549            594          47          7.6         130.9       1.2X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite`.

Closes #29277 from c21/codegen.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 05:51:57 +00:00
Gabor Somogyi f6027827a4 [SPARK-32482][SS][TESTS] Eliminate deprecated poll(long) API calls to avoid infinite wait in tests
### What changes were proposed in this pull request?
Structured Streaming Kafka connector tests are now using a deprecated `poll(long)` API which could cause infinite wait. In this PR I've eliminated these calls and replaced them with `AdminClient`.

### Why are the changes needed?
Deprecated `poll(long)` API calls.

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

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

Closes #29289 from gaborgsomogyi/SPARK-32482.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-07-31 13:40:33 +09:00
Kousuke Saruta 9d7b1d935f [SPARK-32175][SPARK-32175][FOLLOWUP] Remove flaky test added in
### What changes were proposed in this pull request?

This PR removes a test added in SPARK-32175(#29002).

### Why are the changes needed?

That test is flaky. It can be mitigated by increasing the timeout but it would rather be simpler to remove the test.
See also the [discussion](https://github.com/apache/spark/pull/29002#issuecomment-666746857).

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

No.

Closes #29314 from sarutak/remove-flaky-test.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-07-31 10:37:05 +09:00
Devesh Agrawal 6032c5b032 [SPARK-32417] Fix flakyness of BlockManagerDecommissionIntegrationSuite
### What changes were proposed in this pull request?

This test tries to fix the flakyness of BlockManagerDecommissionIntegrationSuite.

### Description of the problem

Make the block manager decommissioning test be less flaky

An interesting failure happens when migrateDuring = true (and persist or shuffle is true):
- We schedule the job with tasks on executors 0, 1, 2.
- We wait 300 ms and decommission executor 0.
- If the task is not yet done on executor 0, it will now fail because
   the block manager won't be able to save the block. This condition is
   easy to trigger on a loaded machine where the github checks run.
- The task with retry on a different executor (1 or 2) and its shuffle
   blocks will land there.
- No actual block migration happens here because the decommissioned
   executor technically failed before it could even produce a block.

To remove the above race, this change replaces the fixed wait for 300 ms to wait for an actual task to succeed. When a task has succeeded, we know its blocks would have been written for sure and thus its executor would certainly be forced to migrate those blocks when it is decommissioned.

The change always decommissions an executor on which a real task finished successfully instead of picking the first executor. Because the system may choose to schedule nothing on the first executor and instead run the two tasks on one executor.

### Why are the changes needed?

I have had bad luck with BlockManagerDecommissionIntegrationSuite and it has failed several times on my PRs. So fixing it.

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

No, unit test only change.

### How was this patch tested?

Github checks. Ran this test 100 times, 10 at a time in parallel in a script.

Closes #29226 from agrawaldevesh/block-manager-decom-flaky.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-30 12:00:19 -07:00
Devesh Agrawal 366a178933 [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning
### What changes were proposed in this pull request?

This PR reduces the prospect of a job loss during decommissioning. It
fixes two holes in the current decommissioning framework:

- (a) Loss of decommissioned executors is not treated as a job failure:
We know that the decommissioned executor would be dying soon, so its death is
clearly not caused by the application.

- (b) Shuffle files on the decommissioned host are cleared when the
first fetch failure is detected from a decommissioned host: This is a
bit tricky in terms of when to clear the shuffle state ? Ideally you
want to clear it the millisecond before the shuffle service on the node
dies (or the executor dies when there is no external shuffle service) --
too soon and it could lead to some wastage and too late would lead to
fetch failures.

  The approach here is to do this clearing when the very first fetch
failure is observed on the decommissioned block manager, without waiting for
other blocks to also signal a failure.

### Why are the changes needed?

Without them decommissioning a lot of executors at a time leads to job failures.

### Code overview

The task scheduler tracks the executors that were decommissioned along with their
`ExecutorDecommissionInfo`. This information is used by: (a) For handling a `ExecutorProcessLost` error, or (b) by the `DAGScheduler` when handling a fetch failure.

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

No

### How was this patch tested?

Added a new unit test `DecommissionWorkerSuite` to test the new behavior by exercising the Master-Worker decommissioning. I chose to add a new test since the setup logic was quite different from the existing `WorkerDecommissionSuite`. I am open to changing the name of the newly added test suite :-)

### Questions for reviewers
- Should I add a feature flag to guard these two behaviors ? They seem safe to me that they should only get triggered by decommissioning, but you never know :-).

Closes #29014 from agrawaldevesh/decom_harden.

Authored-by: Devesh Agrawal <devesh.agrawal@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-07-30 11:58:11 -07:00
Dongjoon Hyun 7cf3b54a2a [SPARK-32489][CORE] Pass core module UTs in Scala 2.13
### What changes were proposed in this pull request?

So far, we fixed many stuffs in `core` module. This PR fixes the remaining UT failures in Scala 2.13.

- `OneApplicationResource.environmentInfo` will return a deterministic result for `sparkProperties`, `hadoopProperties`, `systemProperties`, and `classpathEntries`.
- `SubmitRestProtocolSuite` has Scala 2.13 answer in addition to the existing Scala 2.12 answer, and uses the expected answer based on the Scala runtime version.

### Why are the changes needed?

To support Scala 2.13.

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

Yes, `environmentInfo` is changed, but this fixes the indeterministic behavior.

### How was this patch tested?

- Scala 2.12: Pass the Jenkins or GitHub Action
- Scala 2.13: Do the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13
```

**BEFORE**
```
Tests: succeeded 2612, failed 3, canceled 1, ignored 8, pending 0
*** 3 TESTS FAILED ***
```

**AFTER**
```
Tests: succeeded 2615, failed 0, canceled 1, ignored 8, pending 0
All tests passed.
```

Closes #29298 from dongjoon-hyun/SPARK-32489.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-30 10:59:26 -07:00
HyukjinKwon 32f4ef005f [SPARK-32497][INFRA] Installs qpdf package for CRAN check in GitHub Actions
### What changes were proposed in this pull request?

CRAN check fails due to the size of the generated PDF docs as below:

```
...
 WARNING
‘qpdf’ is needed for checks on size reduction of PDFs
...
Status: 1 WARNING, 1 NOTE
See
  ‘/home/runner/work/spark/spark/R/SparkR.Rcheck/00check.log’
for details.
```

This PR proposes to install `qpdf` in GitHub Actions.

Note that I cannot reproduce in my local with the same R version so I am not documenting it for now.

Also, while I am here, I piggyback to install SparkR when the module includes `sparkr`. it is rather a followup of SPARK-32491.

### Why are the changes needed?

To fix SparkR CRAN check failure.

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

No, dev-only.

### How was this patch tested?

GitHub Actions will test it out.

Closes #29306 from HyukjinKwon/SPARK-32497.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 00:57:24 +09:00
Warren Zhu 7437720952 [SPARK-32227] Fix regression bug in load-spark-env.cmd with Spark 3.0.0
### What changes were proposed in this pull request?
Fix regression bug in load-spark-env.cmd with Spark 3.0.0

### Why are the changes needed?
cmd doesn't support set env twice. So set `SPARK_ENV_CMD=%SPARK_CONF_DIR%\%SPARK_ENV_CMD%` doesn't take effect, which caused regression.

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

### How was this patch tested?
Manually tested.
1. Create a spark-env.cmd under conf folder. Inside this, `echo spark-env.cmd`
2. Run old load-spark-env.cmd, nothing printed in the output
2. Run fixed load-spark-env.cmd, `spark-env.cmd` showed in the output.

Closes #29044 from warrenzhu25/32227.

Lead-authored-by: Warren Zhu <zhonzh@microsoft.com>
Co-authored-by: Warren Zhu <warren.zhu25@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 21:44:49 +09:00
HyukjinKwon 12f443cd99 [SPARK-32496][INFRA] Include GitHub Action file as the changes in testing
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/26556 excluded `.github/workflows/master.yml`. So tests are skipped if the GitHub Actions configuration file is changed.

As of SPARK-32245, we now run the regular tests via the testing script. We should include it to test to make sure GitHub Actions build does not break due to some changes such as Python versions.

### Why are the changes needed?

For better test coverage in GitHub Actions build.

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

No, dev-only.

### How was this patch tested?

GitHub Actions in this PR will test.

Closes #29305 from HyukjinKwon/SPARK-32496.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 21:07:31 +09:00
HyukjinKwon e0c8bd07af [SPARK-32493][INFRA] Manually install R instead of using setup-r in GitHub Actions
### What changes were proposed in this pull request?

This PR proposes to manually install R instead of using `setup-r` which seems broken. Currently, GitHub Actions uses its default R 3.4.4 installed, which we dropped as of SPARK-32073.

While I am here, I am also upgrading R version to 4.0. Jenkins will test the old version and GitHub Actions tests the new version. AppVeyor uses R 4.0 but it does not check CRAN which is important when we make a release.

### Why are the changes needed?

To recover GitHub Actions build.

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

No, dev-only

### How was this patch tested?

Manually tested at https://github.com/HyukjinKwon/spark/pull/15

Closes #29302 from HyukjinKwon/SPARK-32493.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 20:06:35 +09:00
HyukjinKwon 1f7fe5415e [SPARK-32491][INFRA] Do not install SparkR in test-only mode in testing script
### What changes were proposed in this pull request?

This PR proposes to skip SparkR installation that is to run R linters (see SPARK-8505) in the test-only mode at `dev/run-tests.py` script.

As of SPARK-32292, the test-only mode in `dev/run-tests.py` was introduced, for example:

```
dev/run-tests.py --modules sql,core
```

which only runs the relevant tests and does not run other tests such as linters. Therefore, we don't need to install SparkR when `--modules` are specified.

### Why are the changes needed?

GitHub Actions build is currently failed as below:

```
ERROR: this R is version 3.4.4, package 'SparkR' requires R >= 3.5
[error] running /home/runner/work/spark/spark/R/install-dev.sh ; received return code 1
##[error]Process completed with exit code 10.
```

For some reasons, looks GitHub Actions started to have R 3.4.4 installed by default; however, R 3.4 was dropped as of SPARK-32073.  When SparkR tests are not needed, GitHub Actions still builds SparkR with a low R version and it causes the test failure.

This PR partially fixes it by avoid the installation of SparkR.

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

No, dev-only.

### How was this patch tested?

GitHub Actions tests should run to confirm this fix is correct.

Closes #29300 from HyukjinKwon/install-r.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 19:28:34 +09:00
Takeshi Yamamuro 30e3042dc5 [SPARK-32488][SQL] Use @parser::members and @lexer::members to avoid generating unused code
### What changes were proposed in this pull request?

This PR aims to update `SqlBse.g4` for avoiding generating unused code.
Currently, ANTLR generates unused methods and variables; `isValidDecimal` and `isHint` are only used in the generated lexer. This PR changed the code to use `parser::members` and `lexer::members` to avoid it.

### Why are the changes needed?

To reduce unnecessary code.

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

No.

### How was this patch tested?

Existing tests.

Closes #29296 from maropu/UpdateSqlBase.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 07:51:27 +00:00
Kent Yao 510a1656e6 [SPARK-32412][SQL] Unify error handling for spark thrift server operations
### What changes were proposed in this pull request?

Log error/warn message only once at the server-side for both sync and async modes

### Why are the changes needed?

In b151194299 we make the error logging for  SparkExecuteStatementOperation with `runInBackground=true` not duplicated, but the operations with runInBackground=false and other metadata operation still will be log twice which happened in the operation's `runInternal` method and ThriftCLIService.

In this PR, I propose to reflect the logic to get a unified error handling approach.

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

Yes, when spark.sql.hive.thriftServer.async=false and people call sync APIs the error message will be logged only once at server-side.
### How was this patch tested?

locally verified the result in target/unit-test.log

add unit tests.

Closes #29204 from yaooqinn/SPARK-32412.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 06:44:13 +00:00
HyukjinKwon e1d7321034 [SPARK-32478][R][SQL] Error message to show the schema mismatch in gapply with Arrow vectorization
### What changes were proposed in this pull request?

This PR proposes to:

1. Fix the error message when the output schema is misbatched with R DataFrame from the given function. For example,

    ```R
    df <- createDataFrame(list(list(a=1L, b="2")))
    count(gapply(df, "a", function(key, group) { group }, structType("a int, b int")))
    ```

    **Before:**

    ```
    Error in handleErrors(returnStatus, conn) :
      ...
      java.lang.UnsupportedOperationException
	    ...
    ```

    **After:**

    ```
    Error in handleErrors(returnStatus, conn) :
     ...
     java.lang.AssertionError: assertion failed: Invalid schema from gapply: expected IntegerType, IntegerType, got IntegerType, StringType
        ...
    ```

2. Update documentation about the schema matching for `gapply` and `dapply`.

### Why are the changes needed?

To show which schema is not matched, and let users know what's going on.

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

Yes, error message is updated as above, and documentation is updated.

### How was this patch tested?

Manually tested and unitttests were added.

Closes #29283 from HyukjinKwon/r-vectorized-error.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 15:16:02 +09:00
Max Gekk 99a855575c [SPARK-32431][SQL] Check duplicate nested columns in read from in-built datasources
### What changes were proposed in this pull request?
When `spark.sql.caseSensitive` is `false` (by default), check that there are not duplicate column names on the same level (top level or nested levels) in reading from in-built datasources Parquet, ORC, Avro and JSON. If such duplicate columns exist, throw the exception:
```
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema:
```

### Why are the changes needed?
To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error when `spark.sql.caseSensitive` is `false`:
```Scala
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`
```

Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758.

### Does this PR introduce _any_ user-facing change?
Yes. For the example from SPARK-32431:

ORC:
```scala
java.io.IOException: Error reading file: file:/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-c02c2f9a-0cdc-4859-94fc-b9c809ca58b1/part-00001-63e8c3f0-7131-4ec9-be02-30b3fdd276f4-c000.snappy.orc
	at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1329)
	at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78)
...
Caused by: java.io.EOFException: Read past end of RLE integer from compressed stream Stream for column 3 kind DATA position: 6 length: 6 range: 0 offset: 12 limit: 12 range 0 = 0 to 6 uncompressed: 3 to 3
	at org.apache.orc.impl.RunLengthIntegerReaderV2.readValues(RunLengthIntegerReaderV2.java:61)
	at org.apache.orc.impl.RunLengthIntegerReaderV2.next(RunLengthIntegerReaderV2.java:323)
```

JSON:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

Parquet:
```scala
+------------+
|StructColumn|
+------------+
|     [0,, 1]|
+------------+
```

Avro:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

After the changes, Parquet, ORC, JSON and Avro output the same error:
```scala
Found duplicate column(s) in the data schema: `camelcase`;
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`;
	at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:112)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:51)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:67)
```

### How was this patch tested?
Run modified test suites:
```
$ build/sbt "sql/test:testOnly org.apache.spark.sql.FileBasedDataSourceSuite"
$ build/sbt "avro/test:testOnly org.apache.spark.sql.avro.*"
```
and added new UT to `SchemaUtilsSuite`.

Closes #29234 from MaxGekk/nested-case-insensitive-column.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 06:05:55 +00:00
zhengruifeng 81b0785fb2 [SPARK-32455][ML] LogisticRegressionModel prediction optimization
### What changes were proposed in this pull request?
for binary `LogisticRegressionModel`:
1, keep variables `_threshold` and `_rawThreshold` instead of computing them on each instance;
2, in `raw2probabilityInPlace`, make use of the characteristic that the sum of probability is 1.0;

### Why are the changes needed?
for better performance

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

### How was this patch tested?
existing testsuite and performace test in REPL

Closes #29255 from zhengruifeng/pred_opt.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Huaxin Gao <huaxing@us.ibm.com>
2020-07-29 19:53:28 -07:00
HyukjinKwon 89d9b7cc64 [SPARK-32010][PYTHON][CORE] Add InheritableThread for local properties and fixing a thread leak issue in pinned thread mode
### What changes were proposed in this pull request?

This PR proposes:

1. To introduce `InheritableThread` class, that works identically with `threading.Thread` but it can inherit the inheritable attributes of a JVM thread such as `InheritableThreadLocal`.

    This was a problem from the pinned thread mode, see also https://github.com/apache/spark/pull/24898. Now it works as below:

    ```python
    import pyspark

    spark.sparkContext.setLocalProperty("a", "hi")
    def print_prop():
        print(spark.sparkContext.getLocalProperty("a"))

    pyspark.InheritableThread(target=print_prop).start()
    ```

    ```
    hi
    ```

2. Also, it adds the resource leak fix into `InheritableThread`. Py4J leaks the thread and does not close the connection from Python to JVM. In `InheritableThread`, it manually closes the connections when PVM garbage collection happens. So, JVM threads finish safely. I manually verified by profiling but there's also another easy way to verify:

    ```bash
    PYSPARK_PIN_THREAD=true ./bin/pyspark
    ```

    ```python
    >>> from threading import Thread
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> spark._jvm._gateway_client.deque
    deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, <py4j.clientserver.ClientServerConnection object at 0x11a015358>, <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>])
    >>> Thread(target=lambda: spark.range(1000).collect()).start()
    >>> spark._jvm._gateway_client.deque
    deque([<py4j.clientserver.ClientServerConnection object at 0x119f7aba8>, <py4j.clientserver.ClientServerConnection object at 0x119fc9b70>, <py4j.clientserver.ClientServerConnection object at 0x119fc9e10>, <py4j.clientserver.ClientServerConnection object at 0x11a015358>, <py4j.clientserver.ClientServerConnection object at 0x119fc08d0>, <py4j.clientserver.ClientServerConnection object at 0x119fc00f0>])
    ```

    This issue is fixed now.

3. Because now we have a fix for the issue here, it also proposes to deprecate `collectWithJobGroup` which was a temporary workaround added to avoid this leak issue.

### Why are the changes needed?

To support pinned thread mode properly without a resource leak, and a proper inheritable local properties.

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

Yes, it adds an API `InheritableThread` class for pinned thread mode.

### How was this patch tested?

Manually tested as described above, and unit test was added as well.

Closes #28968 from HyukjinKwon/SPARK-32010.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 10:15:25 +09:00
Dongjoon Hyun 08a66f8fd0 [SPARK-32248][BUILD] Recover Java 11 build in Github Actions
### What changes were proposed in this pull request?

This PR aims to recover Java 11 build in `GitHub Action`.

### Why are the changes needed?

This test coverage is removed before. Now, it's time to recover it.

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

No.

### How was this patch tested?

Pass the GitHub Action.

Closes #29295 from dongjoon-hyun/SPARK-32248.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-29 18:05:53 -07:00
Dongjoon Hyun 163867435a [SPARK-32487][CORE] Remove j.w.r.NotFoundException from import in [Stages|OneApplication]Resource
### What changes were proposed in this pull request?

This PR aims to remove `java.ws.rs.NotFoundException` from two problematic `import` statements. All the other use cases are correct.

### Why are the changes needed?

In `StagesResource` and `OneApplicationResource`, there exist two `NotFoundException`s.
- javax.ws.rs.NotFoundException
- org.apache.spark.status.api.v1.NotFoundException

To use `org.apache.spark.status.api.v1.NotFoundException` correctly, we should not import `java.ws.rs.NotFoundException`. This causes UT failures in Scala 2.13 environment.

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

No.

### How was this patch tested?

- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Do the following manually.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.history.HistoryServerSuite
```

**BEFORE**
```
*** 4 TESTS FAILED ***
```

**AFTER**
```
*** 1 TEST FAILED ***
```

Closes #29293 from dongjoon-hyun/SPARK-32487.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-29 17:57:46 -07:00
Holden Karau 50911df08e [SPARK-32397][BUILD] Allow specifying of time for build to keep time consistent between modules
### What changes were proposed in this pull request?

Upgrade codehaus maven build helper to allow people to specify a time during the build to avoid snapshot artifacts with different version strings.

### Why are the changes needed?

During builds of snapshots the maven may assign different versions to different artifacts based on the time each individual sub-module starts building.

The timestamp is used as part of the version string when run `maven deploy` on a snapshot build. This results in different sub-modules having different version strings.

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

No

### How was this patch tested?

Manual build while specifying the current time, ensured the time is consistent in the sub components.

Open question: Ideally I'd like to backport this as well since it's sort of a bug fix and while it does change a dependency version it's not one that is propagated. I'd like to hear folks thoughts about this.

Closes #29274 from holdenk/SPARK-32397-snapshot-artifact-timestamp-differences.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-07-29 21:39:14 +00:00
Wenchen Fan a025a89f4e [SPARK-32332][SQL] Support columnar exchanges
### What changes were proposed in this pull request?

This PR adds abstract classes for shuffle and broadcast, so that users can provide their columnar implementations.

This PR updates several places to use the abstract exchange classes, and also update `AdaptiveSparkPlanExec` so that the columnar rules can see exchange nodes.

This is an alternative of https://github.com/apache/spark/pull/29134 .
Close https://github.com/apache/spark/pull/29134

### Why are the changes needed?

To allow columnar exchanges.

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

no

### How was this patch tested?

new tests

Closes #29262 from cloud-fan/columnar.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-29 14:21:47 -05:00
Thomas Graves e926d419d3 [SPARK-30322][DOCS] Add stage level scheduling docs
### What changes were proposed in this pull request?

Document the stage level scheduling feature.

### Why are the changes needed?

Document the stage level scheduling feature.

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

Documentation.

### How was this patch tested?

n/a docs only

Closes #29292 from tgravescs/SPARK-30322.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-29 13:46:28 -05:00
Dongjoon Hyun 9dc0237851 [SPARK-32476][CORE] ResourceAllocator.availableAddrs should be deterministic
### What changes were proposed in this pull request?

This PR aims to make `ResourceAllocator.availableAddrs` deterministic.

### Why are the changes needed?

Currently, this function returns indeterministically due to the underlying `HashMap`. So, the test case itself is creating a list `[0, 1, 2]` initially, but ends up with comparing `[2, 1, 0]`.

Not only this happens in the 3.0.0, but also this causes UT failures on Scala 2.13 environment.

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

Yes, but this fixes the in-deterministic behavior.

### How was this patch tested?

- Scala 2.12: This should pass the UT with the modified test case.
- Scala 2.13: This can be tested like the following (at least `JsonProtocolSuite`)

```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.deploy.JsonProtocolSuite
```

**BEFORE**
```
*** 2 TESTS FAILED ***
```

**AFTER**
```
All tests passed.
```

Closes #29281 from dongjoon-hyun/SPARK-32476.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-29 10:47:32 -07:00
Max Gekk d897825d2d [SPARK-32346][SQL] Support filters pushdown in Avro datasource
### What changes were proposed in this pull request?
In the PR, I propose to support pushed down filters in Avro datasource V1 and V2.
1. Added new SQL config `spark.sql.avro.filterPushdown.enabled` to control filters pushdown to Avro datasource. It is on by default.
2. Renamed `CSVFilters` to `OrderedFilters`.
3. `OrderedFilters` is used in `AvroFileFormat` (DSv1) and in `AvroPartitionReaderFactory` (DSv2)
4. Modified `AvroDeserializer` to return None from the `deserialize` method when pushdown filters return `false`.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks up to **2** times on JDK 11:
```
OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                        9614           9669          54          0.1        9614.1       1.0X
pushdown disabled                                 10077          10141          66          0.1       10077.2       1.0X
w/ filters                                         4681           4713          29          0.2        4681.5       2.1X
```

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

### How was this patch tested?
- Added UT to `AvroCatalystDataConversionSuite` and `AvroSuite`
- Re-running `AvroReadBenchmark` using Amazon EC2:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`|

and `./dev/run-benchmarks`:
```python
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
  ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #29145 from MaxGekk/avro-filters-pushdown.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-07-30 01:37:42 +08:00
Huaxin Gao 40e6a5bbb0 [SPARK-32449][ML][PYSPARK] Add summary to MultilayerPerceptronClassificationModel
### What changes were proposed in this pull request?
Add training summary to MultilayerPerceptronClassificationModel...

### Why are the changes needed?
so that user can get the training process status, such as loss value of each iteration and total iteration number.

### Does this PR introduce _any_ user-facing change?
Yes
MultilayerPerceptronClassificationModel.summary
MultilayerPerceptronClassificationModel.evaluate

### How was this patch tested?
new tests

Closes #29250 from huaxingao/mlp_summary.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-07-29 09:58:25 -05:00
Dongjoon Hyun 5eab8d27e6 [SPARK-32477][CORE] JsonProtocol.accumulablesToJson should be deterministic
### What changes were proposed in this pull request?

This PR aims to make `JsonProtocol.accumulablesToJson` deterministic.

### Why are the changes needed?

Currently, `JsonProtocol.accumulablesToJson` is indeterministic. So, `JsonProtocolSuite` itself is also using mixed test cases in terms of `"Accumulables": [ ... ]`.

Not only this is indeterministic, but also this causes a UT failure in `JsonProtocolSuite` in Scala 2.13.

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

Yes. However, this is a fix on indeterministic behavior.

### How was this patch tested?

- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Do the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.JsonProtocolSuite
```

**BEFORE**
```
*** 1 TEST FAILED ***
```

**AFTER**
```
All tests passed.
```

Closes #29282 from dongjoon-hyun/SPARK-32477.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-29 07:48:23 -07:00
Kousuke Saruta 9be088357e [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread
### What changes were proposed in this pull request?

This PR changes the order between initialization for ExecutorPlugin and starting heartbeat thread in Executor.

### Why are the changes needed?

In the current master, heartbeat thread in a executor starts after plugin initialization so if the initialization takes long time, heartbeat is not sent to driver and the executor will be removed from cluster.

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

Yes. Plugins for executors will be allowed to take long time for initialization.

### How was this patch tested?

New testcase.

Closes #29002 from sarutak/fix-heartbeat-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-29 08:44:56 -05:00
Dongjoon Hyun 77987a222c [SPARK-32473][CORE][TESTS] Use === instead IndexSeqView
### What changes were proposed in this pull request?

This PR aims to fix `SorterSuite` and `RadixSortSuite` in Scala 2.13 by using `===` instead of `IndexSeqView`.
```
$ git grep "\.view =="
core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala:    assert(data0.view === data1.view)
core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala:    assert(data0.view === data2.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(ref.view == result.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(res1.view == res2.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(ref.view == result.view)
core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala:      assert(res1.view == res2.view)
```

### Why are the changes needed?

Scala 2.13 reimplements `IndexSeqView` and the behavior is different.
- https://docs.scala-lang.org/overviews/core/collections-migration-213.html

**Scala 2.12**
```scala
Welcome to Scala 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> Seq(1,2,3).toArray.view == Seq(1,2,3).toArray.view
res0: Boolean = true
```

**Scala 2.13**
```scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> Seq(1,2,3).toArray.view == Seq(1,2,3).toArray.view
val res0: Boolean = false
```

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

No. This is a test-only fix.

### How was this patch tested?

- Scala 2.12: Pass the GitHub Action or Jenkins.
- Scala 2.13: Manually test the following.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.collection.unsafe.sort.RadixSortSuite
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.util.collection.SorterSuite
```

**BEFORE**
```
Tests: succeeded 9, failed 36, canceled 0, ignored 0, pending 0
*** 36 TESTS FAILED ***
Tests: succeeded 3, failed 1, canceled 0, ignored 2, pending 0
*** 1 TEST FAILED ***
```

**AFTER**
```
Tests: succeeded 45, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
Tests: succeeded 4, failed 0, canceled 0, ignored 2, pending 0
All tests passed.
```

Closes #29280 from dongjoon-hyun/SPARK-32473.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-28 22:31:39 -07:00
LantaoJin 26e6574d58 [SPARK-32283][CORE] Kryo should support multiple user registrators
### What changes were proposed in this pull request?
`spark.kryo.registrator` in 3.0 has a regression problem. From [SPARK-12080](https://issues.apache.org/jira/browse/SPARK-12080), it supports multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it donsn't work in 3.0. Fix it by `toSequence` in `Kryo.scala`

### Why are the changes needed?
In previous Spark version (2.x), it supported multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it doesn't work in 3.0. It's should be a regression.

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

### How was this patch tested?
Existed unit tests.

Closes #29123 from LantaoJin/SPARK-32283.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:58:03 +00:00
Terry Kim 45b7212fd3 [SPARK-32401][SQL] Migrate function related commands to use UnresolvedFunc to resolve function identifier
### What changes were proposed in this pull request?

This PR proposes to migrate the following function related commands to use `UnresolvedFunc` to resolve function identifier:
- DROP FUNCTION
- DESCRIBE FUNCTION
- SHOW FUNCTIONS

`DropFunctionStatement`, `DescribeFunctionStatement` and `ShowFunctionsStatement` logical plans are replaced with `DropFunction`, `DescribeFunction` and `ShowFunctions` logical plans respectively, and each contains `UnresolvedFunc` as its child so that it can be resolved in `Analyzer`.

### Why are the changes needed?

Migrating to the new resolution framework, which resolves `UnresolvedFunc` in `Analyzer`.

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

The message of exception thrown when a catalog is resolved to v2 has been merged to:
`function is only supported in v1 catalog`

Previously, it printed out the command used. E.g.,:
`CREATE FUNCTION is only supported in v1 catalog`

### How was this patch tested?

Updated existing tests.

Closes #29198 from imback82/function_framework.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:55:48 +00:00
Max Gekk b2180c0950 [SPARK-32471][SQL][DOCS][TESTS][PYTHON][SS] Describe JSON option allowNonNumericNumbers
### What changes were proposed in this pull request?
1. Describe the JSON option `allowNonNumericNumbers` which is used in read
2. Add new test cases for allowed JSON field values: NaN, +INF, +Infinity, Infinity, -INF and -Infinity

### Why are the changes needed?
To improve UX with Spark SQL and to provide users full info about the supported option.

### Does this PR introduce _any_ user-facing change?
Yes, in PySpark.

### How was this patch tested?
Added new test to `JsonParsingOptionsSuite`

Closes #29275 from MaxGekk/allowNonNumericNumbers-doc.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-29 12:14:13 +09:00
HyukjinKwon 5491c08bf1 Revert "[SPARK-31525][SQL] Return an empty list for df.head() when df is empty"
This reverts commit 44a5258ac2.
2020-07-29 12:07:35 +09:00
Michael Munday a3d80564ad [SPARK-32458][SQL][TESTS] Fix incorrectly sized row value reads
### What changes were proposed in this pull request?
Updates to tests to use correctly sized `getInt` or `getLong` calls.

### Why are the changes needed?
The reads were incorrectly sized (i.e. `putLong` paired with `getInt` and `putInt` paired with `getLong`). This causes test failures on big-endian systems.

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

### How was this patch tested?
Tests were run on a big-endian system (s390x). This change is unlikely to have any practical effect on little-endian systems.

Closes #29258 from mundaym/fix-row.

Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-28 10:36:20 -07:00
Xiaochang Wu 44c868b73a [SPARK-32339][ML][DOC] Improve MLlib BLAS native acceleration docs
### What changes were proposed in this pull request?
Rewrite a clearer and complete BLAS native acceleration enabling guide.

### Why are the changes needed?
The document of enabling BLAS native acceleration in ML guide (https://spark.apache.org/docs/latest/ml-guide.html#dependencies) is incomplete and unclear to the user.

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

### How was this patch tested?
N/A

Closes #29139 from xwu99/blas-doc.

Lead-authored-by: Xiaochang Wu <xiaochang.wu@intel.com>
Co-authored-by: Wu, Xiaochang <xiaochang.wu@intel.com>
Signed-off-by: Huaxin Gao <huaxing@us.ibm.com>
2020-07-28 08:36:11 -07:00
Max Gekk c28da672f8 [SPARK-32382][SQL] Override table renaming in JDBC dialects
### What changes were proposed in this pull request?
Override the default implementation of `JdbcDialect.renameTable()`:
```scala
s"ALTER TABLE $oldTable RENAME TO $newTable"
```
in the following JDBC dialects according to official documentation:
- DB2
- Derby
- MS SQL Server
- Teradata

Other dialects follow the default implementation:
- MySQL: https://dev.mysql.com/doc/refman/8.0/en/alter-table.html
- Oracle: https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/ALTER-TABLE.html#GUID-552E7373-BF93-477D-9DA3-B2C9386F2877
- PostgreSQL: https://www.postgresql.org/docs/12/sql-altertable.html

### Why are the changes needed?
To have correct implementation of table renaming for all supported JDBC dialects.

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

### How was this patch tested?
Manually

Closes #29237 from MaxGekk/jdbc-rename-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 12:34:10 +00:00
yi.wu ca1ecf7f9f [SPARK-32459][SQL] Support WrappedArray as customCollectionCls in MapObjects
### What changes were proposed in this pull request?

This PR supports `WrappedArray` as `customCollectionCls` in `MapObjects`.

### Why are the changes needed?

This helps fix the regression caused by SPARK-31826.  For the following test, it can pass in branch-3.0 but fail in master branch:

```scala
test("WrappedArray") {
    val myUdf = udf((a: WrappedArray[Int]) =>
      WrappedArray.make[Int](Array(a.head + 99)))
    checkAnswer(Seq(Array(1))
      .toDF("col")
      .select(myUdf(Column("col"))),
      Row(ArrayBuffer(100)))
  }
```

In SPARK-31826, we've changed the catalyst-to-scala converter from `CatalystTypeConverters` to `ExpressionEncoder.deserializer`. However, `CatalystTypeConverters` supports `WrappedArray` while `ExpressionEncoder.deserializer` doesn't.

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

No,  SPARK-31826 is merged into master and branch-3.1, which haven't been released.

### How was this patch tested?

Added a new test for `WrappedArray` in `UDFSuite`; Also updated `ObjectExpressionsSuite` for `MapObjects`.

Closes #29261 from Ngone51/fix-wrappedarray.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 12:24:15 +00:00
xuewei.linxuewei 12b9787a7f [SPARK-32290][SQL] SingleColumn Null Aware Anti Join Optimize
### What changes were proposed in this pull request?
Normally, a Null aware anti join will be planed into BroadcastNestedLoopJoin which is very time consuming, for instance, in TPCH Query 16.

```
select
    p_brand,
    p_type,
    p_size,
    count(distinct ps_suppkey) as supplier_cnt
from
    partsupp,
    part
where
    p_partkey = ps_partkey
    and p_brand <> 'Brand#45'
    and p_type not like 'MEDIUM POLISHED%'
    and p_size in (49, 14, 23, 45, 19, 3, 36, 9)
    and ps_suppkey not in (
        select
            s_suppkey
        from
            supplier
        where
            s_comment like '%Customer%Complaints%'
    )
group by
    p_brand,
    p_type,
    p_size
order by
    supplier_cnt desc,
    p_brand,
    p_type,
    p_size
```

In above query, will planed into

LeftAnti
    condition Or((ps_suppkey=s_suppkey), IsNull(ps_suppkey=s_suppkey))

Inside BroadcastNestedLoopJoinExec will perform O(M\*N), BUT if there is only single column in NAAJ, we can always change buildSide into a HashSet, and streamedSide just need to lookup in the HashSet, then the calculation will be optimized into O(M).

But this optimize is only targeting on null aware anti join with single column case, because multi-column support is much more complicated, we might be able to support multi-column in future.
After apply this patch, the TPCH Query 16 performance decrease from 41mins to 30s

The semantic of null-aware anti join is:

![image](https://user-images.githubusercontent.com/17242071/88077041-66a39a00-cbad-11ea-8fb6-c235c4d219b4.png)

### Why are the changes needed?
TPCH is a common benchmark for distributed compute engine, all other 21 Query works fine on Spark, except for Query 16, apply this patch will make Spark more competitive among all these popular engine. BTW, this patch has restricted rules and only apply on NAAJ Single Column case, which is safe enough.

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

### How was this patch tested?
1. SQLQueryTestSuite with NOT IN keyword SQL, add CONFIG_DIM with spark.sql.optimizeNullAwareAntiJoin on and off
2. added case in org.apache.spark.sql.JoinSuite.
3. added case in org.apache.spark.sql.SubquerySuite.
3. Compare performance before and after applying this patch against TPCH Query 16.
4. config combination against e2e test with following

```
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "sspark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "true"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "true"
)
```

Closes #29104 from leanken/leanken-SPARK-32290.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 04:42:15 +00:00
Tianshi Zhu 44a5258ac2 [SPARK-31525][SQL] Return an empty list for df.head() when df is empty
### What changes were proposed in this pull request?

return an empty list instead of None when calling `df.head()`

### Why are the changes needed?

`df.head()` and `df.head(1)` are inconsistent when df is empty.

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

Yes. If a user relies on `df.head()` to return None, things like `if df.head() is None:` will be broken.

### How was this patch tested?

Closes #29214 from tianshizz/SPARK-31525.

Authored-by: Tianshi Zhu <zhutianshirea@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-28 12:32:19 +09:00
Shantanu 77f2ca6cce [MINOR][PYTHON] Fix spacing in error message
### What changes were proposed in this pull request?
Fixes spacing in an error message

### Why are the changes needed?
Makes error messages easier to read

### Does this PR introduce _any_ user-facing change?
Yes, it changes the error message

### How was this patch tested?
This patch doesn't affect any logic, so existing tests should cover it

Closes #29264 from hauntsaninja/patch-1.

Authored-by: Shantanu <12621235+hauntsaninja@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-28 11:22:18 +09:00
Frank Yin 8323c8eb56 [SPARK-32059][SQL] Allow nested schema pruning thru window/sort plans
### What changes were proposed in this pull request?
This PR is intended to solve schema pruning not working with window functions, as described in SPARK-32059. It also solved schema pruning not working with `Sort`. It also generalizes with `Project->Filter->[any node can be pruned]`.

### Why are the changes needed?
This is needed because of performance issues with nested structures with querying using window functions as well as sorting.

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

### How was this patch tested?
Introduced two tests: 1) optimizer planning level 2) end-to-end tests with SQL queries.

Closes #28898 from frankyin-factual/master.

Authored-by: Frank Yin <frank@factual.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-28 10:00:21 +09:00
GuoPhilipse 8de43338be [SPARK-31753][SQL][DOCS] Add missing keywords in the SQL docs
### What changes were proposed in this pull request?
update sql-ref docs, the following key words will be added in this PR.

CASE/ELSE
WHEN/THEN
MAP KEYS TERMINATED BY
NULL DEFINED AS
LINES TERMINATED BY
ESCAPED BY
COLLECTION ITEMS TERMINATED BY
PIVOT
LATERAL VIEW OUTER?
ROW FORMAT SERDE
ROW FORMAT DELIMITED
FIELDS TERMINATED BY
IGNORE NULLS
FIRST
LAST

### Why are the changes needed?
let more users know the sql key words usage

### Does this PR introduce _any_ user-facing change?
![image](https://user-images.githubusercontent.com/46367746/88148830-c6dc1f80-cc31-11ea-81ea-13bc9dc34550.png)
![image](https://user-images.githubusercontent.com/46367746/88148968-fb4fdb80-cc31-11ea-8649-e8297cf5813e.png)
![image](https://user-images.githubusercontent.com/46367746/88149000-073b9d80-cc32-11ea-9aa4-f914ecd72663.png)
![image](https://user-images.githubusercontent.com/46367746/88149021-0f93d880-cc32-11ea-86ed-7db8672b5aac.png)

### How was this patch tested?
No

Closes #29056 from GuoPhilipse/add-missing-keywords.

Lead-authored-by: GuoPhilipse <guofei_ok@126.com>
Co-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-28 09:41:53 +09:00
zhengruifeng f7542d3b61 [SPARK-32457][ML] logParam thresholds in DT/GBT/FM/LR/MLP
### What changes were proposed in this pull request?
logParam `thresholds` in DT/GBT/FM/LR/MLP

### Why are the changes needed?
param `thresholds` is logged in NB/RF, but not in other ProbabilisticClassifier

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

### How was this patch tested?
existing testsuites

Closes #29257 from zhengruifeng/instr.logParams_add_thresholds.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Huaxin Gao <huaxing@us.ibm.com>
2020-07-27 12:05:29 -07:00
HyukjinKwon c1140661bf [SPARK-32443][CORE] Use POSIX-compatible command -v in testCommandAvailable
### What changes were proposed in this pull request?

This PR aims to use `command -v` in non-Window operating systems instead of executing the given command.

### Why are the changes needed?

1. `command` is POSIX-compatible
    - **POSIX.1-2017**:  https://pubs.opengroup.org/onlinepubs/9699919799/utilities/command.html
2. `command` is faster and safer than the direct execution
    - `command` doesn't invoke another process.
```scala
scala> sys.process.Process("ls").run().exitValue()
LICENSE
NOTICE
bin
doc
lib
man
res1: Int = 0
```

3. The existing way behaves inconsistently.
    - `rm` cannot be checked.

**AS-IS**
```scala
scala> sys.process.Process("rm").run().exitValue()
usage: rm [-f | -i] [-dPRrvW] file ...
       unlink file
res0: Int = 64
```

**TO-BE**
```
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.
scala> sys.process.Process(Seq("sh", "-c", s"command -v ls")).run().exitValue()
/bin/ls
val res1: Int = 0
```

4. The existing logic is already broken in Scala 2.13 environment because it hangs like the following.
```scala
$ bin/scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> sys.process.Process("cat").run().exitValue() // hang here.
```

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

No. Although this is inside `main` source directory, this is used for testing purpose.

```
$ git grep testCommandAvailable | grep -v 'def testCommandAvailable'
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("wc"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(!TestUtils.testCommandAvailable("some_nonexistent_command"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:  private lazy val isPythonAvailable: Boolean = TestUtils.testCommandAvailable(pythonExec)
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:    if (TestUtils.testCommandAvailable(pythonExec)) {
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("python"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("echo | sed"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
```

### How was this patch tested?

- **Scala 2.12**: Pass the Jenkins with the existing tests and one modified test.
- **Scala 2.13**: Do the following manually. It should pass instead of `hang`.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.PipedRDDSuite
...
Tests: succeeded 12, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29241 from dongjoon-hyun/SPARK-32443.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-27 12:02:43 -07:00
Kent Yao d315ebf3a7 [SPARK-32424][SQL] Fix silent data change for timestamp parsing if overflow happens
### What changes were proposed in this pull request?

When using `Seconds.toMicros` API to convert epoch seconds to microseconds,

```scala
 /**
     * Equivalent to
     * {link #convert(long, TimeUnit) MICROSECONDS.convert(duration, this)}.
     * param duration the duration
     * return the converted duration,
     * or {code Long.MIN_VALUE} if conversion would negatively
     * overflow, or {code Long.MAX_VALUE} if it would positively overflow.
     */
```
This PR change it to `Math.multiplyExact(epochSeconds, MICROS_PER_SECOND)`

### Why are the changes needed?

fix silent data change between 3.x and 2.x
```
 ~/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200722   bin/spark-sql -S -e "select to_timestamp('300000', 'y');"
+294247-01-10 12:00:54.775807
```
```
 kentyaohulk  ~/Downloads/spark/spark-2.4.5-bin-hadoop2.7  bin/spark-sql -S  -e "select to_timestamp('300000', 'y');"
284550-10-19 15:58:1010.448384
```

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

Yes, we will raise `ArithmeticException` instead of giving the wrong answer if overflow.

### How was this patch tested?

add unit test

Closes #29220 from yaooqinn/SPARK-32424.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 17:03:14 +00:00
Cheng Su 548b7db345 [SPARK-32420][SQL] Add handling for unique key in non-codegen hash join
### What changes were proposed in this pull request?

`HashRelation` has two separate code paths for unique key look up and non-unique key look up E.g. in its subclass [`UnsafeHashedRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala#L144-L177), unique key look up is more efficient as it does not have e.g. extra `Iterator[UnsafeRow].hasNext()/next()` overhead per row.

`BroadcastHashJoinExec` has handled unique key vs non-unique key separately in [code-gen path](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala#L289-L321). But the non-codegen path for broadcast hash join and shuffled hash join do not separate it yet, so adding the support here.

### Why are the changes needed?

Shuffled hash join and non-codegen broadcast hash join still rely on this code path for execution. So this PR will help save CPU for executing this two type of join. Adding codegen for shuffled hash join would be a different topic and I will add it in https://issues.apache.org/jira/browse/SPARK-32421 .

Ran the same query as [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153-L167), with enabling and disabling this feature. Verified 20% wall clock time improvement (switch control and test group order as well to verify the improvement to not be the noise).

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join unique key SHJ off
  Stopped after 5 iterations, 4039 ms
  Running case: shuffle hash join unique key SHJ on
  Stopped after 5 iterations, 2898 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join unique key SHJ off                707            808          81          5.9         168.6       1.0X
shuffle hash join unique key SHJ on                 547            580          50          7.7         130.4       1.3X
```

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join unique key SHJ on
  Stopped after 5 iterations, 3333 ms
  Running case: shuffle hash join unique key SHJ off
  Stopped after 5 iterations, 4268 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join unique key SHJ on                 565            667          60          7.4         134.8       1.0X
shuffle hash join unique key SHJ off                774            854          85          5.4         184.4       0.7X
```

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

No.

### How was this patch tested?

* Added test in `OuterJoinSuite` to cover left outer and right outer join.
* Added test in `ExistenceJoinSuite` to cover left semi join, and existence join.
* [Existing `joinSuite` already covered inner join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala#L182)
* [Existing `ExistenceJoinSuite` already covered left anti join, and existence join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala#L228)

Closes #29216 from c21/unique-key.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 17:01:03 +00:00
HyukjinKwon ea58e52823 [SPARK-32434][CORE][FOLLOW-UP] Fix load-spark-env.cmd to be able to run in Windows properly
### What changes were proposed in this pull request?

This PR is basically a followup of SPARK-26132 and SPARK-32434. You can't define an environment variable within an-if to use it within the block. See also https://superuser.com/questions/78496/variables-in-batch-file-not-being-set-when-inside-if

### Why are the changes needed?

For Windows users to use Spark and fix the build in AppVeyor.

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

No, it's only in unreleased branches.

### How was this patch tested?

Manually tested on a local Windows machine, and AppVeyor build at https://github.com/HyukjinKwon/spark/pull/13. See https://ci.appveyor.com/project/HyukjinKwon/spark/builds/34316409

Closes #29254 from HyukjinKwon/SPARK-32434.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-27 22:37:08 +09:00
Warren Zhu 998086c9a1 [SPARK-30794][CORE] Stage Level scheduling: Add ability to set off heap memory
### What changes were proposed in this pull request?
Support set off heap memory in `ExecutorResourceRequests`

### Why are the changes needed?
Support stage level scheduling

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

### How was this patch tested?
Added UT in `ResourceProfileSuite` and `DAGSchedulerSuite`

Closes #28972 from warrenzhu25/30794.

Authored-by: Warren Zhu <zhonzh@microsoft.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-27 08:16:13 -05:00
HyukjinKwon a82aee0441 [SPARK-32435][PYTHON] Remove heapq3 port from Python 3
### What changes were proposed in this pull request?

This PR removes the manual port of `heapq3.py` introduced from SPARK-3073. The main reason of this was to support Python 2.6 and 2.7 because Python 2's `heapq.merge()` doesn't not support `key` and `reverse`.

See
- https://docs.python.org/2/library/heapq.html#heapq.merge in Python 2
- https://docs.python.org/3.8/library/heapq.html#heapq.merge in Python 3

Since we dropped the Python 2 at SPARK-32138, we can remove this away.

### Why are the changes needed?

To remove unnecessary codes. Also, we can leverage bug fixes made in Python 3.x at `heapq`.

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

No, dev-only.

### How was this patch tested?

Existing tests should cover. I locally ran and verified:

```bash
./python/run-tests --python-executable=python3 --testname="pyspark.tests.test_shuffle"
./python/run-tests --python-executable=python3 --testname="pyspark.shuffle ExternalSorter"
./python/run-tests --python-executable=python3 --testname="pyspark.tests.test_rdd RDDTests.test_external_group_by_key"
```

Closes #29229 from HyukjinKwon/SPARK-32435.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-27 20:10:13 +09:00
HyukjinKwon 6ab29b37cf [SPARK-32179][SPARK-32188][PYTHON][DOCS] Replace and redesign the documentation base
### What changes were proposed in this pull request?

This PR proposes to redesign the PySpark documentation.

I made a demo site to make it easier to review: https://hyukjin-spark.readthedocs.io/en/stable/reference/index.html.

Here is the initial draft for the final PySpark docs shape: https://hyukjin-spark.readthedocs.io/en/latest/index.html.

In more details, this PR proposes:
1. Use [pydata_sphinx_theme](https://github.com/pandas-dev/pydata-sphinx-theme) theme - [pandas](https://pandas.pydata.org/docs/) and [Koalas](https://koalas.readthedocs.io/en/latest/) use this theme. The CSS overwrite is ported from Koalas. The colours in the CSS were actually chosen by designers to use in Spark.
2. Use the Sphinx option to separate `source` and `build` directories as the documentation pages will likely grow.
3. Port current API documentation into the new style. It mimics Koalas and pandas to use the theme most effectively.

    One disadvantage of this approach is that you should list up APIs or classes; however, I think this isn't a big issue in PySpark since we're being conservative on adding APIs. I also intentionally listed classes only instead of functions in ML and MLlib to make it relatively easier to manage.

### Why are the changes needed?

Often I hear the complaints, from the users, that current PySpark documentation is pretty messy to read - https://spark.apache.org/docs/latest/api/python/index.html compared other projects such as [pandas](https://pandas.pydata.org/docs/) and [Koalas](https://koalas.readthedocs.io/en/latest/).

It would be nicer if we can make it more organised instead of just listing all classes, methods and attributes to make it easier to navigate.

Also, the documentation has been there from almost the very first version of PySpark. Maybe it's time to update it.

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

Yes, PySpark API documentation will be redesigned.

### How was this patch tested?

Manually tested, and the demo site was made to show.

Closes #29188 from HyukjinKwon/SPARK-32179.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-27 17:49:21 +09:00
SaurabhChawla 99f33ec30f [SPARK-32234][FOLLOWUP][SQL] Update the description of utility method
### What changes were proposed in this pull request?
As the part of this PR https://github.com/apache/spark/pull/29045 added the helper method. This PR is the FOLLOWUP PR to update the description of helper method.

### Why are the changes needed?
For better readability and understanding of the code

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

### How was this patch tested?
Since its only change of updating the description , So ran the Spark shell

Closes #29232 from SaurabhChawla100/SPARK-32234-Desc.

Authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 08:14:02 +00:00