Commit graph

3664 commits

Author SHA1 Message Date
mingbo.pb 66f5a42ca5 [SPARK-27638][SQL] Cast string to date/timestamp in binary comparisons with dates/timestamps
## What changes were proposed in this pull request?

The below example works with both Mysql and Hive, however not with spark.

```
mysql> select * from date_test where date_col >= '2000-1-1';
+------------+
| date_col   |
+------------+
| 2000-01-01 |
+------------+
```
The reason is that Spark casts both sides to String type during date and string comparison for partial date support. Please find more details in https://issues.apache.org/jira/browse/SPARK-8420.

Based on some tests, the behavior of Date and String comparison in Hive and Mysql:
Hive: Cast to Date, partial date is not supported
Mysql: Cast to Date, certain "partial date" is supported by defining certain date string parse rules. Check out str_to_datetime in https://github.com/mysql/mysql-server/blob/5.5/sql-common/my_time.c

As below date patterns have been supported, the PR is to cast string to date when comparing string and date:
```
`yyyy`
`yyyy-[m]m`
`yyyy-[m]m-[d]d`
`yyyy-[m]m-[d]d `
`yyyy-[m]m-[d]d *`
`yyyy-[m]m-[d]dT*
```

## How was this patch tested?
UT has been added

Closes #24567 from pengbo/SPARK-27638.

Authored-by: mingbo.pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-14 17:10:36 +08:00
Liang-Chi Hsieh 8b0bdaa8e0 [SPARK-27671][SQL] Fix error when casting from a nested null in a struct
## What changes were proposed in this pull request?

When a null in a nested field in struct, casting from the struct throws error, currently.

```scala
scala> sql("select cast(struct(1, null) as struct<a:int,b:int>)").show
scala.MatchError: NullType (of class org.apache.spark.sql.types.NullType$)
  at org.apache.spark.sql.catalyst.expressions.Cast.castToInt(Cast.scala:447)
  at org.apache.spark.sql.catalyst.expressions.Cast.cast(Cast.scala:635)
  at org.apache.spark.sql.catalyst.expressions.Cast.$anonfun$castStruct$1(Cast.scala:603)
```

Similarly, inline table, which casts null in nested field under the hood, also throws an error.

```scala
scala> sql("select * FROM VALUES (('a', (10, null))), (('b', (10, 50))), (('c', null)) AS tab(x, y)").show
org.apache.spark.sql.AnalysisException: failed to evaluate expression named_struct('col1', 10, 'col2', NULL): NullType (of class org.apache.spark.sql.t
ypes.NullType$); line 1 pos 14
  at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:47)
  at org.apache.spark.sql.catalyst.analysis.ResolveInlineTables.$anonfun$convert$6(ResolveInlineTables.scala:106)
```

This fixes the issue.

## How was this patch tested?

Added tests.

Closes #24576 from viirya/cast-null.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-13 12:40:46 -07:00
Liang-Chi Hsieh d169b0aac3 [SPARK-27653][SQL] Add max_by() and min_by() SQL aggregate functions
## What changes were proposed in this pull request?

This PR goes to add `max_by()` and `min_by()` SQL aggregate functions.

Quoting from the [Presto docs](https://prestodb.github.io/docs/current/functions/aggregate.html#max_by)

> max_by(x, y) → [same as x]
> Returns the value of x associated with the maximum value of y over all input values.

`min_by()` works similarly.

## How was this patch tested?

Added tests.

Closes #24557 from viirya/SPARK-27653.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-13 22:37:34 +08:00
zhoukang 126310ca68 [SPARK-26601][SQL] Make broadcast-exchange thread pool configurable
## What changes were proposed in this pull request?

Currently,thread number of broadcast-exchange thread pool is fixed and keepAliveSeconds is also fixed as 60s.

```
object BroadcastExchangeExec {
  private[execution] val executionContext = ExecutionContext.fromExecutorService(
    ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128))
}

 /**
   * Create a cached thread pool whose max number of threads is `maxThreadNumber`. Thread names
   * are formatted as prefix-ID, where ID is a unique, sequentially assigned integer.
   */
  def newDaemonCachedThreadPool(
      prefix: String, maxThreadNumber: Int, keepAliveSeconds: Int = 60): ThreadPoolExecutor = {
    val threadFactory = namedThreadFactory(prefix)
    val threadPool = new ThreadPoolExecutor(
      maxThreadNumber, // corePoolSize: the max number of threads to create before queuing the tasks
      maxThreadNumber, // maximumPoolSize: because we use LinkedBlockingDeque, this one is not used
      keepAliveSeconds,
      TimeUnit.SECONDS,
      new LinkedBlockingQueue[Runnable],
      threadFactory)
    threadPool.allowCoreThreadTimeOut(true)
    threadPool
  }
```

But some times, if the Thead object do not GC quickly it may caused server(driver) OOM. In such case,we need to make this thread pool configurable.
A case has described in https://issues.apache.org/jira/browse/SPARK-26601

## How was this patch tested?
UT

Closes #23670 from caneGuy/zhoukang/make-broadcat-config.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-13 20:40:21 +09:00
HyukjinKwon c71f217de1 [SPARK-27673][SQL] Add since info to random, regex, null expressions
## What changes were proposed in this pull request?

We should add since info to all expressions.

SPARK-7886 Rand / Randn
af3746ce0d RLike, Like (I manually checked that it exists from 1.0.0)
SPARK-8262 Split
SPARK-8256 RegExpReplace
SPARK-8255 RegExpExtract
9aadcffabd Coalesce / IsNull / IsNotNull (I manually checked that it exists from 1.0.0)
SPARK-14541 IfNull / NullIf / Nvl / Nvl2
SPARK-9080 IsNaN
SPARK-9168 NaNvl

## How was this patch tested?

N/A

Closes #24579 from HyukjinKwon/SPARK-27673.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-10 09:24:04 -07:00
HyukjinKwon 3442fcaa9b [SPARK-27672][SQL] Add since info to string expressions
## What changes were proposed in this pull request?

This PR adds since information to the all string expressions below:

SPARK-8241 ConcatWs
SPARK-16276 Elt
SPARK-1995 Upper / Lower
SPARK-20750 StringReplace
SPARK-8266 StringTranslate
SPARK-8244 FindInSet
SPARK-8253 StringTrimLeft
SPARK-8260 StringTrimRight
SPARK-8267 StringTrim
SPARK-8247 StringInstr
SPARK-8264 SubstringIndex
SPARK-8249 StringLocate
SPARK-8252 StringLPad
SPARK-8259 StringRPad
SPARK-16281 ParseUrl
SPARK-9154 FormatString
SPARK-8269 Initcap
SPARK-8257 StringRepeat
SPARK-8261 StringSpace
SPARK-8263 Substring
SPARK-21007 Right
SPARK-21007 Left
SPARK-8248 Length
SPARK-20749 BitLength
SPARK-20749 OctetLength
SPARK-8270 Levenshtein
SPARK-8271 SoundEx
SPARK-8238 Ascii
SPARK-20748 Chr
SPARK-8239 Base64
SPARK-8268 UnBase64
SPARK-8242 Decode
SPARK-8243 Encode
SPARK-8245 format_number
SPARK-16285 Sentences

## How was this patch tested?

N/A

Closes #24578 from HyukjinKwon/SPARK-27672.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-10 09:11:12 -07:00
Marco Gaido 78748b5752 [SPARK-27625][SQL] ScalaReflection support for annotated types
## What changes were proposed in this pull request?

If a type is annotated, `ScalaReflection` can fail if the datatype is an `Option`, a `Seq`, a `Map` and other similar types. This is because it assumes we are dealing with `TypeRef`, while types with annotations are `AnnotatedType`.

The PR deals with the case the annotation is present.

## How was this patch tested?

added UT

Closes #24564 from mgaido91/SPARK-27625.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-10 22:48:36 +08:00
pgandhi 0969d7aa0c [SPARK-27207][SQL] : Ensure aggregate buffers are initialized again for So…
…rtBasedAggregate

Normally, the aggregate operations that are invoked for an aggregation buffer for User Defined Aggregate Functions(UDAF) follow the order like initialize(), update(), eval() OR initialize(), merge(), eval(). However, after a certain threshold configurable by spark.sql.objectHashAggregate.sortBased.fallbackThreshold is reached, ObjectHashAggregate falls back to SortBasedAggregator which invokes the merge or update operation without calling initialize() on the aggregate buffer.

## What changes were proposed in this pull request?

The fix here is to initialize aggregate buffers again when fallback to SortBasedAggregate operator happens.

## How was this patch tested?

The patch was tested as part of [SPARK-24935](https://issues.apache.org/jira/browse/SPARK-24935) as documented in PR https://github.com/apache/spark/pull/23778.

Closes #24149 from pgandhi999/SPARK-27207.

Authored-by: pgandhi <pgandhi@verizonmedia.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-09 11:12:20 +08:00
Jose Torres 83f628b57d [SPARK-27253][SQL][FOLLOW-UP] Add a legacy flag to restore old session init behavior
## What changes were proposed in this pull request?

Add a legacy flag to restore the old session init behavior, where SparkConf defaults take precedence over configs in a parent session.

Closes #24540 from jose-torres/oss.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-07 20:04:09 -07:00
Ryan Blue 303ee3fce0 [SPARK-24252][SQL] Add TableCatalog API
## What changes were proposed in this pull request?

This adds the TableCatalog API proposed in the [Table Metadata API SPIP](https://docs.google.com/document/d/1zLFiA1VuaWeVxeTDXNg8bL6GP3BVoOZBkewFtEnjEoo/edit#heading=h.m45webtwxf2d).

For `TableCatalog` to use `Table`, it needed to be moved into the catalyst module where the v2 catalog API is located. This also required moving `TableCapability`. Most of the files touched by this PR are import changes needed by this move.

## How was this patch tested?

This adds a test implementation and contract tests.

Closes #24246 from rdblue/SPARK-24252-add-table-catalog-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-08 10:31:06 +08:00
Adi Muraru 8ef4da753d [SPARK-27610][YARN] Shade netty native libraries
## What changes were proposed in this pull request?

Fixed the `spark-<version>-yarn-shuffle.jar` artifact packaging to shade the native netty libraries:
- shade the `META-INF/native/libnetty_*` native libraries when packagin
the yarn shuffle service jar. This is required as netty library loader
derives that based on shaded package name.
- updated the `org/spark_project` shade package prefix to `org/sparkproject`
(i.e. removed underscore) as the former breaks the netty native lib loading.

This was causing the yarn external shuffle service to fail
when spark.shuffle.io.mode=EPOLL

## How was this patch tested?
Manual tests

Closes #24502 from amuraru/SPARK-27610_master.

Authored-by: Adi Muraru <amuraru@adobe.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 10:47:36 -07:00
gaoweikang 3859ca37d9 [SPARK-27586][SQL] Improve binary comparison: replace Scala's for-comprehension if statements with while loop
## What changes were proposed in this pull request?

This PR replaces for-comprehension if statement with while loop to gain better performance in `TypeUtils.compareBinary`.

## How was this patch tested?

Add UT to test old version and new version comparison result

Closes #24494 from woudygao/opt_binary_compare.

Authored-by: gaoweikang <gaoweikang@bytedance.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-02 20:33:27 -07:00
Marco Gaido 7a8cc8e071 [SPARK-27607][SQL] Improve Row.toString performance
## What changes were proposed in this pull request?

`Row.toString` is currently causing the useless creation of an `Array` containing all the values in the row before generating the string containing it. This operation adds a considerable overhead.

The PR proposes to avoid this operation in order to get a faster implementation.

## How was this patch tested?

Run

```scala
test("Row toString perf test") {
    val n = 100000
    val rows = (1 to n).map { i =>
      Row(i, i.toDouble, i.toString, i.toShort, true, null)
    }
    // warmup
    (1 to 10).foreach { _ => rows.foreach(_.toString) }

    val times = (1 to 100).map { _ =>
      val t0 = System.nanoTime()
      rows.foreach(_.toString)
      val t1 = System.nanoTime()
      t1 - t0
    }
    // scalastyle:off println
    println(s"Avg time on ${times.length} iterations for $n toString:" +
      s" ${times.sum.toDouble / times.length / 1e6} ms")
    // scalastyle:on println
  }
