Commit graph

25338 commits

Author SHA1 Message Date
s71955 ee66890f30 [SPARK-28084][SQL] Resolving the partition column name based on the resolver in sql load command
### What changes were proposed in this pull request?

LOAD DATA command resolves the partition column name as case sensitive manner,
where as in insert commandthe partition column name will be resolved using
the SQLConf resolver where the names will be resolved based on `spark.sql.caseSensitive` property. Same logic can be applied for resolving the partition column names in LOAD COMMAND.

### Why are the changes needed?

It's to handle the partition column name correctly according to the configuration.

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

No.

### How was this patch tested?

Existing UT and manual testing.

Closes #24903 from sujith71955/master_paritionColName.

Lead-authored-by: s71955 <sujithchacko.2010@gmail.com>
Co-authored-by: sujith71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-03 01:11:48 -07:00
HyukjinKwon 40485f4656 [SPARK-29317][SQL][PYTHON] Avoid inheritance hierarchy in pandas CoGroup arrow runner and its plan
### What changes were proposed in this pull request?

This PR proposes to avoid abstract classes introduced at https://github.com/apache/spark/pull/24965 but instead uses trait and object.

- `abstract class BaseArrowPythonRunner` -> `trait PythonArrowOutput` to allow mix-in

    **Before:**

    ```
    BasePythonRunner
    ├── BaseArrowPythonRunner
    │   ├── ArrowPythonRunner
    │   └── CoGroupedArrowPythonRunner
    ├── PythonRunner
    └── PythonUDFRunner
    ```

    **After:**

    ```
    └── BasePythonRunner
        ├── ArrowPythonRunner
        ├── CoGroupedArrowPythonRunner
        ├── PythonRunner
        └── PythonUDFRunner
    ```
- `abstract class BasePandasGroupExec ` -> `object PandasGroupUtils` to decouple

    **Before:**

    ```
    └── BasePandasGroupExec
        ├── FlatMapGroupsInPandasExec
        └── FlatMapCoGroupsInPandasExec
    ```

    **After:**

    ```
    ├── FlatMapGroupsInPandasExec
    └── FlatMapCoGroupsInPandasExec
    ```

### Why are the changes needed?

The problem is that R code path is being matched with Python side:

**Python:**

```
└── BasePythonRunner
    ├── ArrowPythonRunner
    ├── CoGroupedArrowPythonRunner
    ├── PythonRunner
    └── PythonUDFRunner
```

**R:**

```
└── BaseRRunner
    ├── ArrowRRunner
    └── RRunner
```

I would like to match the hierarchy and decouple other stuff for now if possible. Ideally we should deduplicate both code paths. Internal implementation is also similar intentionally.

`BasePandasGroupExec` case is similar as well. R (with Arrow optimization, in particular) has some duplicated codes with Pandas UDFs.

`FlatMapGroupsInRWithArrowExec` <> `FlatMapGroupsInPandasExec`
`MapPartitionsInRWithArrowExec` <> `ArrowEvalPythonExec`

In order to prepare deduplication here as well, it might better avoid changing hierarchy alone in Python side.

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

No.

### How was this patch tested?

Locally tested existing tests. Jenkins tests should verify this too.

Closes #25989 from HyukjinKwon/SPARK-29317.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-03 16:42:37 +09:00
angerszhu 178a1f3558 [SPARK-29305][BUILD] Update LICENSE and NOTICE for Hadoop 3.2
### What changes were proposed in this pull request?
This PR update LICENSE and NOTICE for Hadoop 3.2. Hadoop 3.2 newly added jars:

```
com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.9.5
com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.9.5
com.fasterxml.woodstox:woodstox-core:5.0.3
com.github.stephenc.jcip:jcip-annotations:1.0-1
com.google.re2j:re2j:1.1
com.microsoft.sqlserver:mssql-jdbc:6.2.1.jre7
com.nimbusds:nimbus-jose-jwt:4.41.1
dnsjava:dnsjava:2.1.7
net.minidev:accessors-smart:1.2
net.minidev:json-smart:2.3
org.apache.commons:commons-configuration2:2.1.1
org.apache.geronimo.specs:geronimo-jcache_1.0_spec:1.0-alpha-1
org.apache.hadoop:hadoop-hdfs-client:3.2.0
org.apache.kerby:kerb-admin:1.0.1
org.apache.kerby:kerb-client:1.0.1
org.apache.kerby:kerb-common:1.0.1
org.apache.kerby:kerb-core:1.0.1
org.apache.kerby:kerb-crypto:1.0.1
org.apache.kerby:kerb-identity:1.0.1
org.apache.kerby:kerb-server:1.0.1
org.apache.kerby:kerb-simplekdc:1.0.1
org.apache.kerby:kerb-util:1.0.1
org.apache.kerby:kerby-asn1:1.0.1
org.apache.kerby:kerby-config:1.0.1
org.apache.kerby:kerby-pkix:1.0.1
org.apache.kerby:kerby-util:1.0.1
org.apache.kerby:kerby-xdr:1.0.1
org.apache.kerby:token-provider:1.0.1
org.codehaus.woodstox:stax2-api:3.1.4
org.ehcache:ehcache:3.3.1
```
### Why are the changes needed?
We will distribute a binary release based on Hadoop 3.2 / Hive 2.3 in future.

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

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

Closes #25978 from AngersZhuuuu/SPARK-29035.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-03 01:02:41 -05:00
Jungtaek Lim (HeartSaVioR) e44d191bf7 [SPARK-29322][CORE] Enable closeFrameOnFlush on ZstdOutputStream for event log file
### What changes were proposed in this pull request?

This patch proposes to enable `closeFrameOnFlush` for ZstdOutputStream specific to event logger, so that continuous input stream of zstd is not stuck when reading "inprogress" event log file.

