Commit graph

7639 commits

Author SHA1 Message Date
Shahid 8b5224097b [SPARK-27145][MINOR] Close store in the SQLAppStatusListenerSuite after test
## What changes were proposed in this pull request?
We create many stores in the SQLAppStatusListenerSuite, but we need to the close store after test.

## How was this patch tested?
Existing tests

Closes #24079 from shahidki31/SPARK-27145.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-14 13:08:41 -07:00
Yuming Wang da7db9abf6 [SPARK-23749][SQL] Replace built-in Hive API (isSub/toKryo) and remove OrcProto.Type usage
## What changes were proposed in this pull request?

In order to make the upgrade built-in Hive changes smaller.
This pr workaround the simplest 3 API changes first.

## How was this patch tested?

manual tests

Closes #24018 from wangyum/SPARK-23749.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 11:41:40 -07:00
Takeshi Yamamuro 66c5cd2d9c [SPARK-27151][SQL] ClearCacheCommand extends IgnoreCahedData to avoid plan node copys
## What changes were proposed in this pull request?
In SPARK-27011, we introduced `IgnoreCahedData` to avoid plan node copys in `CacheManager`.
Since `ClearCacheCommand` has no argument, it also can extend `IgnoreCahedData`.

## How was this patch tested?
Pass Jenkins.

Closes #24081 from maropu/SPARK-27011-2.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 11:36:16 -07:00
Takeshi Yamamuro bacffb8810 [SPARK-23264][SQL] Make INTERVAL keyword optional in INTERVAL clauses when ANSI mode enabled
## What changes were proposed in this pull request?
This pr updated parsing rules in `SqlBase.g4` to support a SQL query below when ANSI mode enabled;
```
SELECT CAST('2017-08-04' AS DATE) + 1 days;
```
The current master cannot parse it though, other dbms-like systems support the syntax (e.g., hive and mysql). Also, the syntax is frequently used in the official TPC-DS queries.

This pr added new tokens as follows;
```
YEAR | YEARS | MONTH | MONTHS | WEEK | WEEKS | DAY | DAYS | HOUR | HOURS | MINUTE
MINUTES | SECOND | SECONDS | MILLISECOND | MILLISECONDS | MICROSECOND | MICROSECONDS
```
Then, it registered the keywords below as the ANSI reserved (this follows SQL-2011);
```
 DAY | HOUR | MINUTE | MONTH | SECOND | YEAR
```

## How was this patch tested?
Added tests in `SQLQuerySuite`, `ExpressionParserSuite`, and `TableIdentifierParserSuite`.

Closes #20433 from maropu/SPARK-23264.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-14 10:45:29 +09:00
Dongjoon Hyun 250946ff93 [SPARK-27123][SQL][FOLLOWUP] Use isRenaming check for limit too.
## What changes were proposed in this pull request?

This is a followup for https://github.com/apache/spark/pull/24049 to reduce the scope of pattern based on the review comments.

## How was this patch tested?

Pass the existing test.

Closes #24082 from dongjoon-hyun/SPARK-27123-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-13 15:01:01 -07:00
Jungtaek Lim (HeartSaVioR) 733f2c0b98 [MINOR][SQL] Deduplicate huge if statements in get between specialized getters
## What changes were proposed in this pull request?

This patch deduplicates the huge if statements regarding getting value between specialized getters.

## How was this patch tested?

Existing UT.

Closes #24016 from HeartSaVioR/MINOR-deduplicate-get-from-specialized-getters.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-13 15:52:21 -05:00
Dongjoon Hyun 3221bf4cd5 [SPARK-27034][SPARK-27123][SQL][FOLLOWUP] Update Nested Schema Pruning BM result with EC2
## What changes were proposed in this pull request?

This is a follow up PR for #23943 in order to update the benchmark result with EC2 `r3.xlarge` instance.

## How was this patch tested?

N/A. (Manually compare the diff)

Closes #24078 from dongjoon-hyun/SPARK-27034.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-13 20:27:10 +00:00
Wenchen Fan 2a80a4cd39 [SPARK-27106][SQL] merge CaseInsensitiveStringMap and DataSourceOptions
## What changes were proposed in this pull request?

It's a little awkward to have 2 different classes(`CaseInsensitiveStringMap` and `DataSourceOptions`) to present the options in data source and catalog API.

This PR merges these 2 classes, while keeping the name `CaseInsensitiveStringMap`, which is more precise.

## How was this patch tested?

existing tests

Closes #24025 from cloud-fan/option.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-14 01:23:27 +08:00
Dave DeCaprio 812ad55461 [SPARK-26103][SQL] Limit the length of debug strings for query plans
## What changes were proposed in this pull request?

The PR puts in a limit on the size of a debug string generated for a tree node.  Helps to fix out of memory errors when large plans have huge debug strings.   In addition to SPARK-26103, this should also address SPARK-23904 and SPARK-25380.  AN alternative solution was proposed in #23076, but that solution doesn't address all the cases that can cause a large query.  This limit is only on calls treeString that don't pass a Writer, which makes it play nicely with #22429, #23018 and #23039.  Full plans can be written to files, but truncated plans will be used when strings are held in memory, such as for the UI.

