Commit graph

7223 commits

Author SHA1 Message Date
Imran Rashid 8fbc1830f9 [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue
JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never try to allocate an array that big.  This commit changes some defaults & configs to gracefully fallover to something that doesn't require one large array in some cases; in other cases it simply improves an error message for cases which will still fail.

Closes #22818 from squito/SPARK-25827.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-11-07 13:18:52 +01:00
Yuming Wang 9e9fa2f69f
[SPARK-25098][SQL] Trim the string when cast stringToTimestamp and stringToDate
## What changes were proposed in this pull request?

**Hive** and **Oracle** trim the string when cast `stringToTimestamp` and `stringToDate`. this PR support this feature:
![image](https://user-images.githubusercontent.com/5399861/47979721-793b1e80-e0ff-11e8-97c8-24b10950ee9e.png)
![image](https://user-images.githubusercontent.com/5399861/47979725-7dffd280-e0ff-11e8-87d4-5767a00ed46e.png)

## How was this patch tested?

unit tests

Closes https://github.com/apache/spark/pull/22089

Closes #22943 from wangyum/SPARK-25098.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-06 21:26:28 -08:00
Maxim Gekk 76813cfa1e [SPARK-25950][SQL] from_csv should respect to spark.sql.columnNameOfCorruptRecord
## What changes were proposed in this pull request?

Fix for `CsvToStructs` to take into account SQL config `spark.sql.columnNameOfCorruptRecord` similar to `from_json`.

## How was this patch tested?

Added new test where `spark.sql.columnNameOfCorruptRecord` is set to corrupt column name different from default.

Closes #22956 from MaxGekk/csv-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-07 11:26:17 +08:00
yucai 63ca4bbe79
[SPARK-25676][SQL][TEST] Rename and refactor BenchmarkWideTable to use main method
## What changes were proposed in this pull request?

Refactor BenchmarkWideTable to use main method.
Generate benchmark result:

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.WideTableBenchmark"
```

## How was this patch tested?

manual tests

Closes #22823 from yucai/BenchmarkWideTable.

Lead-authored-by: yucai <yyu1@ebay.com>
Co-authored-by: Yucai Yu <yucai.yu@foxmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-06 15:40:56 -08:00
DB Tsai 3ed91c9b89
[SPARK-25946][BUILD] Upgrade ASM to 7.x to support JDK11
## What changes were proposed in this pull request?

Upgrade ASM to 7.x to support JDK11

## How was this patch tested?

Existing tests.

Closes #22953 from dbtsai/asm7.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-11-06 05:38:59 +00:00
Takuya UESHIN 78fa1be29b [SPARK-25926][CORE] Move config entries in core module to internal.config.
## What changes were proposed in this pull request?

Currently definitions of config entries in `core` module are in several files separately. We should move them into `internal/config` to be easy to manage.

## How was this patch tested?

Existing tests.

Closes #22928 from ueshin/issues/SPARK-25926/single_config_file.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-06 09:18:17 +08:00
Sean Owen c0d1bf0322 [MINOR] Fix typos and misspellings
## What changes were proposed in this pull request?

Fix typos and misspellings, per https://github.com/apache/spark-website/pull/158#issuecomment-435790366

## How was this patch tested?

Existing tests.

Closes #22950 from srowen/Typos.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-05 17:34:23 -06:00
Shahid fc65b4af00 [SPARK-25900][WEBUI] When the page number is more than the total page size, then fall back to the first page
## What changes were proposed in this pull request?

When we give the page number more than the maximum page number, webui is throwing an exception. It would be better if fall back to the default page, instead of throwing the exception in the web ui.

## How was this patch tested?
Before PR:
![screenshot from 2018-10-31 23-41-37](https://user-images.githubusercontent.com/23054875/47816448-354fbe80-dd79-11e8-83d8-6aab196642f7.png)

After PR:
![screenshot from 2018-10-31 23-54-23](https://user-images.githubusercontent.com/23054875/47816461-3ed92680-dd79-11e8-959d-0c531b3a6b2d.png)

Closes #22914 from shahidki31/pageFallBack.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-05 09:13:53 -06:00
yucai e017cb3964 [SPARK-25850][SQL] Make the split threshold for the code generated function configurable
## What changes were proposed in this pull request?
As per the discussion in [#22823](https://github.com/apache/spark/pull/22823/files#r228400706), add a new configuration to make the split threshold for the code generated function configurable.

When the generated Java function source code exceeds `spark.sql.codegen.methodSplitThreshold`, it will be split into multiple small functions.

## How was this patch tested?
manual tests

Closes #22847 from yucai/splitThreshold.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-05 20:09:39 +08:00
Takuya UESHIN 4afb350334 [SPARK-25884][SQL][FOLLOW-UP] Add sample.json back.
## What changes were proposed in this pull request?

This is a follow-up pr of #22892 which moved `sample.json` from hive module to sql module, but we still need the file in hive module.

## How was this patch tested?

Existing tests.

Closes #22942 from ueshin/issues/SPARK-25884/sample.json.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-05 15:53:06 +08:00
Maxim Gekk 950e7374a8 [SPARK-25913][SQL] Extend UnaryExecNode by unary SparkPlan nodes
## What changes were proposed in this pull request?

In the PR, I propose to extend `UnaryExecNode` instead of `SparkPlan` by unary nodes.

Closes #22925 from MaxGekk/unary-exec-node.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-04 17:41:42 -08:00
Maxim Gekk 39399f40b8 [SPARK-25638][SQL] Adding new function - to_csv()
## What changes were proposed in this pull request?

New functions takes a struct and converts it to a CSV strings using passed CSV options. It accepts the same CSV options as CSV data source does.

## How was this patch tested?

Added `CsvExpressionsSuite`, `CsvFunctionsSuite` as well as R, Python and SQL tests similar to tests for `to_json()`

Closes #22626 from MaxGekk/to_csv.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-04 14:57:38 +08:00
Maxim Gekk 42b6c1fb05
[SPARK-25931][SQL] Benchmarking creation of Jackson parser
## What changes were proposed in this pull request?

Added new benchmark which forcibly invokes Jackson parser to check overhead of its creation for short and wide JSON strings. Existing benchmarks do not allow to check that due to an optimisation introduced by #21909 for empty schema pushed down to JSON datasource. The `count()` action passes empty schema as required schema to the datasource, and Jackson parser is not created at all in that case.

Besides of new benchmark I also refactored existing benchmarks:
- Added `numIters` to control number of iteration in each benchmark
- Renamed `JSON per-line parsing` -> `count a short column`, `JSON parsing of wide lines` -> `count a wide column`, and `Count a dataset with 10 columns` -> `Select a subset of 10 columns`.

Closes #22920 from MaxGekk/json-benchmark-follow-up.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-03 09:09:39 -07:00
Shahid ed0c57e10d [SPARK-25861][MINOR][WEBUI] Remove unused refreshInterval parameter from the headerSparkPage method.
## What changes were proposed in this pull request?
'refreshInterval' is not used any where in the headerSparkPage method. So, we don't need to pass the parameter while calling the  'headerSparkPage' method.

## How was this patch tested?
Existing tests

Closes #22864 from shahidki31/unusedCode.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-02 17:17:48 -05:00
Dongjoon Hyun e91b607719
[SPARK-25918][SQL] LOAD DATA LOCAL INPATH should handle a relative path
## What changes were proposed in this pull request?

Unfortunately, it seems that we missed this in 2.4.0. In Spark 2.4, if the default file system is not the local file system, `LOAD DATA LOCAL INPATH` only works in case of absolute paths. This PR aims to fix it to support relative paths. This is a regression in 2.4.0.

```scala
$ ls kv1.txt
kv1.txt

scala> spark.sql("LOAD DATA LOCAL INPATH 'kv1.txt' INTO TABLE t")
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: kv1.txt;
```

## How was this patch tested?

Pass the Jenkins

Closes #22927 from dongjoon-hyun/SPARK-LOAD.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-01 23:18:20 -07:00
Wenchen Fan cd92f25be5 [SPARK-25746][SQL][FOLLOWUP] do not add unnecessary If expression
## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/22749.

When we construct the new serializer in `ExpressionEncoder.tuple`, we don't need to add `if(isnull ...)` check for each field. They are either simple expressions that can propagate null correctly(e.g. `GetStructField(GetColumnByOrdinal(0, schema), index)`), or complex expression that already have the isnull check.

## How was this patch tested?

existing tests

Closes #22898 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-01 12:47:32 +08:00
Takuya UESHIN cc82b9fed8 [SPARK-25884][SQL] Add TBLPROPERTIES and COMMENT, and use LOCATION when SHOW CREATE TABLE.
## What changes were proposed in this pull request?

When `SHOW CREATE TABLE` for Datasource tables, we are missing `TBLPROPERTIES` and `COMMENT`, and we should use `LOCATION` instead of path in `OPTION`.

## How was this patch tested?

Splitted `ShowCreateTableSuite` to confirm to work with both `InMemoryCatalog` and `HiveExternalCatalog`, and  added some tests.

Closes #22892 from ueshin/issues/SPARK-25884/show_create_table.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-01 10:00:14 +08:00
hyukjinkwon c9667aff4f [SPARK-25672][SQL] schema_of_csv() - schema inference from an example
## What changes were proposed in this pull request?

In the PR, I propose to add new function - *schema_of_csv()* which infers schema of CSV string literal. The result of the function is a string containing a schema in DDL format. For example:

```sql
select schema_of_csv('1|abc', map('delimiter', '|'))
```
```
struct<_c0:int,_c1:string>
```

## How was this patch tested?

Added new tests to `CsvFunctionsSuite`, `CsvExpressionsSuite` and SQL tests to `csv-functions.sql`

Closes #22666 from MaxGekk/schema_of_csv-function.

Lead-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-01 09:14:16 +08:00
Anton Okolnychyi bc9f9b4d6e
[SPARK-25860][SQL] Replace Literal(null, _) with FalseLiteral whenever possible
## What changes were proposed in this pull request?

This PR proposes a new optimization rule that replaces `Literal(null, _)` with `FalseLiteral` in conditions in `Join` and `Filter`, predicates in `If`, conditions in `CaseWhen`.

The idea is that some expressions evaluate to `false` if the underlying expression is `null` (as an example see `GeneratePredicate$create` or `doGenCode` and `eval` methods in `If` and `CaseWhen`). Therefore, we can replace `Literal(null, _)` with `FalseLiteral`, which can lead to more optimizations later on.

Let’s consider a few examples.

```
val df = spark.range(1, 100).select($"id".as("l"), ($"id" > 50).as("b"))
df.createOrReplaceTempView("t")
df.createOrReplaceTempView("p")
```

**Case 1**
```
spark.sql("SELECT * FROM t WHERE if(l > 10, false, NULL)").explain(true)

// without the new rule
…
== Optimized Logical Plan ==
Project [id#0L AS l#2L, cast(id#0L as string) AS s#3]
+- Filter if ((id#0L > 10)) false else null
   +- Range (1, 100, step=1, splits=Some(12))

== Physical Plan ==
*(1) Project [id#0L AS l#2L, cast(id#0L as string) AS s#3]
+- *(1) Filter if ((id#0L > 10)) false else null
   +- *(1) Range (1, 100, step=1, splits=12)

// with the new rule
…
== Optimized Logical Plan ==
LocalRelation <empty>, [l#2L, s#3]

== Physical Plan ==
LocalTableScan <empty>, [l#2L, s#3]
```

**Case 2**
```
spark.sql("SELECT * FROM t WHERE CASE WHEN l < 10 THEN null WHEN l > 40 THEN false ELSE null END”).explain(true)

// without the new rule
...
== Optimized Logical Plan ==
Project [id#0L AS l#2L, cast(id#0L as string) AS s#3]
+- Filter CASE WHEN (id#0L < 10) THEN null WHEN (id#0L > 40) THEN false ELSE null END
   +- Range (1, 100, step=1, splits=Some(12))

== Physical Plan ==
*(1) Project [id#0L AS l#2L, cast(id#0L as string) AS s#3]
+- *(1) Filter CASE WHEN (id#0L < 10) THEN null WHEN (id#0L > 40) THEN false ELSE null END
   +- *(1) Range (1, 100, step=1, splits=12)

// with the new rule
...
== Optimized Logical Plan ==
LocalRelation <empty>, [l#2L, s#3]

== Physical Plan ==
LocalTableScan <empty>, [l#2L, s#3]
```

**Case 3**
```
spark.sql("SELECT * FROM t JOIN p ON IF(t.l > p.l, null, false)").explain(true)

// without the new rule
...
== Optimized Logical Plan ==
Join Inner, if ((l#2L > l#37L)) null else false
:- Project [id#0L AS l#2L, cast(id#0L as string) AS s#3]
:  +- Range (1, 100, step=1, splits=Some(12))
+- Project [id#0L AS l#37L, cast(id#0L as string) AS s#38]
   +- Range (1, 100, step=1, splits=Some(12))

== Physical Plan ==
BroadcastNestedLoopJoin BuildRight, Inner, if ((l#2L > l#37L)) null else false
:- *(1) Project [id#0L AS l#2L, cast(id#0L as string) AS s#3]
:  +- *(1) Range (1, 100, step=1, splits=12)
+- BroadcastExchange IdentityBroadcastMode
   +- *(2) Project [id#0L AS l#37L, cast(id#0L as string) AS s#38]
      +- *(2) Range (1, 100, step=1, splits=12)

// with the new rule
...
== Optimized Logical Plan ==
LocalRelation <empty>, [l#2L, s#3, l#37L, s#38]
```

## How was this patch tested?

This PR comes with a set of dedicated tests.

Closes #22857 from aokolnychyi/spark-25860.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-10-31 18:35:33 +00:00
Dongjoon Hyun b3af917e76
[SPARK-25893][SQL] Show a directional error message for unsupported Hive Metastore versions
## What changes were proposed in this pull request?

When `spark.sql.hive.metastore.version` is misconfigured, we had better give a directional error message.

**BEFORE**
```scala
scala> sql("show databases").show
scala.MatchError: 2.4 (of class java.lang.String)
```

**AFTER**
```scala
scala> sql("show databases").show
java.lang.UnsupportedOperationException: Unsupported Hive Metastore version (2.4).
Please set spark.sql.hive.metastore.version with a valid version.
```

## How was this patch tested?

Manual.

Closes #22902 from dongjoon-hyun/SPARK-25893.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-31 09:20:19 -07:00
Gengliang Wang 57eddc7182 [SPARK-25886][SQL][MINOR] Improve error message of FailureSafeParser and from_avro in FAILFAST mode
## What changes were proposed in this pull request?

Currently in `FailureSafeParser` and `from_avro`, the exception is created with such code
```
throw new SparkException("Malformed records are detected in record parsing. " +
s"Parse Mode: ${FailFastMode.name}.", e.cause)
```

1. The cause part should be `e` instead of `e.cause`
2. If `e` contains non-null message, it should be shown in `from_json`/`from_csv`/`from_avro`, e.g.
```
com.fasterxml.jackson.core.JsonParseException: Unexpected character ('1' (code 49)): was expecting a colon to separate field name and value
at [Source: (InputStreamReader); line: 1, column: 7]
```
3.Kindly show hint for trying PERMISSIVE in error message.

## How was this patch tested?
Unit test.

Closes #22895 from gengliangwang/improve_error_msg.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-31 20:22:57 +08:00
caoxuewen 3c0e9ce944 [SPARK-24901][SQL] Merge the codegen of RegularHashMap and fastHashMap to reduce compiler maxCodesize when VectorizedHashMap is false.
## What changes were proposed in this pull request?

Currently, Generate code of update UnsafeRow in hash aggregation.
FastHashMap and RegularHashMap are two separate codes,These two separate codes need only when VectorizedHashMap is true. but other cases, we can merge together to reduce compiler maxCodesize. thanks.
```
import org.apache.spark.sql.execution.debug._
sparkSession.range(1).selectExpr("id AS key", "id AS value").groupBy("key").sum("value").debugCodegen

```
Generate code like:
 **Before modified:**
```
Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
...............
/* 420 */     if (agg_fastAggBuffer_0 != null) {
/* 421 */       // common sub-expressions
/* 422 */
/* 423 */       // evaluate aggregate function
/* 424 */       agg_agg_isNull_14_0 = true;
/* 425 */       long agg_value_15 = -1L;
/* 426 */       do {
/* 427 */         boolean agg_isNull_15 = agg_fastAggBuffer_0.isNullAt(0);
/* 428 */         long agg_value_16 = agg_isNull_15 ?
/* 429 */         -1L : (agg_fastAggBuffer_0.getLong(0));
/* 430 */         if (!agg_isNull_15) {
/* 431 */           agg_agg_isNull_14_0 = false;
/* 432 */           agg_value_15 = agg_value_16;
/* 433 */           continue;
/* 434 */         }
/* 435 */
/* 436 */         // This comment is added for manually tracking reference of 0, false
/* 437 */
/* 438 */         boolean agg_isNull_16 = false;
/* 439 */         long agg_value_17 = -1L;
/* 440 */         if (!false) {
/* 441 */           agg_value_17 = (long) 0;
/* 442 */         }
/* 443 */         if (!agg_isNull_16) {
/* 444 */           agg_agg_isNull_14_0 = false;
/* 445 */           agg_value_15 = agg_value_17;
/* 446 */           continue;
/* 447 */         }
/* 448 */
/* 449 */       } while (false);
/* 450 */
/* 451 */       long agg_value_14 = -1L;
/* 452 */       agg_value_14 = agg_value_15 + agg_expr_1_0;
/* 453 */       // update fast row
/* 454 */       agg_fastAggBuffer_0.setLong(0, agg_value_14);
/* 455 */     } else {
/* 456 */       // common sub-expressions
/* 457 */
/* 458 */       // evaluate aggregate function
/* 459 */       agg_agg_isNull_8_0 = true;
/* 460 */       long agg_value_9 = -1L;
/* 461 */       do {
/* 462 */         boolean agg_isNull_9 = agg_unsafeRowAggBuffer_0.isNullAt(0);
/* 463 */         long agg_value_10 = agg_isNull_9 ?
/* 464 */         -1L : (agg_unsafeRowAggBuffer_0.getLong(0));
/* 465 */         if (!agg_isNull_9) {
/* 466 */           agg_agg_isNull_8_0 = false;
/* 467 */           agg_value_9 = agg_value_10;
/* 468 */           continue;
/* 469 */         }
/* 470 */
/* 471 */         // This comment is added for manually tracking reference of 0, false
/* 472 */
/* 473 */         boolean agg_isNull_10 = false;
/* 474 */         long agg_value_11 = -1L;
/* 475 */         if (!false) {
/* 476 */           agg_value_11 = (long) 0;
/* 477 */         }
/* 478 */         if (!agg_isNull_10) {
/* 479 */           agg_agg_isNull_8_0 = false;
/* 480 */           agg_value_9 = agg_value_11;
/* 481 */           continue;
/* 482 */         }
/* 483 */
/* 484 */       } while (false);
/* 485 */
/* 486 */       long agg_value_8 = -1L;
/* 487 */       agg_value_8 = agg_value_9 + agg_expr_1_0;
/* 488 */       // update unsafe row buffer
/* 489 */       agg_unsafeRowAggBuffer_0.setLong(0, agg_value_8);
/* 490 */
/* 491 */     }
......................

```

 **After modified:**
```
Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
.............
/* 423 */     // Updates the proper row buffer
/* 424 */     UnsafeRow agg_aggBuffer_0 = null;
/* 425 */     if (agg_fastAggBuffer_0 != null) {
/* 426 */       agg_aggBuffer_0 = agg_fastAggBuffer_0;
/* 427 */     } else {
/* 428 */       agg_aggBuffer_0 = agg_unsafeRowAggBuffer_0;
/* 429 */     }
/* 430 */
/* 431 */     // common sub-expressions
/* 432 */
/* 433 */     // evaluate aggregate function
/* 434 */     agg_agg_isNull_8_0 = true;
/* 435 */     long agg_value_9 = -1L;
/* 436 */     do {
/* 437 */       boolean agg_isNull_9 = agg_aggBuffer_0.isNullAt(0);
/* 438 */       long agg_value_10 = agg_isNull_9 ?
/* 439 */       -1L : (agg_aggBuffer_0.getLong(0));
/* 440 */       if (!agg_isNull_9) {
/* 441 */         agg_agg_isNull_8_0 = false;
/* 442 */         agg_value_9 = agg_value_10;
/* 443 */         continue;
/* 444 */       }
/* 445 */
/* 446 */       // This comment is added for manually tracking reference of 0, false
/* 447 */
/* 448 */       boolean agg_isNull_10 = false;
/* 449 */       long agg_value_11 = -1L;
/* 450 */       if (!false) {
/* 451 */         agg_value_11 = (long) 0;
/* 452 */       }
/* 453 */       if (!agg_isNull_10) {
/* 454 */         agg_agg_isNull_8_0 = false;
/* 455 */         agg_value_9 = agg_value_11;
/* 456 */         continue;
/* 457 */       }
/* 458 */
/* 459 */     } while (false);
/* 460 */
/* 461 */     long agg_value_8 = -1L;
/* 462 */     agg_value_8 = agg_value_9 + agg_expr_1_0;
/* 463 */     // update unsafe row buffer
/* 464 */     agg_aggBuffer_0.setLong(0, agg_value_8);
...........

```
## How was this patch tested?

the Existed test cases.

Closes #21860 from heary-cao/fastHashMap.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-31 18:39:15 +08:00
yucai f8484e49ef
[SPARK-25663][SPARK-25661][SQL][TEST] Refactor BuiltInDataSourceWriteBenchmark, DataSourceWriteBenchmark and AvroWriteBenchmark to use main method
## What changes were proposed in this pull request?

Refactor BuiltInDataSourceWriteBenchmark, DataSourceWriteBenchmark and AvroWriteBenchmark to use main method.

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.BuiltInDataSourceWriteBenchmark"

SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "avro/test:runMain org.apache.spark.sql.execution.benchmark.AvroWriteBenchmark"
```
## How was this patch tested?

manual tests

Closes #22861 from yucai/BuiltInDataSourceWriteBenchmark.

Lead-authored-by: yucai <yyu1@ebay.com>
Co-authored-by: Yucai Yu <yucai.yu@foxmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-31 03:03:42 -07:00
Reynold Xin 9cf9a83afa [SPARK-25862][SQL] Remove rangeBetween APIs introduced in SPARK-21608
## What changes were proposed in this pull request?
This patch removes the rangeBetween functions introduced in SPARK-21608. As explained in SPARK-25841, these functions are confusing and don't quite work. We will redesign them and introduce better ones in SPARK-25843.

## How was this patch tested?
Removed relevant test cases as well. These test cases will need to be added back in SPARK-25843.

Closes #22870 from rxin/SPARK-25862.

Lead-authored-by: Reynold Xin <rxin@databricks.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-30 21:27:17 -07:00
caoxuewen f6ff6329ee [SPARK-25847][SQL][TEST] Refactor JSONBenchmarks to use main method
## What changes were proposed in this pull request?

Refactor JSONBenchmark to use main method

use spark-submit:
`bin/spark-submit --class org.apache.spark.sql.execution.datasources.json.JSONBenchmark --jars ./core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar,./sql/catalyst/target/spark-catalyst_2.11-3.0.0-SNAPSHOT-tests.jar ./sql/core/target/spark-sql_2.11-3.0.0-SNAPSHOT-tests.jar`

Generate benchmark result:
`SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.datasources.json.JSONBenchmark"`

## How was this patch tested?

manual tests

Closes #22844 from heary-cao/JSONBenchmarks.

Lead-authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Co-authored-by: heary <cao.xuewen@zte.com.cn>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-31 10:28:17 +08:00
Marco Gaido 891032da6f [SPARK-25691][SQL] Use semantic equality in AliasViewChild in order to compare attributes
## What changes were proposed in this pull request?

When we compare attributes, in general, we should always refer to semantic equality, as the default `equal` method can return false when there are "cosmetic" differences between them, but still they are the same thing; at least we have to consider them so when analyzing/optimizing queries.

The PR focuses on the usage and comparison of the `output` of a `LogicalPlan`, which is a `Seq[Attribute]` in `AliasViewChild`. In this case, using equality implicitly fails to check the semantic equality. This results in the operator failing to stabilize.

## How was this patch tested?

running the tests with the patch provided by maryannxue in https://github.com/apache/spark/pull/22060

Closes #22713 from mgaido91/SPARK-25691.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-31 09:18:53 +08:00
caoxuewen 94de5609be
[SPARK-25848][SQL][TEST] Refactor CSVBenchmarks to use main method
## What changes were proposed in this pull request?

use spark-submit:
`bin/spark-submit --class org.apache.spark.sql.execution.datasources.csv.CSVBenchmark --jars ./core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar,./sql/catalyst/target/spark-catalyst_2.11-3.0.0-SNAPSHOT-tests.jar ./sql/core/target/spark-sql_2.11-3.0.0-SNAPSHOT-tests.jar`

Generate benchmark result:
`SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.datasources.csv.CSVBenchmark"`

## How was this patch tested?

manual tests

Closes #22845 from heary-cao/CSVBenchmarks.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-30 09:18:55 -07:00
caoxuewen eab39f79e4 [SPARK-25755][SQL][TEST] Supplementation of non-CodeGen unit tested for BroadcastHashJoinExec
## What changes were proposed in this pull request?

Currently, the BroadcastHashJoinExec physical plan supports CodeGen and non-codegen, but only CodeGen code is tested in the unit tests of InnerJoinSuite、OuterJoinSuite、ExistenceJoinSuite, and non-codegen code is not tested. This PR supplements this part of the test.

## How was this patch tested?

add new unit tested.

Closes #22755 from heary-cao/AddTestToBroadcastHashJoinExec.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-30 20:13:18 +08:00
hyukjinkwon 5bd5e1b9c8 [MINOR][SQL] Avoid hardcoded configuration keys in SQLConf's doc
## What changes were proposed in this pull request?

This PR proposes to avoid hardcorded configuration keys in SQLConf's `doc.

## How was this patch tested?

Manually verified.

Closes #22877 from HyukjinKwon/minor-conf-name.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-30 07:38:26 +08:00
Dilip Biswal 5e5d886a2b [SPARK-25856][SQL][MINOR] Remove AverageLike and CountLike classes
## What changes were proposed in this pull request?
These two classes were added for regr_ expression support (SPARK-23907). These have been removed and hence we can remove these base classes and inline the logic in the concrete classes.
## How was this patch tested?
Existing tests.

Closes #22856 from dilipbiswal/average_cleanup.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-29 12:56:06 -05:00
Peter Toth 7fe5cff058 [SPARK-25767][SQL] Fix lazily evaluated stream of expressions in code generation
## What changes were proposed in this pull request?

Code generation is incorrect if `outputVars` parameter of `consume` method in `CodegenSupport` contains a lazily evaluated stream of expressions.
This PR fixes the issue by forcing the evaluation of `inputVars` before generating the code for UnsafeRow.

## How was this patch tested?

Tested with the sample program provided in https://issues.apache.org/jira/browse/SPARK-25767

Closes #22789 from peter-toth/SPARK-25767.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2018-10-29 16:47:50 +01:00
yucai 409d688fb6 [SPARK-25864][SQL][TEST] Make main args accessible for BenchmarkBase's subclass
## What changes were proposed in this pull request?

Set main args correctly in BenchmarkBase, to make it accessible for its subclass.
It will benefit:
- BuiltInDataSourceWriteBenchmark
- AvroWriteBenchmark

## How was this patch tested?

manual tests

Closes #22872 from yucai/main_args.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-29 20:00:31 +08:00
Bruce Robbins 4e990d9dd2 [DOC] Fix doc for spark.sql.parquet.recordLevelFilter.enabled
## What changes were proposed in this pull request?

Updated the doc string value for spark.sql.parquet.recordLevelFilter.enabled to indicate that spark.sql.parquet.enableVectorizedReader must be disabled.

The code in ParquetFileFormat uses spark.sql.parquet.recordLevelFilter.enabled only after falling back to parquet-mr (see else for this if statement): d5573c578a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala (L412)
d5573c578a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala (L427-L430)

Tests also bear this out.

## How was this patch tested?

This is just a doc string fix: I built Spark and ran a single test.

Closes #22865 from bersprockets/confdocfix.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-29 13:44:58 +08:00
Peter Toth ca2fca1432 [SPARK-25816][SQL] Fix attribute resolution in nested extractors
## What changes were proposed in this pull request?

Extractors are made of 2 expressions, one of them defines the the value to be extract from (called `child`) and the other defines the way of extraction (called `extraction`). In this term extractors have 2 children so they shouldn't be `UnaryExpression`s.

`ResolveReferences` was changed in this commit: 36b826f5d1 which resulted a regression with nested extractors. An extractor need to define its children as the set of both `child` and `extraction`; and should try to resolve both in `ResolveReferences`.

This PR changes `UnresolvedExtractValue` to a `BinaryExpression`.

## How was this patch tested?

added UT

Closes #22817 from peter-toth/SPARK-25816.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-28 17:51:35 -07:00
liuxian 4427a96bce [SPARK-25806][SQL] The instance of FileSplit is redundant
## What changes were proposed in this pull request?

 The instance of `FileSplit` is redundant for   `ParquetFileFormat` and `hive\orc\OrcFileFormat` class.

## How was this patch tested?
Existing unit tests in `ParquetQuerySuite.scala` and `HiveOrcQuerySuite.scala`

Closes #22802 from 10110346/FileSplitnotneed.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-28 17:39:16 -05:00
Xingbo Jiang a7ab7f2348 [SPARK-25845][SQL] Fix MatchError for calendar interval type in range frame left boundary
## What changes were proposed in this pull request?

WindowSpecDefinition checks start < last, but CalendarIntervalType is not comparable, so it would throw the following exception at runtime:

```
 scala.MatchError: CalendarIntervalType (of class org.apache.spark.sql.types.CalendarIntervalType$)      at
 org.apache.spark.sql.catalyst.util.TypeUtils$.getInterpretedOrdering(TypeUtils.scala:58) at
 org.apache.spark.sql.catalyst.expressions.BinaryComparison.ordering$lzycompute(predicates.scala:592) at
 org.apache.spark.sql.catalyst.expressions.BinaryComparison.ordering(predicates.scala:592) at
 org.apache.spark.sql.catalyst.expressions.GreaterThan.nullSafeEval(predicates.scala:797) at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:496) at org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame.isGreaterThan(windowExpressions.scala:245) at
 org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame.checkInputDataTypes(windowExpressions.scala:216) at
 org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:171) at
 org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:171) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 scala.collection.IndexedSeqOptimized$class.prefixLengthImpl(IndexedSeqOptimized.scala:38) at scala.collection.IndexedSeqOptimized$class.forall(IndexedSeqOptimized.scala:43) at scala.collection.mutable.ArrayBuffer.forall(ArrayBuffer.scala:48) at
 org.apache.spark.sql.catalyst.expressions.Expression.childrenResolved(Expression.scala:183) at
 org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition.resolved$lzycompute(windowExpressions.scala:48) at
 org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition.resolved(windowExpressions.scala:48) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 scala.collection.LinearSeqOptimized$class.forall(LinearSeqOptimized.scala:83)
```

We fix the issue by only perform the check on boundary expressions that are AtomicType.

## How was this patch tested?

Add new test case in `DataFrameWindowFramesSuite`

Closes #22853 from jiangxb1987/windowBoundary.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2018-10-28 18:15:47 +08:00
Wenchen Fan ff4bb836aa [SPARK-25817][SQL] Dataset encoder should support combination of map and product type
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/22745 , Dataset encoder supports the combination of java bean and map type. This PR is to fix the Scala side.

The reason why it didn't work before is, `CatalystToExternalMap` tries to get the data type of the input map expression, while it can be unresolved and its data type is known. To fix it, we can follow `UnresolvedMapObjects`, to create a `UnresolvedCatalystToExternalMap`, and only create `CatalystToExternalMap` when the input map expression is resolved and the data type is known.

## How was this patch tested?

enable a old test case

Closes #22812 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-28 13:33:26 +08:00
Dilip Biswal e545811346 [SPARK-19851][SQL] Add support for EVERY and ANY (SOME) aggregates
## What changes were proposed in this pull request?

Implements Every, Some, Any aggregates in SQL. These new aggregate expressions are analyzed in normal way and rewritten to equivalent existing aggregate expressions in the optimizer.

Every(x) => Min(x)  where x is boolean.
Some(x) => Max(x) where x is boolean.

Any is a synonym for Some.
SQL
```
explain extended select every(v) from test_agg group by k;
```
Plan :
```
== Parsed Logical Plan ==
'Aggregate ['k], [unresolvedalias('every('v), None)]
+- 'UnresolvedRelation `test_agg`

== Analyzed Logical Plan ==
every(v): boolean
Aggregate [k#0], [every(v#1) AS every(v)#5]
+- SubqueryAlias `test_agg`
   +- Project [k#0, v#1]
      +- SubqueryAlias `test_agg`
         +- LocalRelation [k#0, v#1]

== Optimized Logical Plan ==
Aggregate [k#0], [min(v#1) AS every(v)#5]
+- LocalRelation [k#0, v#1]

== Physical Plan ==
*(2) HashAggregate(keys=[k#0], functions=[min(v#1)], output=[every(v)#5])
+- Exchange hashpartitioning(k#0, 200)
   +- *(1) HashAggregate(keys=[k#0], functions=[partial_min(v#1)], output=[k#0, min#7])
      +- LocalTableScan [k#0, v#1]
Time taken: 0.512 seconds, Fetched 1 row(s)
```

## How was this patch tested?
Added tests in SQLQueryTestSuite, DataframeAggregateSuite

Closes #22809 from dilipbiswal/SPARK-19851-specific-rewrite.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-28 09:38:38 +08:00
laskfla 6f05669e4e [MINOR][DOC] Fix comment error of HiveUtils
## What changes were proposed in this pull request?
Change the version number in comment of `HiveUtils.newClientForExecution` from `13` to `1.2.1` .

## How was this patch tested?

N/A

Closes #22850 from laskfla/HiveUtils-Comment.

Authored-by: laskfla <wwlsax11@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-27 08:09:59 -05:00
Sean Owen ca545f7941 [SPARK-25821][SQL] Remove SQLContext methods deprecated in 1.4
## What changes were proposed in this pull request?

Remove SQLContext methods deprecated in 1.4

## How was this patch tested?

Existing tests.

Closes #22815 from srowen/SPARK-25821.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-26 16:49:48 -05:00
Gengliang Wang d325ffbf3a [SPARK-25851][SQL][MINOR] Fix deprecated API warning in SQLListener
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/21596, Jackson is upgraded to 2.9.6.
There are some deprecated API warnings in SQLListener.
Create a trivial PR to fix them.

```
[warn] SQLListener.scala:92: method uncheckedSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] val objectType = typeFactory.uncheckedSimpleType(classOf[Object])
[warn]
[warn] SQLListener.scala:93: method constructSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(objectType, objectType))
[warn]
[warn] SQLListener.scala:97: method uncheckedSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] val longType = typeFactory.uncheckedSimpleType(classOf[Long])
[warn]
[warn] SQLListener.scala:98: method constructSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(longType, longType))
```

## How was this patch tested?

Existing unit tests.

Closes #22848 from gengliangwang/fixSQLListenerWarning.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-26 16:45:56 -05:00
hyukjinkwon 33e337c118 [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's input json as literal only
## What changes were proposed in this pull request?

The main purpose of `schema_of_json` is the usage of combination with `from_json` (to make up the leak of schema inference) which takes its schema only as literal; however, currently `schema_of_json` allows JSON input as non-literal expressions (e.g, column).

This was mistakenly allowed - we don't have to take other usages rather then the main purpose into account for now.

This PR makes a followup to only allow literals for `schema_of_json`'s JSON input. We can allow non literal expressions later when it's needed or there are some usecase for it.

## How was this patch tested?

Unit tests were added.

Closes #22775 from HyukjinKwon/SPARK-25447-followup.

Lead-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-26 22:14:43 +08:00
Reynold Xin 89d748b33c [SPARK-25842][SQL] Deprecate rangeBetween APIs introduced in SPARK-21608
## What changes were proposed in this pull request?
See the detailed information at https://issues.apache.org/jira/browse/SPARK-25841 on why these APIs should be deprecated and redesigned.

This patch also reverts 8acb51f08b which applies to 2.4.

## How was this patch tested?
Only deprecation and doc changes.

Closes #22841 from rxin/SPARK-25842.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-26 13:17:24 +08:00
Shixiong Zhu 86d469aeaa [SPARK-25822][PYSPARK] Fix a race condition when releasing a Python worker
## What changes were proposed in this pull request?

There is a race condition when releasing a Python worker. If `ReaderIterator.handleEndOfDataSection` is not running in the task thread, when a task is early terminated (such as `take(N)`), the task completion listener may close the worker but "handleEndOfDataSection" can still put the worker into the worker pool to reuse.

0e07b483d2 is a patch to reproduce this issue.

I also found a user reported this in the mail list: http://mail-archives.apache.org/mod_mbox/spark-user/201610.mbox/%3CCAAUq=H+YLUEpd23nwvq13Ms5hOStkhX3ao4f4zQV6sgO5zM-xAmail.gmail.com%3E

This PR fixes the issue by using `compareAndSet` to make sure we will never return a closed worker to the work pool.

## How was this patch tested?

Jenkins.

Closes #22816 from zsxwing/fix-socket-closed.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-10-26 13:53:51 +09:00
Wenchen Fan 72a23a6c43 [SPARK-25772][SQL][FOLLOWUP] remove GetArrayFromMap
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/22745 we introduced the `GetArrayFromMap` expression. Later on I realized this is duplicated as we already have `MapKeys` and `MapValues`.

This PR removes `GetArrayFromMap`

## How was this patch tested?

existing tests

Closes #22825 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-26 10:19:35 +08:00
Peter Toth ccd07b7366
[SPARK-25665][SQL][TEST] Refactor ObjectHashAggregateExecBenchmark to…
## What changes were proposed in this pull request?

Refactor ObjectHashAggregateExecBenchmark to use main method

## How was this patch tested?

Manually tested:
```
bin/spark-submit --class org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark --jars sql/catalyst/target/spark-catalyst_2.11-3.0.0-SNAPSHOT-tests.jar,core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar,sql/hive/target/spark-hive_2.11-3.0.0-SNAPSHOT.jar --packages org.spark-project.hive:hive-exec:1.2.1.spark2 sql/hive/target/spark-hive_2.11-3.0.0-SNAPSHOT-tests.jar
```
Generated results with:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark"
```

Closes #22804 from peter-toth/SPARK-25665.

Lead-authored-by: Peter Toth <peter.toth@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-25 12:42:31 -07:00
Liang-Chi Hsieh cb5ea201df [SPARK-25746][SQL] Refactoring ExpressionEncoder to get rid of flat flag
## What changes were proposed in this pull request?

This is inspired during implementing #21732. For now `ScalaReflection` needs to consider how `ExpressionEncoder` uses generated serializers and deserializers. And `ExpressionEncoder` has a weird `flat` flag. After discussion with cloud-fan, it seems to be better to refactor `ExpressionEncoder`. It should make SPARK-24762 easier to do.

To summarize the proposed changes:

1. `serializerFor` and `deserializerFor` return expressions for serializing/deserializing an input expression for a given type. They are private and should not be called directly.
2. `serializerForType` and `deserializerForType` returns an expression for serializing/deserializing for an object of type T to/from Spark SQL representation. It assumes the input object/Spark SQL representation is located at ordinal 0 of a row.

So in other words, `serializerForType` and `deserializerForType` return expressions for atomically serializing/deserializing JVM object to/from Spark SQL value.

A serializer returned by `serializerForType` will serialize an object at `row(0)` to a corresponding Spark SQL representation, e.g. primitive type, array, map, struct.

A deserializer returned by `deserializerForType` will deserialize an input field at `row(0)` to an object with given type.

3. The construction of `ExpressionEncoder` takes a pair of serializer and deserializer for type `T`. It uses them to create serializer and deserializer for T <-> row serialization. Now `ExpressionEncoder` dones't need to remember if serializer is flat or not. When we need to construct new `ExpressionEncoder` based on existing ones, we only need to change input location in the atomic serializer and deserializer.

## How was this patch tested?

Existing tests.

Closes #22749 from viirya/SPARK-24762-refactor.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-25 19:27:45 +08:00
adrian555 ddd1b1e8ae [SPARK-24572][SPARKR] "eager execution" for R shell, IDE
## What changes were proposed in this pull request?

Check the `spark.sql.repl.eagerEval.enabled` configuration property in SparkDataFrame `show()` method. If the `SparkSession` has eager execution enabled, the data will be returned to the R client when the data frame is created. So instead of seeing this
```
> df <- createDataFrame(faithful)
> df
SparkDataFrame[eruptions:double, waiting:double]
```
you will see
```
> df <- createDataFrame(faithful)
> df
+---------+-------+
|eruptions|waiting|
+---------+-------+
|      3.6|   79.0|
|      1.8|   54.0|
|    3.333|   74.0|
|    2.283|   62.0|
|    4.533|   85.0|
|    2.883|   55.0|
|      4.7|   88.0|
|      3.6|   85.0|
|     1.95|   51.0|
|     4.35|   85.0|
|    1.833|   54.0|
|    3.917|   84.0|
|      4.2|   78.0|
|     1.75|   47.0|
|      4.7|   83.0|
|    2.167|   52.0|
|     1.75|   62.0|
|      4.8|   84.0|
|      1.6|   52.0|
|     4.25|   79.0|
+---------+-------+
only showing top 20 rows
```

## How was this patch tested?
Manual tests as well as unit tests (one new test case is added).

Author: adrian555 <v2ave10p>

Closes #22455 from adrian555/eager_execution.
2018-10-24 23:42:06 -07:00
Maxim Gekk 4d6704db4d [SPARK-25243][SQL] Use FailureSafeParser in from_json
## What changes were proposed in this pull request?

In the PR, I propose to switch `from_json` on `FailureSafeParser`, and to make the function compatible to `PERMISSIVE` mode by default, and to support the `FAILFAST` mode as well. The `DROPMALFORMED` mode is not supported by `from_json`.

## How was this patch tested?

It was tested by existing `JsonSuite`/`CSVSuite`, `JsonFunctionsSuite` and `JsonExpressionsSuite` as well as new tests for `from_json` which checks different modes.

Closes #22237 from MaxGekk/from_json-failuresafe.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-24 19:09:15 +08:00
Vladimir Kuriatkov 584e767d37 [SPARK-25772][SQL] Fix java map of structs deserialization
This is a follow-up PR for #22708. It considers another case of java beans deserialization: java maps with struct keys/values.

When deserializing values of MapType with struct keys/values in java beans, fields of structs get mixed up. I suggest using struct data types retrieved from resolved input data instead of inferring them from java beans.

## What changes were proposed in this pull request?

Invocations of "keyArray" and "valueArray" functions are used to extract arrays of keys and values. Struct type of keys or values is also inferred from java bean structure and ends up with mixed up field order.
I created a new UnresolvedInvoke expression as a temporary substitution of Invoke expression while no actual data is available. It allows to provide the resulting data type during analysis based on the resolved input data, not on the java bean (similar to UnresolvedMapObjects).

Key and value arrays are then fed to MapObjects expression which I replaced with UnresolvedMapObjects, just like in case of ArrayType.

Finally I added resolution of UnresolvedInvoke expressions in Analyzer.resolveExpression method as an additional pattern matching case.

## How was this patch tested?

Added a test case.
Built complete project on travis.

viirya kiszk cloud-fan michalsenkyr marmbrus liancheng

Closes #22745 from vofque/SPARK-21402-FOLLOWUP.

Lead-authored-by: Vladimir Kuriatkov <vofque@gmail.com>
Co-authored-by: Vladimir Kuriatkov <Vladimir_Kuriatkov@epam.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-24 09:29:40 +08:00