The issue seems to be introduced from [SPARK-26283](https://issues.apache.org/jira/browse/SPARK-26283) which addressed some bug via reading event log file with enabling continuous mode, but it changed the behavior of input stream to read open frame, which seem to wait for frame to be closed. Enabling `closeFrameOnFlush` would close frame whenever flush is called, so input stream could read the frame sooner.

As a pair of `compressedContinuousInputStream`, this patch adds `compressedContinuousOutputStream` which will be only used for event logging.

### Why are the changes needed?

Without this patch, the reader thread in SHS is stuck on reading inprogress event log file compressed with zstd until the application becomes finished.

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

It might bring some overhead on each flush when writing zstd compressed event log, so some sort of performance hit could be introduced. I've restricted the case to only event logging.

### How was this patch tested?

Manually tested, via setting Spark configuration as below:

```
spark.eventLog.enabled                     true
spark.eventLog.compress                  true
spark.eventLog.compression.codec zstd
```

and start Spark application. While the application is running, load the application in SHS webpage.

Before this patch, it may succeed to replay the event log, but high likely it will be stuck and loading page will be also stuck. After this patch, SHS can properly reads the inprogress event log file.

Closes #25996 from HeartSaVioR/SPARK-29322.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-02 20:48:38 -07:00
Henry D 51d6ba7490 [SPARK-28962][SQL] Provide index argument to filter lambda functions
### What changes were proposed in this pull request?

Lambda functions to array `filter` can now take as input the index as well as the element. This behavior matches array `transform`.

### Why are the changes needed?
See JIRA. It's generally useful, and particularly so if you're working with fixed length arrays.

### Does this PR introduce any user-facing change?
Previously filter lambdas had to look like
`filter(arr, el -> whatever)`

Now, lambdas can take an index argument as well
`filter(array, (el, idx) -> whatever)`

### How was this patch tested?
I added unit tests to `HigherOrderFunctionsSuite`.

Closes #25666 from henrydavidge/filter-idx.

Authored-by: Henry D <henrydavidge@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2019-10-02 13:03:06 -07:00
Nik Vanderhoof 730a17823f [SPARK-27297][SQL] Add higher order functions to scala API
## What changes were proposed in this pull request?

There is currently no existing Scala API equivalent for the higher order functions introduced in Spark 2.4.0.
 * transform
 * aggregate
 * filter
 * exists
 * forall
 * zip_with
 * map_zip_with
 * map_filter
 * transform_values
 * transform_keys

Equivalent column based functions should be added to the Scala API for org.apache.spark.sql.functions with the following signatures:

 
```scala
def transform(column: Column, f: Column => Column): Column = ???

def transform(column: Column, f: (Column, Column) => Column): Column = ???

def exists(column: Column, f: Column => Column): Column = ???

def filter(column: Column, f: Column => Column): Column = ???

def aggregate(
expr: Column,
zero: Column,
merge: (Column, Column) => Column,
finish: Column => Column): Column = ???

def aggregate(
expr: Column,
zero: Column,
merge: (Column, Column) => Column): Column = ???

def zip_with(
left: Column,
right: Column,
f: (Column, Column) => Column): Column = ???

def transform_keys(expr: Column, f: (Column, Column) => Column): Column = ???

def transform_values(expr: Column, f: (Column, Column) => Column): Column = ???

def map_filter(expr: Column, f: (Column, Column) => Column): Column = ???

def map_zip_with(left: Column, right: Column, f: (Column, Column, Column) => Column): Column = ???
```

## How was this patch tested?

I've mimicked the existing tests for the higher order functions in `org.apache.spark.sql.DataFrameFunctionsSuite` that use `expr` to test the higher order functions.

As an example of an existing test:
```scala
  test("map_zip_with function - map of primitive types") {
    val df = Seq(
      (Map(8 -> 6L, 3 -> 5L, 6 -> 2L), Map[Integer, Integer]((6, 4), (8, 2), (3, 2))),
      (Map(10 -> 6L, 8 -> 3L), Map[Integer, Integer]((8, 4), (4, null))),
      (Map.empty[Int, Long], Map[Integer, Integer]((5, 1))),
      (Map(5 -> 1L), null)
    ).toDF("m1", "m2")

    checkAnswer(df.selectExpr("map_zip_with(m1, m2, (k, v1, v2) -> k == v1 + v2)"),
      Seq(
        Row(Map(8 -> true, 3 -> false, 6 -> true)),
        Row(Map(10 -> null, 8 -> false, 4 -> null)),
        Row(Map(5 -> null)),
        Row(null)))
}
```

I've added this test that performs the same logic, but with the new column based API I've added.
```scala
    checkAnswer(df.select(map_zip_with(df("m1"), df("m2"), (k, v1, v2) => k === v1 + v2)),
      Seq(
        Row(Map(8 -> true, 3 -> false, 6 -> true)),
        Row(Map(10 -> null, 8 -> false, 4 -> null)),
        Row(Map(5 -> null)),
        Row(null)))
```

Closes #24232 from nvander1/feature/add_higher_order_functions_to_scala_api.

Lead-authored-by: Nik Vanderhoof <nikolasrvanderhoof@gmail.com>
Co-authored-by: Nik <nikolasrvanderhoof@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2019-10-02 12:53:39 -07:00
Dongjoon Hyun 9a84fae216 [SPARK-29332][BUILD] Update zstd-jni to 1.4.3-1
### What changes were proposed in this pull request?

This PR aims to update zstd-jni library to 1.4.3-1.

### Why are the changes needed?

This will bring the latest bug fixes in zstd itself. This is independent from another on-going Spark fix.
- https://github.com/facebook/zstd/releases/tag/v1.4.3

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

No.

### How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #26002 from dongjoon-hyun/SPARK-29332.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-02 11:37:02 -07:00
huangweiyi 85dafabeb4 [SPARK-29273][CORE] Save peakExecutionMemory value when writing task end to event log
in TaskMetrics, there is a exposed metrics peakExecutionMemory, but the value never been set. when a task is finished, it generate a SparkListenerTaskEnd event info, incuding the metrics value. actually the peakExecutionMemory is stored in the Accumulables which is a member of TaskInfo.

so when parse the SparkListenerTaskEnd event, we can get the `internal.metrics.peakExecutionMemory` value from the parsed taskInfo object, and set back to the TaskMetricsInfo with supply a setPeakExecutionMemory method.

Closes #25949 from 012huang/fix-peakExecutionMemory-metrics-value.

Lead-authored-by: huangweiyi <huangweiyi_2006@qq.com>
Co-authored-by: hwy <huangweiyi_2006@qq.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-02 08:45:12 -07:00
Terry Kim f2ead4d0b5 [SPARK-28970][SQL] Implement USE CATALOG/NAMESPACE for Data Source V2
### What changes were proposed in this pull request?
This PR exposes USE CATALOG/USE SQL commands as described in this [SPIP](https://docs.google.com/document/d/1jEcvomPiTc5GtB9F7d2RTVVpMY64Qy7INCA_rFEd9HQ/edit#)

It also exposes `currentCatalog` in `CatalogManager`.

Finally, it changes `SHOW NAMESPACES` and `SHOW TABLES` to use the current catalog if no catalog is specified (instead of default catalog).

### Why are the changes needed?
There is currently no mechanism to change current catalog/namespace thru SQL commands.

### Does this PR introduce any user-facing change?
Yes, you can perform the following:
```scala
// Sets the current catalog to 'testcat'
spark.sql("USE CATALOG testcat")

// Sets the current catalog to 'testcat' and current namespace to 'ns1.ns2'.
spark.sql("USE ns1.ns2 IN testcat")

// Now, the following will use 'testcat' as the current catalog and 'ns1.ns2' as the current namespace.
spark.sql("SHOW NAMESPACES")
```

### How was this patch tested?
Added new unit tests.

Closes #25771 from imback82/use_namespace.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-02 21:55:21 +08:00
Maxim Gekk 3b1674cb1f [SPARK-29313][SQL] Fix failure on writing to noop in benchmarks
### What changes were proposed in this pull request?
In the PR, I propose to specify the save mode explicitly while writing to the `noop` datasource in benchmarks. I set `Overwrite` mode in the following benchmarks:
- JsonBenchmark
- CSVBenchmark
- UDFBenchmark
- MakeDateTimeBenchmark
- ExtractBenchmark
- DateTimeBenchmark
- NestedSchemaPruningBenchmark

### Why are the changes needed?
Otherwise writing to `noop` fails with:
```
[error] Exception in thread "main" org.apache.spark.sql.AnalysisException: TableProvider implementation noop cannot be written with ErrorIfExists mode, please use Append or Overwrite modes instead.;
[error] 	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:284)
```
most likely due to https://github.com/apache/spark/pull/25876

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

### How was this patch tested?
I generated results of `ExtractBenchmark` via the command:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.ExtractBenchmark"
```

Closes #25988 from MaxGekk/noop-overwrite-mode.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-01 21:04:56 -07:00
Josh Rosen c6938eab57 [SPARK-29310][CORE][TESTS] TestMemoryManager should implement getExecutionMemoryUsageForTask()
### What changes were proposed in this pull request?

This PR updates `TestMemoryManager`, a class used only in unit tests, to override the `getExecutionMemoryUsageForTask()` and `releaseAllExecutionMemoryForTask()` methods. I also `synchronized` its state-accessing methods (to make the class thread-safe) and added some additional assertions to guard against freeing memory memory than has been allocated.

### Why are the changes needed?

Spark uses a `TestMemoryManager` class to mock out memory manager functionality in tests, allowing test authors to exercise control over certain behaviors (e.g. to simulate OOMs).

Several of Spark's test suites have memory-leak detection to ensure that all allocated memory is cleaned up at the end of each test case; this helps to guard against bugs that could cause production memory leaks. For example, see `testWithMemoryLeakDetection` in `UnsafeFixedWidthAggregationMapSuite`.

Unfortunately, however, this leak-detection logic is broken for tests which use TestMemoryManager because it does not override the `getExecutionMemoryUsageForTask()` method that is used by the leak-detection checks.

This PR fixes that problem, thereby strengthening our existing tests.

I spotted this problem while reviewing #25953: I tried introducing a change to remove a `freePage()` call (purposely inducing a memory leak) but no tests failed.

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

No.

### How was this patch tested?

Added a new `TestMemoryManagerSuite`, with tests covering `TestMemoryManager` itself.

Ran a subset of existing tests on my laptop and uncovered a bug in one test's `free()` calls, plus missing cleanup calls in another test suite; both of these issues are fixed in this PR.

Closes #25985 from JoshRosen/SPARK-29310-testmemorymanager-getExecutionMemoryUsageForTask.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: joshrosen-stripe <48632449+joshrosen-stripe@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-02 11:07:19 +08:00
Sean Owen 2ec3265ae7 [MINOR][BUILD] Decode output of commands during merge script as UTF-8 consistently
### What changes were proposed in this pull request?

In the PR merge script, decode the raw output of subprocess commands like `git` using UTF-8 encoding, consistently.

### Why are the changes needed?

The merge script occasionally fails if run with Python 2 and the output of a command like `git` contains non-ASCII characters. I think this most usually happens when a user name, for example, contains Chinese characters.

This is because the output is decoded according to `sys.getdefaultencoding()`, which is ASCII in Python 2. It's UTF-8 in Python 3, by default.

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

No.

### How was this patch tested?

The change caused a merge that failed before to succeed.

Closes #25991 from srowen/MergePRUTF8.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-02 11:28:55 +09:00
Maxim Gekk e13880128d [SPARK-29311][SQL] Return seconds with fraction from date_part() and extract
### What changes were proposed in this pull request?

Added new expression `SecondWithFraction` which produces the `seconds` part of timestamps/dates with fractional part containing microseconds. This expression is used only in the `DatePart` expression. As the result, `date_part()` and `extract` return seconds and microseconds as the fractional part of the seconds part when `field` is `SECOND` (or synonyms).

### Why are the changes needed?

The `date_part()` and `extract` were added to maintain feature parity with PostgreSQL which has different behavior for the `SECOND` value of the `field` parameter. The fix is needed to behave in the same way. Here is PostgreSQL's output:
```sql
# SELECT date_part('SECONDS', timestamp'2019-10-01 00:00:01.000001');
 date_part
-----------
  1.000001
(1 row)
```

### Does this PR introduce any user-facing change?
Yes, type of `date_part('SECOND', ...)` is changed from `INT` to `DECIMAL(8, 6)`.
Before:
```sql
spark-sql> SELECT date_part('SECONDS', '2019-10-01 00:00:01.000001');
1
```
After:
```sql
spark-sql> SELECT date_part('SECONDS', '2019-10-01 00:00:01.000001');
1.000001
```

### How was this patch tested?
- Added new tests to `DateExpressionSuite` for the `SecondWithFraction` expression
- Regenerated results of `date_part.sql`, `extract.sql` and `timestamp.sql`
- Updated results of `ExtractBenchmark`

Closes #25986 from MaxGekk/extract-seconds-from-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-02 11:16:31 +09:00
Liang-Chi Hsieh 0cd436be66 [SPARK-29244][CORE] Prevent freed page in BytesToBytesMap free again
### What changes were proposed in this pull request?

When BytesToBytesMap cannot allocate a page, allocated page was freed by TaskMemoryManager. In this case, we should not keep it in longArray field of BytesToBytesMap. Otherwise it might be freed again in task completion listener of UnsafeFixedWidthAggregationMap and cause confusing error.

Note that because we catch Throwable when invoking completion listeners, this error should not affect other listeners, except for the current listener. In the completion listener of UnsafeFixedWidthAggregationMap, it only performs BytesToBytesMap.free().

BytesToBytesMap.free() does two things: freeing allocated pages and deleting spilled files. When it tries to free a freed page, this error hits and skips remaining pages (Executor.cleanUpAllAllocatedMemory will guard memory leak) and spilled files.

### Why are the changes needed?

By chance, it is possibly that we free an already freed page in BytesToBytesMap. Because we have some guards when freeing a page, a confusing error would be hit:

```
16:07:33.550 ERROR org.apache.spark.TaskContextImpl: Error in TaskCompletionListener
java.lang.AssertionError: Called freePage() on a memory block that has already been freed
        at org.apache.spark.memory.TaskMemoryManager.freePage(TaskMemoryManager.java:332)
        at org.apache.spark.memory.MemoryConsumer.freePage(MemoryConsumer.java:129)
        at org.apache.spark.memory.MemoryConsumer.freeArray(MemoryConsumer.java:107)
        at org.apache.spark.unsafe.map.BytesToBytesMap.free(BytesToBytesMap.java:806)
        at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.free(UnsafeFixedWidthAggregationMap.java:226)
        at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.lambda$new$0(UnsafeFixedWidthAggregationMap.java:112)
        at org.apache.spark.TaskContextImpl.$anonfun$markTaskCompleted$1(TaskContextImpl.scala:119)
        at org.apache.spark.TaskContextImpl.$anonfun$markTaskCompleted$1$adapted(TaskContextImpl.scala:119)
        at org.apache.spark.TaskContextImpl.$anonfun$invokeListeners$1(TaskContextImpl.scala:132)
        at org.apache.spark.TaskContextImpl.$anonfun$invokeListeners$1$adapted(TaskContextImpl.scala:130)
        at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
        at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
        at org.apache.spark.TaskContextImpl.invokeListeners(TaskContextImpl.scala:130)
        at org.apache.spark.TaskContextImpl.markTaskCompleted(TaskContextImpl.scala:119)
        at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMapSuite.$anonfun$new$19(UnsafeFixedWidthAggregationMapSuite.scala:425)
        at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMapSuite.$anonfun$testWithMemoryLeakDetection$1(UnsafeFixedWidthAggregationMapSuite.scala:87)
        at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
        at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
        at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
        at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
        at org.scalatest.Transformer.apply(Transformer.scala:22)
        at org.scalatest.Transformer.apply(Transformer.scala:20)
        at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
        at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:149)
        at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
        at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196)
        at org.scalatest.SuperEngine.runTestImpl(Engine.scala:289)
        at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196)
        at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178)
        at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:56)
        at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221)
        at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214)
        at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:56)
        at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229)
        at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:396)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:384)
        at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:379)
        at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:461)
        at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229)
        at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228)
        at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
        at org.scalatest.Suite.run(Suite.scala:1147)
        at org.scalatest.Suite.run$(Suite.scala:1129)
        at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
        at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233)
        at org.scalatest.SuperEngine.runImpl(Engine.scala:521)
        at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233)
        at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232)
        at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:56)
        at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
        at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
        at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
        at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:56)
        at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:314)
        at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:507)
        at sbt.ForkMain$Run$2.call(ForkMain.java:296)
        at sbt.ForkMain$Run$2.call(ForkMain.java:286)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