```
Before the PR:
```
Avg time on 100 iterations for 100000 toString: 61.08408419 ms
```
After the PR:
```
Avg time on 100 iterations for 100000 toString: 38.16539432 ms
```
This means the new implementation is about 1.60X faster than the original one.

Closes #24505 from mgaido91/SPARK-27607.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-02 07:20:33 -07:00
HyukjinKwon df8aa7ba8a [SPARK-27606][SQL] Deprecate 'extended' field in ExpressionDescription/ExpressionInfo
## What changes were proposed in this pull request?

After we added other fields, `arguments`, `examples`, `note` and `since` at SPARK-21485 and `deprecated` at SPARK-27328, we have nicer way to separately describe extended usages.

`extended` field and method at `ExpressionDescription`/`ExpressionInfo` is now pretty useless - it's not used in Spark side and only exists to keep backward compatibility.

This PR proposes to deprecate it.

## How was this patch tested?

Manually checked the deprecation waring is properly shown.

Closes #24500 from HyukjinKwon/SPARK-27606.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-02 21:10:00 +09:00
gatorsmile 2da406cae5 [SPARK-27618][SQL][FOLLOW-UP] Unnecessary access to externalCatalog
## What changes were proposed in this pull request?
This PR is to add test cases for ensuring that we do not have unnecessary access to externalCatalog.

In the future, we can follow these examples to improve our test coverage in this area.

## How was this patch tested?
N/A

Closes #24511 from gatorsmile/addTestcaseSpark-27618.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-01 20:09:46 -07:00
HyukjinKwon 3670826af6 [SPARK-26921][R][DOCS] Document Arrow optimization and vectorized R APIs
## What changes were proposed in this pull request?

This PR adds SparkR with Arrow optimization documentation.

Note that looks CRAN issue in Arrow side won't look likely fixed soon, IMHO, even after Spark 3.0.
If it happen to be fixed, I will fix this doc too later.

Another note is that Arrow R package itself requires R 3.5+. So, I intentionally didn't note this.

## How was this patch tested?

Manually built and checked.

Closes #24506 from HyukjinKwon/SPARK-26924.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-02 10:02:14 +09:00
Artem Kalchenko a35043c9e2 [SPARK-27591][SQL] Fix UnivocityParser for UserDefinedType
## What changes were proposed in this pull request?

Fix bug in UnivocityParser. makeConverter method didn't work correctly for UsedDefinedType

## How was this patch tested?

A test suite for UnivocityParser has been extended.

Closes #24496 from kalkolab/spark-27591.

Authored-by: Artem Kalchenko <artem.kalchenko@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-01 08:27:51 +09:00
Xiangrui Meng 618d6bff71 [SPARK-27588] Binary file data source fails fast and doesn't attempt to read very large files
## What changes were proposed in this pull request?

If a file is too big (>2GB), we should fail fast and do not try to read the file.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #24483 from mengxr/SPARK-27588.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-04-29 16:24:49 -07:00
Sean Owen 8a17d26784 [SPARK-27536][CORE][ML][SQL][STREAMING] Remove most use of scala.language.existentials
## What changes were proposed in this pull request?

I want to get rid of as much use of `scala.language.existentials` as possible for 3.0. It's a complicated language feature that generates warnings unless this value is imported. It might even be on the way out of Scala: https://contributors.scala-lang.org/t/proposal-to-remove-existential-types-from-the-language/2785

For Spark, it comes up mostly where the code plays fast and loose with generic types, not the advanced situations you'll often see referenced where this feature is explained. For example, it comes up in cases where a function returns something like `(String, Class[_])`. Scala doesn't like matching this to any other instance of `(String, Class[_])` because doing so requires inferring the existence of some type that satisfies both. Seems obvious if the generic type is a wildcard, but, not technically something Scala likes to let you get away with.

This is a large PR, and it only gets rid of _most_ instances of `scala.language.existentials`. The change should be all compile-time and shouldn't affect APIs or logic.

Many of the changes simply touch up sloppiness about generic types, making the known correct value explicit in the code.

Some fixes involve being more explicit about the existence of generic types in methods. For instance, `def foo(arg: Class[_])` seems innocent enough but should really be declared `def foo[T](arg: Class[T])` to let Scala select and fix a single type when evaluating calls to `foo`.

For kind of surprising reasons, this comes up in places where code evaluates a tuple of things that involve a generic type, but is OK if the two parts of the tuple are evaluated separately.

One key change was altering `Utils.classForName(...): Class[_]` to the more correct `Utils.classForName[T](...): Class[T]`. This caused a number of small but positive changes to callers that otherwise had to cast the result.

In several tests, `Dataset[_]` was used where `DataFrame` seems to be the clear intent.

Finally, in a few cases in MLlib, the return type `this.type` was used where there are no subclasses of the class that uses it. This really isn't needed and causes issues for Scala reasoning about the return type. These are just changed to be concrete classes as return types.

After this change, we have only a few classes that still import `scala.language.existentials` (because modifying them would require extensive rewrites to fix) and no build warnings.

## How was this patch tested?

Existing tests.

Closes #24431 from srowen/SPARK-27536.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:02:01 -05:00
Jash Gala 90085a1847 [SPARK-23619][DOCS] Add output description for some generator expressions / functions
## What changes were proposed in this pull request?

This PR addresses SPARK-23619: https://issues.apache.org/jira/browse/SPARK-23619

It adds additional comments indicating the default column names for the `explode` and `posexplode`
functions in Spark-SQL.

Functions for which comments have been updated so far:
* stack
* inline
* explode
* posexplode
* explode_outer
* posexplode_outer

## How was this patch tested?

This is just a change in the comments. The package builds and tests successfullly after the change.

Closes #23748 from jashgala/SPARK-23619.

Authored-by: Jash Gala <jashgala@amazon.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-27 10:30:12 +09:00
uncleGen 6328be78f9 [MINOR][TEST][DOC] Execute action miss name message
## What changes were proposed in this pull request?

some minor updates:
- `Execute` action miss `name` message
-  typo in SS document
-  typo in SQLConf

## How was this patch tested?

N/A

Closes #24466 from uncleGen/minor-fix.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-27 09:28:31 +08:00
Liang-Chi Hsieh 8b86326521 [SPARK-27551][SQL] Improve error message of mismatched types for CASE WHEN
## What changes were proposed in this pull request?

When there are mismatched types among cases or else values in case when expression, current error message is hard to read to figure out what and where the mismatch is.

This patch simply improves the error message for mismatched types for case when.

Before:
```scala
scala> spark.range(100).select(when('id === 1, array(struct('id * 123456789 + 123456789 as "x"))).otherwise(array(struct('id * 987654321 + 987654321 as
 "y"))))
org.apache.spark.sql.AnalysisException: cannot resolve 'CASE WHEN (`id` = CAST(1 AS BIGINT)) THEN array(named_struct('x', ((`id` * CAST(123456789 AS BI
GINT)) + CAST(123456789 AS BIGINT)))) ELSE array(named_struct('y', ((`id` * CAST(987654321 AS BIGINT)) + CAST(987654321 AS BIGINT)))) END' due to data
type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type;;
```

After:
```scala
scala> spark.range(100).select(when('id === 1, array(struct('id * 123456789 + 123456789 as "x"))).otherwise(array(struct('id * 987654321 + 987654321 as
 "y"))))
org.apache.spark.sql.AnalysisException: cannot resolve 'CASE WHEN (`id` = CAST(1 AS BIGINT)) THEN array(named_struct('x', ((`id` * CAST(123456789 AS BI
GINT)) + CAST(123456789 AS BIGINT)))) ELSE array(named_struct('y', ((`id` * CAST(987654321 AS BIGINT)) + CAST(987654321 AS BIGINT)))) END' due to data
type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN array<struct<x:bigint>> ELSE arr
ay<struct<y:bigint>> END;;
```

## How was this patch tested?

Added unit test.

Closes #24453 from viirya/SPARK-27551.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-25 08:47:19 -07:00
HyukjinKwon a30983db57 [SPARK-27512][SQL] Avoid to replace ',' in CSV's decimal type inference for backward compatibility
## What changes were proposed in this pull request?

The code below currently infers as decimal but previously it was inferred as string.

**In branch-2.4**, type inference path for decimal and parsing data are different.

2a8343121e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala (L153)

c284c4e1f6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala (L125)

So the code below:

```scala
scala> spark.read.option("delimiter", "|").option("inferSchema", "true").csv(Seq("1,2").toDS).printSchema()
```

produced string as its type.

```
root
 |-- _c0: string (nullable = true)
```

**In the current master**, it now infers decimal as below:

```
root
 |-- _c0: decimal(2,0) (nullable = true)
```

It happened after https://github.com/apache/spark/pull/22979 because, now after this PR, we only have one way to parse decimal:

7a83d71403/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala (L92)

**After the fix:**

```
root
 |-- _c0: string (nullable = true)
```

This PR proposes to restore the previous behaviour back in `CSVInferSchema`.

## How was this patch tested?

Manually tested and unit tests were added.

Closes #24437 from HyukjinKwon/SPARK-27512.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 16:22:07 +09:00
Gengliang Wang 00f2f311f7 [SPARK-27128][SQL] Migrate JSON to File Data Source V2
## What changes were proposed in this pull request?
Migrate JSON to File Data Source V2

## How was this patch tested?

Unit test

Closes #24058 from gengliangwang/jsonV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-23 22:39:59 +08:00
pengbo d9b2ce0f0f [SPARK-27539][SQL] Fix inaccurate aggregate outputRows estimation with column containing null values
## What changes were proposed in this pull request?
This PR is follow up of https://github.com/apache/spark/pull/24286. As gatorsmile pointed out that column with null value is inaccurate as well.

```
> select key from test;
2
NULL
1
spark-sql> desc extended test key;
col_name key
data_type int
comment NULL
min 1
max 2
num_nulls 1
distinct_count 2
```

The distinct count should be distinct_count + 1 when column contains null value.
## How was this patch tested?

Existing tests & new UT added.

Closes #24436 from pengbo/aggregation_estimation.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-22 20:30:08 -07:00
Maxim Gekk 43a73e387c [SPARK-27528][SQL] Use Parquet logical type TIMESTAMP_MICROS by default
## What changes were proposed in this pull request?

In the PR, I propose to use the `TIMESTAMP_MICROS` logical type for timestamps written to parquet files. The type matches semantically to Catalyst's `TimestampType`, and stores microseconds since epoch in UTC time zone. This will allow to avoid conversions of microseconds to nanoseconds and to Julian calendar. Also this will reduce sizes of written parquet files.

## How was this patch tested?

By existing test suites.

Closes #24425 from MaxGekk/parquet-timestamp_micros.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-23 11:06:39 +09:00
Maxim Gekk 79d3bc0409 [SPARK-27438][SQL] Parse strings with timestamps by to_timestamp() in microsecond precision
## What changes were proposed in this pull request?

In the PR, I propose to parse strings to timestamps in microsecond precision by the ` to_timestamp()` function if the specified pattern contains a sub-pattern for seconds fractions.

Closes #24342

## How was this patch tested?

By `DateFunctionsSuite` and `DateExpressionsSuite`

Closes #24420 from MaxGekk/to_timestamp-microseconds3.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-22 19:41:32 +08:00
Maxim Gekk d61b3bc875 [SPARK-27527][SQL][DOCS] Improve descriptions of Timestamp and Date types
## What changes were proposed in this pull request?

In the PR, I propose more precise description of `TimestampType` and `DateType`, how they store timestamps and dates internally.

Closes #24424 from MaxGekk/timestamp-date-type-doc.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-21 16:53:11 +09:00
Yifei Huang 163a6e2982 [SPARK-27514] Skip collapsing windows with empty window expressions
## What changes were proposed in this pull request?

A previous change moved the removal of empty window expressions to the RemoveNoopOperations rule, which comes after the CollapseWindow rule. Therefore, by the time we get to CollapseWindow, we aren't guaranteed that empty windows have been removed. This change checks that the window expressions are not empty, and only collapses the windows if both windows are non-empty.

A lengthier description and repro steps here: https://issues.apache.org/jira/browse/SPARK-27514

## How was this patch tested?

A unit test, plus I reran the breaking case mentioned in the Jira ticket.

Closes #24411 from yifeih/yh/spark-27514.

Authored-by: Yifei Huang <yifeih@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-19 14:04:44 +08:00
Kris Mok 50bdc9befa [SPARK-27423][SQL][FOLLOWUP] Minor polishes to Cast codegen templates for Date <-> Timestamp
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/24332 introduced an unnecessary `import` statement and two slight issues in the codegen templates in `Cast` for `Date` <-> `Timestamp`.
This PR removes the unused import statement and fixes the slight codegen issue.

The issue in those two codegen templates is this pattern:
```scala
val zid = JavaCode.global(
  ctx.addReferenceObj("zoneId", zoneId, "java.time.ZoneId"),
  zoneId.getClass)
