Commit graph

30236 commits

Author SHA1 Message Date
Takuya UESHIN c0b52da89e [SPARK-35388][INFRA] Allow the PR source branch to include slashes
### What changes were proposed in this pull request?

This PR allows the PR source branch to include slashes.

### Why are the changes needed?

There are PRs whose source branches include slashes, like `issues/SPARK-35119/gha` here or #32523.

Before the fix, the PR build fails in `Sync the current branch with the latest in Apache Spark` phase.
For example, at #32523, the source branch is `issues/SPARK-35382/nested_higher_order_functions`:

```
...
fatal: couldn't find remote ref nested_higher_order_functions
Error: Process completed with exit code 128.
```

(https://github.com/ueshin/apache-spark/runs/2569356241)

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

No, this is a dev-only change.

### How was this patch tested?

This PR source branch includes slashes and #32525 doesn't.

Closes #32524 from ueshin/issues/SPARK-35119/gha.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-13 10:59:30 +09:00
Takeshi Yamamuro 3241aeb7f4 [SPARK-35385][SQL][TESTS] Skip duplicate queries in the TPCDS-related tests
### What changes were proposed in this pull request?

This PR proposes to skip the "q6", "q34", "q64", "q74", "q75", "q78" queries in the TPCDS-related tests because the TPCDS v2.7 queries have almost the same ones; the only differences in these queries are ORDER BY columns.

### Why are the changes needed?

To improve test performance.

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

No, dev only.

### How was this patch tested?

Existing tests.

Closes #32520 from maropu/SkipDupQueries.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-13 09:46:25 +09:00
Luca Canali ae0579a945 [SPARK-35369][DOC] Document ExecutorAllocationManager metrics
### What changes were proposed in this pull request?
This proposes to document the available metrics for ExecutorAllocationManager in the Spark monitoring documentation.

### Why are the changes needed?
The ExecutorAllocationManager is instrumented with metrics using the Spark metrics system.
The relevant work is in SPARK-7007 and SPARK-33763
ExecutorAllocationManager metrics are currently undocumented.

### Does this PR introduce _any_ user-facing change?
This PR adds documentation only.

### How was this patch tested?
na

Closes #32500 from LucaCanali/followupMetricsDocSPARK33763.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 13:02:00 -07:00
shahid b3c916e5a5 [SPARK-35013][CORE] Don't allow to set spark.driver.cores=0
### What changes were proposed in this pull request?
Currently spark is not allowing to set spark.driver.memory, spark.executor.cores, spark.executor.memory to 0, but allowing driver cores to 0. This PR checks for driver core size as well. Thanks Oleg Lypkan for finding this.

### Why are the changes needed?
To make the configuration check consistent.

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

### How was this patch tested?
Manual testing

Closes #32504 from shahidki31/shahid/drivercore.

Lead-authored-by: shahid <shahidki31@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 12:45:55 -07:00
Chao Sun bc95c3a69b [SPARK-35361][SQL][FOLLOWUP] Switch to use while loop
### What changes were proposed in this pull request?

Switch to plain `while` loop following Spark [style guide](https://github.com/databricks/scala-style-guide#traversal-and-zipwithindex).

### Why are the changes needed?

`while` loop may yield better performance comparing to `foreach`.

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

No

### How was this patch tested?

N/A

Closes #32522 from sunchao/SPARK-35361-follow-up.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 12:41:12 -07:00
Dongjoon Hyun 77b7fe19e1 [SPARK-35383][CORE] Improve s3a magic committer support by inferring missing configs
### What changes were proposed in this pull request?

This PR aims to improve S3A magic committer support by inferring all missing configs from a single minimum configuration, `spark.hadoop.fs.s3a.bucket.<bucket>.committer.magic.enabled=true`.

Given that AWS S3 provides a [strong read-after-write consistency](https://aws.amazon.com/blogs/aws/amazon-s3-update-strong-read-after-write-consistency/) since December 2020, we can ignore DynamoDB-related configurations. As a result, the minimum set of configuration are the following:

```
spark.hadoop.fs.s3a.committer.magic.enabled=true
spark.hadoop.fs.s3a.bucket.<bucket>.committer.magic.enabled=true
spark.hadoop.fs.s3a.committer.name=magic
spark.hadoop.mapreduce.outputcommitter.factory.scheme.s3a=org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory
spark.sql.parquet.output.committer.class=org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter
spark.sql.sources.commitProtocolClass=org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
```

### Why are the changes needed?

To use S3A magic committer in Apache Spark, the users need to setup a set of configurations. And, if something is missed, it will end up with the error messages like the following.
```
Exception in thread "main" org.apache.hadoop.fs.s3a.commit.PathCommitException:
`s3a://my-spark-bucket`: Filesystem does not have support for 'magic' committer enabled in configuration option fs.s3a.committer.magic.enabled
	at org.apache.hadoop.fs.s3a.commit.CommitUtils.verifyIsMagicCommitFS(CommitUtils.java:74)
	at org.apache.hadoop.fs.s3a.commit.CommitUtils.getS3AFileSystem(CommitUtils.java:109)
```

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

Yes, after this improvement PR, all Spark users can use S3A committer by using a single configuration.
```
spark.hadoop.fs.s3a.bucket.<bucket>.committer.magic.enabled=true
```

This PR is going to inferring the missing configurations. So, there is no side-effect if the existing users who have all configurations already.

### How was this patch tested?

Pass the CIs with the newly added test cases.

Closes #32518 from dongjoon-hyun/SPARK-35383.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 11:53:28 -07:00
Gengliang Wang dac6f175a6 [SPARK-35387][INFRA] Increase the JVM stack size for Java 11 build test
### What changes were proposed in this pull request?

After merging https://github.com/apache/spark/pull/32439, there is flaky error from the Github action job "Java 11 build with Maven":

```
Error:  ## Exception when compiling 473 sources to /home/runner/work/spark/spark/sql/catalyst/target/scala-2.12/classes
java.lang.StackOverflowError
scala.reflect.internal.Trees.itransform(Trees.scala:1376)
scala.reflect.internal.Trees.itransform$(Trees.scala:1374)
scala.reflect.internal.SymbolTable.itransform(SymbolTable.scala:28)
scala.reflect.internal.SymbolTable.itransform(SymbolTable.scala:28)
scala.reflect.api.Trees$Transformer.transform(Trees.scala:2563)
scala.tools.nsc.transform.TypingTransformers$TypingTransformer.transform(TypingTransformers.scala:51)
```
We can resolve it by increasing the stack size of JVM to 256M. The container for Github action jobs has 7G memory so this should be fine.

### Why are the changes needed?

Fix flaky test failure in Java 11 build test

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

No

### How was this patch tested?

Github action test

Closes #32521 from gengliangwang/increaseStackSize.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 10:49:09 -07:00
Liang-Chi Hsieh f156a95641 [SPARK-35347][SQL][FOLLOWUP] Throw exception with an explicit exception type when cannot find the method instead of sys.error
### What changes were proposed in this pull request?

A simple follow-up of #32474 to throw exception instead of sys.error.

### Why are the changes needed?

An exception only fails the query, instead of sys.error.

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

Yes, if `Invoke` or `StaticInvoke` cannot find the method, instead of original `sys.error` now we only throw an exception.

### How was this patch tested?

Existing tests.

Closes #32519 from viirya/SPARK-35347-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-12 09:56:08 -07:00
Cheng Su 7bcadedbd2 [SPARK-35349][SQL] Add code-gen for left/right outer sort merge join
### What changes were proposed in this pull request?

This PR is to add code-gen support for LEFT OUTER / RIGHT OUTER sort merge join. Currently sort merge join only supports inner join type (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala#L374 ). There's no fundamental reason why we cannot support code-gen for other join types. Here we add code-gen for LEFT OUTER / RIGHT OUTER join. Will submit followup PRs to add LEFT SEMI, LEFT ANTI and FULL OUTER code-gen separately.

The change is to extend current sort merge join logic to work with LEFT OUTER and RIGHT OUTER (should work with LEFT SEMI/ANTI as well, but FULL OUTER join needs some other more code change). Replace left/right with streamed/buffered to make code extendable to other join types besides inner join.

Example query:

```
val df1 = spark.range(10).select($"id".as("k1"), $"id".as("k3"))
val df2 = spark.range(4).select($"id".as("k2"), $"id".as("k4"))
df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2" && $"k3" + 1 < $"k4", "left_outer").explain("codegen")
```

Example generated code:

```
== Subtree 5 / 5 (maxMethodCodeSize:396; maxConstantPoolSize:159(0.24% used); numInnerClasses:0) ==
*(5) SortMergeJoin [k1#2L], [k2#8L], LeftOuter, ((k3#3L + 1) < k4#9L)
:- *(2) Sort [k1#2L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(k1#2L, 5), ENSURE_REQUIREMENTS, [id=#26]
:     +- *(1) Project [id#0L AS k1#2L, id#0L AS k3#3L]
:        +- *(1) Range (0, 10, step=1, splits=2)
+- *(4) Sort [k2#8L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(k2#8L, 5), ENSURE_REQUIREMENTS, [id=#32]
      +- *(3) Project [id#6L AS k2#8L, id#6L AS k4#9L]
         +- *(3) Range (0, 4, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage5(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=5
/* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator smj_streamedInput_0;
/* 010 */   private scala.collection.Iterator smj_bufferedInput_0;
/* 011 */   private InternalRow smj_streamedRow_0;
/* 012 */   private InternalRow smj_bufferedRow_0;
/* 013 */   private long smj_value_2;
/* 014 */   private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0;
/* 015 */   private long smj_value_3;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage5(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 */     smj_streamedInput_0 = inputs[0];
/* 026 */     smj_bufferedInput_0 = inputs[1];
/* 027 */
/* 028 */     smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(2147483632, 2147483647);
/* 029 */     smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(4, 0);
/* 030 */
/* 031 */   }
/* 032 */
/* 033 */   private boolean findNextJoinRows(
/* 034 */     scala.collection.Iterator streamedIter,
/* 035 */     scala.collection.Iterator bufferedIter) {
/* 036 */     smj_streamedRow_0 = null;
/* 037 */     int comp = 0;
/* 038 */     while (smj_streamedRow_0 == null) {
/* 039 */       if (!streamedIter.hasNext()) return false;
/* 040 */       smj_streamedRow_0 = (InternalRow) streamedIter.next();
/* 041 */       long smj_value_0 = smj_streamedRow_0.getLong(0);
/* 042 */       if (false) {
/* 043 */         if (!smj_matches_0.isEmpty()) {
/* 044 */           smj_matches_0.clear();
/* 045 */         }
/* 046 */         return false;
/* 047 */
/* 048 */       }
/* 049 */       if (!smj_matches_0.isEmpty()) {
/* 050 */         comp = 0;
/* 051 */         if (comp == 0) {
/* 052 */           comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0);
/* 053 */         }
/* 054 */
/* 055 */         if (comp == 0) {
/* 056 */           return true;
/* 057 */         }
/* 058 */         smj_matches_0.clear();
/* 059 */       }
/* 060 */
/* 061 */       do {
/* 062 */         if (smj_bufferedRow_0 == null) {
/* 063 */           if (!bufferedIter.hasNext()) {
/* 064 */             smj_value_3 = smj_value_0;
/* 065 */             return !smj_matches_0.isEmpty();
/* 066 */           }
/* 067 */           smj_bufferedRow_0 = (InternalRow) bufferedIter.next();
/* 068 */           long smj_value_1 = smj_bufferedRow_0.getLong(0);
/* 069 */           if (false) {
/* 070 */             smj_bufferedRow_0 = null;
/* 071 */             continue;
/* 072 */           }
/* 073 */           smj_value_2 = smj_value_1;
/* 074 */         }
/* 075 */
/* 076 */         comp = 0;
/* 077 */         if (comp == 0) {
/* 078 */           comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0);
/* 079 */         }
/* 080 */
/* 081 */         if (comp > 0) {
/* 082 */           smj_bufferedRow_0 = null;
/* 083 */         } else if (comp < 0) {
/* 084 */           if (!smj_matches_0.isEmpty()) {
/* 085 */             smj_value_3 = smj_value_0;
/* 086 */             return true;
/* 087 */           } else {
/* 088 */             return false;
/* 089 */           }
/* 090 */         } else {
/* 091 */           smj_matches_0.add((UnsafeRow) smj_bufferedRow_0);
/* 092 */           smj_bufferedRow_0 = null;
/* 093 */         }
/* 094 */       } while (smj_streamedRow_0 != null);
/* 095 */     }
/* 096 */     return false; // unreachable
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     while (smj_streamedInput_0.hasNext()) {
/* 101 */       findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0);
/* 102 */       long smj_value_4 = -1L;
/* 103 */       long smj_value_5 = -1L;
/* 104 */       boolean smj_loaded_0 = false;
/* 105 */       smj_value_5 = smj_streamedRow_0.getLong(1);
/* 106 */       scala.collection.Iterator<UnsafeRow> smj_iterator_0 = smj_matches_0.generateIterator();
/* 107 */       boolean smj_foundMatch_0 = false;
/* 108 */
/* 109 */       // the last iteration of this loop is to emit an empty row if there is no matched rows.
/* 110 */       while (smj_iterator_0.hasNext() || !smj_foundMatch_0) {
/* 111 */         InternalRow smj_bufferedRow_1 = smj_iterator_0.hasNext() ?
/* 112 */         (InternalRow) smj_iterator_0.next() : null;
/* 113 */         boolean smj_isNull_5 = true;
/* 114 */         long smj_value_9 = -1L;
/* 115 */         if (smj_bufferedRow_1 != null) {
/* 116 */           long smj_value_8 = smj_bufferedRow_1.getLong(1);
/* 117 */           smj_isNull_5 = false;
/* 118 */           smj_value_9 = smj_value_8;
/* 119 */         }
/* 120 */         if (smj_bufferedRow_1 != null) {
/* 121 */           boolean smj_isNull_6 = true;
/* 122 */           boolean smj_value_10 = false;
/* 123 */           long smj_value_11 = -1L;
/* 124 */
/* 125 */           smj_value_11 = smj_value_5 + 1L;
/* 126 */
/* 127 */           if (!smj_isNull_5) {
/* 128 */             smj_isNull_6 = false; // resultCode could change nullability.
/* 129 */             smj_value_10 = smj_value_11 < smj_value_9;
/* 130 */
/* 131 */           }
/* 132 */           if (smj_isNull_6 || !smj_value_10) {
/* 133 */             continue;
/* 134 */           }
/* 135 */         }
/* 136 */         if (!smj_loaded_0) {
/* 137 */           smj_loaded_0 = true;
/* 138 */           smj_value_4 = smj_streamedRow_0.getLong(0);
/* 139 */         }
/* 140 */         boolean smj_isNull_3 = true;
/* 141 */         long smj_value_7 = -1L;
/* 142 */         if (smj_bufferedRow_1 != null) {
/* 143 */           long smj_value_6 = smj_bufferedRow_1.getLong(0);
/* 144 */           smj_isNull_3 = false;
/* 145 */           smj_value_7 = smj_value_6;
/* 146 */         }
/* 147 */         smj_foundMatch_0 = true;
/* 148 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 149 */
/* 150 */         smj_mutableStateArray_0[0].reset();
/* 151 */
/* 152 */         smj_mutableStateArray_0[0].zeroOutNullBytes();
/* 153 */
/* 154 */         smj_mutableStateArray_0[0].write(0, smj_value_4);
/* 155 */
/* 156 */         smj_mutableStateArray_0[0].write(1, smj_value_5);
/* 157 */
/* 158 */         if (smj_isNull_3) {
/* 159 */           smj_mutableStateArray_0[0].setNullAt(2);
/* 160 */         } else {
/* 161 */           smj_mutableStateArray_0[0].write(2, smj_value_7);
/* 162 */         }
/* 163 */
/* 164 */         if (smj_isNull_5) {
/* 165 */           smj_mutableStateArray_0[0].setNullAt(3);
/* 166 */         } else {
/* 167 */           smj_mutableStateArray_0[0].write(3, smj_value_9);
/* 168 */         }
/* 169 */         append((smj_mutableStateArray_0[0].getRow()).copy());
/* 170 */
/* 171 */       }
/* 172 */       if (shouldStop()) return;
/* 173 */     }
/* 174 */     ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources();
/* 175 */   }
/* 176 */
/* 177 */ }
```

### Why are the changes needed?

Improve query CPU performance. Example micro benchmark below showed 10% run-time improvement.

```
def sortMergeJoinWithDuplicates(): Unit = {
    val N = 2 << 20
    codegenBenchmark("sort merge join with duplicates", N) {
      val df1 = spark.range(N)
        .selectExpr(s"(id * 15485863) % ${N*10} as k1", "id as k3")
      val df2 = spark.range(N)
        .selectExpr(s"(id * 15485867) % ${N*10} as k2", "id as k4")
      val df = df1.join(df2, col("k1") === col("k2") && col("k3") * 3 < col("k4"), "left_outer")
      assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined)
      df.noop()
    }
 }
```

```
Running benchmark: sort merge join with duplicates
  Running case: sort merge join with duplicates outer-smj-codegen off
  Stopped after 2 iterations, 2696 ms
  Running case: sort merge join with duplicates outer-smj-codegen on
  Stopped after 5 iterations, 6058 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
sort merge join with duplicates:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------------------
sort merge join with duplicates outer-smj-codegen off           1333           1348          21          1.6         635.7       1.0X
sort merge join with duplicates outer-smj-codegen on            1169           1212          47          1.8         557.4       1.1X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite.scala` and `WholeStageCodegenSuite.scala`.

Closes #32476 from c21/smj-outer-codegen.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-12 14:10:15 +00:00
Ludovic Henry b52d47a920 [SPARK-35295][ML] Replace fully com.github.fommil.netlib by dev.ludovic.netlib:2.0
### What changes were proposed in this pull request?

Bump to `dev.ludovic.netlib:2.0` which provides JNI-based wrappers for BLAS, ARPACK, and LAPACK. Theseare not taking dependencies on GPL or LGPL libraries, allowing to provide out-of-the-box support for hardware acceleration when a native library is present (this is still up to the end-user to install such library on their system, like OpenBLAS, Intel MKL, and libarpack2).

### Why are the changes needed?

Great performance improvement for ML-related workload on vanilla-distributions of Spark.

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

Users now take advantage of hardware acceleration as long as a native library is installed (like OpenBLAS, Intel MKL and libarpack2).

### How was this patch tested?

Spark test-suite + dev.ludovic.netlib testsuite.

#### JDK8:
```
[info] OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Linux 5.8.0-50-generic
[info] Intel(R) Xeon(R) E-2276G CPU  3.80GHz
[info]
[info] f2jBLAS    = dev.ludovic.netlib.blas.F2jBLAS
[info] javaBLAS   = dev.ludovic.netlib.blas.Java8BLAS
[info] nativeBLAS = dev.ludovic.netlib.blas.JNIBLAS
[info]
[info] daxpy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        220            226           6        454.9           2.2       1.0X
[info] java                       221            228           5        451.9           2.2       1.0X
[info] native                     209            215           5        478.7           2.1       1.1X
[info]
[info] saxpy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        121            125           3        823.3           1.2       1.0X
[info] java                       121            125           3        824.3           1.2       1.0X
[info] native                     101            105           3        988.4           1.0       1.2X
[info]
[info] dcopy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        212            219           6        470.9           2.1       1.0X
[info] java                       208            212           4        481.0           2.1       1.0X
[info] native                     209            215           5        478.5           2.1       1.0X
[info]
[info] scopy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        114            119           3        878.9           1.1       1.0X
[info] java                        99            105           3       1011.4           1.0       1.2X
[info] native                      97            103           3       1026.7           1.0       1.2X
[info]
[info] ddot:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        108            111           2        925.9           1.1       1.0X
[info] java                        71             73           2       1414.9           0.7       1.5X
[info] native                      54             56           2       1847.0           0.5       2.0X
[info]
[info] sdot:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         96             97           2       1046.8           1.0       1.0X
[info] java                        47             48           1       2129.8           0.5       2.0X
[info] native                      29             30           1       3404.7           0.3       3.3X
[info]
[info] dnrm2:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        139            143           2        718.2           1.4       1.0X
[info] java                        46             47           1       2171.2           0.5       3.0X
[info] native                      44             46           2       2261.8           0.4       3.1X
[info]
[info] snrm2:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        154            157           4        651.0           1.5       1.0X
[info] java                        40             42           1       2469.3           0.4       3.8X
[info] native                      26             27           1       3787.6           0.3       5.8X
[info]
[info] dscal:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        185            195           8        541.0           1.8       1.0X
[info] java                       186            196           7        538.5           1.9       1.0X
[info] native                     177            187           7        564.1           1.8       1.0X
[info]
[info] sscal:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         98            102           3       1016.2           1.0       1.0X
[info] java                        98            102           3       1017.8           1.0       1.0X
[info] native                      87             91           3       1143.2           0.9       1.1X
[info]
[info] dgemv[N]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         68             70           1       1474.7           0.7       1.0X
[info] java                        51             52           1       1973.0           0.5       1.3X
[info] native                      30             32           1       3298.8           0.3       2.2X
[info]
[info] dgemv[T]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         96             99           2       1037.9           1.0       1.0X
[info] java                        50             51           1       1999.6           0.5       1.9X
[info] native                      30             31           1       3368.1           0.3       3.2X
[info]
[info] sgemv[N]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         59             61           1       1688.7           0.6       1.0X
[info] java                        41             42           1       2461.9           0.4       1.5X
[info] native                      15             16           1       6593.0           0.2       3.9X
[info]
[info] sgemv[T]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         90             92           1       1116.2           0.9       1.0X
[info] java                        39             40           1       2565.8           0.4       2.3X
[info] native                      15             16           1       6594.2           0.2       5.9X
[info]
[info] dger:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        192            202           7        520.5           1.9       1.0X
[info] java                       203            214           7        491.9           2.0       0.9X
[info] native                     176            187           7        568.8           1.8       1.1X
[info]
[info] dspmv[U]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         59             61           1        846.1           1.2       1.0X
[info] java                        38             39           1       1313.5           0.8       1.6X
[info] native                      24             27           1       2047.8           0.5       2.4X
[info]
[info] dspr[U]:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         97            101           3        515.4           1.9       1.0X
[info] java                        97            101           2        515.1           1.9       1.0X
[info] native                      88             91           3        569.1           1.8       1.1X
[info]
[info] dsyr[U]:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        169            174           3        295.4           3.4       1.0X
[info] java                       169            174           3        295.4           3.4       1.0X
[info] native                     160            165           4        312.2           3.2       1.1X
[info]
[info] dgemm[N,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        561            577          13       1782.3           0.6       1.0X
[info] java                       225            231           4       4446.2           0.2       2.5X
[info] native                      31             32           3      32473.1           0.0      18.2X
[info]
[info] dgemm[N,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        570            584           9       1754.8           0.6       1.0X
[info] java                       224            230           4       4457.3           0.2       2.5X
[info] native                      31             32           1      32493.4           0.0      18.5X
[info]
[info] dgemm[T,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        855            866           6       1169.2           0.9       1.0X
[info] java                       224            228           3       4466.9           0.2       3.8X
[info] native                      31             32           1      32395.5           0.0      27.7X
[info]
[info] dgemm[T,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                       1328           1344           8        752.8           1.3       1.0X
[info] java                       224            230           4       4458.9           0.2       5.9X
[info] native                      31             32           1      32201.8           0.0      42.8X
[info]
[info] sgemm[N,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        534            541           5       1873.0           0.5       1.0X
[info] java                       220            224           3       4542.8           0.2       2.4X
[info] native                      15             16           1      66803.1           0.0      35.7X
[info]
[info] sgemm[N,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        544            551           6       1839.6           0.5       1.0X
[info] java                       220            224           4       4538.2           0.2       2.5X
[info] native                      15             16           1      65589.9           0.0      35.7X
[info]
[info] sgemm[T,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        833            845          21       1201.0           0.8       1.0X
[info] java                       220            224           3       4548.7           0.2       3.8X
[info] native                      15             16           1      66603.2           0.0      55.5X
[info]
[info] sgemm[T,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        899            907           5       1112.9           0.9       1.0X
[info] java                       221            224           2       4531.6           0.2       4.1X
[info] native                      15             16           1      65944.9           0.0      59.3X
```

#### JDK11:
```
[info] OpenJDK 64-Bit Server VM 11.0.11+9-LTS on Linux 5.8.0-50-generic
[info] Intel(R) Xeon(R) E-2276G CPU  3.80GHz
[info]
[info] f2jBLAS    = dev.ludovic.netlib.blas.F2jBLAS
[info] javaBLAS   = dev.ludovic.netlib.blas.Java11BLAS
[info] nativeBLAS = dev.ludovic.netlib.blas.JNIBLAS
[info]
[info] daxpy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        195            200           3        512.2           2.0       1.0X
[info] java                       197            202           3        507.0           2.0       1.0X
[info] native                     184            189           4        543.0           1.8       1.1X
[info]
[info] saxpy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        108            112           3        921.8           1.1       1.0X
[info] java                       101            105           3        989.4           1.0       1.1X
[info] native                      87             91           3       1147.1           0.9       1.2X
[info]
[info] dcopy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        187            191           3        535.1           1.9       1.0X
[info] java                       182            188           3        548.8           1.8       1.0X
[info] native                     178            182           3        562.2           1.8       1.1X
[info]
[info] scopy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        110            114           3        909.3           1.1       1.0X
[info] java                        86             93           4       1159.3           0.9       1.3X
[info] native                      86             90           3       1162.4           0.9       1.3X
[info]
[info] ddot:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        106            108           2        943.6           1.1       1.0X
[info] java                        70             71           2       1426.8           0.7       1.5X
[info] native                      54             56           2       1835.4           0.5       1.9X
[info]
[info] sdot:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         96             97           1       1047.1           1.0       1.0X
[info] java                        43             44           1       2331.9           0.4       2.2X
[info] native                      29             30           1       3392.1           0.3       3.2X
[info]
[info] dnrm2:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        114            115           2        880.7           1.1       1.0X
[info] java                        42             43           1       2398.1           0.4       2.7X
[info] native                      45             46           1       2233.3           0.4       2.5X
[info]
[info] snrm2:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        140            143           2        714.6           1.4       1.0X
[info] java                        28             29           1       3531.0           0.3       4.9X
[info] native                      26             27           1       3820.0           0.3       5.3X
[info]
[info] dscal:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        156            166           7        641.3           1.6       1.0X
[info] java                       158            167           6        633.2           1.6       1.0X
[info] native                     150            160           7        664.8           1.5       1.0X
[info]
[info] sscal:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         85             88           2       1181.7           0.8       1.0X
[info] java                        85             88           2       1176.0           0.9       1.0X
[info] native                      75             78           2       1333.2           0.8       1.1X
[info]
[info] dgemv[N]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         58             59           1       1731.1           0.6       1.0X
[info] java                        41             43           1       2415.5           0.4       1.4X
[info] native                      30             31           1       3293.9           0.3       1.9X
[info]
[info] dgemv[T]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         94             96           1       1063.4           0.9       1.0X
[info] java                        41             42           1       2435.8           0.4       2.3X
[info] native                      30             30           1       3379.8           0.3       3.2X
[info]
[info] sgemv[N]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         44             45           1       2278.9           0.4       1.0X
[info] java                        37             38           0       2686.8           0.4       1.2X
[info] native                      15             16           1       6555.4           0.2       2.9X
[info]
[info] sgemv[T]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         88             89           1       1142.1           0.9       1.0X
[info] java                        33             34           1       3010.7           0.3       2.6X
[info] native                      15             16           1       6553.9           0.2       5.7X
[info]
[info] dger:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        164            172           4        609.4           1.6       1.0X
[info] java                       163            172           5        612.6           1.6       1.0X
[info] native                     150            159           4        667.0           1.5       1.1X
[info]
[info] dspmv[U]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         49             50           1       1029.4           1.0       1.0X
[info] java                        41             42           1       1209.4           0.8       1.2X
[info] native                      25             27           1       2029.2           0.5       2.0X
[info]
[info] dspr[U]:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         80             85           3        622.2           1.6       1.0X
[info] java                        80             85           3        622.4           1.6       1.0X
[info] native                      75             79           3        668.7           1.5       1.1X
[info]
[info] dsyr[U]:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        137            142           3        364.1           2.7       1.0X
[info] java                       139            142           2        360.4           2.8       1.0X
[info] native                     131            135           3        380.4           2.6       1.0X
[info]
[info] dgemm[N,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        517            525           5       1935.5           0.5       1.0X
[info] java                       213            216           3       4704.8           0.2       2.4X
[info] native                      31             31           1      32705.6           0.0      16.9X
[info]
[info] dgemm[N,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        589            601           6       1698.6           0.6       1.0X
[info] java                       213            217           3       4693.3           0.2       2.8X
[info] native                      31             32           1      32498.9           0.0      19.1X
[info]
[info] dgemm[T,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        851            865           6       1175.3           0.9       1.0X
[info] java                       212            216           3       4717.0           0.2       4.0X
[info] native                      30             32           1      32903.0           0.0      28.0X
[info]
[info] dgemm[T,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                       1301           1316           6        768.4           1.3       1.0X
[info] java                       212            216           2       4717.4           0.2       6.1X
[info] native                      31             32           1      32606.0           0.0      42.4X
[info]
[info] sgemm[N,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        454            460           2       2203.0           0.5       1.0X
[info] java                       208            212           3       4803.8           0.2       2.2X
[info] native                      15             16           0      66586.0           0.0      30.2X
[info]
[info] sgemm[N,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        529            536           4       1889.7           0.5       1.0X
[info] java                       208            212           3       4798.6           0.2       2.5X
[info] native                      15             16           1      66751.4           0.0      35.3X
[info]
[info] sgemm[T,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        830            840           5       1205.1           0.8       1.0X
[info] java                       208            211           2       4814.1           0.2       4.0X
[info] native                      15             15           1      67676.4           0.0      56.2X
[info]
[info] sgemm[T,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        894            907           7       1118.7           0.9       1.0X
[info] java                       208            211           3       4809.6           0.2       4.3X
[info] native                      15             16           1      66675.2           0.0      59.6X
```

#### JDK16:
```
[info] OpenJDK 64-Bit Server VM 16+36 on Linux 5.8.0-50-generic
[info] Intel(R) Xeon(R) E-2276G CPU  3.80GHz
[info]
[info] f2jBLAS    = dev.ludovic.netlib.blas.F2jBLAS
[info] javaBLAS   = dev.ludovic.netlib.blas.VectorBLAS
[info] nativeBLAS = dev.ludovic.netlib.blas.JNIBLAS
[info]
[info] daxpy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        193            199           3        517.5           1.9       1.0X
[info] java                       181            186           4        553.2           1.8       1.1X
[info] native                     181            185           5        553.6           1.8       1.1X
[info]
[info] saxpy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        108            112           2        925.1           1.1       1.0X
[info] java                        88             91           3       1138.6           0.9       1.2X
[info] native                      87             91           3       1144.2           0.9       1.2X
[info]
[info] dcopy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        184            189           3        542.5           1.8       1.0X
[info] java                       181            185           3        552.8           1.8       1.0X
[info] native                     179            183           2        558.0           1.8       1.0X
[info]
[info] scopy:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         97            101           3       1031.6           1.0       1.0X
[info] java                        86             90           2       1163.7           0.9       1.1X
[info] native                      85             88           2       1182.9           0.8       1.1X
[info]
[info] ddot:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        107            109           2        932.4           1.1       1.0X
[info] java                        54             56           2       1846.7           0.5       2.0X
[info] native                      54             56           2       1846.7           0.5       2.0X
[info]
[info] sdot:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         96             97           1       1043.6           1.0       1.0X
[info] java                        29             30           1       3439.3           0.3       3.3X
[info] native                      29             30           1       3423.9           0.3       3.3X
[info]
[info] dnrm2:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        121            123           2        829.8           1.2       1.0X
[info] java                        32             32           1       3171.3           0.3       3.8X
[info] native                      45             46           1       2246.2           0.4       2.7X
[info]
[info] snrm2:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        142            144           2        705.9           1.4       1.0X
[info] java                        15             16           1       6585.8           0.2       9.3X
[info] native                      26             27           1       3839.5           0.3       5.4X
[info]
[info] dscal:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        157            165           5        635.6           1.6       1.0X
[info] java                       151            159           5        664.0           1.5       1.0X
[info] native                     151            160           5        663.6           1.5       1.0X
[info]
[info] sscal:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         85             89           2       1172.3           0.9       1.0X
[info] java                        75             79           3       1337.3           0.7       1.1X
[info] native                      75             79           2       1335.5           0.7       1.1X
[info]
[info] dgemv[N]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         58             59           1       1731.5           0.6       1.0X
[info] java                        28             29           1       3544.2           0.3       2.0X
[info] native                      30             31           1       3306.2           0.3       1.9X
[info]
[info] dgemv[T]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         90             92           1       1108.3           0.9       1.0X
[info] java                        28             28           1       3622.5           0.3       3.3X
[info] native                      30             31           1       3381.3           0.3       3.1X
[info]
[info] sgemv[N]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         44             45           1       2284.7           0.4       1.0X
[info] java                        14             15           1       7034.0           0.1       3.1X
[info] native                      15             16           1       6643.7           0.2       2.9X
[info]
[info] sgemv[T]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         85             86           1       1177.4           0.8       1.0X
[info] java                        15             15           1       6886.1           0.1       5.8X
[info] native                      15             16           1       6560.1           0.2       5.6X
[info]
[info] dger:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        164            173           6        608.1           1.6       1.0X
[info] java                       148            157           5        675.2           1.5       1.1X
[info] native                     152            160           5        659.9           1.5       1.1X
[info]
[info] dspmv[U]:        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         61             63           1        815.4           1.2       1.0X
[info] java                        16             17           1       3104.3           0.3       3.8X
[info] native                      24             27           1       2071.9           0.5       2.5X
[info]
[info] dspr[U]:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                         81             85           2        616.4           1.6       1.0X
[info] java                        81             85           2        614.7           1.6       1.0X
[info] native                      75             78           2        669.5           1.5       1.1X
[info]
[info] dsyr[U]:         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        138            141           3        362.7           2.8       1.0X
[info] java                       137            140           2        365.3           2.7       1.0X
[info] native                     131            134           2        382.9           2.6       1.1X
[info]
[info] dgemm[N,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        525            544           8       1906.2           0.5       1.0X
[info] java                        61             68           3      16358.1           0.1       8.6X
[info] native                      31             32           1      32623.7           0.0      17.1X
[info]
[info] dgemm[N,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        580            598          12       1724.5           0.6       1.0X
[info] java                        61             68           4      16302.5           0.1       9.5X
[info] native                      30             32           1      32962.8           0.0      19.1X
[info]
[info] dgemm[T,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        829            838           4       1206.2           0.8       1.0X
[info] java                        61             69           3      16339.7           0.1      13.5X
[info] native                      30             31           1      33231.9           0.0      27.6X
[info]
[info] dgemm[T,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                       1352           1363           5        739.6           1.4       1.0X
[info] java                        61             69           3      16347.0           0.1      22.1X
[info] native                      31             32           1      32740.3           0.0      44.3X
[info]
[info] sgemm[N,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        482            493           7       2073.1           0.5       1.0X
[info] java                        35             38           2      28315.3           0.0      13.7X
[info] native                      15             15           1      67579.7           0.0      32.6X
[info]
[info] sgemm[N,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        472            482           4       2119.0           0.5       1.0X
[info] java                        36             38           2      28138.1           0.0      13.3X
[info] native                      15             16           1      66616.5           0.0      31.4X
[info]
[info] sgemm[T,N]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        823            830           5       1215.2           0.8       1.0X
[info] java                        35             38           2      28681.4           0.0      23.6X
[info] native                      15             15           1      67908.4           0.0      55.9X
[info]
[info] sgemm[T,T]:      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] -----------------------------------------------------------------------------------------------
[info] f2j                        896            908           7       1115.8           0.9       1.0X
[info] java                        35             38           2      28402.0           0.0      25.5X
[info] native                      15             16           0      66691.2           0.0      59.8X
```

TODO:
- [x] update documentation in `docs/` and `docs/ml-linalg-guide.md` refering `com.github.fommil.netlib`
- [ ] merge https://github.com/luhenry/netlib/pull/1 with all feedback from this PR + remove references to snapshot repositories in `pom.xml` and `project/SparkBuild.scala`.

Closes #32415 from luhenry/master.

Authored-by: Ludovic Henry <git@ludovic.dev>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-12 08:59:36 -05:00
Takeshi Yamamuro 101b0cc313 [SPARK-35253][SQL][BUILD] Bump up the janino version to v3.1.4
### What changes were proposed in this pull request?

This PR proposes to bump up the janino version from 3.0.16 to v3.1.4.
The major changes of this upgrade are as follows:
 - Fixed issue #131: Janino 3.1.2 is 10x slower than 3.0.11: The Compiler's IClassLoader was initialized way too eagerly, thus lots of classes were loaded from the class path, which is very slow.
 - Improved the encoding of stack map frames according to JVMS11 4.7.4: Previously, only "full_frame"s were generated.
 - Fixed issue #107: Janino requires "org.codehaus.commons.compiler.io", but commons-compiler does not export this package
 - Fixed the promotion of the array access index expression (see JLS7 15.13 Array Access Expressions).

For all the changes, please see the change log: http://janino-compiler.github.io/janino/changelog.html

NOTE1: I've checked that there is no obvious performance regression. For all the data, see a link: https://docs.google.com/spreadsheets/d/1srxT9CioGQg1fLKM3Uo8z1sTzgCsMj4pg6JzpdcG6VU/edit?usp=sharing

NOTE2: We upgraded janino to 3.1.2 (#27860) once before, but the commit had been reverted in #29495 because of the correctness issue. Recently, #32374 had checked if Spark could land on v3.1.3 or not, but a new bug was found there. These known issues has been fixed in v3.1.4 by following PRs:
 - janino-compiler/janino#145
 - janino-compiler/janino#146

### Why are the changes needed?

janino v3.0.X  is no longer maintained.

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

No.

### How was this patch tested?

GA passed.

Closes #32455 from maropu/janino_v3.1.4.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-12 08:57:57 -05:00
Enzo Bonnal 402375b59e [SPARK-35357][GRAPHX] Allow to turn off the normalization applied by static PageRank utilities
### What changes were proposed in this pull request?

Overload methods `PageRank.runWithOptions` and  `PageRank.runWithOptionsWithPreviousPageRank` (not to break any user-facing signature) with a `normalized` parameter that describes "whether or not to normalize the rank sum".

### Why are the changes needed?

https://issues.apache.org/jira/browse/SPARK-35357

When dealing with a non negligible proportion of sinks in a graph, algorithm based on incremental update of ranks can get a **precision gain for free** if they are allowed to manipulate non normalized ranks.

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

No

### How was this patch tested?

By adding a unit test that verifies that (even when dealing with a graph containing a sink) we end up with the same result for both these scenarios:
a)
  - Run **6 iterations** of pagerank in a row using `PageRank.runWithOptions` with **normalization enabled**

b)
  - Run **2 iterations** using `PageRank.runWithOptions` with **normalization disabled**
  - Resume from the `preRankGraph1` and run **2 more iterations** using `PageRank.runWithOptionsWithPreviousPageRank` with **normalization disabled**
  - Finally resume from the `preRankGraph2` and run **2 more iterations** using `PageRank.runWithOptionsWithPreviousPageRank` with **normalization enabled**

Closes #32485 from bonnal-enzo/make-pagerank-normalization-optional.

Authored-by: Enzo Bonnal <enzobonnal@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-12 08:56:22 -05:00
Angerszhuuuu ed059541eb [SPARK-29145][SQL][FOLLOWUP] Clean up code about support sub-queries in join conditions
### What changes were proposed in this pull request?
According to discuss https://github.com/apache/spark/pull/25854#discussion_r629451135

### Why are the changes needed?
Clean code

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

### How was this patch tested?
Existed UT

Closes #32499 from AngersZhuuuu/SPARK-29145-fix.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-12 13:45:53 +00:00
Yingyi Bu d92018ee35 [SPARK-35298][SQL] Migrate to transformWithPruning for rules in Optimizer.scala
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- ALIAS
- AND_OR
- AVERAGE
- GENERATE
- INTERSECT
- SORT
- SUM
- DISTINCT_LIKE
- PROJECT
- REPARTITION_OPERATION
- UNION

Added tree traversal pruning to the following rules in Optimizer.scala:
- EliminateAggregateFilter
- RemoveRedundantAggregates
- RemoveNoopOperators
- RemoveNoopUnion
- LimitPushDown
- ColumnPruning
- CollapseRepartition
- OptimizeRepartition
- OptimizeWindowFunctions
- CollapseWindow
- TransposeWindow
- InferFiltersFromGenerate
- InferFiltersFromConstraints
- CombineUnions
- CombineFilters
- EliminateSorts
- PruneFilters
- EliminateLimits
- DecimalAggregates
- ConvertToLocalRelation
- ReplaceDistinctWithAggregate
- ReplaceIntersectWithSemiJoin
- ReplaceExceptWithAntiJoin
- RewriteExceptAll
- RewriteIntersectAll
- RemoveLiteralFromGroupExpressions
- RemoveRepetitionFromGroupExpressions
- OptimizeLimitZero

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

perf diff:
Rule name | Total Time (baseline) | Total Time (experiment) | experiment/baseline
RemoveRedundantAggregates | 51290766 | 67070477 | 1.31
RemoveNoopOperators | 192371141 | 196631275 | 1.02
RemoveNoopUnion | 49222561 | 43266681 | 0.88
LimitPushDown | 40885185 | 21672646 | 0.53
ColumnPruning | 2003406120 | 1285562149 | 0.64
CollapseRepartition | 40648048 | 72646515 | 1.79
OptimizeRepartition | 37813850 | 20600803 | 0.54
OptimizeWindowFunctions | 174426904 | 46741409 | 0.27
CollapseWindow | 38959957 | 24542426 | 0.63
TransposeWindow | 33533191 | 20414930 | 0.61
InferFiltersFromGenerate | 21758688 | 15597344 | 0.72
InferFiltersFromConstraints | 518009794 | 493282321 | 0.95
CombineUnions | 67694022 | 70550382 | 1.04
CombineFilters | 35265060 | 29005424 | 0.82
EliminateSorts | 57025509 | 19795776 | 0.35
PruneFilters | 433964815 | 465579200 | 1.07
EliminateLimits | 44275393 | 24476859 | 0.55
DecimalAggregates | 83143172 | 28816090 | 0.35
ReplaceDistinctWithAggregate | 21783760 | 18287489 | 0.84
ReplaceIntersectWithSemiJoin | 22311271 | 16566393 | 0.74
ReplaceExceptWithAntiJoin | 23838520 | 16588808 | 0.70
RewriteExceptAll | 32750296 | 29421957 | 0.90
RewriteIntersectAll | 29760454 | 21243599 | 0.71
RemoveLiteralFromGroupExpressions | 28151861 | 25270947 | 0.90
RemoveRepetitionFromGroupExpressions | 29587030 | 23447041 | 0.79
OptimizeLimitZero | 18081943 | 15597344 | 0.86
**Accumulated | 4129959311 | 3112676285 | 0.75**

### How was this patch tested?

Existing tests.

Closes #32439 from sigmod/optimizer.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-12 20:42:47 +08:00
PengLei 82c520a3e2 [SPARK-35243][SQL] Support columnar execution on ANSI interval types
### What changes were proposed in this pull request?
Columnar execution support for ANSI interval types include YearMonthIntervalType and DayTimeIntervalType

### Why are the changes needed?
support cache tables with ANSI interval types.

### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
run ./dev/lint-java
run ./dev/scalastyle
run test: CachedTableSuite
run test: ColumnTypeSuite

Closes #32452 from Peng-Lei/SPARK-35243.

Lead-authored-by: PengLei <18066542445@189.cn>
Co-authored-by: Lei Peng <peng.8lei@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 20:11:34 +09:00
Hyukjin Kwon ecb48ccb7d [SPARK-35381][R] Fix lambda variable name issues in nested higher order functions at R APIs
### What changes were proposed in this pull request?

This PR fixes the same issue as https://github.com/apache/spark/pull/32424

```r
df <- sql("SELECT array(1, 2, 3) as numbers, array('a', 'b', 'c') as letters")
collect(select(
  df,
  array_transform("numbers", function(number) {
    array_transform("letters", function(latter) {
      struct(alias(number, "n"), alias(latter, "l"))
    })
  })
))
```

**Before:**

```
... a, a, b, b, c, c, a, a, b, b, c, c, a, a, b, b, c, c
```

**After:**

```
... 1, a, 1, b, 1, c, 2, a, 2, b, 2, c, 3, a, 3, b, 3, c
```

### Why are the changes needed?

To produce the correct results.

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

Yes, it fixes the results to be correct as mentioned above.

### How was this patch tested?

Manually tested as above, and unit test was added.

Closes #32517 from HyukjinKwon/SPARK-35381.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 16:52:39 +09:00
Kousuke Saruta 7e3446a204 [SPARK-35377][INFRA] Add JS linter to GA
### What changes were proposed in this pull request?

SPARK-35175 (#32274) added a linter for JS so let's add it to GA.

### Why are the changes needed?

To JS code keep clean.

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

No.

### How was this patch tested?

GA

Closes #32512 from sarutak/ga-lintjs.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 16:00:55 +09:00
Sean Owen a189be8754 [MINOR][DOCS] Avoid some python docs where first sentence has "e.g." or similar
### What changes were proposed in this pull request?

Avoid some python docs where first sentence has "e.g." or similar as the period causes the docs to show only half of the first sentence as the summary.

### Why are the changes needed?

See for example https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.ml.regression.LinearRegressionModel.html?highlight=linearregressionmodel#pyspark.ml.regression.LinearRegressionModel.summary where the method description is clearly truncated.

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

Only changes docs.

### How was this patch tested?

Manual testing of docs.

Closes #32508 from srowen/TruncatedPythonDesc.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 10:38:59 +09:00
Chao Sun 78221bda95 [SPARK-35361][SQL] Improve performance for ApplyFunctionExpression
### What changes were proposed in this pull request?

In `ApplyFunctionExpression`, move `zipWithIndex` out of the loop for each input row.

### Why are the changes needed?

When the `ScalarFunction` is trivial, `zipWithIndex` could incur significant costs, as shown below:

<img width="899" alt="Screen Shot 2021-05-11 at 10 03 42 AM" src="https://user-images.githubusercontent.com/506679/117866421-fb19de80-b24b-11eb-8c94-d5e8c8b1eda9.png">

By removing it out of the loop, I'm seeing sometimes 2x speedup from `V2FunctionBenchmark`. For instance:

Before:
```
scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
native_long_add                                                                         32437          32896         434         15.4          64.9       1.0X
java_long_add_default                                                                   85675          97045         NaN          5.8         171.3       0.4X
```

After:
```
scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
native_long_add                                                                         30182          30387         279         16.6          60.4       1.0X
java_long_add_default                                                                   42862          43009         209         11.7          85.7       0.7X
```

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

No

### How was this patch tested?

Existing tests

Closes #32507 from sunchao/SPARK-35361.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 10:16:35 +09:00
Kousuke Saruta af0d99cce6 [SPARK-35375][INFRA] Use Jinja2 < 3.0.0 for Python linter dependency in GA
### What changes were proposed in this pull request?

From a few hours ago, Python linter fails in GA.
The latest Jinja 3.0.0 seems to cause this failure.
https://pypi.org/project/Jinja2/

```
Run ./dev/lint-python
starting python compilation test...
python compilation succeeded.

starting pycodestyle test...
pycodestyle checks passed.

starting flake8 test...
flake8 checks passed.

starting mypy test...
mypy checks passed.

starting sphinx-build tests...
sphinx-build checks failed:
Running Sphinx v3.0.4
making output directory... done
[autosummary] generating autosummary for: development/contributing.rst, development/debugging.rst, development/index.rst, development/setting_ide.rst, development/testing.rst, getting_started/index.rst, getting_started/install.rst, getting_started/quickstart.ipynb, index.rst, migration_guide/index.rst, ..., reference/pyspark.ml.rst, reference/pyspark.mllib.rst, reference/pyspark.resource.rst, reference/pyspark.rst, reference/pyspark.sql.rst, reference/pyspark.ss.rst, reference/pyspark.streaming.rst, user_guide/arrow_pandas.rst, user_guide/index.rst, user_guide/python_packaging.rst

Exception occurred:
  File "/__w/spark/spark/python/docs/source/_templates/autosummary/class_with_docs.rst", line 26, in top-level template code
    {% if '__init__' in methods %}
jinja2.exceptions.UndefinedError: 'methods' is undefined
The full traceback has been saved in /tmp/sphinx-err-ypgyi75y.log, if you want to report the issue to the developers.
Please also report this if it was a user error, so that a better error message can be provided next time.
A bug report can be filed in the tracker at <https://github.com/sphinx-doc/sphinx/issues>. Thanks!
make: *** [Makefile:20: html] Error 2

re-running make html to print full warning list:
Running Sphinx v3.0.4
making output directory... done
[autosummary] generating autosummary for: development/contributing.rst, development/debugging.rst, development/index.rst, development/setting_ide.rst, development/testing.rst, getting_started/index.rst, getting_started/install.rst, getting_started/quickstart.ipynb, index.rst, migration_guide/index.rst, ..., reference/pyspark.ml.rst, reference/pyspark.mllib.rst, reference/pyspark.resource.rst, reference/pyspark.rst, reference/pyspark.sql.rst, reference/pyspark.ss.rst, reference/pyspark.streaming.rst, user_guide/arrow_pandas.rst, user_guide/index.rst, user_guide/python_packaging.rst

Exception occurred:
  File "/__w/spark/spark/python/docs/source/_templates/autosummary/class_with_docs.rst", line 26, in top-level template code
    {% if '__init__' in methods %}
jinja2.exceptions.UndefinedError: 'methods' is undefined
The full traceback has been saved in /tmp/sphinx-err-fvtmvvwv.log, if you want to report the issue to the developers.
Please also report this if it was a user error, so that a better error message can be provided next time.
A bug report can be filed in the tracker at <https://github.com/sphinx-doc/sphinx/issues>. Thanks!
make: *** [Makefile:20: html] Error 2
Error: Process completed with exit code 2.
```

### Why are the changes needed?

To recover GA build.

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

No.

### How was this patch tested?

GA.

Closes #32509 from sarutak/fix-python-lint-error.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 10:13:38 +09:00
Hyukjin Kwon b59d5ab060 [SPARK-35372][BUILD] Increase stack size for Scala compilation in Maven build
### What changes were proposed in this pull request?

This PR increases the stack size for Scala compilation in Maven build to fix the error:

```
java.lang.StackOverflowError
scala.reflect.internal.Trees$UnderConstructionTransformer.transform(Trees.scala:1741)
scala.reflect.internal.Trees$UnderConstructionTransformer.transform$(Trees.scala:1740)
scala.tools.nsc.transform.ExplicitOuter$OuterPathTransformer.transform(ExplicitOuter.scala:289)
scala.tools.nsc.transform.ExplicitOuter$ExplicitOuterTransformer.transform(ExplicitOuter.scala:477)
scala.tools.nsc.transform.ExplicitOuter$ExplicitOuterTransformer.transform(ExplicitOuter.scala:330)
scala.reflect.api.Trees$Transformer.$anonfun$transformStats$1(Trees.scala:2597)
scala.reflect.api.Trees$Transformer.transformStats(Trees.scala:2595)
scala.reflect.internal.Trees.itransform(Trees.scala:1404)
scala.reflect.internal.Trees.itransform$(Trees.scala:1374)
scala.reflect.internal.SymbolTable.itransform(SymbolTable.scala:28)
scala.reflect.internal.SymbolTable.itransform(SymbolTable.scala:28)
scala.reflect.api.Trees$Transformer.transform(Trees.scala:2563)
scala.tools.nsc.transform.TypingTransformers$TypingTransformer.transform(TypingTransformers.scala:51)
scala.tools.nsc.transform.ExplicitOuter$OuterPathTransformer.scala$reflect$internal$Trees$UnderConstructionTransformer$$super$transform(ExplicitOuter.scala:212)
scala.reflect.internal.Trees$UnderConstructionTransformer.transform(Trees.scala:1745)
scala.reflect.internal.Trees$UnderConstructionTransformer.transform$(Trees.scala:1740)
scala.tools.nsc.transform.ExplicitOuter$OuterPathTransformer.transform(ExplicitOuter.scala:289)
scala.tools.nsc.transform.ExplicitOuter$ExplicitOuterTransformer.transform(ExplicitOuter.scala:477)
scala.tools.nsc.transform.ExplicitOuter$ExplicitOuterTransformer.transform(ExplicitOuter.scala:330)
scala.reflect.internal.Trees.itransform(Trees.scala:1383)
```

See https://github.com/apache/spark/runs/2554067779

### Why are the changes needed?

To recover JDK 11 compilation

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

No, dev-only.

### How was this patch tested?

CI in this PR will test it out.

Closes #32502 from HyukjinKwon/SPARK-35372.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-12 02:20:28 +09:00
Kousuke Saruta 2b6640a169 [SPARK-35229][WEBUI] Limit the maximum number of items on the timeline view
### What changes were proposed in this pull request?

This PR proposes to introduces three new configurations to limit the maximum number of jobs/stages/executors on the timeline view.

### Why are the changes needed?

If the number of items on the timeline view grows +1000, rendering can be significantly slow.
https://issues.apache.org/jira/browse/SPARK-35229

The maximum number of tasks on the timeline is already limited by `spark.ui.timeline.tasks.maximum` so l proposed to mitigate this issue with the same manner.

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

Yes. the maximum number of items shown on the timeline view is limited.
I proposed the default value 500 for jobs and stages, and 250 for executors.
A executor has at most 2 items (added and removed) 250 is chosen.

### How was this patch tested?

I manually confirm this change works with the following procedures.
```
# launch a cluster
$ bin/spark-shell --conf spark.ui.retainedDeadExecutors=300 --master "local-cluster[4, 1, 1024]"

// Confirm the maximum number of jobs
(1 to 1000).foreach { _ => sc.parallelize(List(1)).collect }

// Confirm the maximum number of stages
var df = sc.parallelize(1 to 2)
(1 to 1000).foreach { i =>  df = df.repartition(i % 5 + 1) }
df.collect

// Confirm the maximum number of executors
(1 to 300).foreach { _ => try sc.parallelize(List(1)).foreach { _ => System.exit(0) } catch { case e => }}
```

Screenshots here.
![jobs_limited](https://user-images.githubusercontent.com/4736016/116386937-3e8c4a00-a855-11eb-8f4c-151cf7ddd3b8.png)
![stages_limited](https://user-images.githubusercontent.com/4736016/116386990-49df7580-a855-11eb-9f71-8e129e3336ab.png)
![executors_limited](https://user-images.githubusercontent.com/4736016/116387009-4f3cc000-a855-11eb-8697-a2eb4c9c99e6.png)

Closes #32381 from sarutak/mitigate-timeline-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-11 20:53:11 +08:00
Yingyi Bu 7c9a9ec04f [SPARK-35146][SQL] Migrate to transformWithPruning or resolveWithPruning for rules in finishAnalysis.scala
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- BOOL_AGG
- COUNT_IF
- CURRENT_LIKE
- RUNTIME_REPLACEABLE

Added tree traversal pruning to the following rules:
- ReplaceExpressions
- RewriteNonCorrelatedExists
- ComputeCurrentTime
- GetCurrentDatabaseAndCatalog

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

Performance improvement (org.apache.spark.sql.TPCDSQuerySuite):
Rule name | Total Time (baseline) | Total Time (experiment) | experiment/baseline
ReplaceExpressions | 27546369 | 19753804 | 0.72
RewriteNonCorrelatedExists | 17304883 | 2086194 | 0.12
ComputeCurrentTime | 35751301 | 19984477 | 0.56
GetCurrentDatabaseAndCatalog | 37230787 | 18874013 | 0.51

### How was this patch tested?

Existing tests.

Closes #32461 from sigmod/finish_analysis.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-11 17:11:38 +08:00
Cheng Su c4ca23207b [SPARK-35363][SQL] Refactor sort merge join code-gen be agnostic to join type
### What changes were proposed in this pull request?

This is a pre-requisite of https://github.com/apache/spark/pull/32476, in discussion of https://github.com/apache/spark/pull/32476#issuecomment-836469779 . This is to refactor sort merge join code-gen to depend on streamed/buffered terminology, which makes the code-gen agnostic to different join types and can be extended to support other join types than inner join.

### Why are the changes needed?

Pre-requisite of https://github.com/apache/spark/pull/32476.

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

No.

### How was this patch tested?

Existing unit test in `InnerJoinSuite.scala` for inner join code-gen.

Closes #32495 from c21/smj-refactor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-11 11:21:59 +09:00
gengjiaan 44bd0a8bd3 [SPARK-35088][SQL][FOLLOWUP] Improve the error message for Sequence expression
### What changes were proposed in this pull request?
Sequence expression output a message looks confused.
This PR will fix the issue.

### Why are the changes needed?
Improve the error message for Sequence expression

### Does this PR introduce _any_ user-facing change?
Yes. this PR updates the error message of Sequence expression.

### How was this patch tested?
Tests updated.

Closes #32492 from beliefer/SPARK-35088-followup.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-11 09:45:09 +09:00
“attilapiros” 8b94eff1ca [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests
### What changes were proposed in this pull request?

This PR upgrades Kubernetes and Minikube version for integration tests and removes/updates the old code for this new version.

Details of this changes:

- As [discussed in the mailing list](http://apache-spark-developers-list.1001551.n3.nabble.com/minikube-and-kubernetes-cluster-versions-for-integration-testing-td30856.html): updating Minikube version from v0.34.1 to v1.7.3 and kubernetes version from v1.15.12 to v1.17.3.
- making Minikube version checked and fail with an explanation when the test is started with on a version <  v1.7.3.
- removing minikube status checking code related to old Minikube versions
- in the Minikube backend using fabric8's `Config.autoConfigure()` method to configure the kubernetes client to use the `minikube` k8s context (like it was in [one of the Minikube's example](https://github.com/fabric8io/kubernetes-client/blob/master/kubernetes-examples/src/main/java/io/fabric8/kubernetes/examples/kubectl/equivalents/ConfigUseContext.java#L36))
- Introducing `persistentVolume` test tag: this would be a temporary change to skip PVC tests in the Kubernetes integration test, as currently the PCV tests are blocking the move to Docker as Minikube's driver (for details please check https://issues.apache.org/jira/browse/SPARK-34738).

### Why are the changes needed?

With the current suggestion one can run into several problems without noticing the Minikube/kubernetes version is the problem.

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

No.

### How was this patch tested?

It was tested on Mac with [this script](https://gist.github.com/attilapiros/cd58a16bdde833c80c5803c337fffa94#file-check_minikube_versions-zsh) which installs each Minikube versions from v1.7.2 (including this version to test the negative case of the version check) and runs the integration tests.

It was started with:
```
./check_minikube_versions.zsh > test_log 2>&1
```

And there was only one build failure the rest was successful:

```
$ grep "BUILD SUCCESS" test_log | wc -l
      26
$ grep "BUILD FAILURE" test_log | wc -l
       1
```

It was for Minikube v1.7.2  and the log is:

```
KubernetesSuite:
*** RUN ABORTED ***
  java.lang.AssertionError: assertion failed: Unsupported Minikube version is detected: minikube version: v1.7.2.For integration testing Minikube version 1.7.3 or greater is expected.
  at scala.Predef$.assert(Predef.scala:223)
  at org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube$.getKubernetesClient(Minikube.scala:52)
  at org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend$.initialize(MinikubeTestBackend.scala:33)
  at org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.beforeAll(KubernetesSuite.scala:163)
  at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212)
  at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
  at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
  at org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.org$scalatest$BeforeAndAfter$$super$run(KubernetesSuite.scala:43)
  at org.scalatest.BeforeAndAfter.run(BeforeAndAfter.scala:273)
  at org.scalatest.BeforeAndAfter.run$(BeforeAndAfter.scala:271)
  ...
```

Moreover I made a test with having multiple k8s cluster contexts, too.

Closes #31829 from attilapiros/SPARK-34736.

Lead-authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Co-authored-by: attilapiros <piros.attila.zsolt@gmail.com>
Signed-off-by: attilapiros <piros.attila.zsolt@gmail.com>
2021-05-10 18:56:52 +02:00
Gengliang Wang d2a535f85b [SPARK-34246][FOLLOWUP] Change the definition of findTightestCommonType for backward compatibility
### What changes were proposed in this pull request?

Change the definition of `findTightestCommonType` from
```
def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType]
```
to
```
val findTightestCommonType: (DataType, DataType) => Option[DataType]
```

### Why are the changes needed?

For backward compatibility.
When running a MongoDB connector (built with Spark 3.1.1) with the latest master, there is such an error
```
java.lang.NoSuchMethodError: org.apache.spark.sql.catalyst.analysis.TypeCoercion$.findTightestCommonType()Lscala/Function2
```
from https://github.com/mongodb/mongo-spark/blob/master/src/main/scala/com/mongodb/spark/sql/MongoInferSchema.scala#L150

In the previous release, the function was
```
static public  scala.Function2<org.apache.spark.sql.types.DataType, org.apache.spark.sql.types.DataType, scala.Option<org.apache.spark.sql.types.DataType>> findTightestCommonType ()
```
After https://github.com/apache/spark/pull/31349, the function becomes:
```
static public  scala.Option<org.apache.spark.sql.types.DataType> findTightestCommonType (org.apache.spark.sql.types.DataType t1, org.apache.spark.sql.types.DataType t2)
```

This PR is to reduce the unnecessary API change.
### Does this PR introduce _any_ user-facing change?

Yes, the definition of `TypeCoercion.findTightestCommonType`  is consistent with previous release again.

### How was this patch tested?

Existing unit tests

Closes #32493 from gengliangwang/typecoercion.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-10 23:26:39 +08:00
Angerszhuuuu 7182f8cece [SPARK-35360][SQL] RepairTableCommand respects spark.sql.addPartitionInBatch.size too
### What changes were proposed in this pull request?
RepairTableCommand respects `spark.sql.addPartitionInBatch.size` too

### Why are the changes needed?
Make RepairTableCommand add partition batch size configurable.

### Does this PR introduce _any_ user-facing change?
User can use `spark.sql.addPartitionInBatch.size` to change batch size when repair table.

### How was this patch tested?
Not need

Closes #32489 from AngersZhuuuu/SPARK-35360.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-10 14:53:31 +05:00
Hyukjin Kwon d808956be4 [MINOR][INFRA] Add python/.idea into git ignore
### What changes were proposed in this pull request?

This PR adds `python/.idea` into Git ignore. PyCharm is supposed to be open against `python` directory which contains `pyspark` package as its root package.

This was caused by https://github.com/apache/spark/pull/32337.

### Why are the changes needed?

To ignore `.idea` file for PyCharm.

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

No, dev-only.

### How was this patch tested?

Manually tested by testing with `git` command.

Closes #32490 from HyukjinKwon/minor-python-gitignore.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-10 16:52:59 +09:00
Liang-Chi Hsieh 20d32242a2 [SPARK-35358][BUILD] Increase maximum Java heap used for release build to avoid OOM
### What changes were proposed in this pull request?

This patch proposes to increase the maximum heap memory setting for release build.

### Why are the changes needed?

When I was cutting RCs for 2.4.8, I frequently encountered OOM during building using mvn. It happens many times until I increased the heap memory setting.

I am not sure if other release managers encounter the same issue. So I propose to increase the heap memory setting and see if it looks good for others.

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

No, dev only.

### How was this patch tested?

Manually used it during cutting RCs of 2.4.8.

Closes #32487 from viirya/release-mvn-oom.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-10 00:29:05 -07:00
Chao Sun 245dce1ea1 [SPARK-35261][SQL][TESTS][FOLLOW-UP] Change failOnError to false for NativeAdd in V2FunctionBenchmark
### What changes were proposed in this pull request?

Change `failOnError` to false for `NativeAdd` in `V2FunctionBenchmark`.

### Why are the changes needed?

Since `NativeAdd` is simply doing addition on long it's better to set `failOnError` to false so it will use native long addition instead of `Math.addExact`.

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

No

### How was this patch tested?

N/A

Closes #32481 from sunchao/SPARK-35261-follow-up.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-10 07:20:05 +00:00
Angerszhuuuu 2c8ced9590 [SPARK-35111][SPARK-35112][SQL][FOLLOWUP] Rename ANSI interval patterns and regexps
### What changes were proposed in this pull request?
Rename pattern strings and regexps of year-month and day-time intervals.

### Why are the changes needed?
To improve code maintainability.

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

### How was this patch tested?
By existing test suites.

Closes #32444 from AngersZhuuuu/SPARK-35111-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-10 11:33:27 +05:00
Cheng Su 38eb5a6936 [SPARK-35354][SQL] Replace BaseJoinExec with ShuffledJoin in CoalesceBucketsInJoin
### What changes were proposed in this pull request?

As title. We should use a more restrictive interface `ShuffledJoin` other than `BaseJoinExec` in `CoalesceBucketsInJoin`, as the rule only applies to sort merge join and shuffled hash join (i.e. `ShuffledJoin`).

### Why are the changes needed?

Code cleanup.

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

No.

### How was this patch tested?

Existing unit test in `CoalesceBucketsInJoinSuite`.

Closes #32480 from c21/minor-cleanup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-10 10:04:49 +09:00
Ruifeng Zheng 620f0727e3 [SPARK-35231][SQL] logical.Range override maxRowsPerPartition
### What changes were proposed in this pull request?
when `numSlices` is avaiable, `logical.Range` should compute a exact `maxRowsPerPartition`

### Why are the changes needed?
`maxRowsPerPartition` is used in optimizer, we should provide an exact value if possible

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

### How was this patch tested?
existing testsuites

Closes #32350 from zhengruifeng/range_maxRowsPerPartition.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-09 21:44:49 +09:00
Liang-Chi Hsieh 5b65d8a129 [SPARK-35347][SQL] Use MethodUtils for looking up methods in Invoke and StaticInvoke
### What changes were proposed in this pull request?

This patch proposes to use `MethodUtils` for looking up methods `Invoke` and `StaticInvoke` expressions.

### Why are the changes needed?

Currently we wrote our logic in `Invoke` and `StaticInvoke` expressions for looking up methods. It is tricky to consider all the cases and there is already existing utility package for this purpose. We should reuse the utility package.

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

No, internal change only.

### How was this patch tested?

Existing tests.

Closes #32474 from viirya/invoke-util.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-08 15:17:30 -07:00
Dongjoon Hyun e31bef1ed4 Revert "[SPARK-35321][SQL] Don't register Hive permanent functions when creating Hive client"
This reverts commit b4ec9e2304.
2021-05-08 13:01:17 -07:00
Takeshi Yamamuro 06c40091a6 [SPARK-35327][SQL][TESTS] Filters out the TPC-DS queries that can cause flaky test results
### What changes were proposed in this pull request?

This PR proposes to filter out TPCDS v1.4 q6 and q75 in `TPCDSQueryTestSuite`.

I saw`TPCDSQueryTestSuite` failed nondeterministically because output row orders were different with those in the golden files. For example, the failure in the GA job, https://github.com/linhongliu-db/spark/runs/2507928605?check_suite_focus=true, happened because the `tpcds/q6.sql` query output rows were only sorted by `cnt`:

a0c76a8755/sql/core/src/test/resources/tpcds/q6.sql (L20)
Actually, `tpcds/q6.sql`  and `tpcds-v2.7.0/q6.sql` are almost the same and the only difference is that `tpcds-v2.7.0/q6.sql` sorts both `cnt` and `a.ca_state`:
a0c76a8755/sql/core/src/test/resources/tpcds-v2.7.0/q6.sql (L22)
So, I think it's okay just to test `tpcds-v2.7.0/q6.sql` in this case (q75 has the same issue).

### Why are the changes needed?

For stable testing.

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

No, dev-only.

### How was this patch tested?

GA passed.

Closes #32454 from maropu/CleanUpTpcdsQueries.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-08 21:43:39 +09:00
Kent Yao b0257801d5 [SPARK-35331][SQL] Support resolving missing attrs for distribute/cluster by/repartition hint
### What changes were proposed in this pull request?

This PR makes the below case work well.

```sql
select a b from values(1) t(a) distribute by a;
```

```logtalk
== Parsed Logical Plan ==
'RepartitionByExpression ['a]
+- 'Project ['a AS b#42]
   +- 'SubqueryAlias t
      +- 'UnresolvedInlineTable [a], [List(1)]

== Analyzed Logical Plan ==
org.apache.spark.sql.AnalysisException: cannot resolve 'a' given input columns: [b]; line 1 pos 62;
'RepartitionByExpression ['a]
+- Project [a#48 AS b#42]
   +- SubqueryAlias t
      +- LocalRelation [a#48]
```
### Why are the changes needed?

bugfix

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

yes, the original attributes can be used in `distribute by` / `cluster by` and hints like `/*+ REPARTITION(3, c) */`

### How was this patch tested?

new tests

Closes #32465 from yaooqinn/SPARK-35331.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-08 05:00:51 -07:00
Chao Sun 323a6e848e [SPARK-35232][SQL] Nested column pruning should retain column metadata
### What changes were proposed in this pull request?

Retain column metadata during the process of nested column pruning, when constructing `StructField`.

To test the above change, this also added the logic of column projection in `InMemoryTable`. Without the fix `DSV2CharVarcharDDLTestSuite` will fail.

### Why are the changes needed?

The column metadata is used in a few places such as re-constructing CHAR/VARCHAR information such as in [SPARK-33901](https://issues.apache.org/jira/browse/SPARK-33901). Therefore, we should retain the info during nested column pruning.

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

No

### How was this patch tested?

Existing tests.

Closes #32354 from sunchao/SPARK-35232.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-07 22:37:54 -07:00
Chao Sun f47e0f8379 [SPARK-35261][SQL] Support static magic method for stateless Java ScalarFunction
### What changes were proposed in this pull request?

This allows `ScalarFunction` implemented in Java to optionally specify the magic method `invoke` to be static, which can be used if the UDF is stateless. Comparing to the non-static method, it can potentially give better performance due to elimination of dynamic dispatch, etc.

Also added a benchmark to measure performance of: the default `produceResult`, non-static magic method and static magic method.

### Why are the changes needed?

For UDFs that are stateless (e.g., no need to maintain intermediate state between each function call), it's better to allow users to implement the UDF function as static method which could potentially give better performance.

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

Yes. Spark users can now have the choice to define static magic method for `ScalarFunction` when it is written in Java and when the UDF is stateless.

### How was this patch tested?

Added new UT.

Closes #32407 from sunchao/SPARK-35261.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-07 20:34:51 -07:00
Chao Sun b4ec9e2304 [SPARK-35321][SQL] Don't register Hive permanent functions when creating Hive client
### What changes were proposed in this pull request?

Instantiate a new Hive client through `Hive.getWithFastCheck(conf, false)` instead of `Hive.get(conf)`.

### Why are the changes needed?

[HIVE-10319](https://issues.apache.org/jira/browse/HIVE-10319) introduced a new API `get_all_functions` which is only supported in Hive 1.3.0/2.0.0 and up. As result, when Spark 3.x talks to a HMS service of version 1.2 or lower, the following error will occur:
```
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.hadoop.hive.ql.metadata.Hive.getAllFunctions(Hive.java:3897)
        at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:248)
        at org.apache.hadoop.hive.ql.metadata.Hive.registerAllFunctionsOnce(Hive.java:231)
        ... 96 more
Caused by: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_all_functions(ThriftHiveMetastore.java:3845)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_all_functions(ThriftHiveMetastore.java:3833)
```

The `get_all_functions` is called only when `doRegisterAllFns` is set to true:
```java
  private Hive(HiveConf c, boolean doRegisterAllFns) throws HiveException {
    conf = c;
    if (doRegisterAllFns) {
      registerAllFunctionsOnce();
    }
  }
```

what this does is to register all Hive permanent functions defined in HMS in Hive's `FunctionRegistry` class, via iterating through results from `get_all_functions`. To Spark, this seems unnecessary as it loads Hive permanent (not built-in) UDF via directly calling the HMS API, i.e., `get_function`. The `FunctionRegistry` is only used in loading Hive's built-in function that is not supported by Spark. At this time, it only applies to `histogram_numeric`.

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

Yes with this fix Spark now should be able to talk to HMS server with Hive 1.2.x and lower (with HIVE-24608 too)

### How was this patch tested?

Manually started a HMS server of Hive version 1.2.2, with patched Hive 2.3.8 using HIVE-24608. Without the PR it failed with the above exception. With the PR the error disappeared and I can successfully perform common operations such as create table, create database, list tables, etc.

Closes #32446 from sunchao/SPARK-35321.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-07 15:06:04 -07:00
Liang-Chi Hsieh 33fbf5647b [SPARK-35288][SQL] StaticInvoke should find the method without exact argument classes match
### What changes were proposed in this pull request?

This patch proposes to make StaticInvoke able to find method with given method name even the parameter types do not exactly match to argument classes.

### Why are the changes needed?

Unlike `Invoke`, `StaticInvoke` only tries to get the method with exact argument classes. If the calling method's parameter types are not exactly matched with the argument classes, `StaticInvoke` cannot find the method.

`StaticInvoke` should be able to find the method under the cases too.

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

Yes. `StaticInvoke` can find a method even the argument classes are not exactly matched.

### How was this patch tested?

Unit test.

Closes #32413 from viirya/static-invoke.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-07 09:07:57 -07:00
RoryQi 6f0ef93f9a [SPARK-35297][CORE][DOC][MINOR] Modify the comment about the executor
### What changes were proposed in this pull request?
Now Spark Executor already can be used in Kubernetes scheduler. So we should modify the annotation in the Executor.scala.

### Why are the changes needed?
only comment

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

### How was this patch tested?
no

Closes #32426 from jerqi/master.

Authored-by: RoryQi <1242949407@qq.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-08 00:03:02 +09:00
Kousuke Saruta 2634dbac35 [SPARK-35175][BUILD] Add linter for JavaScript source files
### What changes were proposed in this pull request?

This PR proposes to add linter for JavaScript source files.
[ESLint](https://eslint.org/) seems to be a popular linter for JavaScript so I choose it.

### Why are the changes needed?

Linter enables us to check style and keeps code clean.

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

No.

### How was this patch tested?

Manually run `dev/lint-js` (Node.js and npm are required).

In this PR, mainly indentation style is also fixed an linter passes.

Closes #32274 from sarutak/introduce-eslint.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-07 21:55:08 +09:00
beliefer d3b92eec45 [SPARK-35021][SQL] Group exception messages in connector/catalog
### What changes were proposed in this pull request?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32377 from beliefer/SPARK-35021.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 10:54:43 +00:00
Yingyi Bu 72d32662d4 [SPARK-35144][SQL] Migrate to transformWithPruning for object rules
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- APPEND_COLUMNS
- DESERIALIZE_TO_OBJECT
- LAMBDA_VARIABLE
- MAP_OBJECTS
- SERIALIZE_FROM_OBJECT
- PROJECT
- TYPED_FILTER

Added tree traversal pruning to the following rules dealing with objects:
- EliminateSerialization
- CombineTypedFilters
- EliminateMapObjects
- ObjectSerializerPruning

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

### How was this patch tested?

Existing tests.

Closes #32451 from sigmod/object.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-07 18:36:28 +08:00
Wenchen Fan 9aa18dfe19 [SPARK-35333][SQL] Skip object null check in Invoke if possible
### What changes were proposed in this pull request?

If `targetObject` is not nullable, we don't need the object null check in `Invoke`.

### Why are the changes needed?

small perf improvement

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

no

### How was this patch tested?

existing tests

Closes #32466 from cloud-fan/invoke.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 10:27:28 +00:00
gengjiaan cf2c4ba584 [SPARK-35020][SQL] Group exception messages in catalyst/util
### What changes were proposed in this pull request?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32367 from beliefer/SPARK-35020.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 08:30:30 +00:00
Wenchen Fan e83910f1f8 [SPARK-26164][SQL][FOLLOWUP] WriteTaskStatsTracker should know which file the row is written to
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/32198

Before https://github.com/apache/spark/pull/32198, in `WriteTaskStatsTracker.newRow`, we know that the row is written to the current file. After https://github.com/apache/spark/pull/32198 , we no longer know this connection.

This PR adds the file path parameter in `WriteTaskStatsTracker.newRow` to bring back the connection.

### Why are the changes needed?

To not break some custom `WriteTaskStatsTracker` implementations.

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

no

### How was this patch tested?

N/A

Closes #32459 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 08:28:42 +00:00
Terry Kim 33c1034315 [SPARK-34701][SQL][FOLLOW-UP] Children/innerChildren should be mutually exclusive for AnalysisOnlyCommand
### What changes were proposed in this pull request?

This is a follow up to https://github.com/apache/spark/pull/32032#discussion_r620928086. Basically, `children`/`innerChildren` should be mutually exclusive for `AlterViewAsCommand` and `CreateViewCommand`, which extend `AnalysisOnlyCommand`. Otherwise, there could be an issue in the `EXPLAIN` command. Currently, this is not an issue, because these commands will be analyzed (children will always be empty) when the `EXPLAIN` command is run.

### Why are the changes needed?

To be future-proof where these commands are directly used.

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

No.

### How was this patch tested?

Added new tsts

Closes #32447 from imback82/SPARK-34701-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 06:07:53 +00:00