[info] - SPARK-29244 *** FAILED *** (16 milliseconds)
[info]   org.apache.spark.util.TaskCompletionListenerException: Called freePage() on a memory block that has already been freed
[info]
[info] Previous exception in task: Unable to acquire 4096 bytes of memory, got 0
[info]  org.apache.spark.memory.MemoryConsumer.throwOom(MemoryConsumer.java:157)
[info]  org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:97)
[info]  org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:790)
[info]  org.apache.spark.unsafe.map.BytesToBytesMap.reset(BytesToBytesMap.java:893)
[info]  org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:170)
[info]  org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:249)
[info]  org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMapSuite.$anonfun$new$19(UnsafeFixedWidthAggregationMapSuite.scala:421)
[info]  org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMapSuite.$anonfun$testWithMemoryLeakDetection$1(UnsafeFixedWidthAggregationMapSuite.scala:87)
```

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

No

### How was this patch tested?

Added unit test.

Closes #25953 from viirya/SPARK-29244.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2019-10-01 11:50:31 -07:00
Jungtaek Lim (HeartSaVioR) a4601cb44e [SPARK-29055][CORE] Update driver/executors' storage memory when block is removed from BlockManager
### What changes were proposed in this pull request?

This patch proposes to fix the issue that storage memory is not decreasing even block is removed in BlockManager. Originally the issue is found while removed broadcast doesn't reflect the storage memory on driver/executors.

AppStatusListener expects the value of memory in events on block update as "delta" so that it adjusts driver/executors' storage memory based on delta, but when removing block BlockManager reports the delta as 0, so the storage memory is not decreased. `BlockManager.dropFromMemory` deals with this correctly, so some of path of freeing memory has been updated correctly.

### Why are the changes needed?

The storage memory in metrics in AppStatusListener is now out of sync which lets end users easy to confuse as memory leak is happening.

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

No.

### How was this patch tested?

Modified UTs. Also manually tested via running simple query repeatedly and observe executor page of Spark UI to see the value of storage memory is decreasing as well.

Please refer the description of [SPARK-29055](https://issues.apache.org/jira/browse/SPARK-29055) to get simple reproducer.

Closes #25973 from HeartSaVioR/SPARK-29055.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-01 09:41:51 -07:00
angerszhu 0cf2f48dfe [SPARK-29022][SQL] Fix SparkSQLCLI can not add jars by AddJarCommand
### What changes were proposed in this pull request?
For issue mentioned in [SPARK-29022](https://issues.apache.org/jira/browse/SPARK-29022)
Spark SQL CLI can't use class as serde class in jars add by SQL `ADD JAR`.

When we create table with `serde` class contains by jar added by SQL 'ADD JAR'.
We can create table with `serde` class construct success since we call `HiveClientImpl.createTable` under `withHiveState` method, it will add `clientLoader.classLoader` to `HiveClientImpl.state.getConf.classLoader`.

Jars added by SQL `ADD JAR` will be add to
1. `sparkSession.sharedState.jarClassLoader`.
2. 'HiveClientLoader.clientLoader.classLoader'

In Current spark-sql MODE,  `HiveClientImpl.state`  will use CliSessionState created when initialize
SparkSQLCliDriver,  When we select data from table, it will check `serde` class, when call method `HiveTableScanExec#addColumnMetadataToConf()` to check for table desc serde class.