```
`zoneId` can refer to an instance of a non-public class, e.g. `java.time.ZoneRegion`, and while this code correctly puts in the 3rd argument to `ctx.addReferenceObj()`, it's still passing `zoneId.getClass` to `JavaCode.global()` which is not desirable, but doesn't cause any immediate bugs in this particular case, because `zid` is used in an expression immediately afterwards.
If this `zid` ever needs to spill to any explicitly typed variables, e.g. a local variable, and if the spill handling uses the `javaType` on this `GlobalVariable`, it'd generate code that looks like:
```java
java.time.ZoneRegion value1 = ((java.time.ZoneId) references[2] /* literal */);
```
which would then be a real bug:
- a non-accessible type `java.time.ZoneRegion` is referenced in the generated code, and
- `ZoneId` -> `ZoneRegion` requires an explicit downcast.

## How was this patch tested?

Existing tests. This PR does not change behavior, and the original PR won't cause any real behavior bug to begin with.

Closes #24392 from rednaxelafx/spark-27423-followup.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-18 14:27:33 +08:00
Dilip Biswal e1c90d66bb [SPARK-19712][SQL] Pushdown LeftSemi/LeftAnti below join
## What changes were proposed in this pull request?
This PR adds support for pushing down LeftSemi and LeftAnti joins below the Join operator.
This is a prerequisite work thats needed for the subsequent task of moving the subquery rewrites to the beginning of optimization phase.

The larger  PR is [here](https://github.com/apache/spark/pull/23211) . This PR addresses the comment at [link](https://github.com/apache/spark/pull/23211#issuecomment-445705922).
## How was this patch tested?
Added tests under LeftSemiAntiJoinPushDownSuite.

Closes #24331 from dilipbiswal/SPARK-19712-pushleftsemi-belowjoin.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 20:30:20 +08:00
pengbo 54b0d1e0ef [SPARK-27416][SQL] UnsafeMapData & UnsafeArrayData Kryo serialization …
## What changes were proposed in this pull request?
Finish the rest work of https://github.com/apache/spark/pull/24317, https://github.com/apache/spark/pull/9030
a. Implement Kryo serialization for UnsafeArrayData
b. fix UnsafeMapData Java/Kryo Serialization issue when two machines have different Oops size
c. Move the duplicate code "getBytes()" to Utils.

## How was this patch tested?
According Units has been added & tested

Closes #24357 from pengbo/SPARK-27416_new.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 13:03:00 +08:00
liwensun 26ed65f415 [SPARK-27453] Pass partitionBy as options in DataFrameWriter
## What changes were proposed in this pull request?

Pass partitionBy columns as options and feature-flag this behavior.

## How was this patch tested?

A new unit test.

Closes #24365 from liwensun/partitionby.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2019-04-16 15:03:16 -07:00
Liang-Chi Hsieh b404e02574 [SPARK-27476][SQL] Refactoring SchemaPruning rule to remove duplicate code
## What changes were proposed in this pull request?

In SchemaPruning rule, there is duplicate code for data source v1 and v2. Their logic is the same and we can refactor the rule to remove duplicate code.

## How was this patch tested?

Existing tests.

Closes #24383 from viirya/SPARK-27476.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-16 14:50:37 -07:00
pengbo c58a4fed8d [SPARK-27351][SQL] Wrong outputRows estimation after AggregateEstimation wit…
## What changes were proposed in this pull request?
The upper bound of group-by columns row number is to multiply distinct counts of group-by columns. However, column with only null value will cause the output row number to be 0 which is incorrect.
Ex:
col1 (distinct: 2, rowCount 2)
col2 (distinct: 0, rowCount 2)
=> group by col1, col2
Actual: output rows: 0
Expected: output rows: 2

## How was this patch tested?
According unit test has been added, plus manual test has been done in our tpcds benchmark environement.

Closes #24286 from pengbo/master.

Lead-authored-by: pengbo <bo.peng1019@gmail.com>
Co-authored-by: mingbo_pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-15 15:37:07 -07:00
Wenchen Fan 0407070945 [SPARK-27444][SQL] multi-select can be used in subquery
## What changes were proposed in this pull request?

This is a regression caused by https://github.com/apache/spark/pull/24150

`select * from (from a select * select *)` is supported in 2.4, and we should keep supporting it.

This PR merges the parser rule for single and multi select statements, as they are very similar.

## How was this patch tested?

a new test case

Closes #24348 from cloud-fan/parser.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 20:57:34 +08:00
Kris Mok bbbe54aa79 [SPARK-27199][SQL][FOLLOWUP] Fix bug in codegen templates in UnixTime and FromUnixTime
## What changes were proposed in this pull request?

SPARK-27199 introduced the use of `ZoneId` instead of `TimeZone` in a few date/time expressions.
There were 3 occurrences of `ctx.addReferenceObj("zoneId", zoneId)` in that PR, which had a bug because while the `java.time.ZoneId` base type is public, the actual concrete implementation classes are not public, so using the 2-arg version of `CodegenContext.addReferenceObj` would incorrectly generate code that reference non-public types (`java.time.ZoneRegion`, to be specific). The 3-arg version should be used, with the class name of the referenced object explicitly specified to the public base type.

One of such occurrences was caught in testing in the main PR of SPARK-27199 (https://github.com/apache/spark/pull/24141), for `DateFormatClass`. But the other 2 occurrences slipped through because there were no test cases that covered them.

Example of this bug in the current Apache Spark master, in a Spark Shell:
```
scala> Seq(("2016-04-08", "yyyy-MM-dd")).toDF("s", "f").repartition(1).selectExpr("to_unix_timestamp(s, f)").show
...
java.lang.IllegalAccessError: tried to access class java.time.ZoneRegion from class org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1
```

This PR fixes the codegen issues and adds the corresponding unit tests.

## How was this patch tested?

Enhanced tests in `DateExpressionsSuite` for `to_unix_timestamp` and `from_unixtime`.

Closes #24352 from rednaxelafx/fix-spark-27199.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 13:31:18 +08:00
maryannxue 43da473c1c [SPARK-27225][SQL] Implement join strategy hints
## What changes were proposed in this pull request?

This PR extends the existing BROADCAST join hint (for both broadcast-hash join and broadcast-nested-loop join) by implementing other join strategy hints corresponding to the rest of Spark's existing join strategies: shuffle-hash, sort-merge, cartesian-product. The hint names: SHUFFLE_MERGE, SHUFFLE_HASH, SHUFFLE_REPLICATE_NL are partly different from the code names in order to make them clearer to users and reflect the actual algorithms better.

The hinted strategy will be used for the join with which it is associated if it is applicable/doable.

Conflict resolving rules in case of multiple hints:
1. Conflicts within either side of the join: take the first strategy hint specified in the query, or the top hint node in Dataset. For example, in "select /*+ merge(t1) */ /*+ broadcast(t1) */ k1, v2 from t1 join t2 on t1.k1 = t2.k2", take "merge(t1)"; in ```df1.hint("merge").hint("shuffle_hash").join(df2)```, take "shuffle_hash". This is a general hint conflict resolving strategy, not specific to join strategy hint.
2. Conflicts between two sides of the join:
  a) In case of different strategy hints, hints are prioritized as ```BROADCAST``` over ```SHUFFLE_MERGE``` over ```SHUFFLE_HASH``` over ```SHUFFLE_REPLICATE_NL```.
  b) In case of same strategy hints but conflicts in build side, choose the build side based on join type and size.

## How was this patch tested?

Added new UTs.

Closes #24164 from maryannxue/join-hints.

Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-12 00:14:37 +08:00
chakravarthiT 074533334d [SPARK-27088][SQL] Add a configuration to set log level for each batch at RuleExecutor
## What changes were proposed in this pull request?

Similar to #22406 , which has made log level for plan changes by each rule configurable ,this PR is to make log level for plan changes by each batch configurable,and I have reused the same configuration: "spark.sql.optimizer.planChangeLog.level".

Config proposed in this PR ,
spark.sql.optimizer.planChangeLog.batches - enable plan change logging only for a set of specified batches, separated by commas.

## How was this patch tested?

Added UT , also tested manually and attached screenshots below.

1)Setting spark.sql.optimizer.planChangeLog.leve to warn.

![settingLogLevelToWarn](https://user-images.githubusercontent.com/45845595/54556730-8803dd00-49df-11e9-95ab-ebb0c8d735ef.png)

2)setting spark.sql.optimizer.planChangeLog.batches to Resolution and Subquery.
![settingBatchestoLog](https://user-images.githubusercontent.com/45845595/54556740-8cc89100-49df-11e9-80ab-fbbbe1ff2cdf.png)

3)  plan change logging enabled only for a set of specified batches(Resolution and Subquery)
![batchloggingOp](https://user-images.githubusercontent.com/45845595/54556788-ab2e8c80-49df-11e9-9ae0-57815f552896.png)

Closes #24136 from chakravarthiT/logBatches.

Lead-authored-by: chakravarthiT <45845595+chakravarthiT@users.noreply.github.com>
Co-authored-by: chakravarthiT <tcchakra@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-11 10:02:27 +09:00
ocaballero 181d190c60 [MINOR][SQL] Unnecessary access to externalCatalog
Necessarily access the external catalog without having to do it

## What changes were proposed in this pull request?

The existsFunction function has been changed because it unnecessarily accessed the externalCatalog to find if the database exists in cases where the function is in the functionRegistry

## How was this patch tested?

It has been tested through spark-shell and accessing the metastore logs of hive.

Inside spark-shell we use spark.table (% tableA%). SelectExpr ("trim (% columnA%)") in the current version and it appears every time:

org.apache.hadoop.hive.metastore.HiveMetaStore.audit: cmd = get_database: default

Once the change is made, no record appears

Closes #24312 from OCaballero/master.

Authored-by: ocaballero <oliver.caballero.alvarez@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-11 10:00:09 +09:00
Maxim Gekk ab8710b579 [SPARK-27423][SQL] Cast DATE <-> TIMESTAMP according to the SQL standard
## What changes were proposed in this pull request?

According to SQL standard, value of `DATE` type is union of year, month, dayInMonth, and it is independent from any time zones. To convert it to Catalyst's `TIMESTAMP`, `DATE` value should be "extended" by the time at midnight - `00:00:00`. The resulted local date+time should be considered as a timestamp in the session time zone, and casted to microseconds since epoch in `UTC` accordingly.

The reverse casting from `TIMESTAMP` to `DATE` should be performed in the similar way. `TIMESTAMP` values should be represented as a local date+time in the session time zone. And the time component should be just removed. For example, `TIMESTAMP 2019-04-10 00:10:12` -> `DATE 2019-04-10`. The resulted date is converted to days since epoch `1970-01-01`.

## How was this patch tested?

The changes were tested by existing test suites - `DateFunctionsSuite`, `DateExpressionsSuite` and `CastSuite`.

Closes #24332 from MaxGekk/cast-timestamp-to-date2.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 22:41:19 +08:00
Maxim Gekk 1470f23ec9 [SPARK-27422][SQL] current_date() should return current date in the session time zone
## What changes were proposed in this pull request?

In the PR, I propose to revert 2 commits 06abd06112 and 61561c1c2d, and take current date via `LocalDate.now` in the session time zone. The result is stored as days since epoch `1970-01-01`.

## How was this patch tested?

It was tested by `DateExpressionsSuite`, `DateFunctionsSuite`, `DateTimeUtilsSuite`, and `ComputeCurrentTimeSuite`.

Closes #24330 from MaxGekk/current-date2.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 21:54:50 +08:00
韩田田00222924 85e5d4f141 [SPARK-24872] Replace taking the $symbol with $sqlOperator in BinaryOperator's toString method
## What changes were proposed in this pull request?

For BinaryOperator's toString method, it's better to use `$sqlOperator` instead of `$symbol`.

## How was this patch tested?

We can test this patch  with unit tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #21826 from httfighter/SPARK-24872.

Authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 16:58:01 +08:00
Wenchen Fan 2e90574dd0 [SPARK-27414][SQL] make it clear that date type is timezone independent
## What changes were proposed in this pull request?

In SQL standard, date type is a union of the `year`, `month` and `day` fields. It's timezone independent, which means it does not represent a specific point in the timeline.

Spark SQL follows the SQL standard, this PR is to make it clear that date type is timezone independent
1. improve the doc to highlight that date is timezone independent.
2. when converting string to date,  uses the java time API that can directly parse a `LocalDate` from a string, instead of converting `LocalDate` to a `Instant` at UTC first.
3. when converting date to string, uses the java time API that can directly format a `LocalDate` to a string, instead of converting `LocalDate` to a `Instant` at UTC first.

2 and 3 should not introduce any behavior changes.

## How was this patch tested?

existing tests

Closes #24325 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 16:39:28 +08:00
Ryan Blue 58674d54ba [SPARK-27181][SQL] Add public transform API
## What changes were proposed in this pull request?

This adds a public Expression API that can be used to pass partition transformations to data sources.

## How was this patch tested?

Existing tests to validate no regressions. Added transform cases to DDL suite and v1 conversions suite.

Closes #24117 from rdblue/add-public-transform-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-10 14:30:39 +08:00
Maxim Gekk 63e4bf42c2 [SPARK-27401][SQL] Refactoring conversion of Timestamp to/from java.sql.Timestamp
## What changes were proposed in this pull request?

In the PR, I propose simpler implementation of `toJavaTimestamp()`/`fromJavaTimestamp()` by reusing existing functions of `DateTimeUtils`. This will allow to:
- Simply implementation of `toJavaTimestamp()`, and handle properly negative inputs.
- Detect `Long` overflow in conversion of milliseconds (`java.sql.Timestamp`) to microseconds (Catalyst's Timestamp).

## How was this patch tested?

By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `CastSuite`. And by new benchmark for export/import timestamps added to `DateTimeBenchmark`:

Before:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             290            335          49         17.2          58.0       1.0X
Collect longs                                      1234           1681         487          4.1         246.8       0.2X
Collect timestamps                                 1718           1755          63          2.9         343.7       0.2X
```