- A new configuration parameter called spark.sql.debug.maxPlanLength was added to control the length of the plans.
- When plans are truncated, "..." is printed to indicate that it isn't a full plan
- A warning is printed out the first time a truncated plan is displayed. The warning explains what happened and how to adjust the limit.

## How was this patch tested?

Unit tests were created for the new SizeLimitedWriter.  Also a unit test for TreeNode was created that checks that a long plan is correctly truncated.

Closes #23169 from DaveDeCaprio/text-plan-size.

Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-13 09:58:43 -07:00
Wenchen Fan d3813d8b21 [SPARK-27064][SS] create StreamingWrite at the beginning of streaming execution
## What changes were proposed in this pull request?

According to the [design](https://docs.google.com/document/d/1vI26UEuDpVuOjWw4WPoH2T6y8WAekwtI7qoowhOFnI4/edit?usp=sharing), the life cycle of `StreamingWrite` should be the same as the read side `MicroBatch/ContinuousStream`, i.e. each run of the stream query, instead of each epoch.

This PR fixes it.

## How was this patch tested?

existing tests

Closes #23981 from cloud-fan/dsv2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-13 19:47:54 +08:00
Liang-Chi Hsieh f55c760df6 [SPARK-27034][SQL][FOLLOWUP] Rename ParquetSchemaPruning to SchemaPruning
## What changes were proposed in this pull request?

This is a followup to #23943. This proposes to rename ParquetSchemaPruning to SchemaPruning as ParquetSchemaPruning supports both Parquet and ORC v1 now.

## How was this patch tested?

Existing tests.

Closes #24077 from viirya/nested-schema-pruning-orc-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-13 20:12:01 +09:00
Jungtaek Lim (HeartSaVioR) 1b06cda532 [MINOR][SQL] Refactor RowEncoder to use existing (De)serializerBuildHelper methods
## What changes were proposed in this pull request?

This patch proposes to reuse existing methods in (De)serializerBuildHelper in RowEncoder to achieve deduplication as well as consistent creation of serialization/deserialization of same type.

## How was this patch tested?

Existing UT.

Closes #24014 from HeartSaVioR/SPARK-27092.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-13 10:54:47 +08:00
Takeshi Yamamuro 1e9469bb7a [SPARK-26976][SQL] Forbid reserved keywords as identifiers when ANSI mode is on
## What changes were proposed in this pull request?
This pr added code to forbid reserved keywords as identifiers when ANSI mode is on.
This is a follow-up of SPARK-26215(#23259).

## How was this patch tested?
Added tests in `TableIdentifierParserSuite`.

Closes #23880 from maropu/SPARK-26976.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-13 11:20:27 +09:00
Ajith e60d8fce0b [SPARK-27045][SQL] SQL tab in UI shows actual SQL instead of callsite in case of SparkSQLDriver
## What changes were proposed in this pull request?

When we run sql in spark via SparkSQLDriver (thrift server, spark-sql), SQL string is siet via ``setJobDescription``. the SparkUI SQL tab must show SQL instead of stacktrace in case ``setJobDescription`` is set which is more useful to end user. Instead it currently shows in description column the callsite shortform which is less useful

![image](https://user-images.githubusercontent.com/22072336/53734682-aaa7d900-3eaa-11e9-957b-0e5006db417e.png)

## How was this patch tested?

Manually:
![image](https://user-images.githubusercontent.com/22072336/53734657-9f54ad80-3eaa-11e9-8dc5-2b38f6970f4e.png)

Closes #23958 from ajithme/sqlui.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-12 16:14:29 -07:00
Liang-Chi Hsieh b0c2b3bfd9 [SPARK-27034][SQL] Nested schema pruning for ORC
## What changes were proposed in this pull request?

We only supported nested schema pruning for Parquet previously. This proposes to support nested schema pruning for ORC too.

Note: This only covers ORC v1. For ORC v2, the necessary change is at the schema pruning rule. We should deal with ORC v2 as a TODO item, in order to reduce review burden.

## How was this patch tested?

Added tests.

Closes #23943 from viirya/nested-schema-pruning-orc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-12 15:39:16 -07:00
Dongjoon Hyun 78314af580 [SPARK-27123][SQL] Improve CollapseProject to handle projects cross limit/repartition/sample
## What changes were proposed in this pull request?

`CollapseProject` optimizer rule simplifies some plans by merging the adjacent projects and performing alias substitutions.
```scala
scala> sql("SELECT b c FROM (SELECT a b FROM t)").explain
== Physical Plan ==
*(1) Project [a#5 AS c#1]
+- Scan hive default.t [a#5], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#5]
```

We can do that more complex cases like the following. This PR aims to handle adjacent projects across limit/repartition/sample. Here, repartition means `Repartition`, not `RepartitionByExpression`.

**BEFORE**
```scala
scala> sql("SELECT b c FROM (SELECT /*+ REPARTITION(1) */ a b FROM t)").explain
== Physical Plan ==
*(2) Project [b#0 AS c#1]
+- Exchange RoundRobinPartitioning(1)
   +- *(1) Project [a#5 AS b#0]
      +- Scan hive default.t [a#5], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#5]
```

**AFTER**
```scala
scala> sql("SELECT b c FROM (SELECT /*+ REPARTITION(1) */ a b FROM t)").explain
== Physical Plan ==
Exchange RoundRobinPartitioning(1)
+- *(1) Project [a#11 AS c#7]
   +- Scan hive default.t [a#11], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#11]
```

## How was this patch tested?

Pass the Jenkins with the newly added and updated test cases.

Closes #24049 from dongjoon-hyun/SPARK-27123.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-12 21:45:40 +00:00
zuotingbing 3f9247de1e [SPARK-27010][SQL] Find out the actual port number when hive.server2.thrift.port=0
## What changes were proposed in this pull request?
Currently, if we set hive.server2.thrift.port=0, it hard to find out the actual port number which one we should use when using beeline to connect.

before:
![2019-02-28_170942](https://user-images.githubusercontent.com/24823338/53557240-779ad800-3b80-11e9-9567-175f28aa61da.png)

after:
![2019-02-28_170904](https://user-images.githubusercontent.com/24823338/53557255-7f5a7c80-3b80-11e9-8ba6-9764c03e5407.png)

use beeline to connect success:
![2019-02-28_170844](https://user-images.githubusercontent.com/24823338/53557267-85e8f400-3b80-11e9-90a5-f7f53a51cc32.png)

## How was this patch tested?
 manual tests

Closes #23917 from zuotingbing/SPARK-27010.

Authored-by: zuotingbing <zuo.tingbing9@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:38:41 -05:00
shivusondur 4b6d39d85d [SPARK-27090][CORE] Removing old LEGACY_DRIVER_IDENTIFIER ("<driver>")
## What changes were proposed in this pull request?
LEGACY_DRIVER_IDENTIFIER and its reference are removed.
corresponding references test are updated.

## How was this patch tested?
tested  UT test cases

Closes #24026 from shivusondur/newjira2.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:29:39 -05:00
Shahid 1853db3186 [SPARK-27125][SQL][TEST] Add test suite for sql execution page
## What changes were proposed in this pull request?
Added test suite for AllExecutionsPage class. Checked the scenarios for SPARK-27019 and SPARK-27075.

## How was this patch tested?
Added UT, manually tested

Closes #24052 from shahidki31/SPARK-27125.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 10:15:28 -05:00
Ajith b8dd84b9e4 [SPARK-27011][SQL] reset command fails with cache
## What changes were proposed in this pull request?

When cache is enabled ( i.e once cache table command is executed), any following sql will trigger
 CacheManager#lookupCachedData which will create a copy of the tree node, which inturn calls TreeNode#makeCopy. Here the problem is it will try to create a copy instance. But as ResetCommand is a case object this will fail

## How was this patch tested?

Added UT to reproduce the issue

Closes #23918 from ajithme/reset.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-12 11:02:09 +08:00
Maxim Gekk 60be6d2ea3 [SPARK-27109][SQL] Refactoring of TimestampFormatter and DateFormatter
## What changes were proposed in this pull request?

In PR, I propose to refactor the `parse()` method of `Iso8601DateFormatter`/`Iso8601DateFormatter` and `toInstantWithZoneId` of `toInstantWithZoneId` to achieve the following:
- Avoid unnecessary conversion of parsed input to `java.time.Instant` before converting it to micros and days. Necessary information exists in `ZonedDateTime` already, and micros/days can be extracted from the former one.
- Avoid additional extraction of LocalTime from parsed object, more precisely, double query of `TemporalQueries.localTime` from `temporalAccessor`.
- Avoid additional extraction of zone id from parsed object, in particular, double query of `TemporalQueries.offset()`.
- Using `ZoneOffset.UTC` instead of `ZoneId.of` in `DateFormatter`. This allows to avoid looking for zone offset by zone id.

## How was this patch tested?

By existing test suite `DateTimeUtilsSuite`, `TimestampFormatterSuite` and `DateFormatterSuite`.

Closes #24030 from MaxGekk/query-localtime.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 19:02:30 -05:00
Hyukjin Kwon 3725b1324f [SPARK-26923][SQL][R] Refactor ArrowRRunner and RRunner to share one BaseRRunner
## What changes were proposed in this pull request?

This PR proposes to have one base R runner.

In the high level,

Previously, it had `ArrowRRunner` and it inherited `RRunner`:

```
└── RRunner
    └── ArrowRRunner
```

After this PR, now it has a `BaseRRunner`, and `ArrowRRunner` and `RRunner` inherit `BaseRRunner`:

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

This way is consistent with Python's.

In more details, see below:

```scala
class BaseRRunner[IN, OUT] {

  def compute: Iterator[OUT] = {
    ...
    newWriterThread(...).start()
    ...
    newReaderIterator(...)
    ...
  }

  // Make a thread that writes data from JVM to R process
  abstract protected def newWriterThread(..., iter: Iterator[IN], ...): WriterThread

  // Make an iterator that reads data from the R process to JVM
  abstract protected def newReaderIterator(...): ReaderIterator

  abstract class WriterThread(..., iter: Iterator[IN], ...) extends Thread {
    override def run(): Unit {
      ...
      writeIteratorToStream(...)
      ...
    }

    // Actually writing logic to the socket stream.
    abstract protected def writeIteratorToStream(dataOut: DataOutputStream): Unit
  }

  abstract class ReaderIterator extends Iterator[OUT] {
    override def hasNext(): Boolean = {
      ...
      read(...)
      ...
    }

    override def next(): OUT = {
      ...
      hasNext()
      ...
    }

    // Actually reading logic from the socket stream.
    abstract protected def read(...): OUT
  }
}
```

```scala
case [Arrow]RRunner extends BaseRRunner {
  override def newWriterThread(...) {
    new WriterThread(...) {
      override def writeIteratorToStream(...) {
        ...
      }
    }
  }

  override def newReaderIterator(...) {
    new ReaderIterator(...) {
      override def read(...) {
        ...
      }
    }
  }
}
```

## How was this patch tested?

Manually tested and existing tests should cover.

Closes #23977 from HyukjinKwon/SPARK-26923.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-12 08:45:29 +09:00
Wenchen Fan 31878c9daa [SPARK-27119][SQL] Do not infer schema when reading Hive serde table with native data source
## What changes were proposed in this pull request?

In Spark 2.1, we hit a correctness bug. When reading a Hive serde parquet table with the native parquet data source, and the actual file schema doesn't match the table schema in Hive metastore(only upper/lower case difference), the query returns 0 results.

The reason is that, the parquet reader is case sensitive. If we push down filters with column names that don't match the file physical schema case-sensitively, no data will be returned.

To fix this bug, there were 2 solutions proposed at that time:
1. Add a config to optionally disable parquet filter pushdown, and make parquet column pruning case insensitive.
https://github.com/apache/spark/pull/16797

2. Infer the actual schema from data files, when reading Hive serde table with native data source. A config is provided to disable it.
https://github.com/apache/spark/pull/17229

Solution 2 was accepted and merged to Spark 2.1.1

In Spark 2.4, we refactored the parquet data source a little:
1. do parquet filter pushdown with the actual file schema.
https://github.com/apache/spark/pull/21696

2. make parquet filter pushdown case insensitive.
https://github.com/apache/spark/pull/22197

3. make parquet column pruning case insensitive.
https://github.com/apache/spark/pull/22148

With these patches, the correctness bug in Spark 2.1 no longer exists, and the schema inference becomes unnecessary.

To be safe, this PR just changes the default value to NEVER_INFER, so that users can set it back to INFER_AND_SAVE. If we don't receive any bug reports for it, we can remove the related code in the next release.

## How was this patch tested?

existing tests

Closes #24041 from cloud-fan/infer.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-11 09:44:29 -07:00
Jagadesh Kiran d9978fb4e4 [SPARK-26860][PYSPARK][SPARKR] Fix for RangeBetween and RowsBetween docs to be in sync with spark documentation
The docs describing RangeBetween & RowsBetween for pySpark & SparkR are not in sync with Spark description.

a. Edited PySpark and SparkR docs  and made description same for both RangeBetween and RowsBetween
b. created executable examples in both pySpark and SparkR documentation
c. Locally tested the patch for scala Style checks and UT for checking no testcase failures

Closes #23946 from jagadesh-kiran/master.

Authored-by: Jagadesh Kiran <jagadesh.n@in.verizon.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 08:53:09 -05:00
Wenchen Fan 8114b63d56 [SPARK-27117][SQL] current_date/current_timestamp should not refer to columns with ansi parser mode
## What changes were proposed in this pull request?

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

It revisits https://issues.apache.org/jira/browse/SPARK-27117 , which should be an invalid use case according to the SQL standard.

`current_date/current_timestamp` are reserved keywords, if users want to access columns named `current_date/current_timestamp`, they should quote the name like ```select `current_date` from tbl```

If ansi mode is not enabled(which is the default), this PR won't introduce any changes.

## How was this patch tested?

a new test case

Closes #24039 from cloud-fan/current_datetime.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 20:40:03 +08:00
Dilip Biswal 1b9fd67904 [SPARK-27096][SQL] Reconcile the join types between data frame and sql interface
## What changes were proposed in this pull request?
Currently in the grammar file, we have the joinType rule defined as following :
```
joinType
    : INNER?
   ....
   ....
    | LEFT SEMI
    | LEFT? ANTI
    ;
```
The keyword LEFT is optional for ANTI join even though its not optional for SEMI join. When
using data frame interface join type "anti" is not allowed. The allowed types are "left_anti" or
"leftanti" for anti joins. ~~In this PR, i am making the LEFT keyword mandatory for ANTI joins so
it aligns better with the LEFT SEMI join in the grammar file and also the join types allowed from dataframe api.~~

This PR makes LEFT optional for SEMI join in .g4 and add "semi" and "anti" join types from dataframe.

~~I have not opened any JIRA for this as probably we may need some discussion to see if
we are going to address this or not.~~

## How was this patch tested?
Modified the join type tests.

Closes #23982 from dilipbiswal/join_fix.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 14:02:21 +08:00
Takeshi Yamamuro 7a9537c338 [SPARK-21351][SQL] Remove the UpdateAttributeNullability rule from the optimizer
## What changes were proposed in this pull request?
This pr removed `UpdateAttributeNullability` from the optimizer because the same logic happens in the analyzer. See SPARK-26459(#23390) for more detailed discussion.

## How was this patch tested?
N/A

Closes #23508 from maropu/SPARK-21351.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 13:34:14 +08:00
Jungtaek Lim (HeartSaVioR) f0bde69ebc [MINOR][SQL] Throw better exception for Encoder with tuple more than 22 elements
## What changes were proposed in this pull request?

This patch proposes to throw better exception with better error message when encoding to tuple which elements are more than 22.

**BEFORE**
```scala
scala> import org.apache.spark.sql.catalyst.encoders._
scala> val encoders = (0 to 22).map(_ => org.apache.spark.sql.Encoders.scalaInt.asInstanceOf[ExpressionEncoder[_]])
scala> ExpressionEncoder.tuple(encoders)
java.lang.ClassNotFoundException: scala.Tuple23
```

**AFTER**
```scala
scala> ExpressionEncoder.tuple(encoders)
java.lang.UnsupportedOperationException: Due to Scala's limited support of tuple, tuple with more than 22 elements are not supported.
```

## How was this patch tested?

Added UT.

Closes #24046 from HeartSaVioR/MINOR-throw-better-exception-for-tuple-more-than-22.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-11 13:44:45 +09:00
Takeshi Yamamuro f0927d8ac4 [SPARK-27110][SQL] Moves some functions from AnalyzeColumnCommand to command/CommandUtils
## What changes were proposed in this pull request?
To reuse some common logics for improving `Analyze` commands (See the description of `SPARK-25196` for details), this pr moved some functions from `AnalyzeColumnCommand` to `command/CommandUtils`.  A follow-up pr will add code to extend `Analyze` commands for cached tables.

## How was this patch tested?
Existing tests.

Closes #22204 from maropu/SPARK-25196.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-10 15:17:46 -07:00
Yuming Wang 470313e660 [SPARK-27118][SQL] Upgrade Hive Metastore Client to the latest versions for Hive 1.0.x/1.1.x
## What changes were proposed in this pull request?

Hive 1.1.1 and Hive 1.0.1 released. We should upgrade Hive Metastore Client version.

https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329444&styleName=Text&projectId=12310843
https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329557&styleName=Text&projectId=12310843

## How was this patch tested?

N/A

Closes #24040 from wangyum/SPARK-27118.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:50:10 -08:00
Yuming Wang f732647ae4 [SPARK-27054][BUILD][SQL] Remove the Calcite dependency
## What changes were proposed in this pull request?

Calcite is only used for [runSqlHive](02bbe977ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L699-L705)) when `hive.cbo.enable=true`([SemanticAnalyzer](https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java#L278-L280)).
So we can disable `hive.cbo.enable` and remove Calcite dependency.

## How was this patch tested?

Exist tests

Closes #23970 from wangyum/SPARK-27054.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:34:24 -08:00
Shixiong Zhu 6e1c0827ec
[SPARK-27111][SS] Fix a race that a continuous query may fail with InterruptedException
## What changes were proposed in this pull request?

Before a Kafka consumer gets assigned with partitions, its offset will contain 0 partitions. However, runContinuous will still run and launch a Spark job having 0 partitions. In this case, there is a race that epoch may interrupt the query execution thread after `lastExecution.toRdd`, and either `epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` or the next `runContinuous` will get interrupted unintentionally.

To handle this case, this PR has the following changes:

- Clean up the resources in `queryExecutionThread.runUninterruptibly`. This may increase the waiting time of `stop` but should be minor because the operations here are very fast (just sending an RPC message in the same process and stopping a very simple thread).
- Clear the interrupted status at the end so that it won't impact the `runContinuous` call. We may clear the interrupted status set by `stop`, but it doesn't affect the query termination because `runActivatedStream` will check `state` and exit accordingly.

I also updated the clean up codes to make sure exceptions thrown from `epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` won't stop the clean up.

## How was this patch tested?

Jenkins

Closes #24034 from zsxwing/SPARK-27111.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-03-09 14:26:58 -08:00
CodeGod a29df5fa02 [SPARK-27080][SQL] bug fix: mergeWithMetastoreSchema with uniform lower case comparison
## What changes were proposed in this pull request?
When reading parquet file with merging metastore schema and file schema, we should compare field names using uniform case. In current implementation, lowercase is used but one omission. And this patch fix it.

## How was this patch tested?
Unit test

Closes #24001 from codeborui/mergeSchemaBugFix.

Authored-by: CodeGod <>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-09 21:28:10 +08:00
Kris Mok 57ae251f75 [SPARK-27097] Avoid embedding platform-dependent offsets literally in whole-stage generated code
## What changes were proposed in this pull request?

Spark SQL performs whole-stage code generation to speed up query execution. There are two steps to it:
- Java source code is generated from the physical query plan on the driver. A single version of the source code is generated from a query plan, and sent to all executors.
  - It's compiled to bytecode on the driver to catch compilation errors before sending to executors, but currently only the generated source code gets sent to the executors. The bytecode compilation is for fail-fast only.
- Executors receive the generated source code and compile to bytecode, then the query runs like a hand-written Java program.

In this model, there's an implicit assumption about the driver and executors being run on similar platforms. Some code paths accidentally embedded platform-dependent object layout information into the generated code, such as:
```java
Platform.putLong(buffer, /* offset */ 24, /* value */ 1);
```
This code expects a field to be at offset +24 of the `buffer` object, and sets a value to that field.
But whole-stage code generation generally uses platform-dependent information from the driver. If the object layout is significantly different on the driver and executors, the generated code can be reading/writing to wrong offsets on the executors, causing all kinds of data corruption.

One code pattern that leads to such problem is the use of `Platform.XXX` constants in generated code, e.g. `Platform.BYTE_ARRAY_OFFSET`.

Bad:
```scala
val baseOffset = Platform.BYTE_ARRAY_OFFSET
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will embed the value of `Platform.BYTE_ARRAY_OFFSET` on the driver into the generated code.

Good:
```scala
val baseOffset = "Platform.BYTE_ARRAY_OFFSET"
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will generate the offset symbolically -- `Platform.putLong(buffer, Platform.BYTE_ARRAY_OFFSET, value)`, which will be able to pick up the correct value on the executors.

Caveat: these offset constants are declared as runtime-initialized `static final` in Java, so they're not compile-time constants from the Java language's perspective. It does lead to a slightly increased size of the generated code, but this is necessary for correctness.

NOTE: there can be other patterns that generate platform-dependent code on the driver which is invalid on the executors. e.g. if the endianness is different between the driver and the executors, and if some generated code makes strong assumption about endianness, it would also be problematic.

## How was this patch tested?

Added a new test suite `WholeStageCodegenSparkSubmitSuite`. This test suite needs to set the driver's extraJavaOptions to force the driver and executor use different Java object layouts, so it's run as an actual SparkSubmit job.

Authored-by: Kris Mok <kris.mokdatabricks.com>

Closes #24031 from gatorsmile/cherrypickSPARK-27097.

Lead-authored-by: Kris Mok <kris.mok@databricks.com>
Co-authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-09 01:20:32 +00:00
Sunitha Kambhampati bd2710bd79 [MINOR][SQL] Fix the typo in the spark.sql.extensions conf doc
## What changes were proposed in this pull request?
Fix the  typo (missing the s)  in the class name (SparkSessionExtensions)  in the doc for Spark conf spark.sql.extensions.

## How was this patch tested?
Verified by checking that the configuration doc shows up correctly in spark-shell using the SET -v

Closes #24020 from skambha/fixnametypo.

Authored-by: Sunitha Kambhampati <skambha@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-09 08:51:19 +09:00
SongYadong 14b1312727 [SPARK-27103][SQL][MINOR] List SparkSql reserved keywords in alphabet order
## What changes were proposed in this pull request?

This PR tries to correct spark-sql reserved keywords' position in list if they are not in alphabetical order.
In test suite some repeated words are removed. Also some comments are added for remind.

## How was this patch tested?

Existing unit tests.

Closes #23985 from SongYadong/sql_reserved_alphabet.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-08 10:51:39 -08:00
wangguangxin.cn d3d9c7bb0a [SPARK-27079][MINOR][SQL] Fix typo & Remove useless imports & Add missing override annotation
## What changes were proposed in this pull request?

1. Fix two typos
2. Remove useless imports in `CSVExprUtils.scala`
3. Add missing `override` annotation

## How was this patch tested?

test by existing uts

Closes #24000 from WangGuangxin/SPARK-27079.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-08 12:14:04 -06:00
Ryan Blue 6170e40c15 [SPARK-24252][SQL] Add v2 catalog plugin system
## What changes were proposed in this pull request?

This adds a v2 API for adding new catalog plugins to Spark.

* Catalog implementations extend `CatalogPlugin` and are loaded via reflection, similar to data sources
* `Catalogs` loads and initializes catalogs using configuration from a `SQLConf`
* `CaseInsensitiveStringMap` is used to pass configuration to `CatalogPlugin` via `initialize`

Catalogs are configured by adding config properties starting with `spark.sql.catalog.(name)`. The name property must specify a class that implements `CatalogPlugin`. Other properties under the namespace (`spark.sql.catalog.(name).(prop)`) are passed to the provider during initialization along with the catalog name.

This replaces #21306, which will be implemented in two multiple parts: the catalog plugin system (this commit) and specific catalog APIs, like `TableCatalog`.

## How was this patch tested?

Added test suites for `CaseInsensitiveStringMap` and for catalog loading.

Closes #23915 from rdblue/SPARK-24252-add-v2-catalog-plugins.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 19:31:49 +08:00
Yuming Wang 2036074b99 [SPARK-26004][SQL] InMemoryTable support StartsWith predicate push down
## What changes were proposed in this pull request?

[SPARK-24638](https://issues.apache.org/jira/browse/SPARK-24638) adds support for Parquet file `StartsWith` predicate push down.
`InMemoryTable` can also support this feature.

This is an example to explain how it works, Imagine that the `id` column stored as below:

Partition ID | lowerBound | upperBound
-- | -- | --
p1 | '1' | '9'
p2 | '10' | '19'
p3 | '20' | '29'
p4 | '30' | '39'
p5 | '40' | '49'

A filter ```df.filter($"id".startsWith("2"))``` or ```id like '2%'```
then we substr lowerBound and upperBound:

Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2"))
-- | -- | --
p1 | '1' | '9'
p2 | '1' | '1'
p3 | '2' | '2'
p4 | '3' | '3'
p5 | '4' | '4'

We can see that we only need to read `p1` and `p3`.

## How was this patch tested?

 unit tests and benchmark tests

benchmark test result:
```
================================================================================================
Pushdown benchmark for StringStartsWith
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.12.6
Intel(R) Core(TM) i7-7820HQ CPU  2.90GHz
StringStartsWith filter: (value like '10%'): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InMemoryTable Vectorized                    12068 / 14198          1.3         767.3       1.0X
InMemoryTable Vectorized (Pushdown)           5457 / 8662          2.9         347.0       2.2X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.12.6
Intel(R) Core(TM) i7-7820HQ CPU  2.90GHz
StringStartsWith filter: (value like '1000%'): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InMemoryTable Vectorized                      5246 / 5355          3.0         333.5       1.0X
InMemoryTable Vectorized (Pushdown)           2185 / 2346          7.2         138.9       2.4X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.12.6
Intel(R) Core(TM) i7-7820HQ CPU  2.90GHz
StringStartsWith filter: (value like '786432%'): Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InMemoryTable Vectorized                      5112 / 5312          3.1         325.0       1.0X
InMemoryTable Vectorized (Pushdown)           2292 / 2522          6.9         145.7       2.2X
```

Closes #23004 from wangyum/SPARK-26004.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 19:18:32 +08:00
Sean Owen 5ebb4b5723 [SPARK-24783][SQL] spark.sql.shuffle.partitions=0 should throw exception
## What changes were proposed in this pull request?

Throw an exception if spark.sql.shuffle.partitions=0
This takes over https://github.com/apache/spark/pull/23835

## How was this patch tested?

Existing tests.

Closes #24008 from srowen/SPARK-24783.2.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: WindCanDie <491237260@qq.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-08 14:09:53 +09:00
Jungtaek Lim (HeartSaVioR) d8f77e11a4 [SPARK-27001][SQL][FOLLOWUP] Address primitive array type for serializer
## What changes were proposed in this pull request?

This is follow-up PR which addresses review comment in PR for SPARK-27001:
https://github.com/apache/spark/pull/23908#discussion_r261511454

This patch proposes addressing primitive array type for serializer - instead of handling it to generic one, Spark now handles it efficiently as primitive array.

## How was this patch tested?

UT modified to include primitive array.

Closes #24015 from HeartSaVioR/SPARK-27001-FOLLOW-UP-java-primitive-array.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 11:54:04 +08:00
Yuming Wang 43dcb91a4c [SPARK-19678][FOLLOW-UP][SQL] Add behavior change test when table statistics are incorrect
## What changes were proposed in this pull request?

Since Spark 2.2.0 ([SPARK-19678](https://issues.apache.org/jira/browse/SPARK-19678)), the below SQL changed from `broadcast join` to `sort merge join`:
```sql
-- small external table with incorrect statistics
CREATE EXTERNAL TABLE t1(c1 int)
ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
WITH SERDEPROPERTIES (
  'serialization.format' = '1'
)
STORED AS
  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION 'file:///tmp/t1'
TBLPROPERTIES (
'rawDataSize'='-1', 'numFiles'='0', 'totalSize'='0', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='-1'
);

-- big table
CREATE TABLE t2 (c1 int)
LOCATION 'file:///tmp/t2'
TBLPROPERTIES (
'rawDataSize'='23437737', 'numFiles'='12222', 'totalSize'='333442230', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='443442223'
);

explain SELECT t1.c1 FROM t1 INNER JOIN t2 ON t1.c1 = t2.c1;
```
This pr add a test case for this behavior change.

## How was this patch tested?

unit tests

Closes #24003 from wangyum/SPARK-19678.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 11:47:49 +08:00
Yuming Wang d70b6a39e1 [MINOR][BUILD] Add 2 maven properties(hive.classifier and hive.parquet.group)
## What changes were proposed in this pull request?

This pr adds 2 maven properties to help us upgrade the built-in Hive.

| Property Name | Default | In future |
| ------ | ------ | ------ |
| hive.classifier | (none) | core |
| hive.parquet.group | com.twitter | org.apache.parquet |

## How was this patch tested?

existing tests

Closes #23996 from wangyum/add_2_maven_properties.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-07 16:46:07 -06:00
Shahid 713646ddc2 [SPARK-27075] Remove duplicate execution tag parameters from the url, when accessing the execution table in the SQL page
## What changes were proposed in this pull request?

When we sort any columns in the execution table of the SQL page in the WEBUI, it throws IllegalArgumentException. The root cause is that,  in the url, we are duplicating the execution tag parameters in the 'parameterPath'. Actually we should filter out the executionTag related entries while getting the 'parameterOtherTable'
e9e8bb33ef/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala (L161-L163)
e9e8bb33ef/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala (L241)
e9e8bb33ef/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala (L263-L266)

## How was this patch tested?
Manually tested
Test steps:
Sort any column in the sql page execution table
Before fix:
![screenshot from 2019-03-07 01-38-17](https://user-images.githubusercontent.com/23054875/53913261-f0b69580-4080-11e9-88ea-f238b47a21d5.png)

After fix:
![screenshot from 2019-03-07 02-01-40](https://user-images.githubusercontent.com/23054875/53913285-01670b80-4081-11e9-81b6-78cdbf5a0817.png)

Closes #23994 from shahidki31/SPARK-27075.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-07 12:52:46 -08:00
Takeshi Yamamuro 315c95c399 [SPARK-25863][SPARK-21871][SQL] Check if code size statistics is empty or not in updateAndGetCompilationStats
## What changes were proposed in this pull request?
`CodeGenerator.updateAndGetCompilationStats` throws an unsupported exception for empty code size statistics. This pr added code to check if it is empty or not.

## How was this patch tested?
Pass Jenkins.

Closes #23947 from maropu/SPARK-21871-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-07 17:25:22 +09:00
Dilip Biswal a0e26cffc5 [MINOR][SQL][TEST] Include usage example for generating output for single test in SQLQueryTestSuite
## What changes were proposed in this pull request?
This is a very minor pr to include the usage example to generate output for single test in SQLQueryTestSuite. I tried to deduce it from the existing example and ran into a scenario
where sbt is simply looping to run the same test over and over again. Here is the example
of running a single test.

```
build/sbt "~sql/test-only *SQLQueryTestSuite -- -z inline-table.sql"
```
I tried to generate the output for a single test by prepending `SPARK_GENERATE_GOLDEN_FILES=1` like following
```
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "~sql/test-only *SQLQueryTestSuite -- -z describe.sql"
```
In this case i found that sbt is looping trying to run describe.sql over and over again as we are running the test in on continuous mode (because of `~` prefix ) where it detects a change in
the generated result file which in turn triggers a build and test. I have included an example where
we don't run it in continuous mode when generating the output. Hopefully it saves other developers some time.
## How was this patch tested?
Verified manually in my dev setup.

Closes #23995 from dilipbiswal/dkb_sqlquerytest_usage.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-07 13:06:23 +09:00
Gengliang Wang a543f917e0 [SPARK-27049][SQL] Create util class to support handling partition values in file source V2
## What changes were proposed in this pull request?

While I am migrating other data sources, I find that we should abstract the logic that:
1. converting safe `InternalRow`s into `UnsafeRow`s
2. appending partition values to the end of the result row if existed

This PR proposes to support handling partition values in file source v2 abstraction by adding a util class `PartitionReaderWithPartitionValues`.

## How was this patch tested?

Existing unit tests

Closes #23987 from gengliangwang/SPARK-27049.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-07 11:24:15 +08:00
Yuming Wang 32848eecc5 [SPARK-27078][SQL] Fix NoSuchFieldError when read Hive materialized views
## What changes were proposed in this pull request?

This pr fix `NoSuchFieldError` when reading Hive materialized views from Hive 2.3.4.

How to reproduce:
Hive side:
```sql
CREATE TABLE materialized_view_tbl (key INT);
CREATE MATERIALIZED VIEW view_1 DISABLE REWRITE AS SELECT * FROM materialized_view_tbl;
```
Spark side:
```java
bin/spark-sql --conf spark.sql.hive.metastore.version=2.3.4 --conf spark.sql.hive.metastore.jars=maven

spark-sql> select * from view_1;
19/03/05 19:55:37 ERROR SparkSQLDriver: Failed in [select * from view_1]
java.lang.NoSuchFieldError: INDEX_TABLE
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$3(HiveClientImpl.scala:438)
	at scala.Option.map(Option.scala:163)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$1(HiveClientImpl.scala:370)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:277)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:215)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:214)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:260)
	at org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:368)
```

## How was this patch tested?

unit tests

Closes #23984 from wangyum/SPARK-24360.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-06 16:56:32 -08:00
Maxim Gekk 9513d82edd [SPARK-27057][SQL] Common trait for limit exec operators
## What changes were proposed in this pull request?

I would like to refactor `limit.scala` slightly and introduce common trait `LimitExec` for `CollectLimitExec` and `BaseLimitExec` (`LocalLimitExec` and `GlobalLimitExec`). This will allow to distinguish those operators from others, and to get the `limit` value without casting to concrete class.

## How was this patch tested?

by existing test suites.

Closes #23976 from MaxGekk/limit-exec.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-07 08:47:52 +08:00
Shahid 62fd133f74 [SPARK-27019][SQL][WEBUI] onJobStart happens after onExecutionEnd shouldn't overwrite kvstore
## What changes were proposed in this pull request?
Currently, when the event reordering happens, especially onJobStart event come after onExecutionEnd event, SQL page in the UI displays weirdly.(for eg:test mentioned in JIRA and also this issue randomly occurs when the TPCDS query  fails due to broadcast timeout etc.)

The reason is that, In the SQLAppstatusListener, we remove the liveExecutions entry once the execution ends. So, if a jobStart event come after that, then we create a new liveExecution entry corresponding to the execId. Eventually this will overwrite the kvstore and UI displays confusing entries.

## How was this patch tested?

Added UT, Also manually tested with the eventLog, provided in the jira, of the failed query.

Before fix:
![screenshot from 2019-03-03 03-05-52](https://user-images.githubusercontent.com/23054875/53687929-53e2b800-3d61-11e9-9dca-620fa41e605c.png)

After fix:
![screenshot from 2019-03-03 02-40-18](https://user-images.githubusercontent.com/23054875/53687928-4f1e0400-3d61-11e9-86aa-584646ac68f9.png)

Closes #23939 from shahidki31/SPARK-27019.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-06 14:02:30 -08:00