```
val deserializer = tableDesc.getDeserializerClass.getConstructor().newInstance()
    deserializer.initialize(hiveConf, tableDesc.getProperties)
```

`getDeserializer` will  use CliSessionState's hiveConf's classLoader in `Spark SQL CLI` mode.
But when we call `ADD JAR` in spark, the jar won't be added to `Classloader of  CliSessionState' conf `, then `ClassNotFound` error happen.

So we reset `CliSessionState conf's classLoader ` to `sharedState.jarClassLoader` when `sharedState.jarClassLoader` has added jar passed by `HIVEAUXJARS`
Then when we use  `ADD JAR `  to add jar,  jar path will be added to CliSessionState's conf's ClassLoader

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
ADD UT

Closes #25729 from AngersZhuuuu/SPARK-29015.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-01 10:09:29 -05:00
Dongjoon Hyun bd031c2173 [SPARK-29307][BUILD][TESTS] Remove scalatest deprecation warnings
### What changes were proposed in this pull request?

This PR aims to remove `scalatest` deprecation warnings with the following changes.
- `org.scalatest.mockito.MockitoSugar` -> `org.scalatestplus.mockito.MockitoSugar`
- `org.scalatest.selenium.WebBrowser` -> `org.scalatestplus.selenium.WebBrowser`
- `org.scalatest.prop.Checkers` -> `org.scalatestplus.scalacheck.Checkers`
- `org.scalatest.prop.GeneratorDrivenPropertyChecks` -> `org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks`

### Why are the changes needed?

According to the Jenkins logs, there are 118 warnings about this.
```
 grep "is deprecated" ~/consoleText | grep scalatest | wc -l
     118
```

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

No.

### How was this patch tested?

After Jenkins passes, we need to check the Jenkins log.

Closes #25982 from dongjoon-hyun/SPARK-29307.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-30 21:00:11 -07:00
Jeff Evans d841b33ba3 [SPARK-25153][SQL] Improve error messages for columns with dots/periods
### What changes were proposed in this pull request?

Check schema fields to see if they contain the exact column name, add to error message in DataSet#resolve

Add test for extra error message piece

Adds an additional check in `DataSet#resolve`, in the else clause (i.e. column not resolved), that appends a suffix to the error message for the `AnalysisException` if that column name is literally found in the schema fields, to suggest to the user that it might need to be quoted via backticks.

### Why are the changes needed?

Forgetting to quote such column names is a common occurrence for new Spark users.

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

No (other than the extra suffix on the error message).

### How was this patch tested?

`test` was run for `core` in `sbt`, and passed.

Closes #25807 from jeff303/SPARK-25153.

Authored-by: Jeff Evans <jeffrey.wayne.evans@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2019-09-30 18:34:44 -07:00
Dongjoon Hyun a0b3d7a323 [SPARK-29300][TESTS] Compare catalyst and avro module benchmark in JDK8/11
### What changes were proposed in this pull request?

This PR regenerate the benchmark results in `catalyst` and `avro` module in order to compare JDK8/JDK11 result.

### Why are the changes needed?

This PR aims to verify that there is no regression on JDK11.

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

No.

### How was this patch tested?

This is a test-only update. We need to run the benchmark manually.

Closes #25972 from dongjoon-hyun/SPARK-29300.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-30 17:59:43 -07:00
Sean Owen e1ea806b30 [SPARK-29291][CORE][SQL][STREAMING][MLLIB] Change procedure-like declaration to function + Unit for 2.13
### What changes were proposed in this pull request?

Scala 2.13 emits a deprecation warning for procedure-like declarations:

```
def foo() {
 ...
```

This is equivalent to the following, so should be changed to avoid a warning:

```
def foo(): Unit = {
  ...
```

### Why are the changes needed?

It will avoid about a thousand compiler warnings when we start to support Scala 2.13. I wanted to make the change in 3.0 as there are less likely to be back-ports from 3.0 to 2.4 than 3.1 to 3.0, for example, minimizing that downside to touching so many files.

Unfortunately, that makes this quite a big change.

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

No behavior change at all.

### How was this patch tested?

Existing tests.

Closes #25968 from srowen/SPARK-29291.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-30 10:03:23 -07:00
Chris Martin 76791b89f5 [SPARK-27463][PYTHON][FOLLOW-UP] Miscellaneous documentation and code cleanup of cogroup pandas UDF
Follow up from https://github.com/apache/spark/pull/24981 incorporating some comments from HyukjinKwon.

Specifically:

- Adding `CoGroupedData` to `pyspark/sql/__init__.py __all__` so that documentation is generated.
- Added pydoc, including example, for the use case whereby the user supplies a cogrouping function including a key.
- Added the boilerplate for doctests to cogroup.py.  Note that cogroup.py only contains the apply() function which has doctests disabled as per the  other Pandas Udfs.
- Restricted the newly exposed RelationalGroupedDataset constructor parameters to access only by the sql package.
- Some minor  formatting tweaks.

This was tested by running the appropriate unit tests.  I'm unsure as to how to check that my change will cause the documentation to be generated correctly, but it someone can describe how I can do this I'd be happy to check.

Closes #25939 from d80tb7/SPARK-27463-fixes.

Authored-by: Chris Martin <chris@cmartinit.co.uk>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-30 22:25:35 +09:00
Jungtaek Lim (HeartSaVioR) 39eb79ac4b [SPARK-28074][SS] Log warn message on possible correctness issue for multiple stateful operations in single query
## What changes were proposed in this pull request?