After:
```
To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
From java.sql.Timestamp                             283            301          19         17.7          56.6       1.0X
Collect longs                                      1048           1087          36          4.8         209.6       0.3X
Collect timestamps                                 1425           1479          56          3.5         285.1       0.2X
```

Closes #24311 from MaxGekk/conv-java-sql-date-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-09 15:42:27 -07:00
mingbo_pb 3e4cfe9dbc [SPARK-27406][SQL] UnsafeArrayData serialization breaks when two machines have different Oops size
## What changes were proposed in this pull request?
ApproxCountDistinctForIntervals holds the UnsafeArrayData data to initialize endpoints. When the UnsafeArrayData is serialized with Java serialization, the BYTE_ARRAY_OFFSET in memory can change if two machines have different pointer width (Oops in JVM).

This PR fixes this issue by using the same way in https://github.com/apache/spark/pull/9030

## How was this patch tested?
Manual test has been done in our tpcds environment and regarding unit test case has been added as well

Closes #24317 from pengbo/SPARK-27406.

Authored-by: mingbo_pb <mingbo.pb@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 15:41:42 +08:00
Hyukjin Kwon f16dfb9129 [SPARK-27328][SQL] Add 'deprecated' in ExpressionDescription for extended usage and SQL doc
## What changes were proposed in this pull request?

This PR proposes to two things:

1. Add `deprecated` field to `ExpressionDescription` so that it can be shown in our SQL function documentation (https://spark.apache.org/docs/latest/api/sql/), and it can be shown via `DESCRIBE FUNCTION EXTENDED`.

2. While I am here, add some more restrictions for `note()` and `since()`. Looks some documentations are broken due to malformed `note`:

    ![Screen Shot 2019-03-31 at 3 00 53 PM](https://user-images.githubusercontent.com/6477701/55285518-a3e88500-53c8-11e9-9e99-41d857794fbe.png)

    It should start with 4 spaces and end with a newline. I added some asserts, and fixed the instances together while I am here. This is technically a breaking change but I think it's too trivial to note somewhere (and we're in Spark 3.0.0).

This PR adds `deprecated` property into `from_utc_timestamp` and `to_utc_timestamp` (it's deprecated as of #24195) as examples of using this field.

Now it shows the deprecation information as below:

- **SQL documentation is shown as below:**

    ![Screen Shot 2019-03-31 at 3 07 31 PM](https://user-images.githubusercontent.com/6477701/55285537-2113fa00-53c9-11e9-9932-f5693a03332d.png)

- **`DESCRIBE FUNCTION EXTENDED from_utc_timestamp;`**:

    ```
    Function: from_utc_timestamp
    Class: org.apache.spark.sql.catalyst.expressions.FromUTCTimestamp
    Usage: from_utc_timestamp(timestamp, timezone) - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 03:40:00.0'.
    Extended Usage:
        Examples:
          > SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul');
           2016-08-31 09:00:00

        Since: 1.5.0

        Deprecated:
          Deprecated since 3.0.0. See SPARK-25496.

    ```

## How was this patch tested?

Manually tested via:

- For documentation verification:

    ```
    $ cd sql
    $ sh create-docs.sh
    ```

- For checking description:

    ```
    $ ./bin/spark-sql
    ```
    ```
    spark-sql> DESCRIBE FUNCTION EXTENDED from_utc_timestamp;
    spark-sql> DESCRIBE FUNCTION EXTENDED to_utc_timestamp;
    ```

Closes #24259 from HyukjinKwon/SPARK-27328.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-09 13:49:42 +08:00
Gengliang Wang d50603a37c [SPARK-27271][SQL] Migrate Text to File Data Source V2
## What changes were proposed in this pull request?

Migrate Text source to File Data Source V2

## How was this patch tested?

Unit test

Closes #24207 from gengliangwang/textV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-08 10:15:22 -07:00
Maxim Gekk 00241733a6 [SPARK-27405][SQL][TEST] Restrict the range of generated random timestamps
## What changes were proposed in this pull request?

In the PR, I propose to restrict the range of random timestamp literals generated in `LiteralGenerator. timestampLiteralGen`. The generator creates instances of `java.sql.Timestamp` by passing milliseconds since epoch as `Long` type. Converting the milliseconds to microseconds can cause arithmetic overflow of Long type because Catalyst's Timestamp type stores microseconds since epoch in `Long` type internally as well. Proposed interval of random milliseconds is `[Long.MinValue / 1000, Long.MaxValue / 1000]`.

For example, generated timestamp `new java.sql.Timestamp(-3948373668011580000)` causes `Long` overflow at the method:
```scala
  def fromJavaTimestamp(t: Timestamp): SQLTimestamp = {
  ...
      MILLISECONDS.toMicros(t.getTime()) + NANOSECONDS.toMicros(t.getNanos()) % NANOS_PER_MICROS
  ...
  }
```
because `t.getTime()` returns `-3948373668011580000` which is multiplied by `1000` at `MILLISECONDS.toMicros`, and the result `-3948373668011580000000` is less than `Long.MinValue`.

## How was this patch tested?

By `DateExpressionsSuite` in the PR https://github.com/apache/spark/pull/24311

Closes #24316 from MaxGekk/random-timestamps-gen.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-08 09:53:00 -07:00
Dongjoon Hyun 982c4c8e3c [SPARK-27390][CORE][SQL][TEST] Fix package name mismatch
## What changes were proposed in this pull request?

This PR aims to clean up package name mismatches.

## How was this patch tested?

Pass the Jenkins.

Closes #24300 from dongjoon-hyun/SPARK-27390.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-05 11:50:37 -07:00
gatorsmile 5678e687c6 [SPARK-27393][SQL] Show ReusedSubquery in the plan when the subquery is reused
## What changes were proposed in this pull request?
With this change, we can easily identify the plan difference when subquery is reused.

When the reuse is enabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery240 + ReusedSubquery Subquery subquery240) AS (scalarsubquery() + scalarsubquery())#253]
   :  :- Subquery subquery240
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#250])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#256, count#257L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- ReusedSubquery Subquery subquery240
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

When the reuse is disabled, the plan looks like
```
== Physical Plan ==
CollectLimit 1
+- *(1) Project [(Subquery subquery286 + Subquery subquery287) AS (scalarsubquery() + scalarsubquery())#299]
   :  :- Subquery subquery286
   :  :  +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#296])
   :  :     +- Exchange SinglePartition
   :  :        +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#302, count#303L])
   :  :           +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :  :              +- Scan[obj#12]
   :  +- Subquery subquery287
   :     +- *(2) HashAggregate(keys=[], functions=[avg(cast(key#13 as bigint))], output=[avg(key)#298])
   :        +- Exchange SinglePartition
   :           +- *(1) HashAggregate(keys=[], functions=[partial_avg(cast(key#13 as bigint))], output=[sum#306, count#307L])
   :              +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13]
   :                 +- Scan[obj#12]
   +- *(1) SerializeFromObject
      +- Scan[obj#12]
```

## How was this patch tested?
Modified the existing test.

Closes #24258 from gatorsmile/followupSPARK-27279.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-05 08:31:41 -07:00
Aayushmaan Jain 04e53d2e3c [SPAR-27342][SQL] Optimize Limit 0 queries
## What changes were proposed in this pull request?
With this change, unnecessary file scans are avoided in case of Limit 0 queries.

I added a case (rule) to `PropagateEmptyRelation` to replace `GlobalLimit 0` and `LocalLimit 0` nodes with an empty `LocalRelation`. This prunes the subtree under the Limit 0 node and further allows other rules of `PropagateEmptyRelation` to optimize the Logical Plan - while remaining semantically consistent with the Limit 0 query.

For instance:
**Query:**
`SELECT * FROM table1 INNER JOIN (SELECT * FROM table2 LIMIT 0) AS table2 ON table1.id = table2.id`

**Optimized Plan without fix:**
```
Join Inner, (id#79 = id#87)
:- Filter isnotnull(id#79)
:  +- Relation[id#79,num1#80] parquet
+- Filter isnotnull(id#87)
   +- GlobalLimit 0
      +- LocalLimit 0
         +- Relation[id#87,num2#88] parquet
```

**Optimized Plan with fix:**
`LocalRelation <empty>, [id#75, num1#76, id#77, num2#78]`

## How was this patch tested?
Added unit tests to verify Limit 0 optimization for:
- Simple query containing Limit 0
- Inner Join, Left Outer Join, Right Outer Join, Full Outer Join queries containing Limit 0 as one of their children
- Nested Inner Joins between 3 tables with one of them having a Limit 0 clause.
- Intersect query wherein one of the subqueries was a Limit 0 query.

Closes #24271 from aayushmaanjain/optimize-limit0.

Authored-by: Aayushmaan Jain <aayushmaan.jain42@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-04 21:19:40 -07:00
Ruben Fiszel 0e44a51f2e [SPARK-24345][SQL] Improve ParseError stop location when offending symbol is a token
In the case where the offending symbol is a CommonToken, this PR increases the accuracy of the start and stop origin by leveraging the start and stop index information from CommonToken.

Closes #21334 from rubenfiszel/patch-1.

Lead-authored-by: Ruben Fiszel <rubenfiszel@gmail.com>
Co-authored-by: rubenfiszel <rfiszel@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-04 18:20:34 -05:00
Dongjoon Hyun b51763612a Revert "[SPARK-27278][SQL] Optimize GetMapValue when the map is a foldable and the key is not"
This reverts commit 5888b15d9c.
2019-04-03 09:41:13 -07:00
Wenchen Fan ffb362a705 [SPARK-19712][SQL][FOLLOW-UP] reduce code duplication
## What changes were proposed in this pull request?

abstract some common code into a method.

## How was this patch tested?

existing tests

Closes #24281 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 00:37:57 +08:00
Maxim Gekk 1bc672366d [SPARK-27344][SQL][TEST] Support the LocalDate and Instant classes in Java Bean encoders
## What changes were proposed in this pull request?

- Added new test for Java Bean encoder of the classes: `java.time.LocalDate` and `java.time.Instant`.
- Updated comment for `Encoders.bean`
- New Row getters: `getLocalDate` and `getInstant`
- Extended `inferDataType` to infer types for `java.time.LocalDate` -> `DateType` and `java.time.Instant` -> `TimestampType`.

## How was this patch tested?

By `JavaBeanDeserializationSuite`

Closes #24273 from MaxGekk/bean-instant-localdate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 17:45:59 +08:00
Dilip Biswal 3286bff942 [SPARK-27255][SQL] Report error when illegal expressions are hosted by a plan operator.
## What changes were proposed in this pull request?
In the PR, we raise an AnalysisError when we detect the presense of aggregate expressions in where clause. Here is the problem description from the JIRA.

Aggregate functions should not be allowed in WHERE clause. But Spark SQL throws an exception when generating codes. It is supposed to throw an exception during parsing or analyzing.

Here is an example:
```
val df = spark.sql("select * from t where sum(ta) > 0")
df.explain(true)
df.show()
```
Resulting exception:
```
Exception in thread "main" java.lang.UnsupportedOperationException: Cannot generate code for expression: sum(cast(input[0, int, false] as bigint))
	at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode(Expression.scala:291)
	at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode$(Expression.scala:290)
	at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87)
	at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:138)
	at scala.Option.getOrElse(Option.scala:138)
```
Checked the behaviour of other database and all of them return an exception:
**Postgress**
```
select * from foo where max(c1) > 0;
Error
ERROR: aggregate functions are not allowed in WHERE Position: 25
```
**DB2**
```
db2 => select * from foo where max(c1) > 0;
SQL0120N  Invalid use of an aggregate function or OLAP function.
```
**Oracle**
```
select * from foo where max(c1) > 0;
ORA-00934: group function is not allowed here
```
**MySql**
```
select * from foo where max(c1) > 0;
Invalid use of group function
```

**Update**
This PR has been enhanced to report error when expressions such as Aggregate, Window, Generate are hosted by operators where they are invalid.
## How was this patch tested?
Added tests in AnalysisErrorSuite and group-by.sql

Closes #24209 from dilipbiswal/SPARK-27255.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 13:05:06 +08:00
Maxim Gekk 1d20d13149 [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 10:55:56 +08:00
Dilip Biswal b8b5acdd41 [SPARK-19712][SQL][FOLLOW-UP] Don't do partial pushdown when pushing down LeftAnti joins below Aggregate or Window operators.
## What changes were proposed in this pull request?
After [23750](https://github.com/apache/spark/pull/23750), we may pushdown left anti joins below aggregate and window operators with a partial join condition. This is not correct and was pointed out by hvanhovell and cloud-fan [here](https://github.com/apache/spark/pull/23750#discussion_r270017097). This pr addresses their comments.
## How was this patch tested?
Added two new tests to verify the behaviour.

Closes #24253 from dilipbiswal/SPARK-19712-followup.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 09:56:27 +08:00
Hyukjin Kwon 949d712839 [SPARK-27346][SQL] Loosen the newline assert condition on 'examples' field in ExpressionInfo
## What changes were proposed in this pull request?

I haven't tested by myself on Windows and I am not 100% sure if this is going to cause an actual problem.

However, this one line:

827383a97c/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java (L82)

made me to investigate a lot today.

Given my speculation, if Spark is built in Linux and it's executed on Windows, it looks possible for multiline strings, like,

5264164a67/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala (L146-L150)

to throw an exception because the newline in the binary is `\n` but `System.lineSeparator` returns `\r\n`.

I think this is not yet found because this particular codes are not released yet (see SPARK-26426).

Looks just better to loosen the condition and forget about this stuff.

This should be backported into branch-2.4 as well.

## How was this patch tested?

N/A

Closes #24274 from HyukjinKwon/SPARK-27346.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-03 08:27:41 +09:00
Sean Owen d4420b455a [SPARK-27323][CORE][SQL][STREAMING] Use Single-Abstract-Method support in Scala 2.12 to simplify code
## What changes were proposed in this pull request?

Use Single Abstract Method syntax where possible (and minor related cleanup). Comments below. No logic should change here.

## How was this patch tested?

Existing tests.

Closes #24241 from srowen/SPARK-27323.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-02 07:37:05 -07:00
Dongjoon Hyun d575a453db Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp"
This reverts commit c5e83ab92c.
2019-04-02 01:05:54 -07:00
Maxim Gekk c5e83ab92c [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-02 10:20:06 +08:00
Liang-Chi Hsieh eaf008ad0e [SPARK-27329][SQL] Pruning nested field in map of map key and value from object serializers
## What changes were proposed in this pull request?

If object serializer has map of map key/value, pruning nested field should work.

Previously object serializer pruner don't recursively prunes nested fields if it is deeply located in map key or value. This patch proposed to address it by slightly factoring the pruning logic.

## How was this patch tested?

Added tests.

Closes #24260 from viirya/SPARK-27329.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-01 13:53:55 -07:00
Marco Gaido 5888b15d9c [SPARK-27278][SQL] Optimize GetMapValue when the map is a foldable and the key is not
## What changes were proposed in this pull request?

When `GetMapValue` contains a foldable Map and a non-foldable key, `SimplifyExtractValueOps` fails to optimize it transforming it into case when statements.
The PR adds a case for covering this situation too.

## How was this patch tested?

added UT

Closes #24223 from mgaido91/SPARK-27278.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-01 09:09:06 -07:00
Maxim Gekk d332958109 [SPARK-27325][SQL] Add implicit encoders for LocalDate and Instant
## What changes were proposed in this pull request?

Added implicit encoders for the `java.time.LocalDate` and `java.time.Instant` classes. This allows creation of datasets from instances of the types.

## How was this patch tested?

Added new tests to `JavaDatasetSuite` and `DatasetSuite`.

Closes #24249 from MaxGekk/instant-localdate-encoders.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-01 23:02:48 +08:00
Marco Gaido 8012f55a9b [SPARK-26812][SQL] Report correct nullability for complex datatypes in Union
## What changes were proposed in this pull request?

When there is a `Union`, the reported output datatypes are the ones of the first plan and the nullability is updated according to all the plans. For complex types, though, the nullability of their elements is not updated using the types from the other plans. This means that the nullability of the inner elements is the one of the first plan. If this is not compatible with the one of other plans, errors can happen (as reported in the JIRA).

The PR proposes to update the nullability of the inner elements of complex datatypes according to most permissive value of all the plans.

## How was this patch tested?

added UT

Closes #23726 from mgaido91/SPARK-26812.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-01 22:22:10 +08:00
Takuya UESHIN f176dd3f28 [SPARK-27314][SQL] Deduplicate exprIds for Union.
## What changes were proposed in this pull request?

We have been having a potential problem with `Union` when the children have the same expression id in their outputs, which happens when self-union.

## How was this patch tested?

Modified some tests to adjust plan changes.

Closes #24236 from ueshin/issues/SPARK-27314/dedup_union.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-29 14:05:38 -07:00
Maxim Gekk 61561c1c2d [SPARK-27252][SQL][FOLLOWUP] Calculate min and max days independently from time zone in ComputeCurrentTimeSuite
## What changes were proposed in this pull request?

This fixes the `analyzer should replace current_date with literals` test in `ComputeCurrentTimeSuite` by making calculation of `min` and `max` days independent from time zone.

## How was this patch tested?

by `ComputeCurrentTimeSuite`.

Closes #24240 from MaxGekk/current-date-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-29 14:28:36 -05:00
Maxim Gekk 06abd06112 [SPARK-27252][SQL] Make current_date() independent from time zones
## What changes were proposed in this pull request?

This makes the `CurrentDate` expression and `current_date` function independent from time zone settings. New result is number of days since epoch in `UTC` time zone. Previously, Spark shifted the current date (in `UTC` time zone) according the session time zone which violets definition of `DateType` - number of days since epoch (which is an absolute point in time, midnight of Jan 1 1970 in UTC time).

The changes makes `CurrentDate` consistent to `CurrentTimestamp` which is independent from time zone too.

## How was this patch tested?

The changes were tested by existing test suites like `DateExpressionsSuite`.

Closes #24185 from MaxGekk/current-date.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-28 18:44:08 -07:00
Liang-Chi Hsieh 93ff69003b [SPARK-27288][SQL] Pruning nested field in complex map key from object serializers
## What changes were proposed in this pull request?

In the original PR #24158, pruning nested field in complex map key was not supported, because some methods in schema pruning did't support it at that moment. This is a followup to add it.

## How was this patch tested?

Added tests.

Closes #24220 from viirya/SPARK-26847-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-27 19:40:14 +09:00
liuxian fac31104f6 [SPARK-27083][SQL] Add a new conf to control subqueryReuse
## What changes were proposed in this pull request?
Subquery Reuse and Exchange Reuse are not the same feature, if we don't want to reuse subqueries,and we just want to reuse exchanges,only one configuration that cannot be done.

This PR adds a new configuration `spark.sql.subquery.reuse` to control subqueryReuse.

## How was this patch tested?

N/A

Closes #23998 from 10110346/SUBQUERY_REUSE.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 23:37:58 -07:00
Dilip Biswal 6c0e13b456 [SPARK-27285] Support describing output of CTE
## What changes were proposed in this pull request?
SPARK-26982 allows users to describe output of a query. However, it had a limitation of not supporting CTEs due to limitation of the grammar having a single rule to parse both select and inserts. After SPARK-27209, which splits select and insert parsing to two different rules, we can now support describing output of the CTEs easily.

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

Closes #24224 from dilipbiswal/describe_support_cte.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 16:00:56 -07:00
Maxim Gekk 69035684d4 [SPARK-27242][SQL] Make formatting TIMESTAMP/DATE literals independent from the default time zone
## What changes were proposed in this pull request?

In the PR, I propose to use the SQL config `spark.sql.session.timeZone` in formatting `TIMESTAMP` literals, and make formatting `DATE` literals independent from time zone. The changes make parsing and formatting `TIMESTAMP`/`DATE` literals consistent, and independent from the default time zone of current JVM.

Also this PR ports `TIMESTAMP`/`DATE` literals formatting on Proleptic Gregorian Calendar via using `TimestampFormatter`/`DateFormatter`.

## How was this patch tested?

Added new tests to `LiteralExpressionSuite`

Closes #24181 from MaxGekk/timezone-aware-literals.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 15:29:59 -07:00
Takuya UESHIN 529a061168 [SPARK-26103][SQL][FOLLOW-UP] Use string-interpolation to show the config key.
## What changes were proposed in this pull request?

This is a follow-up of #23169.
We should've used string-interpolation to show the config key in the warn message.

## How was this patch tested?

Existing tests.

Closes #24217 from ueshin/issues/SPARK-26103/s.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-26 20:56:00 +09:00
Dilip Biswal 9cc925cda2 [SPARK-27209][SQL] Split parsing of SELECT and INSERT into two top-level rules in the grammar file.
## What changes were proposed in this pull request?
Currently in the grammar file the rule `query` is responsible to parse both select and insert statements. As a result, we need to have more semantic checks in the code to guard against in-valid insert constructs in a query. Couple of examples are in the `visitCreateView` and `visitAlterView` functions. One other issue is that, we don't catch the `invalid insert constructs` in all the places until checkAnalysis (the errors we raise can be confusing as well). Here are couple of examples :

```SQL
select * from (insert into bar values (2));
```
```
Error in query: unresolved operator 'Project [*];
'Project [*]
+- SubqueryAlias `__auto_generated_subquery_name`
   +- InsertIntoHiveTable `default`.`bar`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, false, false, [c1]
      +- Project [cast(col1#18 as int) AS c1#20]
         +- LocalRelation [col1#18]
```

```SQL
select * from foo where c1 in (insert into bar values (2))
```
```
Error in query: cannot resolve '(default.foo.`c1` IN (listquery()))' due to data type mismatch:
The number of columns in the left hand side of an IN subquery does not match the
number of columns in the output of subquery.
#columns in left hand side: 1.
#columns in right hand side: 0.

Left side columns:
[default.foo.`c1`].
Right side columns:
[].;;
'Project [*]
+- 'Filter c1#6 IN (list#5 [])
   :  +- InsertIntoHiveTable `default`.`bar`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, false, false, [c1]
   :     +- Project [cast(col1#7 as int) AS c1#9]
   :        +- LocalRelation [col1#7]
   +- SubqueryAlias `default`.`foo`
      +- HiveTableRelation `default`.`foo`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#6]
```

For both the cases above, we should reject the syntax at parser level.

In this PR, we create two top-level parser rules to parse `SELECT` and `INSERT` respectively.
I will create a small PR to allow CTEs in DESCRIBE QUERY after this PR is in.
## How was this patch tested?
Added tests to PlanParserSuite and removed the semantic check tests from SparkSqlParserSuites.

Closes #24150 from dilipbiswal/split-query-insert.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-25 17:43:03 -07:00
sandeep-katta 0bc030c859 [SPARK-27246][SQL] Add an assert on invalid Scalar subquery plan with no column
## What changes were proposed in this pull request?

This PR proposes to add an assert on `ScalarSubquery`'s `dataType` because there's a possibility that `dataType` can be called alone before throwing analysis exception.

This was found while working on [SPARK-27088](https://issues.apache.org/jira/browse/SPARK-27088). This change calls `treeString` for logging purpose, and the specific test "scalar subquery with no column" under `AnalysisErrorSuite` was being failed with:

```
Caused by: sbt.ForkMain$ForkError: java.util.NoSuchElementException: next on empty iterator
	...
	at scala.collection.mutable.ArrayOps$ofRef.head(ArrayOps.scala:198)
	at org.apache.spark.sql.catalyst.expressions.ScalarSubquery.dataType(subquery.scala:251)
	at org.apache.spark.sql.catalyst.expressions.Alias.dataType(namedExpressions.scala:163)
        ...
	at org.apache.spark.sql.catalyst.trees.TreeNode.simpleString(TreeNode.scala:465)
        ...
	at org.apache.spark.sql.catalyst.rules.RuleExecutor$PlanChangeLogger.logRule(RuleExecutor.scala:176)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:116)
	...
```

The reason is that `treeString` for logging happened to call `dataType` on `ScalarSubquery` but one test has empty column plan. So, it happened to throw `NoSuchElementException` before checking analysis.

## How was this patch tested?

Manually tested.

```scala
ScalarSubquery(LocalRelation()).treeString
```

```
An exception or error caused a run to abort: assertion failed: Scala subquery should have only one column
java.lang.AssertionError: assertion failed: Scala subquery should have only one column
	at scala.Predef$.assert(Predef.scala:223)
	at org.apache.spark.sql.catalyst.expressions.ScalarSubquery.dataType(subquery.scala:252)
	at org.apache.spark.sql.catalyst.analysis.AnalysisErrorSuite.<init>(AnalysisErrorSuite.scala:116)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at java.lang.Class.newInstance(Class.java:442)
	at org.scalatest.tools.Runner$.genSuiteConfig(Runner.scala:1428)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$8(Runner.scala:1236)
	at scala.collection.immutable.List.map(List.scala:286)
	at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1235)
```

Closes #24182 from sandeep-katta/subqueryissue.

Authored-by: sandeep-katta <sandeep.katta2007@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-26 09:25:57 +09:00
Liang-Chi Hsieh 8433ff6607 [SPARK-26847][SQL] Pruning nested serializers from object serializers: MapType support
## What changes were proposed in this pull request?

In SPARK-26837, we prune nested fields from object serializers if they are unnecessary in the query execution. SPARK-26837 leaves the support of MapType as a TODO item. This proposes to support map type.

## How was this patch tested?

Added tests.

Closes #24158 from viirya/SPARK-26847.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-25 15:36:58 -07:00
Liang-Chi Hsieh 5a36cf66ed [SPARK-27268][SQL] Add map_keys and map_values support in nested schema pruning.
## What changes were proposed in this pull request?

We need to add `map_keys` and `map_values` into `ProjectionOverSchema` to support those methods in nested schema pruning. This also adds end-to-end tests to SchemaPruningSuite.

## How was this patch tested?

Added tests.

Closes #24202 from viirya/SPARK-27268.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-25 15:32:01 -07:00
liuxian e4b36df2c0 [SPARK-27256][CORE][SQL] If the configuration is used to set the number of bytes, we'd better use bytesConf'.
## What changes were proposed in this pull request?
Currently, if we want to configure `spark.sql.files.maxPartitionBytes` to 256 megabytes, we must set  `spark.sql.files.maxPartitionBytes=268435456`, which is very unfriendly to users.

And if we set it like this:`spark.sql.files.maxPartitionBytes=256M`, we will  encounter this exception:
```
Exception in thread "main" java.lang.IllegalArgumentException:
 spark.sql.files.maxPartitionBytes should be long, but was 256M
        at org.apache.spark.internal.config.ConfigHelpers$.toNumber(ConfigBuilder.scala)
```
This PR use `bytesConf` to replace `longConf` or `intConf`,  if the configuration is used to set the number of bytes.
Configuration change list:
`spark.files.maxPartitionBytes`
`spark.files.openCostInBytes`
`spark.shuffle.sort.initialBufferSize`
`spark.shuffle.spill.initialMemoryThreshold`
`spark.sql.autoBroadcastJoinThreshold`
`spark.sql.files.maxPartitionBytes`
`spark.sql.files.openCostInBytes`
`spark.sql.defaultSizeInBytes`
## How was this patch tested?
1.Existing unit tests
2.Manual testing

Closes #24187 from 10110346/bytesConf.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-25 14:47:40 -07:00
Maxim Gekk 52671d631d [SPARK-27008][SQL][FOLLOWUP] Fix typo from *_EANBLED to *_ENABLED
## What changes were proposed in this pull request?

This fixes a typo in the SQL config value: DATETIME_JAVA8API_**EANBLED** -> DATETIME_JAVA8API_**ENABLED**.

## How was this patch tested?

This was tested by `RowEncoderSuite` and `LiteralExpressionSuite`.

Closes #24194 from MaxGekk/date-localdate-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-24 17:16:33 -07:00
Liang-Chi Hsieh 6f18ac9e99 [SPARK-27241][SQL] Support map_keys and map_values in SelectedField
## What changes were proposed in this pull request?

`SelectedField` doesn't support map_keys and map_values for now. When map key or value is complex struct, we should be able to prune unnecessary fields from keys/values. This proposes to add map_keys and map_values support to `SelectedField`.

## How was this patch tested?

Added tests.

Closes #24179 from viirya/SPARK-27241.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-23 23:13:31 -07:00
Gengliang Wang 624288556d [SPARK-27085][SQL] Migrate CSV to File Data Source V2
## What changes were proposed in this pull request?

Migrate CSV to File Data Source V2.

## How was this patch tested?

Unit test

Closes #24005 from gengliangwang/CSVDataSourceV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-23 15:43:46 -07:00
Maxim Gekk 027ed2d11b [SPARK-23643][CORE][SQL][ML] Shrinking the buffer in hashSeed up to size of the seed parameter
## What changes were proposed in this pull request?

The hashSeed method allocates 64 bytes instead of 8. Other bytes are always zeros (thanks to default behavior of ByteBuffer). And they could be excluded from hash calculation because they don't differentiate inputs.

## How was this patch tested?

By running the existing tests - XORShiftRandomSuite

Closes #20793 from MaxGekk/hash-buff-size.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-23 11:26:09 -05:00
Marco Gaido fe317dc74e [SPARK-27243][SQL] RuleExecutor.dumpTimeSpent should not throw exception when empty
## What changes were proposed in this pull request?

`RuleExecutor.dumpTimeSpent` currently throws an exception when invoked before any rule is run or immediately after `RuleExecutor.reset`. The PR makes it returning an empty summary, which is the expected output instead.

## How was this patch tested?

added UT

Closes #24180 from mgaido91/SPARK-27243.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-23 09:49:20 +09:00
Ryan Blue 34e3cc7060 [SPARK-27108][SQL] Add parsed SQL plans for create, CTAS.
## What changes were proposed in this pull request?

This moves parsing `CREATE TABLE ... USING` statements into catalyst. Catalyst produces logical plans with the parsed information and those plans are converted to v1 `DataSource` plans in `DataSourceAnalysis`.

This prepares for adding v2 create plans that should receive the information parsed from SQL without being translated to v1 plans first.

This also makes it possible to parse in catalyst instead of breaking the parser across the abstract `AstBuilder` in catalyst and `SparkSqlParser` in core.

For more information, see the [mailing list thread](https://lists.apache.org/thread.html/54f4e1929ceb9a2b0cac7cb058000feb8de5d6c667b2e0950804c613%3Cdev.spark.apache.org%3E).

## How was this patch tested?

This uses existing tests to catch regressions. This introduces no behavior changes.

Closes #24029 from rdblue/SPARK-27108-add-parsed-create-logical-plans.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-22 13:58:54 -07:00
Martin Junghanns 8efc5ec72e [SPARK-27174][SQL] Add support for casting integer types to binary
Co-authored-by: Philip Stutz <philip.stutzgmail.com>

## What changes were proposed in this pull request?

This PR adds support for casting

* `ByteType`
* `ShortType`
* `IntegerType`
* `LongType`

to `BinaryType`.

## How was this patch tested?

We added unit tests for casting instances of the above types. For validation, we used Javas `DataOutputStream` to compare the resulting byte array with the result of `Cast`.

We state that the contribution is our original work and that we license the work to the project under the project’s open source license.

cloud-fan we'd appreciate a review if you find the time, thx

Closes #24107 from s1ck/cast_to_binary.

Authored-by: Martin Junghanns <martin.junghanns@neotechnology.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-22 10:09:35 -07:00
Maxim Gekk a529be2930 [SPARK-27212][SQL] Eliminate TimeZone to ZoneId conversion in stringToTimestamp
## What changes were proposed in this pull request?

In the PR, I propose to avoid the `TimeZone` to `ZoneId` conversion in `DateTimeUtils.stringToTimestamp` by changing signature of the method, and require a parameter of `ZoneId` type. This will allow to avoid unnecessary conversion (`TimeZone` -> `String` -> `ZoneId`) per each row.

Also the PR avoids creation of `ZoneId` instances from `ZoneOffset` because `ZoneOffset` is a sub-class, and the conversion is unnecessary too.

## How was this patch tested?

It was tested by `DateTimeUtilsSuite` and `CastSuite`.

Closes #24155 from MaxGekk/stringtotimestamp-zoneid.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-22 18:01:29 +09:00
maryannxue 9f58d3b436 [SPARK-27236][TEST] Refactor log-appender pattern in tests
## What changes were proposed in this pull request?

Refactored code in tests regarding the "withLogAppender()" pattern by creating a general helper method in SparkFunSuite.

## How was this patch tested?

Passed existing tests.

Closes #24172 from maryannxue/log-appender.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-21 19:18:30 -07:00
John Zhuge 80565ce253 [SPARK-26946][SQL] Identifiers for multi-catalog
## What changes were proposed in this pull request?

- Support N-part identifier in SQL
- N-part identifier extractor in Analyzer

## How was this patch tested?

- A new unit test suite ResolveMultipartRelationSuite
- CatalogLoadingSuite

rblue cloud-fan mccheah

Closes #23848 from jzhuge/SPARK-26946.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-21 18:04:50 -07:00
Maxim Gekk 0f4f8160e6 [SPARK-27222][SQL] Support Instant and LocalDate in Literal.apply
## What changes were proposed in this pull request?

In the PR, I propose to extend `Literal.apply` to support constructing literals of `TimestampType` and `DateType` from `java.time.Instant` and `java.time.LocalDate`. The java classes have been already supported as external types for `TimestampType` and `DateType` by the PRs #23811  and #23913.

## How was this patch tested?

Added new tests to `LiteralExpressionSuite`.

Closes #24161 from MaxGekk/literal-instant-localdate.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-21 12:57:32 -07:00
gengjiaan 22c9ed6a9c [MINOR][SQL] Put the grammar of database together, because this is good for maintenance and readability.
## What changes were proposed in this pull request?

The SQL grammar `SHOW DATABASES` is mixed in some grammar of table. I think should arrange the grammar of database together.
This is good for maintenance and readability.

## How was this patch tested?

No UT

Closes #24138 from beliefer/arrange-sql-grammar.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-21 06:41:04 -05:00
Venkata krishnan Sowrirajan b1857a4d7d [SPARK-26894][SQL] Handle Alias as well in AggregateEstimation to propagate child stats
## What changes were proposed in this pull request?

Currently aliases are not handled in the Aggregate Estimation due to which stats are not getting propagated. This causes CBO join-reordering to not give optimal join plans. ProjectEstimation is already taking care of aliases, we need same logic for AggregateEstimation as well to properly propagate stats when CBO is enabled.

## How was this patch tested?

This patch is manually tested using the query Q83 of TPCDS benchmark (scale 1000)

Closes #23803 from venkata91/aggstats.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@qubole.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-21 11:21:56 +09:00
Shixiong Zhu c26379b446 [SPARK-27221][SQL] Improve the assert error message in TreeNode.parseToJson
## What changes were proposed in this pull request?

When `TreeNode.parseToJson` may throw an assert error without any error message when a TreeNode is not implemented properly, and it's hard to find the bad TreeNode implementation.

This PR adds the assert message to improve the error, like what `TreeNode.jsonFields` does.

## How was this patch tested?

Jenkins

Closes #24159 from zsxwing/SPARK-27221.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-21 11:15:05 +09:00
Maxim Gekk 1882912cca [SPARK-27199][SQL] Replace TimeZone by ZoneId in TimestampFormatter API
## What changes were proposed in this pull request?

In the PR, I propose to use `ZoneId` instead of `TimeZone` in:
- the `apply` and `getFractionFormatter ` methods of the `TimestampFormatter` object,
- and in implementations of the `TimestampFormatter` trait like `FractionTimestampFormatter`.

The reason of the changes is to avoid unnecessary conversion from `TimeZone` to `ZoneId` because `ZoneId` is used in `TimestampFormatter` implementations internally, and the conversion is performed via `String` which is not for free. Also taking into account that `TimeZone` instances are converted from `String` in some cases, the worse case looks like `String` -> `TimeZone` -> `String` -> `ZoneId`. The PR eliminates the unneeded conversions.

## How was this patch tested?

It was tested by `DateExpressionsSuite`, `DateTimeUtilsSuite` and `TimestampFormatterSuite`.

Closes #24141 from MaxGekk/zone-id.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 21:28:11 +09:00
Huon Wilson b67d369572 [SPARK-27099][SQL] Add 'xxhash64' for hashing arbitrary columns to Long
## What changes were proposed in this pull request?

This introduces a new SQL function 'xxhash64' for getting a 64-bit hash of an arbitrary number of columns.

This is designed to exactly mimic the 32-bit `hash`, which uses
MurmurHash3. The name is designed to be more future-proof than the
'hash', by indicating the exact algorithm used, similar to md5 and the
sha hashes.

## How was this patch tested?

The tests for the existing `hash` function were duplicated to run with `xxhash64`.

Closes #24019 from huonw/hash64.

Authored-by: Huon Wilson <Huon.Wilson@data61.csiro.au>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-20 16:34:34 +08:00
Dongjoon Hyun 257391497b [SPARK-26975][SQL] Support nested-column pruning over limit/sample/repartition
## What changes were proposed in this pull request?

As [SPARK-26958](https://github.com/apache/spark/pull/23862/files) benchmark shows, nested-column pruning has limitations. This PR aims to remove the limitations on `limit/repartition/sample`. Here, repartition means `Repartition`, not `RepartitionByExpression`.

**PREPARATION**
```scala
scala> spark.range(100).map(x => (x, (x, s"$x" * 100))).toDF("col1", "col2").write.mode("overwrite").save("/tmp/p")
scala> sql("set spark.sql.optimizer.nestedSchemaPruning.enabled=true")
scala> spark.read.parquet("/tmp/p").createOrReplaceTempView("t")
```

**BEFORE**
```scala
scala> sql("SELECT col2._1 FROM (SELECT col2 FROM t LIMIT 1000000)").explain
== Physical Plan ==
CollectLimit 1000000
+- *(1) Project [col2#22._1 AS _1#28L]
   +- *(1) FileScan parquet [col2#22] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint>>

scala> sql("SELECT col2._1 FROM (SELECT /*+ REPARTITION(1) */ col2 FROM t)").explain
== Physical Plan ==
*(2) Project [col2#22._1 AS _1#33L]
+- Exchange RoundRobinPartitioning(1)
   +- *(1) Project [col2#22]
      +- *(1) FileScan parquet [col2#22] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint,_2:string>>
```

**AFTER**
```scala
scala> sql("SELECT col2._1 FROM (SELECT /*+ REPARTITION(1) */ col2 FROM t)").explain
== Physical Plan ==
Exchange RoundRobinPartitioning(1)
+- *(1) Project [col2#5._1 AS _1#11L]
   +- *(1) FileScan parquet [col2#5] Batched: false, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/tmp/p], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<col2:struct<_1:bigint>>
```

This supercedes https://github.com/apache/spark/pull/23542 and https://github.com/apache/spark/pull/23873 .

## How was this patch tested?

Pass the Jenkins with a newly added test suite.

Closes #23964 from dongjoon-hyun/SPARK-26975-ALIAS.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: DB Tsai <d_tsai@apple.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-19 20:24:22 -07:00
Takeshi Yamamuro 901c7408a4 [SPARK-27161][SQL][FOLLOWUP] Drops non-keywords from docs/sql-keywords.md
## What changes were proposed in this pull request?
This pr is a follow-up of #24093 and includes fixes below;
 - Lists up all the keywords of Spark only (that is, drops non-keywords there); I listed up all the keywords of ANSI SQL-2011 in the previous commit (SPARK-26215).
 - Sorts the keywords in `SqlBase.g4` in a alphabetical order

## How was this patch tested?
Pass Jenkins.

Closes #24125 from maropu/SPARK-27161-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 20:18:40 +08:00
mwlon d5c08fcaab [SPARK-26555][SQL] make ScalaReflection subtype checking thread safe
## What changes were proposed in this pull request?

Make ScalaReflection subtype checking thread safe by adding a lock. There is a thread safety bug in the <:< operator in all versions of scala (https://github.com/scala/bug/issues/10766).

## How was this patch tested?

Existing tests and a new one for the new subtype checking function.

Closes #24085 from mwlon/SPARK-26555.

Authored-by: mwlon <mloncaric@hmc.edu>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 18:22:01 +08:00
wuyi a8af23d7ab [SPARK-27193][SQL] CodeFormatter should format multiple comment lines correctly
## What changes were proposed in this pull request?

when enable `spark.sql.codegen.comments`,  there will be multiple comment lines. However, CodeFormatter can not handle multiple comment lines currently:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ /**
 * Codegend pipeline for stage (id=1)
 * *(1) Project [(id#0L + 1) AS (id + 1)#3L]
 * +- *(1) Filter (id#0L = 1)
 *    +- *(1) Range (0, 10, step=1, splits=4)
 */
/* 006 */ // codegenStageId=1
/* 007 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
```

After applying this pr:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ /**
/* 006 */  * Codegend pipeline for stage (id=1)
/* 007 */  * *(1) Project [(id#0L + 1) AS (id + 1)#4L]
/* 008 */  * +- *(1) Filter (id#0L = 1)
/* 009 */  *    +- *(1) Range (0, 10, step=1, splits=2)
/* 010 */  */
/* 011 */ // codegenStageId=1
/* 012 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
```

## How was this patch tested?

Tested Manually.

Closes #24133 from Ngone51/fix-codeformatter-for-multi-comment-lines.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 14:47:51 +08:00
Gengliang Wang 28d35c8578 [SPARK-27162][SQL] Add new method asCaseSensitiveMap in CaseInsensitiveStringMap
## What changes were proposed in this pull request?

Currently, DataFrameReader/DataFrameReader supports setting Hadoop configurations via method `.option()`.
E.g, the following test case should be passed in both ORC V1 and V2
```
  class TestFileFilter extends PathFilter {
    override def accept(path: Path): Boolean = path.getParent.getName != "p=2"
  }

  withTempPath { dir =>
      val path = dir.getCanonicalPath

      val df = spark.range(2)
      df.write.orc(path + "/p=1")
      df.write.orc(path + "/p=2")
      val extraOptions = Map(
        "mapred.input.pathFilter.class" -> classOf[TestFileFilter].getName,
        "mapreduce.input.pathFilter.class" -> classOf[TestFileFilter].getName
      )
      assert(spark.read.options(extraOptions).orc(path).count() === 2)
    }
  }
```
While Hadoop Configurations are case sensitive, the current data source V2 APIs are using `CaseInsensitiveStringMap` in the top level entry `TableProvider`.
To create Hadoop configurations correctly, I suggest
1. adding a new method `asCaseSensitiveMap` in `CaseInsensitiveStringMap`.
2. Make `CaseInsensitiveStringMap` read-only to ambiguous conversion in `asCaseSensitiveMap`

## How was this patch tested?

Unit test

Closes #24094 from gengliangwang/originalMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 13:35:47 +08:00
Wenchen Fan dbcb4792f2 [SPARK-27161][SQL] improve the document of SQL keywords
## What changes were proposed in this pull request?

Make it more clear about how Spark categories keywords regarding to the config `spark.sql.parser.ansi.enabled`

## How was this patch tested?

existing tests

Closes #24093 from cloud-fan/parser.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-18 15:19:52 +09:00
Dilip Biswal aea9a574c4 [SPARK-27134][SQL] array_distinct function does not work correctly with columns containing array of array
## What changes were proposed in this pull request?
Correct the logic to compute the distinct.

Below is a small repro snippet.

```
scala> val df = Seq(Seq(Seq(1, 2), Seq(1, 2), Seq(1, 2), Seq(3, 4), Seq(4, 5))).toDF("array_col")
df: org.apache.spark.sql.DataFrame = [array_col: array<array<int>>]

scala> val distinctDF = df.select(array_distinct(col("array_col")))
distinctDF: org.apache.spark.sql.DataFrame = [array_distinct(array_col): array<array<int>>]

scala> df.show(false)
+----------------------------------------+
|array_col                               |
+----------------------------------------+
|[[1, 2], [1, 2], [1, 2], [3, 4], [4, 5]]|
+----------------------------------------+
```
Error
```
scala> distinctDF.show(false)
+-------------------------+
|array_distinct(array_col)|
+-------------------------+
|[[1, 2], [1, 2], [1, 2]] |
+-------------------------+
```
Expected result
```
scala> distinctDF.show(false)
+-------------------------+
|array_distinct(array_col)|
+-------------------------+
|[[1, 2], [3, 4], [4, 5]] |
+-------------------------+
```
## How was this patch tested?
Added an additional test.

Closes #24073 from dilipbiswal/SPARK-27134.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-16 14:30:42 -05:00
Dongjoon Hyun 74d2f04183 [SPARK-27166][SQL] Improve printSchema to print up to the given level
## What changes were proposed in this pull request?

This PR aims to improve `printSchema` to be able to print up to the given level of the schema.

```scala
scala> val df = Seq((1,(2,(3,4)))).toDF
df: org.apache.spark.sql.DataFrame = [_1: int, _2: struct<_1: int, _2: struct<_1: int, _2: int>>]

scala> df.printSchema
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)
| | |-- _1: integer (nullable = false)
| | |-- _2: integer (nullable = false)

scala> df.printSchema(1)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)

scala> df.printSchema(2)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)

scala> df.printSchema(3)
root
|-- _1: integer (nullable = false)
|-- _2: struct (nullable = true)
| |-- _1: integer (nullable = false)
| |-- _2: struct (nullable = true)
| | |-- _1: integer (nullable = false)
| | |-- _2: integer (nullable = false)
```

## How was this patch tested?

Pass the Jenkins with the newly added test case.

Closes #24098 from dongjoon-hyun/SPARK-27166.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 20:27:55 -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
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
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
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
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
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
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
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
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
Udbhav30 9bddf7180e [SPARK-24669][SQL] Invalidate tables in case of DROP DATABASE CASCADE
## What changes were proposed in this pull request?
Before dropping database refresh the tables of that database, so as to refresh all cached entries associated with those tables.
We follow the same when dropping a table.

## How was this patch tested?
UT is added

Closes #23905 from Udbhav30/SPARK-24669.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-06 09:06:10 -08:00
Maxim Gekk 6001258398 [SPARK-27035][SQL] Get more precise current time
## What changes were proposed in this pull request?

In the PR, I propose to replace `System.currentTimeMillis()` by `Instant.now()` in the `CurrentTimestamp` expression. `Instant.now()` uses the best available clock in the system to take current time. See [JDK-8068730](https://bugs.openjdk.java.net/browse/JDK-8068730) for more details. In JDK8, `Instant.now()` provides results with millisecond resolution but starting from JDK9 resolution of results is increased up to microseconds.

## How was this patch tested?

The changes were tested by `DateTimeUtilsSuite` and by `DateFunctionsSuite`.

Closes #23945 from MaxGekk/current-time.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-06 08:32:16 -06:00
Maxim Gekk 9b55722161 [SPARK-27031][SQL] Avoid double formatting in timestampToString
## What changes were proposed in this pull request?

Removed unnecessary conversion of microseconds in `DateTimeUtils.timestampToString` to `java.sql.Timestamp` which aims to output fraction of seconds by casting it to string. This was replaced by special `TimestampFormatter` which appends the fraction formatter to `DateTimeFormatterBuilder`: `appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)`. The former one means trailing zeros in second's fraction should be truncated while formatting.

## How was this patch tested?

By existing test suites like `CastSuite`, `DateTimeUtilsSuite`, `JDBCSuite`, and by new test in `TimestampFormatterSuite`.

Closes #23936 from MaxGekk/timestamp-to-string.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-06 08:26:59 -06:00
Takeshi Yamamuro 4490fd0ff0 [SPARK-27001][SQL][FOLLOW-UP] Drop Serializable in WalkedTypePath
## What changes were proposed in this pull request?
This pr tried to drop `Serializable` in `WalkedTypePath`.

## How was this patch tested?
Pass Jenkins.

Closes #23973 from maropu/SPARK-27001-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-05 23:05:50 +08:00
Anton Okolnychyi 0c23a39384 [SPARK-26205][SQL] Optimize InSet Expression for bytes, shorts, ints, dates
## What changes were proposed in this pull request?

This PR optimizes `InSet` expressions for byte, short, integer, date types. It is a follow-up on PR #21442 from dbtsai.

`In` expressions are compiled into a sequence of if-else statements, which results in O\(n\) time complexity. `InSet` is an optimized version of `In`, which is supposed to improve the performance if all values are literals and the number of elements is big enough. However, `InSet` actually worsens the performance in many cases due to various reasons.

The main idea of this PR is to use Java `switch` statements to significantly improve the performance of `InSet` expressions for bytes, shorts, ints, dates. All `switch` statements are compiled into `tableswitch` and `lookupswitch` bytecode instructions. We will have O\(1\) time complexity if our case values are compact and `tableswitch` can be used. Otherwise, `lookupswitch` will give us O\(log n\).

Locally, I tried Spark `OpenHashSet` and primitive collections from `fastutils` in order to solve the boxing issue in `InSet`. Both options significantly decreased the memory consumption and `fastutils` improved the time compared to `HashSet` from Scala. However, the switch-based approach was still more than two times faster even on 500+ non-compact elements.

I also noticed that applying the switch-based approach on less than 10 elements gives a relatively minor improvement compared to the if-else approach. Therefore, I placed the switch-based logic into `InSet` and added a new config to track when it is applied. Even if we migrate to primitive collections at some point, the switch logic will be still faster unless the number of elements is really big. Another option is to have a separate `InSwitch` expression. However, this would mean we need to modify other places (e.g., `DataSourceStrategy`).

See [here](https://docs.oracle.com/javase/specs/jvms/se7/html/jvms-3.html#jvms-3.10) and [here](https://stackoverflow.com/questions/10287700/difference-between-jvms-lookupswitch-and-tableswitch) for more information.

This PR does not cover long values as Java `switch` statements cannot be used on them. However, we can have a follow-up PR with an approach similar to binary search.

## How was this patch tested?

There are new tests that verify the logic of the proposed optimization.

The performance was evaluated using existing benchmarks. This PR was also tested on an EC2 instance (OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 4.14.77-70.59.amzn1.x86_64, Intel(R) Xeon(R) CPU E5-2686 v4  2.30GHz).

## Notes

- [This link](http://hg.openjdk.java.net/jdk8/jdk8/langtools/file/30db5e0aaf83/src/share/classes/com/sun/tools/javac/jvm/Gen.java#l1153) contains source code that decides between `tableswitch` and `lookupswitch`. The logic was re-used in the benchmarks. See the `isLookupSwitch` method.

Closes #23171 from aokolnychyi/spark-26205.

Lead-authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-04 15:40:04 -08:00
Takeshi Yamamuro 68fbbbea4e [SPARK-26965][SQL] Makes ElementAt nullability more precise for array cases
## What changes were proposed in this pull request?
In master, `ElementAt` nullable is always true;
be1cadf16d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala (L1977)

But, If input is an array and foldable, we could make its nullability more precise.
This fix is based on  SPARK-26637(#23566).

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

Closes #23867 from maropu/SPARK-26965.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-04 21:27:18 +08:00
Dilip Biswal ad4823c99d [SPARK-19712][SQL] Pushing Left Semi and Left Anti joins through Project, Aggregate, Window, Union etc.
## What changes were proposed in this pull request?
This PR adds support for pushing down LeftSemi and LeftAnti joins below operators such as Project, Aggregate, Window, Union etc.  This is the initial piece of work that will be needed for
the subsequent work of moving the subquery rewrites to the beginning of optimization phase.

The larger  PR is [here](https://github.com/apache/spark/pull/23211) . This PR addresses the comment at [link](https://github.com/apache/spark/pull/23211#issuecomment-445705922).
## How was this patch tested?
Added a new test suite LeftSemiAntiJoinPushDownSuite.

Closes #23750 from dilipbiswal/SPARK-19712-pushleftsemi.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-04 19:09:24 +08:00
Jungtaek Lim (HeartSaVioR) 34f606678a [SPARK-27001][SQL] Refactor "serializerFor" method between ScalaReflection and JavaTypeInference
## What changes were proposed in this pull request?

This patch proposes refactoring `serializerFor` method between `ScalaReflection` and `JavaTypeInference`, being consistent with what we refactored for `deserializerFor` in #23854.

This patch also extracts the logic on recording walk type path since the logic is duplicated across `serializerFor` and `deserializerFor` with `ScalaReflection` and `JavaTypeInference`.

## How was this patch tested?

Existing tests.

Closes #23908 from HeartSaVioR/SPARK-27001.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-04 10:45:48 +08:00
Dilip Biswal 04ad559ab6 [SPARK-27016][SQL][BUILD] Treat all antlr warnings as errors while generating parser from the sql grammar file.
## What changes were proposed in this pull request?
Use the maven plugin option `treatWarningsAsErrors` to make sure the warnings are treated as errors while generating the parser file. In the absence of it, we may inadvertently introducing problems while making grammar changes.  Please refer to [PR-23897](https://github.com/apache/spark/pull/23897) to know more about the context.
## How was this patch tested?
We can use two ways to build Spark 1) sbt 2) Maven
This PR, we made a change to configure the maven antlr plugin to include a parameter that makes antlr4 report error on warning. However, when spark is built using sbt, we use the sbt antlr plugin which does not allow us to pass this additional compilation flag.  More info on sbt-antlr plugin can be found at [link](https://github.com/ihji/sbt-antlr4/blob/master/src/main/scala/com/simplytyped/Antlr4Plugin.scala)
In summary, this fix only applicable when we use maven to build.

Closes #23925 from dilipbiswal/antlr_fix.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-03 10:02:25 -06:00
Dilip Biswal 4a486d6716 [SPARK-26982][SQL] Enhance describe framework to describe the output of a query.
## What changes were proposed in this pull request?
Currently we can use `df.printSchema` to discover the schema information for a query. We should have a way to describe the output schema of a query using SQL interface.

Example:

DESCRIBE SELECT * FROM desc_table
DESCRIBE QUERY SELECT * FROM desc_table
```SQL

spark-sql> create table desc_table (c1 int comment 'c1-comment', c2 decimal comment 'c2-comment', c3 string);

spark-sql> desc select * from desc_table;
c1	int	        c1-comment
c2	decimal(10,0)	c2-comment
c3	string	        NULL

```
## How was this patch tested?
Added a new test under SQLQueryTestSuite and SparkSqlParserSuite

Closes #23883 from dilipbiswal/dkb_describe_query.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-02 11:21:23 +08:00
Dilip Biswal 5fd62ca65a [SPARK-26215][SQL][FOLLOW-UP][MINOR] Fix the warning from ANTR4
## What changes were proposed in this pull request?
I see the following new warning from ANTR4 after SPARK-26215 after it added `SCHEMA` keyword in the reserved/unreserved list. This is a minor PR to cleanup the warning.

```
WARNING] warning(125): org/apache/spark/sql/catalyst/parser/SqlBase.g4:784:90: implicit definition of token SCHEMA in parser
[WARNING] .../apache/spark/org/apache/spark/sql/catalyst/parser/SqlBase.g4 [784:90]: implicit definition of token SCHEMA in parser
```
## How was this patch tested?
Manually built catalyst after the fix to verify

Closes #23897 from dilipbiswal/minor_parser_token.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-01 12:34:15 -08:00
liuxian 02bbe977ab [MINOR] Remove unnecessary gets when getting a value from map.
## What changes were proposed in this pull request?

Redundant `get`  when getting a value from `Map` given a key.

## How was this patch tested?

N/A

Closes #23901 from 10110346/removegetfrommap.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-01 11:48:07 -06:00
Maxim Gekk 8e5f9995ca [SPARK-27008][SQL] Support java.time.LocalDate as an external type of DateType
## What changes were proposed in this pull request?

In the PR, I propose to add new Catalyst type converter for `DateType`. It should be able to convert `java.time.LocalDate` to/from `DateType`.

Main motivations for the changes:
- Smoothly support Java 8 time API
- Avoid inconsistency of calendars used inside of Spark 3.0 (Proleptic Gregorian calendar) and `java.sql.Date` (hybrid calendar - Julian + Gregorian).
- Make conversion independent from current system timezone.

By default, Spark converts values of `DateType` to `java.sql.Date` instances but the SQL config `spark.sql.datetime.java8API.enabled` can change the behavior. If it is set to `true`, Spark uses `java.time.LocalDate` as external type for `DateType`.

## How was this patch tested?

Added new testes to `CatalystTypeConvertersSuite` to check conversion of `DateType` to/from `java.time.LocalDate`, `JavaUDFSuite`/ `UDFSuite` to test usage of `LocalDate` type in Scala/Java UDFs.

Closes #23913 from MaxGekk/date-localdate.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-01 11:04:28 +08:00
Gabor Somogyi c4bbfd177b [SPARK-24063][SS] Add maximum epoch queue threshold for ContinuousExecution
## What changes were proposed in this pull request?

Continuous processing is waiting on epochs which are not yet complete (for example one partition is not making progress) and stores pending items in queues. These queues are unbounded and can consume up all the memory easily. In this PR I've added `spark.sql.streaming.continuous.epochBacklogQueueSize` configuration possibility to make them bounded. If the related threshold reached then the query will stop with `IllegalStateException`.

## How was this patch tested?

Existing + additional unit tests.

Closes #23156 from gaborgsomogyi/SPARK-24063.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-27 09:52:43 -08:00
liuxian 7912dbb88f [MINOR] Simplify boolean expression
## What changes were proposed in this pull request?

Comparing whether Boolean expression is equal to true is redundant
For example:
The datatype of `a` is boolean.
Before:
if (a == true)
After:
if (a)

## How was this patch tested?
N/A

Closes #23884 from 10110346/simplifyboolean.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-27 08:38:00 -06:00
Maxim Gekk b0450d07bd [SPARK-26902][SQL] Support java.time.Instant as an external type of TimestampType
## What changes were proposed in this pull request?

In the PR, I propose to add new Catalyst type converter for `TimestampType`. It should be able to convert `java.time.Instant` to/from `TimestampType`.

Main motivations for the changes:
- Smoothly support Java 8 time API
- Avoid inconsistency of calendars used inside of Spark 3.0 (Proleptic Gregorian calendar) and `java.sql.Timestamp` (hybrid calendar - Julian + Gregorian).
- Make conversion independent from current system timezone.

By default, Spark converts values of `TimestampType` to `java.sql.Timestamp` instances but the SQL config `spark.sql.catalyst.timestampType` can change the behavior. It accepts two values `Timestamp` (default) and `Instant`. If the former one is set, Spark returns `java.time.Instant` instances for timestamp values.

## How was this patch tested?

Added new testes to `CatalystTypeConvertersSuite` to check conversion of `TimestampType` to/from `java.time.Instant`.

Closes #23811 from MaxGekk/timestamp-instant.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-27 21:05:19 +08:00
Jungtaek Lim (HeartSaVioR) dea18ee85b [SPARK-22000][SQL] Address missing Upcast in JavaTypeInference.deserializerFor
## What changes were proposed in this pull request?

Spark expects the type of column and the type of matching field is same when deserializing to Object, but Spark hasn't actually restrict it (at least for Java bean encoder) and some users just do it and experience undefined behavior (in SPARK-22000, Spark throws compilation failure on generated code because it calls `.toString()` against primitive type.

It doesn't produce error in Scala side because `ScalaReflection.deserializerFor` properly inject Upcast if necessary. This patch proposes applying same thing to `JavaTypeInference.deserializerFor` as well.

Credit to srowen, maropu, and cloud-fan since they provided various approaches to solve this.

## How was this patch tested?

Added UT which query is slightly modified based on sample code in attachment on JIRA issue.

Closes #23854 from HeartSaVioR/SPARK-22000.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-27 13:47:20 +08:00
Hyukjin Kwon 88bc481b9e [SPARK-26830][SQL][R] Vectorized R dapply() implementation
## What changes were proposed in this pull request?

This PR targets to add vectorized `dapply()` in R, Arrow optimization.

This can be tested as below:

```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
df <- createDataFrame(mtcars)
collect(dapply(df, function(rdf) { data.frame(rdf$gear + 1) }, structType("gear double")))
```

### Requirements
  - R 3.5.x
  - Arrow package 0.12+
    ```bash
    Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
    ```

**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.

### Benchmarks

**Shall**

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false --driver-memory 4g
```

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true --driver-memory 4g
```

**R code**

```r
rdf <- read.csv("500000.csv")
df <- cache(createDataFrame(rdf))
count(df)

test <- function() {
  options(digits.secs = 6) # milliseconds
  start.time <- Sys.time()
  count(cache(dapply(df, function(rdf) { rdf }, schema(df))))
  end.time <- Sys.time()
  time.taken <- end.time - start.time
  print(time.taken)
}

test()
```

**Data (350 MB):**

```r
object.size(read.csv("500000.csv"))
350379504 bytes
```

"500000 Records"  http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/

**Results**

```
Time difference of 13.42037 mins
```

```
Time difference of 30.64156 secs
```

The performance improvement was around **2627%**.

### Limitations

- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.

- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.

## How was this patch tested?

Unit tests were added, and manually tested.

Closes #23787 from HyukjinKwon/SPARK-26830-1.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-27 14:29:58 +09:00
Liang-Chi Hsieh 0f2c0b53e8 [SPARK-26837][SQL] Pruning nested fields from object serializers
## What changes were proposed in this pull request?

In SPARK-26619, we make change to prune unnecessary individual serializers when serializing objects. This is extension to SPARK-26619. We can further prune nested fields from object serializers if they are not used.

For example, in following query, we only use one field in a struct column:

```scala
val data = Seq((("a", 1), 1), (("b", 2), 2), (("c", 3), 3))
val df = data.toDS().map(t => (t._1, t._2 + 1)).select("_1._1")
```

So, instead of having a serializer to create a two fields struct, we can prune unnecessary field from it. This is what this PR proposes to do.

In order to make this change conservative and safer, a SQL config is added to control it. It is disabled by default.

TODO: Support to prune nested fields inside MapType's key and value.

## How was this patch tested?

Added tests.

Closes #23740 from viirya/nested-pruning-serializer-2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-27 12:45:24 +08:00
Maxim Gekk a2a41b7bf2 [SPARK-26978][CORE][SQL] Avoid magic time constants
## What changes were proposed in this pull request?

In the PR, I propose to refactor existing code related to date/time conversions, and replace constants like `1000` and `1000000` by `DateTimeUtils` constants and transformation functions from `java.util.concurrent.TimeUnit._`.

## How was this patch tested?

The changes are tested by existing test suites.

Closes #23878 from MaxGekk/magic-time-constants.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-26 09:08:12 -06:00