Please refer [the link on dev. mailing list](https://lists.apache.org/thread.html/cc6489a19316e7382661d305fabd8c21915e5faf6a928b4869ac2b4a%3Cdev.spark.apache.org%3E) to see rationalization of this patch.

This patch adds the functionality to detect the possible correct issue on multiple stateful operations in single streaming query and logs warning message to inform end users.

This patch also documents some notes to inform caveats when using multiple stateful operations in single query, and provide one known alternative.

## How was this patch tested?

Added new UTs in UnsupportedOperationsSuite to test various combination of stateful operators on streaming query.

Closes #24890 from HeartSaVioR/SPARK-28074.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-30 08:18:23 -05:00
gengjiaan 1018390542 [SPARK-29252][BUILD] Upgrade zookeeper to 3.4.14 and fix vulnerabilities
### What changes were proposed in this pull request?
The current code uses org.apache.zookeeper:zookeeper:jar:3.4.6 and it will cause a security vulnerabilities. We could get some security info from https://www.tenable.com/cve/CVE-2019-0201

This reference remind to upgrate the version of `zookeeper` to 3.4.14 or later.

### Why are the changes needed?
This PR fix the security vulnerabilities.

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

### How was this patch tested?
Exists UT.

Closes #25933 from beliefer/upgrade-zookeeper.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-30 08:16:32 -05:00
Sean Owen 28b8383a6c [SPARK-29289][BUILD] Update scalatest, scalacheck, scopt, clapper, scala-parser-combinators for 2.13
### What changes were proposed in this pull request?

Update scalatest, scalacheck, scopt, clapper, scala-parser-combinators to latest maintenance release that is also cross-published for Scala 2.13.

### Why are the changes needed?

To build in the future for Scala 2.13

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

No

### How was this patch tested?

Existing tests

Closes #25967 from srowen/SPARK-29289.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-30 08:13:57 -05:00
Dongjoon Hyun 989b0c773f [SPARK-29297][TESTS] Compare core/mllib module benchmarks in JDK8/11
### What changes were proposed in this pull request?

This PR regenerate the benchmark results in `core` and `mllib` module in order to compare JDK8/JDK11 result.

### Why are the changes needed?

According to the result, For `PropertiesCloneBenchmark` and `UDTSerializationBenchmark`, JDK11 is slightly faster. In general, there is no regression in JDK11.

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

No.

### How was this patch tested?

This is a test-only PR. Manually run the benchmark.

Closes #25969 from dongjoon-hyun/SPARK-29297.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-29 21:43:58 -07:00
Liang-Chi Hsieh dd92e15301 [SPARK-29186][SQL] AliasIdentifier should be converted to Json in prettyJson
### What changes were proposed in this pull request?

This patch adds AliasIdentifier to the list of classes that should be converted to Json in TreeNode.shouldConvertToJson.

### Why are the changes needed?

When asking prettyJson of an analyzed query plan which contains SubqueryAlias. The field of name of SubqueryAlias is "null", like:

```
[ {
   "class" : "org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias",
   "num-children" : 1,
   "name" : null,
   "child" : 0
 }, {
   "class" : "org.apache.spark.sql.catalyst.plans.logical.Project",
 ...
```

Seems the alias name was in the Json before SPARK-19602.

It is fixed by this patch:

```
[ {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias",
  "num-children" : 1,
  "name" : {
    "product-class" : "org.apache.spark.sql.catalyst.AliasIdentifier",
    "identifier" : "t1"
  },
  "child" : 0
}, {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.Project",
...
```

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

Yes. This patch changes null value of name field of SubqueryAlias in Json string to the alias identifier.

### How was this patch tested?

Added unit test.

Closes #25959 from viirya/SPARK-29186.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2019-09-29 20:00:13 -07:00
Unknown 3ea9d6825b [SPARK-29019][WEBUI] Improve tooltip JDBC/ODBC Server tab
### What changes were proposed in this pull request?
Some of the columns of JDBC/ODBC server tab in Web UI are hard to understand.
We have documented it at SPARK-28373 but I think it is better to have some tooltips in the SQL statistics table to explain the columns
![image](https://user-images.githubusercontent.com/12819544/64489775-38e48980-d257-11e9-868a-5f5f6a0f1e46.png)
The columns with new tooltips are finish time, close time, execution time and duration
![image](https://user-images.githubusercontent.com/12819544/64489858-1141f100-d258-11e9-9e4e-fae3299da465.png)
Improvements in UIUtils can be used in other tables in WebUI to add tooltips

### Why are the changes needed?
It is interesting to improve the undestanding of the WebUI

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

### How was this patch tested?
Unit tests are added and manual test.

Closes #25723 from planga82/feature/SPARK-29019_tooltipjdbcServer.

Lead-authored-by: Unknown <soypab@gmail.com>
Co-authored-by: Pablo <soypab@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-29 18:34:24 -05:00
Dongjoon Hyun df28671800 [SPARK-29282][TESTS] Use the same VM configurations for test/benchmark
### What changes were proposed in this pull request?

This PR aims to specify the JDK8 default configurations `-XX:+UseParallelGC -XX:-UseDynamicNumberOfGCThreads` explicitly. As we see in this PR [here](https://github.com/apache/spark/pull/25966/files#diff-12b89b7ee67c63c2254b749c8f8d0694R10), this will make the comparison between JDK8 and JDK11 easier by removing a misleading regression.

**NOTE THAT THESE JVM CONFS ARE ONLY FOR BENCHMARK COMPARISON, NOT FOR A PRODUCTION**

### Why are the changes needed?

There exists many JVM-level changes between JDK8 and JDK11. For example, the followings are notable changes and it turns out that especially (1) and (2) shows a misleading regression in our micro-benchmark environment because our microbenchmark uses small VM memory.
1. [JEP 248: Make G1 the Default Garbage Collector](https://bugs.openjdk.java.net/browse/JDK-8073273) **JDK9+**
2. [Enable UseDynamicNumberOfGCThreads by default](https://bugs.openjdk.java.net/browse/JDK-8198547) **JDK11+**
3. [Change default value of HeapSizePerGCThread](https://bugs.openjdk.java.net/browse/JDK-8200417) **JDK11+**

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

No.

### How was this patch tested?

This is a test-only JVM configuration change. Manually, run the benchmark.

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

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-29 15:11:46 -07:00
angerszhu 1d4b2f010b [SPARK-29247][SQL] Redact sensitive information in when construct HiveClientHive.state
### What changes were proposed in this pull request?

HiveClientImpl may be log sensitive information. e.g. url, secret and token:
```scala
      logDebug(
        s"""
           |Applying Hadoop/Hive/Spark and extra properties to Hive Conf:
           |$k=${if (k.toLowerCase(Locale.ROOT).contains("password")) "xxx" else v}
         """.stripMargin)
```
So redact it.  Use SQLConf.get.redactOptions.

I add a new overloading function to fit this situation for one by one kv pair situation.

### Why are the changes needed?
Redact sensitive information when construct HiveClientImpl

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

### How was this patch tested?
MT

Run command
` /sbin/start-thriftserver.sh`

In log we can get
```
19/09/28 08:27:02 main DEBUG HiveClientImpl:
Applying Hadoop/Hive/Spark and extra properties to Hive Conf:
hive.druid.metadata.password=*********(redacted)
```

Closes #25954 from AngersZhuuuu/SPARK-29247.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-29 14:30:32 -07:00
Yuming Wang 31700116d2 [SPARK-28476][SQL] Support ALTER DATABASE SET LOCATION
### What changes were proposed in this pull request?
Support the syntax of `ALTER (DATABASE|SCHEMA) database_name SET LOCATION` path. Please note that only Hive 3.x metastore support this syntax.

Ref:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
https://issues.apache.org/jira/browse/HIVE-8472

### Why are the changes needed?
Support more syntax.

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

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

Closes #25883 from wangyum/SPARK-28476.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-09-29 11:31:49 -07:00
TomokoKomiyama 67d5b9b157 [SPARK-29172][SQL] Fix some exception issue of explain commands
### What changes were proposed in this pull request?
Added try exception

### Why are the changes needed?
The behaviors of run commands during exception handling are different depends on explain command. I think it should be unified.
[ >spark.sql("explain cost select * from hoge").show(false) ]
![cost](https://user-images.githubusercontent.com/55128575/65225389-09a80500-db00-11e9-9246-0f1a3a881595.png)

[ >spark.sql("explain extended select * from hoge").show(false) ]
![extemded](https://user-images.githubusercontent.com/55128575/65225430-188eb780-db00-11e9-99bf-ff550b2ffd12.png)

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

### How was this patch tested?
tested manually

Closes #25848 from TomokoKomiyama/fix-explain.

Authored-by: TomokoKomiyama <btkomiyamatm@oss.nttdata.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-29 10:41:57 -05:00
Yuming Wang 8167714cab [SPARK-27831][FOLLOW-UP][SQL][TEST] Should not use maven to add Hive test jars
### What changes were proposed in this pull request?

This PR moves Hive test jars(`hive-contrib-*.jar` and `hive-hcatalog-core-*.jar`) from maven dependency to local file.

### Why are the changes needed?
`--jars` can't be tested since `hive-contrib-*.jar` and `hive-hcatalog-core-*.jar` are already in classpath.

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

### How was this patch tested?
manual test

Closes #25690 from wangyum/SPARK-27831-revert.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-28 16:55:49 -07:00
Maxim Gekk 2409320d8f [SPARK-29237][SQL][FOLLOWUP] Ignore SET commands in expression examples while checking the _FUNC_ pattern
### What changes were proposed in this pull request?

The `SET` commands do not contain the `_FUNC_` pattern a priori. In the PR, I propose filter out such commands in the `using _FUNC_ instead of function names in examples` test.

### Why are the changes needed?
After the merge of https://github.com/apache/spark/pull/25942, examples will require particular settings. Currently, the whole expression example has to be ignored which is so much. It makes sense to ignore only `SET` commands in expression examples.

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

### How was this patch tested?

By running the `using _FUNC_ instead of function names in examples` test.

Closes #25958 from MaxGekk/dont-check-_FUNC_-in-set.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-29 08:51:47 +09:00
Jungtaek Lim (HeartSaVioR) 94946e4836 [SPARK-29281][SQL] Correct example of Like/RLike to test the origin intention correctly
### What changes were proposed in this pull request?

This patch fixes examples of Like/RLike to test its origin intention correctly. The example doesn't consider the default value of spark.sql.parser.escapedStringLiterals: it's false by default.

Please take a look at current example of Like:

d72f39897b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala (L97-L106)

If spark.sql.parser.escapedStringLiterals=false, then it should fail as there's `\U` in pattern (spark.sql.parser.escapedStringLiterals=false by default) but it doesn't fail.

```
The escape character is '\'. If an escape character precedes a special symbol or another
escape character, the following character is matched literally. It is invalid to escape
any other character.
```

For the query

```
SET spark.sql.parser.escapedStringLiterals=false;
SELECT '%SystemDrive%\Users\John' like '\%SystemDrive\%\Users%';
```

SQL parser removes single `\` (not sure that is intended) so the expressions of Like are constructed as following (I've printed out expression of left and right for Like/RLike):

> LIKE - left `%SystemDrive%UsersJohn` / right `\%SystemDrive\%Users%`

which are no longer having origin intention (see left).

Below query tests the origin intention:

```
SET spark.sql.parser.escapedStringLiterals=false;
SELECT '%SystemDrive%\\Users\\John' like '\%SystemDrive\%\\\\Users%';
```

> LIKE - left `%SystemDrive%\Users\John` / right `\%SystemDrive\%\\Users%`

Note that `\\\\` is needed in pattern as `StringUtils.escapeLikeRegex` requires `\\` to represent normal character of `\`.

Same for RLIKE:

```
SET spark.sql.parser.escapedStringLiterals=true;
SELECT '%SystemDrive%\Users\John' rlike '%SystemDrive%\\Users.*';
```

> RLIKE - left `%SystemDrive%\Users\John` / right `%SystemDrive%\\Users.*`

which is OK, but

```
SET spark.sql.parser.escapedStringLiterals=false;
SELECT '%SystemDrive%\Users\John' rlike '%SystemDrive%\Users.*';
```

> RLIKE - left `%SystemDrive%UsersJohn` / right `%SystemDrive%Users.*`

which no longer haves origin intention.

Below query tests the origin intention:
```
SET spark.sql.parser.escapedStringLiterals=true;
SELECT '%SystemDrive%\\Users\\John' rlike '%SystemDrive%\\\\Users.*';
```

> RLIKE - left `%SystemDrive%\Users\John` / right `%SystemDrive%\\Users.*`

### Why are the changes needed?

Because the example doesn't test the origin intention. Spark is now running automated tests from these examples, so now it's not only documentation issue but also test issue.

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

No, as it only corrects documentation.

### How was this patch tested?

Added debug log (like above) and ran queries from `spark-sql`.

Closes #25957 from HeartSaVioR/SPARK-29281.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-29 03:05:49 +09:00
Maxim Gekk ece4213176 [SPARK-21914][FOLLOWUP][TEST-HADOOP3.2][TEST-JAVA11] Clone SparkSession per each function example
### What changes were proposed in this pull request?
In the PR, I propose to clone Spark session per-each expression example. Examples can modify SQL settings, and can influence on each other if they run in the same Spark session in parallel.

### Why are the changes needed?
This should fix test failures like [this](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-3.2-jdk-11/478/testReport/junit/org.apache.spark.sql/SQLQuerySuite/check_outputs_of_expression_examples/) checking of the `Like` example:
```
org.apache.spark.sql.AnalysisException: the pattern '\%SystemDrive\%\Users%' is invalid, the escape character is not allowed to precede 'U';
      at org.apache.spark.sql.catalyst.util.StringUtils$.fail$1(StringUtils.scala:48)
      at org.apache.spark.sql.catalyst.util.StringUtils$.escapeLikeRegex(StringUtils.scala:57)
      at org.apache.spark.sql.catalyst.expressions.Like.escape(regexpExpressions.scala:108)
```

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

### How was this patch tested?
By running `check outputs of expression examples` in `org.apache.spark.sql.SQLQuerySuite`

Closes #25956 from MaxGekk/fix-expr-examples-checks.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-29 02:57:55 +09:00
Jungtaek Lim (HeartSaVioR) d72f39897b
[SPARK-27254][SS] Cleanup complete but invalid output files in ManifestFileCommitProtocol if job is aborted
## What changes were proposed in this pull request?

SPARK-27210 enables ManifestFileCommitProtocol to clean up incomplete output files in task level if task is aborted.

This patch extends the area of cleaning up, proposes ManifestFileCommitProtocol to clean up complete but invalid output files in job level if job aborts. Please note that this works as 'best-effort', not kind of guarantee, as we have in HadoopMapReduceCommitProtocol.

## How was this patch tested?

Added UT.

Closes #24186 from HeartSaVioR/SPARK-27254.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-09-27 12:35:26 -07:00
Jeff Evans 233c214a75 [SPARK-29070][CORE] Make SparkLauncher log full spark-submit command line
Log the full spark-submit command in SparkSubmit#launchApplication

Adding .python-version (pyenv file) to RAT exclusion list

### What changes were proposed in this pull request?

Original motivation [here](http://apache-spark-user-list.1001560.n3.nabble.com/Is-it-possible-to-obtain-the-full-command-to-be-invoked-by-SparkLauncher-td35144.html), expanded in the [Jira](https://issues.apache.org/jira/browse/SPARK-29070)..  In essence, we want to be able to log the full `spark-submit` command being constructed by `SparkLauncher`

### Why are the changes needed?

Currently, it is not possible to directly obtain this information from the `SparkLauncher` instance, which makes debugging and customer support more difficult.

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

No

### How was this patch tested?

`core` `sbt` tests were executed.  The `SparkLauncherSuite` (where I added assertions to an existing test) was also checked.  Within that, `testSparkLauncherGetError` is failing, but that appears not to have been caused by this change (failing for me even on the parent commit of c18f849d76).

Closes #25777 from jeff303/SPARK-29070.

Authored-by: Jeff Evans <jeffrey.wayne.evans@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-27 11:32:22 -07:00
Juliusz Sompolski 420abb457d [SPARK-29263][SCHEDULER] Update availableSlots in resourceOffers() before checking available slots for barrier taskSet
### What changes were proposed in this pull request?

availableSlots are computed before the for loop looping over all TaskSets in resourceOffers. But the number of slots changes in every iteration, as in every iteration these slots are taken. The number of available slots checked by a barrier task set has therefore to be recomputed in every iteration from availableCpus.

### Why are the changes needed?

Bugfix.
This could make resourceOffer attempt to start a barrier task set, even though it has not enough slots available. That would then be caught by the `require` in line 519, which will throw an exception, which will get caught and ignored by Dispatcher's MessageLoop, so nothing terrible would happen, but the exception would prevent resourceOffers from considering further TaskSets.
Note that launching the barrier TaskSet can still fail if other requirements are not satisfied, and still can be rolled-back by throwing exception in this `require`. Handling it more gracefully remains a TODO in SPARK-24818, but this fix at least should resolve the situation when it's unable to launch because of insufficient slots.

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

No

### How was this patch tested?

Added UT

Closes #23375

Closes #25946 from juliuszsompolski/SPARK-29263.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-09-27 11:18:32 -07:00
HyukjinKwon fda0e6e48d [SPARK-29240][PYTHON] Pass Py4J column instance to support PySpark column in element_at function
### What changes were proposed in this pull request?

This PR makes `element_at` in PySpark able to take PySpark `Column` instances.

### Why are the changes needed?

To match with Scala side. Seems it was intended but not working correctly as a bug.

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

Yes. See below:

```python
from pyspark.sql import functions as F
x = spark.createDataFrame([([1,2,3],1),([4,5,6],2),([7,8,9],3)],['list','num'])
x.withColumn('aa',F.element_at('list',x.num.cast('int'))).show()
```

Before:

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/functions.py", line 2059, in element_at
    return Column(sc._jvm.functions.element_at(_to_java_column(col), extraction))
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1277, in __call__
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1241, in _build_args
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1228, in _get_args
  File "/.../forked/spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_collections.py", line 500, in convert
  File "/.../spark/python/pyspark/sql/column.py", line 344, in __iter__
    raise TypeError("Column is not iterable")
TypeError: Column is not iterable
```

After:

```
+---------+---+---+
|     list|num| aa|
+---------+---+---+
|[1, 2, 3]|  1|  1|
|[4, 5, 6]|  2|  5|
|[7, 8, 9]|  3|  9|
+---------+---+---+
```

### How was this patch tested?

Manually tested against literal, Python native types, and PySpark column.

Closes #25950 from HyukjinKwon/SPARK-29240.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-27 11:04:55 -07:00
Maxim Gekk 4bffcf5a34 [SPARK-29275][SQL][DOC] Describe special date/timestamp values in the SQL migration guide
### What changes were proposed in this pull request?

Updated the SQL migration guide regarding to recently supported special date and timestamp values, see https://github.com/apache/spark/pull/25716 and https://github.com/apache/spark/pull/25708.

Closes #25834

### Why are the changes needed?
To let users know about new feature in Spark 3.0.

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

Closes #25948 from MaxGekk/special-values-migration-guide.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-27 10:36:20 -07:00
angerszhu cc852d4eec [SPARK-29015][SQL][TEST-HADOOP3.2] Reset class loader after initializing SessionState for built-in Hive 2.3
### What changes were proposed in this pull request?

Hive 2.3 will set a new UDFClassLoader to hiveConf.classLoader when initializing SessionState since HIVE-11878,  and
1. ADDJarCommand will add jars to clientLoader.classLoader.
2. --jar passed jar will be added to clientLoader.classLoader
3.  jar passed by hive conf  `hive.aux.jars`  [SPARK-28954](https://github.com/apache/spark/pull/25653) [SPARK-28840](https://github.com/apache/spark/pull/25542) will be added to clientLoader.classLoader too

For these  reason we cannot load the jars added by ADDJarCommand because of class loader got changed. We reset it to clientLoader.ClassLoader here.

### Why are the changes needed?
support for jdk11

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

### How was this patch tested?
UT
```
export JAVA_HOME=/usr/lib/jdk-11.0.3
export PATH=$JAVA_HOME/bin:$PATH

build/sbt -Phive-thriftserver -Phadoop-3.2

hive/test-only *HiveSparkSubmitSuite -- -z "SPARK-8368: includes jars passed in through --jars"
hive-thriftserver/test-only *HiveThriftBinaryServerSuite -- -z "test add jar"
```

Closes #25775 from AngersZhuuuu/SPARK-29015-STS-JDK11.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-27 10:23:56 -05:00
Maxim Gekk 4dd0066d40 [SPARK-21914][SQL][TESTS] Check results of expression examples
### What changes were proposed in this pull request?

New test compares outputs of expression examples in comments with results of `hiveResultString()`. Also I fixed existing examples where actual and expected outputs are different.

### Why are the changes needed?
This prevents mistakes in expression examples, and fixes existing mistakes in comments.

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

### How was this patch tested?
Add new test to `SQLQuerySuite`.

Closes #25942 from MaxGekk/run-expr-examples.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-27 21:30:37 +09:00
Wang Shuo bd28e8e179 [SPARK-29213][SQL] Generate extra IsNotNull predicate in FilterExec
### What changes were proposed in this pull request?
Currently the behavior of getting output and generating null checks in `FilterExec` is different. Thus some nullable attribute could be treated as not nullable by mistake.

In `FilterExec.ouput`, an attribute is marked as nullable or not by finding its `exprId` in notNullAttributes:
```
a.nullable && notNullAttributes.contains(a.exprId)
```
But in `FilterExec.doConsume`,  a `nullCheck` is generated or not for a predicate is decided by whether there is semantic equal not null predicate:
```
      val nullChecks = c.references.map { r =>
        val idx = notNullPreds.indexWhere { n => n.asInstanceOf[IsNotNull].child.semanticEquals(r)}
        if (idx != -1 && !generatedIsNotNullChecks(idx)) {
          generatedIsNotNullChecks(idx) = true
          // Use the child's output. The nullability is what the child produced.
          genPredicate(notNullPreds(idx), input, child.output)
        } else {
          ""
        }
      }.mkString("\n").trim
```
NPE will happen when run the SQL below:
```
sql("create table table1(x string)")
sql("create table table2(x bigint)")
sql("create table table3(x string)")
sql("insert into table2 select null as x")
sql(
  """
    |select t1.x
    |from (
    |    select x from table1) t1
    |left join (
    |    select x from (
    |        select x from table2
    |        union all
    |        select substr(x,5) x from table3
    |    ) a
    |    where length(x)>0
    |) t3
    |on t1.x=t3.x
  """.stripMargin).collect()
```
NPE Exception:
```
java.lang.NullPointerException
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(generated.java:40)
    at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:726)
    at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
    at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:135)
    at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
    at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:94)
    at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
    at org.apache.spark.scheduler.Task.run(Task.scala:127)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:449)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:452)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748)
```
the generated code:
```
== Subtree 4 / 5 ==
*(2) Project [cast(x#7L as string) AS x#9]
+- *(2) Filter ((length(cast(x#7L as string)) > 0) AND isnotnull(cast(x#7L as string)))
   +- Scan hive default.table2 [x#7L], HiveTableRelation `default`.`table2`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [x#7L]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator inputadapter_input_0;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] filter_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 011 */
/* 012 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 013 */     this.references = references;
/* 014 */   }
/* 015 */
/* 016 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 017 */     partitionIndex = index;
/* 018 */     this.inputs = inputs;
/* 019 */     inputadapter_input_0 = inputs[0];
/* 020 */     filter_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 021 */     filter_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
/* 022 */
/* 023 */   }
/* 024 */
/* 025 */   protected void processNext() throws java.io.IOException {
/* 026 */     while ( inputadapter_input_0.hasNext()) {
/* 027 */       InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next();
/* 028 */
/* 029 */       do {
/* 030 */         boolean inputadapter_isNull_0 = inputadapter_row_0.isNullAt(0);
/* 031 */         long inputadapter_value_0 = inputadapter_isNull_0 ?
/* 032 */         -1L : (inputadapter_row_0.getLong(0));
/* 033 */
/* 034 */         boolean filter_isNull_2 = inputadapter_isNull_0;
/* 035 */         UTF8String filter_value_2 = null;
/* 036 */         if (!inputadapter_isNull_0) {
/* 037 */           filter_value_2 = UTF8String.fromString(String.valueOf(inputadapter_value_0));
/* 038 */         }
/* 039 */         int filter_value_1 = -1;
/* 040 */         filter_value_1 = (filter_value_2).numChars();
/* 041 */
/* 042 */         boolean filter_value_0 = false;
/* 043 */         filter_value_0 = filter_value_1 > 0;
/* 044 */         if (!filter_value_0) continue;
/* 045 */
/* 046 */         boolean filter_isNull_6 = inputadapter_isNull_0;
/* 047 */         UTF8String filter_value_6 = null;
/* 048 */         if (!inputadapter_isNull_0) {
/* 049 */           filter_value_6 = UTF8String.fromString(String.valueOf(inputadapter_value_0));
/* 050 */         }
/* 051 */         if (!(!filter_isNull_6)) continue;
/* 052 */
/* 053 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 054 */
/* 055 */         boolean project_isNull_0 = false;
/* 056 */         UTF8String project_value_0 = null;
/* 057 */         if (!false) {
/* 058 */           project_value_0 = UTF8String.fromString(String.valueOf(inputadapter_value_0));
/* 059 */         }
/* 060 */         filter_mutableStateArray_0[1].reset();
/* 061 */
/* 062 */         filter_mutableStateArray_0[1].zeroOutNullBytes();
/* 063 */
/* 064 */         if (project_isNull_0) {
/* 065 */           filter_mutableStateArray_0[1].setNullAt(0);
/* 066 */         } else {
/* 067 */           filter_mutableStateArray_0[1].write(0, project_value_0);
/* 068 */         }
/* 069 */         append((filter_mutableStateArray_0[1].getRow()));
/* 070 */
/* 071 */       } while(false);
/* 072 */       if (shouldStop()) return;
/* 073 */     }
/* 074 */   }
/* 075 */
/* 076 */ }

```

This PR proposes to use semantic comparison both in `FilterExec.output` and `FilterExec.doConsume` for nullable attribute.

With this PR, the generated code snippet is below:
```
== Subtree 2 / 5 ==
*(3) Project [substring(x#8, 5, 2147483647) AS x#5]
+- *(3) Filter ((length(substring(x#8, 5, 2147483647)) > 0) AND isnotnull(substring(x#8, 5, 2147483647)))
   +- Scan hive default.table3 [x#8], HiveTableRelation `default`.`table3`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [x#8]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage3(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=3
/* 006 */ final class GeneratedIteratorForCodegenStage3 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator inputadapter_input_0;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] filter_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 011 */
/* 012 */   public GeneratedIteratorForCodegenStage3(Object[] references) {
/* 013 */     this.references = references;
/* 014 */   }
/* 015 */
/* 016 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 017 */     partitionIndex = index;
/* 018 */     this.inputs = inputs;
/* 019 */     inputadapter_input_0 = inputs[0];
/* 020 */     filter_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
/* 021 */     filter_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 32);
/* 022 */
/* 023 */   }
/* 024 */
/* 025 */   protected void processNext() throws java.io.IOException {
/* 026 */     while ( inputadapter_input_0.hasNext()) {
/* 027 */       InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next();
/* 028 */
/* 029 */       do {
/* 030 */         boolean inputadapter_isNull_0 = inputadapter_row_0.isNullAt(0);
/* 031 */         UTF8String inputadapter_value_0 = inputadapter_isNull_0 ?
/* 032 */         null : (inputadapter_row_0.getUTF8String(0));
/* 033 */
/* 034 */         boolean filter_isNull_0 = true;
/* 035 */         boolean filter_value_0 = false;
/* 036 */         boolean filter_isNull_2 = true;
/* 037 */         UTF8String filter_value_2 = null;
/* 038 */
/* 039 */         if (!inputadapter_isNull_0) {
/* 040 */           filter_isNull_2 = false; // resultCode could change nullability.
/* 041 */           filter_value_2 = inputadapter_value_0.substringSQL(5, 2147483647);
/* 042 */
/* 043 */         }
/* 044 */         boolean filter_isNull_1 = filter_isNull_2;
/* 045 */         int filter_value_1 = -1;
/* 046 */
/* 047 */         if (!filter_isNull_2) {
/* 048 */           filter_value_1 = (filter_value_2).numChars();
/* 049 */         }
/* 050 */         if (!filter_isNull_1) {
/* 051 */           filter_isNull_0 = false; // resultCode could change nullability.
/* 052 */           filter_value_0 = filter_value_1 > 0;
/* 053 */
/* 054 */         }
/* 055 */         if (filter_isNull_0 || !filter_value_0) continue;
/* 056 */         boolean filter_isNull_8 = true;
/* 057 */         UTF8String filter_value_8 = null;
/* 058 */
/* 059 */         if (!inputadapter_isNull_0) {
/* 060 */           filter_isNull_8 = false; // resultCode could change nullability.
/* 061 */           filter_value_8 = inputadapter_value_0.substringSQL(5, 2147483647);
/* 062 */
/* 063 */         }
/* 064 */         if (!(!filter_isNull_8)) continue;
/* 065 */
/* 066 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 067 */
/* 068 */         boolean project_isNull_0 = true;
/* 069 */         UTF8String project_value_0 = null;
/* 070 */
/* 071 */         if (!inputadapter_isNull_0) {
/* 072 */           project_isNull_0 = false; // resultCode could change nullability.
/* 073 */           project_value_0 = inputadapter_value_0.substringSQL(5, 2147483647);
/* 074 */
/* 075 */         }
/* 076 */         filter_mutableStateArray_0[1].reset();
/* 077 */
/* 078 */         filter_mutableStateArray_0[1].zeroOutNullBytes();
/* 079 */
/* 080 */         if (project_isNull_0) {
/* 081 */           filter_mutableStateArray_0[1].setNullAt(0);
/* 082 */         } else {
/* 083 */           filter_mutableStateArray_0[1].write(0, project_value_0);
/* 084 */         }
/* 085 */         append((filter_mutableStateArray_0[1].getRow()));
/* 086 */
/* 087 */       } while(false);
/* 088 */       if (shouldStop()) return;
/* 089 */     }
/* 090 */   }
/* 091 */
/* 092 */ }
```
### Why are the changes needed?
Fix NPE bug in FilterExec.

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

### How was this patch tested?
new UT

Closes #25902 from wangshuo128/filter-codegen-npe.

Authored-by: Wang Shuo <wangshuo128@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-09-27 15:14:17 +08:00
zhengruifeng aed7ff36f7 [SPARK-29258][ML][PYSPARK] parity between ml.evaluator and mllib.metrics
### What changes were proposed in this pull request?
1, expose `BinaryClassificationMetrics.numBins` in `BinaryClassificationEvaluator`
2, expose `RegressionMetrics.throughOrigin` in `RegressionEvaluator`
3, add metric `explainedVariance` in `RegressionEvaluator`

### Why are the changes needed?
existing function in mllib.metrics should also be exposed in ml

### Does this PR introduce any user-facing change?
yes, this PR add two expert params and one metric option

### How was this patch tested?
existing and added tests

Closes #25940 from zhengruifeng/evaluator_add_param.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2019-09-27 13:30:03 +08:00
Yuanjian Li ada3ad34c6 [SPARK-29175][SQL] Make additional remote maven repository in IsolatedClientLoader configurable
### What changes were proposed in this pull request?
Added a new config "spark.sql.additionalRemoteRepositories", a comma-delimited string config of the optional additional remote maven mirror.

### Why are the changes needed?
We need to connect the Maven repositories in IsolatedClientLoader for downloading Hive jars,
end-users can set this config if the default maven central repo is unreachable.

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

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

Closes #25849 from xuanyuanking/SPARK-29175.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-26 20:57:44 -07:00
uncleGen 570525f886 [SPARK-27715][SQL][UI] SQL query details in UI does not show in correct format
## What changes were proposed in this pull request?

before pr:
![image](https://user-images.githubusercontent.com/7402327/57752168-bb7e9180-771a-11e9-8757-63236ecab753.png)

after pr:
![image](https://user-images.githubusercontent.com/7402327/57752175-c802ea00-771a-11e9-96fd-aef1890b7985.png)

## How was this patch tested?

manual test

Closes #24609 from uncleGen/SPARK-27715.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-26 22:52:22 -05:00
sev7e0 cd04607785 [SPARK-29246][CORE] Remove unnecessary imports in core module
### What changes were proposed in this pull request?

Remove unnecessary imports in `core` module.

### Why are the changes needed?

Clean  code for Apache Spark 3.0.0.

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

No.

### How was this patch tested?

Local test.

Closes #25927 from sev7e0/dev_0925.

Authored-by: sev7e0 <sev7e0@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-26 20:43:08 -07:00
Huaxin Gao bdc4943b9e [SPARK-29142][PYTHON][ML] Pyspark clustering models support column setters/getters/predict
### What changes were proposed in this pull request?
Add the following Params classes in Pyspark clustering
```GaussianMixtureParams```
```KMeansParams```
```BisectingKMeansParams```
```LDAParams```
```PowerIterationClusteringParams```

### Why are the changes needed?
To be consistent with scala side

### Does this PR introduce any user-facing change?
Yes. Add the following changes:
```
GaussianMixtureModel
- get/setMaxIter
- get/setFeaturesCol
- get/setSeed
- get/setPredictionCol
- get/setProbabilityCol
- get/setTol
- predict
```

```
KMeansModel
- get/setMaxIter
- get/setFeaturesCol
- get/setSeed
- get/setPredictionCol
- get/setDistanceMeasure
- get/setTol
- predict
```

```
BisectingKMeansModel
- get/setMaxIter
- get/setFeaturesCol
- get/setSeed
- get/setPredictionCol
- get/setDistanceMeasure
- predict
```

```
LDAModel(HasMaxIter, HasFeaturesCol, HasSeed, HasCheckpointInterval):
- get/setMaxIter
- get/setFeaturesCol
- get/setSeed
- get/setCheckpointInterval
```

### How was this patch tested?
Add doctests

Closes #25859 from huaxingao/spark-29142.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2019-09-27 11:19:02 +08:00
Rahij Ramsharan 9f3c82163a [SPARK-29259][SQL] call fs.exists only when necessary
### What changes were proposed in this pull request?

Call fs.exists only when necessary in InsertIntoHadoopFsRelationCommand.

### Why are the changes needed?

When saving a dataframe into Hadoop, spark first checks if the file exists before inspecting the SaveMode to determine if it should actually insert data. However, the pathExists variable is actually not used in the case of SaveMode.Append. In some file systems, the exists call can be expensive and hence this PR makes that call only when necessary.

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

### How was this patch tested?
Existing unit tests should cover it since this doesn't change the behavior.

Closes #25928 from rahij/rr/exists-upstream.

Authored-by: Rahij Ramsharan <rramsharan@palantir.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-26 15:46:31 -07:00
sandeep katta 103de96059 [SPARK-29202][DEPLOY] Driver java options are not passed to driver process in Yarn client mode
### What changes were proposed in this pull request?
`--driver-java-options` is not passed to driver process if the user runs the application in **Yarn client** mode

Run the below command

```
./bin/spark-sql --master yarn \
--driver-java-options="-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5555"
```

**In Spark 2.4.4**
```
java ... -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5555
org.apache.spark.deploy.SparkSubmit --master yarn --conf spark.driver.extraJavaOptions=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5555 ...
```

**In Spark 3.0**
```
java ...
org.apache.spark.deploy.SparkSubmit --master yarn --conf spark.driver.extraJavaOptions=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5556 ...
```

This issue is caused by [SPARK-28980](https://github.com/apache/spark/pull/25684/files#diff-75e0f814aa3717db995fa701883dc4e1R395)

### Why are the changes needed?
Corrected the `isClientMode`  API implementation

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

### How was this patch tested?
Manually,

![image](https://user-images.githubusercontent.com/35216143/65383114-c92dce80-dd2d-11e9-86c1-60e6d7e09f1e.png)

Closes #25889 from sandeep-katta/yarnmode.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-26 15:37:20 -07:00