Commit graph

3094 commits

Author SHA1 Message Date
Kazuaki Ishizaki 5ad4735bda [SPARK-24529][BUILD][TEST-MAVEN] Add spotbugs into maven build process
## What changes were proposed in this pull request?

This PR enables a Java bytecode check tool [spotbugs](https://spotbugs.github.io/) to avoid possible integer overflow at multiplication. When an violation is detected, the build process is stopped.
Due to the tool limitation, some other checks will be enabled. In this PR, [these patterns](http://spotbugs-in-kengo-toda.readthedocs.io/en/lqc-list-detectors/detectors.html#findpuzzlers) in `FindPuzzlers` can be detected.

This check is enabled at `compile` phase. Thus, `mvn compile` or `mvn package` launches this check.

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21542 from kiszk/SPARK-24529.
2018-07-12 09:52:23 +08:00
Maxim Gekk 3ab48f985c [SPARK-24761][SQL] Adding of isModifiable() to RuntimeConfig
## What changes were proposed in this pull request?

In the PR, I propose to extend `RuntimeConfig` by new method `isModifiable()` which returns `true` if a config parameter can be modified at runtime (for current session state). For static SQL and core parameters, the method returns `false`.

## How was this patch tested?

Added new test to `RuntimeConfigSuite` for checking Spark core and SQL parameters.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21730 from MaxGekk/is-modifiable.
2018-07-11 17:38:43 -07:00
Marco Gaido e008ad1752 [SPARK-24782][SQL] Simplify conf retrieval in SQL expressions
## What changes were proposed in this pull request?

The PR simplifies the retrieval of config in `size`, as we can access them from tasks too thanks to SPARK-24250.

## How was this patch tested?

existing UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21736 from mgaido91/SPARK-24605_followup.
2018-07-11 17:30:43 -07:00
Marco Gaido ebf4bfb966 [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas
## What changes were proposed in this pull request?

A self-join on a dataset which contains a `FlatMapGroupsInPandas` fails because of duplicate attributes. This happens because we are not dealing with this specific case in our `dedupAttr` rules.

The PR fix the issue by adding the management of the specific case

## How was this patch tested?

added UT + manual tests

Author: Marco Gaido <marcogaido91@gmail.com>
Author: Marco Gaido <mgaido@hortonworks.com>

Closes #21737 from mgaido91/SPARK-24208.
2018-07-11 09:29:19 -07:00
Marek Novotny 74a8d6308b [SPARK-24165][SQL] Fixing conditional expressions to handle nullability of nested types
## What changes were proposed in this pull request?
This PR is proposing a fix for the output data type of ```If``` and ```CaseWhen``` expression. Upon till now, the implementation of exprassions has ignored nullability of nested types from different execution branches and returned the type of the first branch.

This could lead to an unwanted ```NullPointerException``` from other expressions depending on a ```If```/```CaseWhen``` expression.

Example:
```
val rows = new util.ArrayList[Row]()
rows.add(Row(true, ("a", 1)))
rows.add(Row(false, (null, 2)))
val schema = StructType(Seq(
  StructField("cond", BooleanType, false),
  StructField("s", StructType(Seq(
    StructField("val1", StringType, true),
    StructField("val2", IntegerType, false)
  )), false)
))

val df = spark.createDataFrame(rows, schema)

df
  .select(when('cond, struct(lit("x").as("val1"), lit(10).as("val2"))).otherwise('s) as "res")
  .select('res.getField("val1"))
  .show()
```
Exception:
```
Exception in thread "main" java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.codegen.UnsafeWriter.write(UnsafeWriter.java:109)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
...
```
Output schema:
```
root
 |-- res.val1: string (nullable = false)
```

## How was this patch tested?
New test cases added into
- DataFrameSuite.scala
- conditionalExpressions.scala

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21687 from mn-mikke/SPARK-24165.
2018-07-11 12:21:03 +08:00
Tathagata Das 6078b891da [SPARK-24730][SS] Add policy to choose max as global watermark when streaming query has multiple watermarks
## What changes were proposed in this pull request?

Currently, when a streaming query has multiple watermark, the policy is to choose the min of them as the global watermark. This is safe to do as the global watermark moves with the slowest stream, and is therefore is safe as it does not unexpectedly drop some data as late, etc. While this is indeed the safe thing to do, in some cases, you may want the watermark to advance with the fastest stream, that is, take the max of multiple watermarks. This PR is to add that configuration. It makes the following changes.

- Adds a configuration to specify max as the policy.
- Saves the configuration in OffsetSeqMetadata because changing it in the middle can lead to unpredictable results.
   - For old checkpoints without the configuration, it assumes the default policy as min (irrespective of the policy set at the session where the query is being restarted). This is to ensure that existing queries are affected in any way.

TODO
- [ ] Add a test for recovery from existing checkpoints.

## How was this patch tested?
New unit test

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #21701 from tdas/SPARK-24730.
2018-07-10 18:03:40 -07:00
Mukul Murthy 32cb50835e [SPARK-24662][SQL][SS] Support limit in structured streaming
## What changes were proposed in this pull request?

Support the LIMIT operator in structured streaming.

For streams in append or complete output mode, a stream with a LIMIT operator will return no more than the specified number of rows. LIMIT is still unsupported for the update output mode.

This change reverts e4fee395ec as part of it because it is a better and more complete implementation.

## How was this patch tested?

New and existing unit tests.

Author: Mukul Murthy <mukul.murthy@gmail.com>

Closes #21662 from mukulmurthy/SPARK-24662.
2018-07-10 11:08:04 -07:00
Xiao Li aec966b05e Revert "[SPARK-24268][SQL] Use datatype.simpleString in error messages"
This reverts commit 1bd3d61f41.
2018-07-09 14:24:23 -07:00
Marco Gaido 1bd3d61f41 [SPARK-24268][SQL] Use datatype.simpleString in error messages
## What changes were proposed in this pull request?

SPARK-22893 tried to unify error messages about dataTypes. Unfortunately, still many places were missing the `simpleString` method in other to have the same representation everywhere.

The PR unified the messages using alway the simpleString representation of the dataTypes in the messages.

## How was this patch tested?

existing/modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21321 from mgaido91/SPARK-24268.
2018-07-09 22:59:05 +08:00
Bruce Robbins 034913b62b [SPARK-23936][SQL] Implement map_concat
## What changes were proposed in this pull request?

Implement map_concat high order function.

This implementation does not pick a winner when the specified maps have overlapping keys. Therefore, this implementation preserves existing duplicate keys in the maps and potentially introduces new duplicates (After discussion with ueshin, we settled on option 1 from [here](https://issues.apache.org/jira/browse/SPARK-23936?focusedCommentId=16464245&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16464245)).

## How was this patch tested?

New tests
Manual tests
Run all sbt SQL tests
Run all pyspark sql tests

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21073 from bersprockets/SPARK-23936.
2018-07-09 21:21:38 +09:00
Liang-Chi Hsieh fc43690d36 [SPARK-24749][SQL] Use sameType to compare Array's element type in ArrayContains
## What changes were proposed in this pull request?

We should use `DataType.sameType` to compare element type in `ArrayContains`, otherwise nullability affects comparison result.

## How was this patch tested?

Added test.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21724 from viirya/SPARK-24749.
2018-07-07 11:34:30 +08:00
Liang-Chi Hsieh 4de0425df8 [SPARK-24569][SQL] Aggregator with output type Option should produce consistent schema
## What changes were proposed in this pull request?

SQL `Aggregator` with output type `Option[Boolean]` creates column of type `StructType`. It's not in consistency with a Dataset of similar java class.

This changes the way `definedByConstructorParams` checks given type. For `Option[_]`, it goes to check its type argument.

## How was this patch tested?

Added test.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21611 from viirya/SPARK-24569.
2018-07-07 10:54:14 +08:00
Takuya UESHIN 01fcba2c68 [SPARK-24737][SQL] Type coercion between StructTypes.
## What changes were proposed in this pull request?

We can support type coercion between `StructType`s where all the internal types are compatible.

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21713 from ueshin/issues/SPARK-24737/structtypecoercion.
2018-07-06 11:10:50 +08:00
Gengliang Wang 33952cfa81 [SPARK-24675][SQL] Rename table: validate existence of new location
## What changes were proposed in this pull request?
If table is renamed to a existing new location, data won't show up.
```
scala>  Seq("hello").toDF("a").write.format("parquet").saveAsTable("t")

scala> sql("select * from t").show()
+-----+
|    a|
+-----+
|hello|
+-----+

scala> sql("alter table t rename to test")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("select * from test").show()
+---+
|  a|
+---+
+---+
```
The file layout is like
```
$ tree test
test
├── gabage
└── t
    ├── _SUCCESS
    └── part-00000-856b0f10-08f1-42d6-9eb3-7719261f3d5e-c000.snappy.parquet
```

In Hive, if the new location exists, the renaming will fail even the location is empty.

We should have the same validation in Catalog, in case of unexpected bugs.

## How was this patch tested?

New unit test.

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21655 from gengliangwang/validate_rename_table.
2018-07-05 09:25:19 -07:00
Liang-Chi Hsieh 32cfd3e75a [SPARK-24361][SQL] Polish code block manipulation API
## What changes were proposed in this pull request?

Current code block manipulation API is immature and hacky. We need a formal API to manipulate code blocks.

The basic idea is making `JavaCode`  as `TreeNode`. So we can use familiar `transform` API to manipulate code blocks and expressions in code blocks.

For example, we can replace `SimpleExprValue` in a code block like this:

```scala
code.transformExprValues {
  case SimpleExprValue("1 + 1", _) => aliasedParam
}
```

The example use case is splitting code to methods.

For example, we have an `ExprCode` containing generated code. But it is too long and we need to split it as method. Because statement-based expressions can't be directly passed into. We need to transform them as variables first:

```scala

def getExprValues(block: Block): Set[ExprValue] = block match {
  case c: CodeBlock =>
    c.blockInputs.collect {
      case e: ExprValue => e
    }.toSet
  case _ => Set.empty
}

def currentCodegenInputs(ctx: CodegenContext): Set[ExprValue] = {
  // Collects current variables in ctx.currentVars and ctx.INPUT_ROW.
  // It looks roughly like...
  ctx.currentVars.flatMap { v =>
    getExprValues(v.code) ++ Set(v.value, v.isNull)
  }.toSet + ctx.INPUT_ROW
}

// A code block of an expression contains too long code, making it as method
if (eval.code.length > 1024) {
  val setIsNull = if (!eval.isNull.isInstanceOf[LiteralValue]) {
    ...
  } else {
    ""
  }

  // Pick up variables and statements necessary to pass in.
  val currentVars = currentCodegenInputs(ctx)
  val varsPassIn = getExprValues(eval.code).intersect(currentVars)
  val aliasedExprs = HashMap.empty[SimpleExprValue, VariableValue]

  // Replace statement-based expressions which can't be directly passed in the method.
  val newCode = eval.code.transform {
    case block =>
      block.transformExprValues {
        case s: SimpleExprValue(_, javaType) if varsPassIn.contains(s) =>
          if (aliasedExprs.contains(s)) {
            aliasedExprs(s)
          } else {
            val aliasedVariable = JavaCode.variable(ctx.freshName("aliasedVar"), javaType)
            aliasedExprs += s -> aliasedVariable
            varsPassIn += aliasedVariable
            aliasedVariable
          }
      }
  }

  val params = varsPassIn.filter(!_.isInstanceOf[SimpleExprValue])).map { variable =>
    s"${variable.javaType.getName} ${variable.variableName}"
  }.mkString(", ")

  val funcName = ctx.freshName("nodeName")
  val javaType = CodeGenerator.javaType(dataType)
  val newValue = JavaCode.variable(ctx.freshName("value"), dataType)
  val funcFullName = ctx.addNewFunction(funcName,
    s"""
      |private $javaType $funcName($params) {
      |  $newCode
      |  $setIsNull
      |  return ${eval.value};
      |}
    """.stripMargin))

  eval.value = newValue
  val args = varsPassIn.filter(!_.isInstanceOf[SimpleExprValue])).map { variable =>
    s"${variable.variableName}"
  }

  // Create a code block to assign statements to aliased variables.
  val createVariables = aliasedExprs.foldLeft(EmptyBlock) { (block, (statement, variable)) =>
    block + code"${statement.javaType.getName} $variable = $statement;"
  }
  eval.code = createVariables + code"$javaType $newValue = $funcFullName($args);"
}
```

## How was this patch tested?

Added unite tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21405 from viirya/codeblock-api.
2018-07-05 20:48:55 +08:00
Wenchen Fan bf764a33be [SPARK-22384][SQL][FOLLOWUP] Refine partition pruning when attribute is wrapped in Cast
## What changes were proposed in this pull request?

As mentioned in https://github.com/apache/spark/pull/21586 , `Cast.mayTruncate` is not 100% safe, string to boolean is allowed. Since changing `Cast.mayTruncate` also changes the behavior of Dataset, here I propose to add a new `Cast.canSafeCast` for partition pruning.

## How was this patch tested?

new test cases

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21712 from cloud-fan/safeCast.
2018-07-04 18:36:09 -07:00
Liang-Chi Hsieh 1a2655a9e7 [SPARK-24635][SQL] Remove Blocks class from JavaCode class hierarchy
## What changes were proposed in this pull request?

The `Blocks` class in `JavaCode` class hierarchy is not necessary. Its function can be taken by `CodeBlock`. We should remove it to make simpler class hierarchy.

## How was this patch tested?

Existing tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21619 from viirya/SPARK-24635.
2018-07-04 20:42:08 +08:00
Takeshi Yamamuro b2deef64f6 [SPARK-24727][SQL] Add a static config to control cache size for generated classes
## What changes were proposed in this pull request?
Since SPARK-24250 has been resolved, executors correctly references user-defined configurations. So, this pr added a static config to control cache size for generated classes in `CodeGenerator`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21705 from maropu/SPARK-24727.
2018-07-04 20:04:18 +08:00
Takuya UESHIN 7c08eb6d61 [SPARK-24732][SQL] Type coercion between MapTypes.
## What changes were proposed in this pull request?

Currently we don't allow type coercion between maps.
We can support type coercion between MapTypes where both the key types and the value types are compatible.

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21703 from ueshin/issues/SPARK-24732/maptypecoercion.
2018-07-04 12:21:26 +08:00
Maxim Gekk 776f299fc8 [SPARK-24709][SQL] schema_of_json() - schema inference from an example
## What changes were proposed in this pull request?

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

One of the use cases is using of *schema_of_json()* in the combination with *from_json()*. Currently, _from_json()_ requires a schema as a mandatory argument. The *schema_of_json()* function will allow to point out an JSON string as an example which has the same schema as the first argument of _from_json()_. For instance:

```sql
select from_json(json_column, schema_of_json('{"c1": [0], "c2": [{"c3":0}]}'))
from json_table;
```

## How was this patch tested?

Added new test to `JsonFunctionsSuite`, `JsonExpressionsSuite` and SQL tests to `json-functions.sql`

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21686 from MaxGekk/infer_schema_json.
2018-07-04 09:38:18 +08:00
Yuanjian Li 8f91c697e2 [SPARK-24665][PYSPARK] Use SQLConf in PySpark to manage all sql configs
## What changes were proposed in this pull request?

Use SQLConf for PySpark to manage all sql configs, drop all the hard code in config usage.

## How was this patch tested?

Existing UT.

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21648 from xuanyuanking/SPARK-24665.
2018-07-02 14:35:37 +08:00
Xiao Li d54d8b8630 simplify rand in dsl/package.scala 2018-06-29 23:51:13 -07:00
maryannxue 797971ed42 [SPARK-24696][SQL] ColumnPruning rule fails to remove extra Project
## What changes were proposed in this pull request?

The ColumnPruning rule tries adding an extra Project if an input node produces fields more than needed, but as a post-processing step, it needs to remove the lower Project in the form of "Project - Filter - Project" otherwise it would conflict with PushPredicatesThroughProject and would thus cause a infinite optimization loop. The current post-processing method is defined as:
```
  private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transform {
    case p1  Project(_, f  Filter(_, p2  Project(_, child)))
      if p2.outputSet.subsetOf(child.outputSet) =>
      p1.copy(child = f.copy(child = child))
  }
```
This method works well when there is only one Filter but would not if there's two or more Filters. In this case, there is a deterministic filter and a non-deterministic filter so they stay as separate filter nodes and cannot be combined together.

An simplified illustration of the optimization process that forms the infinite loop is shown below (F1 stands for the 1st filter, F2 for the 2nd filter, P for project, S for scan of relation, PredicatePushDown as abbrev. of PushPredicatesThroughProject):
```
                             F1 - F2 - P - S
PredicatePushDown      =>    F1 - P - F2 - S
ColumnPruning          =>    F1 - P - F2 - P - S
                       =>    F1 - P - F2 - S        (Project removed)
PredicatePushDown      =>    P - F1 - F2 - S
ColumnPruning          =>    P - F1 - P - F2 - S
                       =>    P - F1 - P - F2 - P - S
                       =>    P - F1 - F2 - P - S    (only one Project removed)
RemoveRedundantProject =>    F1 - F2 - P - S        (goes back to the loop start)
```
So the problem is the ColumnPruning rule adds a Project under a Filter (and fails to remove it in the end), and that new Project triggers PushPredicateThroughProject. Once the filters have been push through the Project, a new Project will be added by the ColumnPruning rule and this goes on and on.
The fix should be when adding Projects, the rule applies top-down, but later when removing extra Projects, the process should go bottom-up to ensure all extra Projects can be matched.

## How was this patch tested?

Added a optimization rule test in ColumnPruningSuite; and a end-to-end test in SQLQuerySuite.

Author: maryannxue <maryannxue@apache.org>

Closes #21674 from maryannxue/spark-24696.
2018-06-29 23:46:12 -07:00
Yuming Wang 03545ce6de [SPARK-24638][SQL] StringStartsWith support push down
## What changes were proposed in this pull request?

`StringStartsWith` support push down. About 50% savings in compute time.

## How was this patch tested?
unit tests, manual tests and performance test:
```scala
cat <<EOF > SPARK-24638.scala
def benchmark(func: () => Unit): Long = {
  val start = System.currentTimeMillis()
  for(i <- 0 until 100) { func() }
  val end = System.currentTimeMillis()
  end - start
}
val path = "/tmp/spark/parquet/string/"
spark.range(10000000).selectExpr("concat(id, 'str', id) as id").coalesce(1).write.mode("overwrite").option("parquet.block.size", 1048576).parquet(path)
val df = spark.read.parquet(path)

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=true")
val pushdownEnable = benchmark(() => df.where("id like '999998%'").count())

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=false")
val pushdownDisable = benchmark(() => df.where("id like '999998%'").count())

val improvements = pushdownDisable - pushdownEnable
println(s"improvements: $improvements")
EOF

bin/spark-shell -i SPARK-24638.scala
```
result:
```scala
Loading SPARK-24638.scala...
benchmark: (func: () => Unit)Long
path: String = /tmp/spark/parquet/string/
df: org.apache.spark.sql.DataFrame = [id: string]
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownEnable: Long = 11608
res2: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownDisable: Long = 31981
improvements: Long = 20373
```

Author: Yuming Wang <yumwang@ebay.com>

Closes #21623 from wangyum/SPARK-24638.
2018-06-30 13:58:50 +08:00
Jose Torres f6e6899a8b [SPARK-24386][SS] coalesce(1) aggregates in continuous processing
## What changes were proposed in this pull request?

Provide a continuous processing implementation of coalesce(1), as well as allowing aggregates on top of it.

The changes in ContinuousQueuedDataReader and such are to use split.index (the ID of the partition within the RDD currently being compute()d) rather than context.partitionId() (the partition ID of the scheduled task within the Spark job - that is, the post coalesce writer). In the absence of a narrow dependency, these values were previously always the same, so there was no need to distinguish.

## How was this patch tested?

new unit test

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21560 from jose-torres/coalesce.
2018-06-28 16:25:40 -07:00
debugger87 c04cb2d1b7 [SPARK-21687][SQL] Spark SQL should set createTime for Hive partition
## What changes were proposed in this pull request?

Set createTime for every hive partition created in Spark SQL, which could be used to manage data lifecycle in Hive warehouse. We found  that almost every partition modified by spark sql has not been set createTime.

```
mysql> select * from partitions where create_time=0 limit 1\G;
*************************** 1. row ***************************
         PART_ID: 1028584
     CREATE_TIME: 0
LAST_ACCESS_TIME: 1502203611
       PART_NAME: date=20170130
           SD_ID: 1543605
          TBL_ID: 211605
  LINK_TARGET_ID: NULL
1 row in set (0.27 sec)
```

## How was this patch tested?
 N/A

Author: debugger87 <yangchaozhong.2009@gmail.com>
Author: Chaozhong Yang <yangchaozhong.2009@gmail.com>

Closes #18900 from debugger87/fix/set-create-time-for-hive-partition.
2018-06-27 11:34:28 -07:00
Yuanjian Li 6a0b77a55d [SPARK-24215][PYSPARK][FOLLOW UP] Implement eager evaluation for DataFrame APIs in PySpark
## What changes were proposed in this pull request?

Address comments in #21370 and add more test.

## How was this patch tested?

Enhance test in pyspark/sql/test.py and DataFrameSuite

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21553 from xuanyuanking/SPARK-24215-follow.
2018-06-27 10:43:06 -07:00
Takuya UESHIN 9a76f23c6a [SPARK-23927][SQL][FOLLOW-UP] Fix a build failure.
## What changes were proposed in this pull request?

This pr is a follow-up pr of #21155.
The #21155 removed unnecessary import at that time, but the import became necessary in another pr.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21646 from ueshin/issues/SPARK-23927/fup1.
2018-06-27 11:52:48 +08:00
Vayda, Oleksandr: IT (PRG) 2669b4de3b [SPARK-23927][SQL] Add "sequence" expression
## What changes were proposed in this pull request?
The PR adds the SQL function ```sequence```.
https://issues.apache.org/jira/browse/SPARK-23927

The behavior of the function is based on Presto's one.
Ref: https://prestodb.io/docs/current/functions/array.html

- ```sequence(start, stop) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```1``` if ```start``` is less than or equal to ```stop```, otherwise ```-1```.
- ```sequence(start, stop, step) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```step```.
- ```sequence(start_date, stop_date) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_date, stop_date, step_interval) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.
- ```sequence(start_timestemp, stop_timestemp) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_timestamp, stop_timestamp, step_interval) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.

## How was this patch tested?

Added unit tests.

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21155 from wajda/feature/array-api-sequence.
2018-06-27 11:52:31 +09:00
Maxim Gekk d08f53dc61 [SPARK-24605][SQL] size(null) returns null instead of -1
## What changes were proposed in this pull request?

In PR, I propose new behavior of `size(null)` under the config flag `spark.sql.legacy.sizeOfNull`. If the former one is disabled, the `size()` function returns `null` for `null` input. By default the `spark.sql.legacy.sizeOfNull` is enabled to keep backward compatibility with previous versions. In that case, `size(null)` returns `-1`.

## How was this patch tested?

Modified existing tests for the `size()` function to check new behavior (`null`) and old one (`-1`).

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21598 from MaxGekk/legacy-size-of-null.
2018-06-27 10:36:51 +08:00
Kris Mok 1b9368f7d4 [SPARK-24659][SQL] GenericArrayData.equals should respect element type differences
## What changes were proposed in this pull request?

Fix `GenericArrayData.equals`, so that it respects the actual types of the elements.
e.g. an instance that represents an `array<int>` and another instance that represents an `array<long>` should be considered incompatible, and thus should return false for `equals`.

`GenericArrayData` doesn't keep any schema information by itself, and rather relies on the Java objects referenced by its `array` field's elements to keep track of their own object types. So, the most straightforward way to respect their types is to call `equals` on the elements, instead of using Scala's `==` operator, which can have semantics that are not always desirable:
```
new java.lang.Integer(123) == new java.lang.Long(123L) // true in Scala
new java.lang.Integer(123).equals(new java.lang.Long(123L)) // false in Scala
```

## How was this patch tested?

Added unit test in `ComplexDataSuite`

Author: Kris Mok <kris.mok@databricks.com>

Closes #21643 from rednaxelafx/fix-genericarraydata-equals.
2018-06-27 10:27:40 +08:00
Yuming Wang dcaa49ff1e [SPARK-24658][SQL] Remove workaround for ANTLR bug
## What changes were proposed in this pull request?

Issue antlr/antlr4#781 has already been fixed, so the workaround of extracting the pattern into a separate rule is no longer needed. The presto already removed it: https://github.com/prestodb/presto/pull/10744.

## How was this patch tested?

Existing tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21641 from wangyum/ANTLR-780.
2018-06-26 14:33:04 -07:00
Marek Novotny e07aee2165 [SPARK-24636][SQL] Type coercion of arrays for array_join function
## What changes were proposed in this pull request?
Presto's implementation accepts arbitrary arrays of primitive types as an input:

```
presto> SELECT array_join(ARRAY [1, 2, 3], ', ');
_col0
---------
1, 2, 3
(1 row)
```

This PR proposes to implement a type coercion rule for ```array_join``` function that converts arrays of primitive as well as non-primitive types to arrays of string.

## How was this patch tested?

New test cases add into:
- sql-tests/inputs/typeCoercion/native/arrayJoin.sql
- DataFrameFunctionsSuite.scala

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21620 from mn-mikke/SPARK-24636.
2018-06-26 09:51:55 +08:00
Bryan Cutler d48803bf64 [SPARK-24324][PYTHON][FOLLOWUP] Grouped Map positional conf should have deprecation note
## What changes were proposed in this pull request?

Followup to the discussion of the added conf in SPARK-24324 which allows assignment by column position only.  This conf is to preserve old behavior and will be removed in future releases, so it should have a note to indicate that.

## How was this patch tested?

NA

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21637 from BryanCutler/arrow-groupedMap-conf-deprecate-followup-SPARK-24324.
2018-06-25 17:08:23 -07:00
Marco Gaido 594ac4f7b8 [SPARK-24633][SQL] Fix codegen when split is required for arrays_zip
## What changes were proposed in this pull request?

In function array_zip, when split is required by the high number of arguments, a codegen error can happen.

The PR fixes codegen for cases when splitting the code is required.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21621 from mgaido91/SPARK-24633.
2018-06-25 23:44:20 +08:00
Bryan Cutler a5849ad9a3 [SPARK-24324][PYTHON] Pandas Grouped Map UDF should assign result columns by name
## What changes were proposed in this pull request?

Currently, a `pandas_udf` of type `PandasUDFType.GROUPED_MAP` will assign the resulting columns based on index of the return pandas.DataFrame.  If a new DataFrame is returned and constructed using a dict, then the order of the columns could be arbitrary and be different than the defined schema for the UDF.  If the schema types still match, then no error will be raised and the user will see column names and column data mixed up.

This change will first try to assign columns using the return type field names.  If a KeyError occurs, then the column index is checked if it is string based. If so, then the error is raised as it is most likely a naming mistake, else it will fallback to assign columns by position and raise a TypeError if the field types do not match.

## How was this patch tested?

Added a test that returns a new DataFrame with column order different than the schema.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21427 from BryanCutler/arrow-grouped-map-mixesup-cols-SPARK-24324.
2018-06-24 09:28:46 +08:00
Maxim Gekk c7e2742f9b [SPARK-24190][SQL] Allow saving of JSON files in UTF-16 and UTF-32
## What changes were proposed in this pull request?

Currently, restrictions in JSONOptions for `encoding` and `lineSep` are the same for read and for write. For example, a requirement for `lineSep` in the code:

```
df.write.option("encoding", "UTF-32BE").json(file)
```
doesn't allow to skip `lineSep` and use its default value `\n` because it throws the exception:
```
equirement failed: The lineSep option must be specified for the UTF-32BE encoding
java.lang.IllegalArgumentException: requirement failed: The lineSep option must be specified for the UTF-32BE encoding
```

In the PR, I propose to separate JSONOptions in read and write, and make JSONOptions in write less restrictive.

## How was this patch tested?

Added new test for blacklisted encodings in read. And the `lineSep` option was removed in write for some tests.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21247 from MaxGekk/json-options-in-write.
2018-06-23 17:40:20 -07:00
Marek Novotny 92c2f00bd2 [SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.

## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`

## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
  Seq((1, 10), (2, 20), (3, 10)),
  Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */           final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */           final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */           if (project_byteArraySize_0 > 2147483632) {
/* 055 */             final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */             final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */             for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */               InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */               project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */               project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */             }
/* 064 */
/* 065 */             project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */           } else {
/* 068 */             final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */             UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */             Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */             Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */             Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */             project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */             ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */             ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */             for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */               InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */               project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */               project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */             }
/* 083 */
/* 084 */             project_value_0 = project_unsafeMapData_0;
/* 085 */           }
/* 086 */
/* 087 */         }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
  Seq(("a", null), ("b", "bb"), ("c", "aa")),
  Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */           final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */           for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */             InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */             if (project_entry_0.isNullAt(0)) {
/* 058 */               throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */             }
/* 060 */
/* 061 */             project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */             project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */           }
/* 064 */
/* 065 */           project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */         }
```

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 16:18:22 +09:00
Wenchen Fan dc8a6befa5 [SPARK-24588][SS] streaming join should require HashClusteredPartitioning from children
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19080 we simplified the distribution/partitioning framework, and make all the join-like operators require `HashClusteredDistribution` from children. Unfortunately streaming join operator was missed.

This can cause wrong result. Think about
```
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]

val df1 = input1.toDF.select('value as 'a, 'value * 2 as 'b)
val df2 = input2.toDF.select('value as 'a, 'value * 2 as 'b).repartition('b)
val joined = df1.join(df2, Seq("a", "b")).select('a)
```

The physical plan is
```
*(3) Project [a#5]
+- StreamingSymmetricHashJoin [a#5, b#6], [a#10, b#11], Inner, condition = [ leftOnly = null, rightOnly = null, both = null, full = null ], state info [ checkpoint = <unknown>, runId = 54e31fce-f055-4686-b75d-fcd2b076f8d8, opId = 0, ver = 0, numPartitions = 5], 0, state cleanup [ left = null, right = null ]
   :- Exchange hashpartitioning(a#5, b#6, 5)
   :  +- *(1) Project [value#1 AS a#5, (value#1 * 2) AS b#6]
   :     +- StreamingRelation MemoryStream[value#1], [value#1]
   +- Exchange hashpartitioning(b#11, 5)
      +- *(2) Project [value#3 AS a#10, (value#3 * 2) AS b#11]
         +- StreamingRelation MemoryStream[value#3], [value#3]
```

The left table is hash partitioned by `a, b`, while the right table is hash partitioned by `b`. This means, we may have a matching record that is in different partitions, which should be in the output but not.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21587 from cloud-fan/join.
2018-06-21 15:38:46 -07:00
Maxim Gekk 54fcaafb09 [SPARK-24571][SQL] Support Char literals
## What changes were proposed in this pull request?

In the PR, I propose to automatically convert a `Literal` with `Char` type to a `Literal` of `String` type. Currently, the following code:
```scala
val df = Seq("Amsterdam", "San Francisco", "London").toDF("city")
df.where($"city".contains('o')).show(false)
```
fails with the exception:
```
Unsupported literal type class java.lang.Character o
java.lang.RuntimeException: Unsupported literal type class java.lang.Character o
at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:78)
```
The PR fixes this issue by converting `char` to `string` of length `1`. I believe it makes sense to does not differentiate `char` and `string(1)` in _a unified, multi-language data platform_ like Spark which supports languages like Python/R.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21578 from MaxGekk/support-char-literals.
2018-06-20 23:38:37 -07:00
Huaxin Gao 9de11d3f90 [SPARK-23912][SQL] add array_distinct
## What changes were proposed in this pull request?

Add array_distinct to remove duplicate value from the array.

## How was this patch tested?

Add unit tests

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #21050 from huaxingao/spark-23912.
2018-06-21 12:24:53 +09:00
aokolnychyi c5a0d1132a [SPARK-24575][SQL] Prohibit window expressions inside WHERE and HAVING clauses
## What changes were proposed in this pull request?

As discussed [before](https://github.com/apache/spark/pull/19193#issuecomment-393726964), this PR prohibits window expressions inside WHERE and HAVING clauses.

## How was this patch tested?

This PR comes with a dedicated unit test.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #21580 from aokolnychyi/spark-24575.
2018-06-20 18:57:13 +02:00
Xiao Li 9a75c18290 [SPARK-24542][SQL] UDF series UDFXPathXXXX allow users to pass carefully crafted XML to access arbitrary files
## What changes were proposed in this pull request?

UDF series UDFXPathXXXX allow users to pass carefully crafted XML to access arbitrary files. Spark does not have built-in access control. When users use the external access control library, users might bypass them and access the file contents.

This PR basically patches the Hive fix to Apache Spark. https://issues.apache.org/jira/browse/HIVE-18879

## How was this patch tested?

A unit test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21549 from gatorsmile/xpathSecurity.
2018-06-18 20:17:04 -07:00
Liang-Chi Hsieh 8f225e055c [SPARK-24548][SQL] Fix incorrect schema of Dataset with tuple encoders
## What changes were proposed in this pull request?

When creating tuple expression encoders, we should give the serializer expressions of tuple items correct names, so we can have correct output schema when we use such tuple encoders.

## How was this patch tested?

Added test.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21576 from viirya/SPARK-24548.
2018-06-18 11:01:17 -07:00
Takeshi Yamamuro e219e692ef [SPARK-23772][SQL] Provide an option to ignore column of all null values or empty array during JSON schema inference
## What changes were proposed in this pull request?
This pr added a new JSON option `dropFieldIfAllNull ` to ignore column of all null values or empty array/struct during JSON schema inference.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Xiangrui Meng <meng@databricks.com>

Closes #20929 from maropu/SPARK-23772.
2018-06-19 00:24:54 +08:00
James Yu c7c0b086a0 add one supported type missing from the javadoc
## What changes were proposed in this pull request?

The supported java.math.BigInteger type is not mentioned in the javadoc of Encoders.bean()

## How was this patch tested?

only Javadoc fix

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

Author: James Yu <james@ispot.tv>

Closes #21544 from yuj/master.
2018-06-15 21:04:04 -07:00
Kazuaki Ishizaki 90da7dc241 [SPARK-24452][SQL][CORE] Avoid possible overflow in int add or multiple
## What changes were proposed in this pull request?

This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)

The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```

To avoid this problem, this PR performs cast from int to long in right hand side.

## How was this patch tested?

Existing UTs.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21481 from kiszk/SPARK-24452.
2018-06-15 13:47:48 -07:00
Maxim Gekk b8f27ae3b3 [SPARK-24543][SQL] Support any type as DDL string for from_json's schema
## What changes were proposed in this pull request?

In the PR, I propose to support any DataType represented as DDL string for the from_json function. After the changes, it will be possible to specify `MapType` in SQL like:
```sql
select from_json('{"a":1, "b":2}', 'map<string, int>')
```
and in Scala (similar in other languages)
```scala
val in = Seq("""{"a": {"b": 1}}""").toDS()
val schema = "map<string, map<string, int>>"
val out = in.select(from_json($"value", schema, Map.empty[String, String]))
```

## How was this patch tested?

Added a couple sql tests and modified existing tests for Python and Scala. The former tests were modified because it is not imported for them in which format schema for `from_json` is provided.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21550 from MaxGekk/from_json-ddl-schema.
2018-06-14 13:27:27 -07:00
Herman van Hovell 299d297e25 [SPARK-24500][SQL] Make sure streams are materialized during Tree transforms.
## What changes were proposed in this pull request?
If you construct catalyst trees using `scala.collection.immutable.Stream` you can run into situations where valid transformations do not seem to have any effect. There are two causes for this behavior:
- `Stream` is evaluated lazily. Note that default implementation will generally only evaluate a function for the first element (this makes testing a bit tricky).
- `TreeNode` and `QueryPlan` use side effects to detect if a tree has changed. Mapping over a stream is lazy and does not need to trigger this side effect. If this happens the node will invalidly assume that it did not change and return itself instead if the newly created node (this is for GC reasons).

This PR fixes this issue by forcing materialization on streams in `TreeNode` and `QueryPlan`.

## How was this patch tested?
Unit tests were added to `TreeNodeSuite` and `LogicalPlanSuite`. An integration test was added to the `PlannerSuite`

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #21539 from hvanhovell/SPARK-24500.
2018-06-13 07:09:48 -07:00
Arun Mahadevan 7703b46d28 [SPARK-24479][SS] Added config for registering streamingQueryListeners
## What changes were proposed in this pull request?

Currently a "StreamingQueryListener" can only be registered programatically. We could have a new config "spark.sql.streamingQueryListeners" similar to  "spark.sql.queryExecutionListeners" and "spark.extraListeners" for users to register custom streaming listeners.

## How was this patch tested?

New unit test and running example programs.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21504 from arunmahadevan/SPARK-24480.
2018-06-13 20:43:16 +08:00
Li Jin 9786ce66c5 [SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.

```
       >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
       >>> from pyspark.sql import Window
       >>> df = spark.createDataFrame(
       ...     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
       ...     ("id", "v"))
       >>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
       ... def mean_udf(v):
       ...     return v.mean()
       >>> w = Window.partitionBy('id')
       >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
       +---+----+------+
       | id|   v|mean_v|
       +---+----+------+
       |  1| 1.0|   1.5|
       |  1| 2.0|   1.5|
       |  2| 3.0|   6.0|
       |  2| 5.0|   6.0|
       |  2|10.0|   6.0|
       +---+----+------+
```

The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)

Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.

## How was this patch tested?

WindowPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-13 09:10:52 +08:00
Kazuaki Ishizaki ada28f2595 [SPARK-23933][SQL] Add map_from_arrays function
## What changes were proposed in this pull request?

The PR adds the SQL function `map_from_arrays`. The behavior of the function is based on Presto's `map`. Since SparkSQL already had a `map` function, we prepared the different name for this behavior.

This function returns returns a map from a pair of arrays for keys and values.

## How was this patch tested?

Added UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21258 from kiszk/SPARK-23933.
2018-06-12 12:31:22 -07:00
DylanGuedes f0ef1b311d [SPARK-23931][SQL] Adds arrays_zip function to sparksql
Signed-off-by: DylanGuedes <djmgguedesgmail.com>

## What changes were proposed in this pull request?

Addition of arrays_zip function to spark sql functions.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Unit tests that checks if the results are correct.

Author: DylanGuedes <djmgguedes@gmail.com>

Closes #21045 from DylanGuedes/SPARK-23931.
2018-06-12 11:57:25 -07:00
Marco Gaido f07c5064a3 [SPARK-24468][SQL] Handle negative scale when adjusting precision for decimal operations
## What changes were proposed in this pull request?

In SPARK-22036 we introduced the possibility to allow precision loss in arithmetic operations (according to the SQL standard). The implementation was drawn from Hive's one, where Decimals with a negative scale are not allowed in the operations.

The PR handles the case when the scale is negative, removing the assertion that it is not.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21499 from mgaido91/SPARK-24468.
2018-06-08 18:51:56 -07:00
Bruce Robbins 1462bba4fd [SPARK-24119][SQL] Add interpreted execution to SortPrefix expression
## What changes were proposed in this pull request?

Implemented eval in SortPrefix expression.

## How was this patch tested?

- ran existing sbt SQL tests
- added unit test
- ran existing Python SQL tests
- manual tests: disabling codegen -- patching code to disable beyond what spark.sql.codegen.wholeStage=false can do -- and running sbt SQL tests

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21231 from bersprockets/sortprefixeval.
2018-06-08 13:27:52 +02:00
aokolnychyi 7297ae04d8 [SPARK-21896][SQL] Fix StackOverflow caused by window functions inside aggregate functions
## What changes were proposed in this pull request?

This PR explicitly prohibits window functions inside aggregates. Currently, this will cause StackOverflow during analysis. See PR #19193 for previous discussion.

## How was this patch tested?

This PR comes with a dedicated unit test.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #21473 from aokolnychyi/fix-stackoverflow-window-funcs.
2018-06-04 13:28:16 -07:00
Yuming Wang 0be5aa2746 [SPARK-23903][SQL] Add support for date extract
## What changes were proposed in this pull request?

Add support for date `extract` function:
```sql
spark-sql> SELECT EXTRACT(YEAR FROM TIMESTAMP '2000-12-16 12:21:13');
2000
```
Supported field same as [Hive](https://github.com/apache/hive/blob/rel/release-2.3.3/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g#L308-L316): `YEAR`, `QUARTER`, `MONTH`, `WEEK`, `DAY`, `DAYOFWEEK`, `HOUR`, `MINUTE`, `SECOND`.

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21479 from wangyum/SPARK-23903.
2018-06-04 10:16:13 -07:00
Xiao Li d2c3de7efc Revert "[SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set"
This reverts commit 1e46f92f95.
2018-06-01 11:51:10 -07:00
Huaxin Gao 98909c398d [SPARK-23920][SQL] add array_remove to remove all elements that equal element from array
## What changes were proposed in this pull request?

add array_remove to remove all elements that equal element from array

## How was this patch tested?

add unit tests

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #21069 from huaxingao/spark-23920.
2018-05-31 22:04:26 -07:00
Yuming Wang cc976f6cb8 [SPARK-23900][SQL] format_number support user specifed format as argument
## What changes were proposed in this pull request?

`format_number` support user specifed format as argument. For example:
```sql
spark-sql> SELECT format_number(12332.123456, '##################.###');
12332.123
```

## How was this patch tested?

unit test

Author: Yuming Wang <yumwang@ebay.com>

Closes #21010 from wangyum/SPARK-23900.
2018-05-31 11:38:23 -07:00
Marco Gaido 24ef7fbfa9 [SPARK-24276][SQL] Order of literals in IN should not affect semantic equality
## What changes were proposed in this pull request?

When two `In` operators are created with the same list of values, but different order, we are considering them as semantically different. This is wrong, since they have the same semantic meaning.

The PR adds a canonicalization rule which orders the literals in the `In` operator so the semantic equality works properly.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21331 from mgaido91/SPARK-24276.
2018-05-30 15:31:40 -07:00
Marco Gaido 1b36f14889 [SPARK-23901][SQL] Add masking functions
## What changes were proposed in this pull request?

The PR adds the masking function as they are described in Hive's documentation: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-DataMaskingFunctions.
This means that only `string`s are accepted as parameter for the masking functions.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21246 from mgaido91/SPARK-23901.
2018-05-30 11:18:04 -07:00
Takeshi Yamamuro 1e46f92f95 [SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set
## What changes were proposed in this pull request?
This pr fixed an issue when having multiple distinct aggregations having the same argument set, e.g.,
```
scala>: paste
val df = sql(
  s"""SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*)
     | FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y)
   """.stripMargin)

java.lang.RuntimeException
You hit a query analyzer bug. Please report your query to Spark user mailing list.
```
The root cause is that `RewriteDistinctAggregates` can't detect multiple distinct aggregations if they have the same argument set. This pr modified code so that `RewriteDistinctAggregates` could count the number of aggregate expressions with `isDistinct=true`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21443 from maropu/SPARK-24369.
2018-05-31 00:23:25 +08:00
DB Tsai 900bc1f7dc
[SPARK-24371][SQL] Added isInCollection in DataFrame API for Scala and Java.
## What changes were proposed in this pull request?

Implemented **`isInCollection `** in DataFrame API for both Scala and Java, so users can do

```scala
val profileDF = Seq(
  Some(1), Some(2), Some(3), Some(4),
  Some(5), Some(6), Some(7), None
).toDF("profileID")

val validUsers: Seq[Any] = Seq(6, 7.toShort, 8L, "3")

val result = profileDF.withColumn("isValid", $"profileID". isInCollection(validUsers))

result.show(10)
"""
+---------+-------+
|profileID|isValid|
+---------+-------+
|        1|  false|
|        2|  false|
|        3|   true|
|        4|  false|
|        5|  false|
|        6|   true|
|        7|   true|
|     null|   null|
+---------+-------+
 """.stripMargin
```
## How was this patch tested?

Several unit tests are added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21416 from dbtsai/optimize-set.
2018-05-29 10:22:18 -07:00
Miles Yucht d440699192 [SPARK-24381][TESTING] Add unit tests for NOT IN subquery around null values
## What changes were proposed in this pull request?
This PR adds several unit tests along the `cols NOT IN (subquery)` pathway. There are a scattering of tests here and there which cover this codepath, but there doesn't seem to be a unified unit test of the correctness of null-aware anti joins anywhere. I have also added a brief explanation of how this expression behaves in SubquerySuite. Lastly, I made some clarifying changes in the NOT IN pathway in RewritePredicateSubquery.

## How was this patch tested?
Added unit tests! There should be no behavioral change in this PR.

Author: Miles Yucht <miles@databricks.com>

Closes #21425 from mgyucht/spark-24381.
2018-05-26 20:42:23 -07:00
Maxim Gekk 1b1528a504 [SPARK-24366][SQL] Improving of error messages for type converting
## What changes were proposed in this pull request?

Currently, users are getting the following error messages on type conversions:

```
scala.MatchError: test (of class java.lang.String)
```

The message doesn't give any clues to the users where in the schema the error happened. In this PR, I would like to improve the error message like:

```
The value (test) of the type (java.lang.String) cannot be converted to struct<f1:int>
```

## How was this patch tested?

Added tests for converting of wrong values to `struct`, `map`, `array`, `string` and `decimal`.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21410 from MaxGekk/type-conv-error.
2018-05-25 15:42:46 -07:00
Maxim Gekk 64fad0b519 [SPARK-24244][SPARK-24368][SQL] Passing only required columns to the CSV parser
## What changes were proposed in this pull request?

uniVocity parser allows to specify only required column names or indexes for [parsing](https://www.univocity.com/pages/parsers-tutorial) like:

```
// Here we select only the columns by their indexes.
// The parser just skips the values in other columns
parserSettings.selectIndexes(4, 0, 1);
CsvParser parser = new CsvParser(parserSettings);
```
In this PR, I propose to extract indexes from required schema and pass them into the CSV parser. Benchmarks show the following improvements in parsing of 1000 columns:

```
Select 100 columns out of 1000: x1.76
Select 1 column out of 1000: x2
```

**Note**: Comparing to current implementation, the changes can return different result for malformed rows in the `DROPMALFORMED` and `FAILFAST` modes if only subset of all columns is requested. To have previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`.

## How was this patch tested?

It was tested by new test which selects 3 columns out of 15, by existing tests and by new benchmarks.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21415 from MaxGekk/csv-column-pruning2.
2018-05-24 21:38:04 -07:00
Gengliang Wang 3b20b34ab7 [SPARK-24367][SQL] Parquet: use JOB_SUMMARY_LEVEL instead of deprecated flag ENABLE_JOB_SUMMARY
## What changes were proposed in this pull request?

In current parquet version,the conf ENABLE_JOB_SUMMARY is deprecated.

When writing to Parquet files, the warning message
```WARN org.apache.parquet.hadoop.ParquetOutputFormat: Setting parquet.enable.summary-metadata is deprecated, please use parquet.summary.metadata.level```
keeps showing up.

From https://github.com/apache/parquet-mr/blame/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java#L164 we can see that we should use JOB_SUMMARY_LEVEL.

## How was this patch tested?

Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21411 from gengliangwang/summaryLevel.
2018-05-25 11:16:35 +08:00
Yuming Wang 0d89943449 [SPARK-24378][SQL] Fix date_trunc function incorrect examples
## What changes were proposed in this pull request?

Fix `date_trunc` function incorrect examples.

## How was this patch tested?

N/A

Author: Yuming Wang <yumwang@ebay.com>

Closes #21423 from wangyum/SPARK-24378.
2018-05-24 23:38:50 +08:00
Vayda, Oleksandr: IT (PRG) 230f144197 [SPARK-24350][SQL] Fixes ClassCastException in the "array_position" function
## What changes were proposed in this pull request?

### Fixes `ClassCastException` in the `array_position` function - [SPARK-24350](https://issues.apache.org/jira/browse/SPARK-24350)
When calling `array_position` function with a wrong type of the 1st argument an `AnalysisException` should be thrown instead of `ClassCastException`

Example:

```sql
select array_position('foo', 'bar')
```

```
java.lang.ClassCastException: org.apache.spark.sql.types.StringType$ cannot be cast to org.apache.spark.sql.types.ArrayType
	at org.apache.spark.sql.catalyst.expressions.ArrayPosition.inputTypes(collectionOperations.scala:1398)
	at org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes$class.checkInputDataTypes(ExpectsInputTypes.scala:44)
	at org.apache.spark.sql.catalyst.expressions.ArrayPosition.checkInputDataTypes(collectionOperations.scala:1401)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:168)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:168)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:256)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:252)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:288)
```

## How was this patch tested?

unit test

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21401 from wajda/SPARK-24350-array_position-error-fix.
2018-05-23 17:22:52 -07:00
Xiao Li 5a5a868dc4 Revert "[SPARK-24244][SQL] Passing only required columns to the CSV parser"
This reverts commit 8086acc2f6.
2018-05-23 11:51:13 -07:00
Liang-Chi Hsieh a40ffc656d [SPARK-23711][SQL] Add fallback generator for UnsafeProjection
## What changes were proposed in this pull request?

Add fallback logic for `UnsafeProjection`. In production we can try to create unsafe projection using codegen implementation. Once any compile error happens, it fallbacks to interpreted implementation.

## How was this patch tested?

Added test.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21106 from viirya/SPARK-23711.
2018-05-23 22:40:52 +08:00
Vayda, Oleksandr: IT (PRG) bc6ea614ad [SPARK-24348][SQL] "element_at" error fix
## What changes were proposed in this pull request?

### Fixes a `scala.MatchError` in the `element_at` operation - [SPARK-24348](https://issues.apache.org/jira/browse/SPARK-24348)

When calling `element_at` with a wrong first operand type an `AnalysisException` should be thrown instead of `scala.MatchError`

*Example:*
```sql
select element_at('foo', 1)
```

results in:
```
scala.MatchError: StringType (of class org.apache.spark.sql.types.StringType$)
	at org.apache.spark.sql.catalyst.expressions.ElementAt.inputTypes(collectionOperations.scala:1469)
	at org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes$class.checkInputDataTypes(ExpectsInputTypes.scala:44)
	at org.apache.spark.sql.catalyst.expressions.ElementAt.checkInputDataTypes(collectionOperations.scala:1478)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:168)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:168)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:256)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:252)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:288)
```

## How was this patch tested?

unit tests

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21395 from wajda/SPARK-24348-element_at-error-fix.
2018-05-22 13:01:07 -07:00
Liang-Chi Hsieh f9f055afa4 [SPARK-24121][SQL] Add API for handling expression code generation
## What changes were proposed in this pull request?

This patch tries to implement this [proposal](https://github.com/apache/spark/pull/19813#issuecomment-354045400) to add an API for handling expression code generation. It should allow us to manipulate how to generate codes for expressions.

In details, this adds an new abstraction `CodeBlock` to `JavaCode`. `CodeBlock` holds the code snippet and inputs for generating actual java code.

For example, in following java code:

```java
  int ${variable} = 1;
  boolean ${isNull} = ${CodeGenerator.defaultValue(BooleanType)};
```

`variable`, `isNull` are two `VariableValue` and `CodeGenerator.defaultValue(BooleanType)` is a string. They are all inputs to this code block and held by `CodeBlock` representing this code.

For codegen, we provide a specified string interpolator `code`, so you can define a code like this:
```scala
  val codeBlock =
    code"""
         |int ${variable} = 1;
         |boolean ${isNull} = ${CodeGenerator.defaultValue(BooleanType)};
        """.stripMargin
  // Generates actual java code.
  codeBlock.toString
```

Because those inputs are held separately in `CodeBlock` before generating code, we can safely manipulate them, e.g., replacing statements to aliased variables, etc..

## How was this patch tested?

Added tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21193 from viirya/SPARK-24121.
2018-05-23 01:50:22 +08:00
Maxim Gekk 8086acc2f6 [SPARK-24244][SQL] Passing only required columns to the CSV parser
## What changes were proposed in this pull request?

uniVocity parser allows to specify only required column names or indexes for [parsing](https://www.univocity.com/pages/parsers-tutorial) like:

```
// Here we select only the columns by their indexes.
// The parser just skips the values in other columns
parserSettings.selectIndexes(4, 0, 1);
CsvParser parser = new CsvParser(parserSettings);
```
In this PR, I propose to extract indexes from required schema and pass them into the CSV parser. Benchmarks show the following improvements in parsing of 1000 columns:

```
Select 100 columns out of 1000: x1.76
Select 1 column out of 1000: x2
```

**Note**: Comparing to current implementation, the changes can return different result for malformed rows in the `DROPMALFORMED` and `FAILFAST` modes if only subset of all columns is requested. To have previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`.

## How was this patch tested?

It was tested by new test which selects 3 columns out of 15, by existing tests and by new benchmarks.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21296 from MaxGekk/csv-column-pruning.
2018-05-22 22:07:32 +08:00
Marco Gaido d3d1807315 [SPARK-24313][SQL] Fix collection operations' interpreted evaluation for complex types
## What changes were proposed in this pull request?

The interpreted evaluation of several collection operations works only for simple datatypes. For complex data types, for instance, `array_contains` it returns always `false`. The list of the affected functions is `array_contains`, `array_position`, `element_at` and `GetMapValue`.

The PR fixes the behavior for all the datatypes.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21361 from mgaido91/SPARK-24313.
2018-05-22 21:08:49 +08:00
Kris Mok 952e4d1c83 [SPARK-24321][SQL] Extract common code from Divide/Remainder to a base trait
## What changes were proposed in this pull request?

Extract common code from `Divide`/`Remainder` to a new base trait, `DivModLike`.

Further refactoring to make `Pmod` work with `DivModLike` is to be done as a separate task.

## How was this patch tested?

Existing tests in `ArithmeticExpressionSuite` covers the functionality.

Author: Kris Mok <kris.mok@databricks.com>

Closes #21367 from rednaxelafx/catalyst-divmod.
2018-05-22 19:12:30 +08:00
Wenchen Fan 03e90f65bf [SPARK-24250][SQL] support accessing SQLConf inside tasks
re-submit https://github.com/apache/spark/pull/21299 which broke build.

A few new commits are added to fix the SQLConf problem in `JsonSchemaInference.infer`, and prevent us to access `SQLConf` in DAGScheduler event loop thread.

## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21376 from cloud-fan/config.
2018-05-22 00:19:18 +08:00
Marek Novotny a6e883feb3 [SPARK-23935][SQL] Adding map_entries function
## What changes were proposed in this pull request?

This PR adds `map_entries` function that returns an unordered array of all entries in the given map.

## How was this patch tested?

New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionsSuite`

## CodeGen examples
### Primitive types
```
val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */
/* 044 */         ArrayData project_value_0 = null;
/* 045 */
/* 046 */         final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */         final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */         final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */         final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 051 */           project_numElements_0,
/* 052 */           32);
/* 053 */         if (project_size_0 > 2147483632) {
/* 054 */           final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 055 */           for (int z = 0; z < project_numElements_0; z++) {
/* 056 */             project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)});
/* 057 */           }
/* 058 */           project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
/* 059 */
/* 060 */         } else {
/* 061 */           final byte[] project_arrayBytes_0 = new byte[(int)project_size_0];
/* 062 */           UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData();
/* 063 */           Platform.putLong(project_arrayBytes_0, 16, project_numElements_0);
/* 064 */           project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0);
/* 065 */
/* 066 */           final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8;
/* 067 */           UnsafeRow project_unsafeRow_0 = new UnsafeRow(2);
/* 068 */           for (int z = 0; z < project_numElements_0; z++) {
/* 069 */             long offset = project_structsOffset_0 + z * 24L;
/* 070 */             project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L);
/* 071 */             project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24);
/* 072 */             project_unsafeRow_0.setInt(0, project_keys_0.getInt(z));
/* 073 */             project_unsafeRow_0.setInt(1, project_values_0.getInt(z));
/* 074 */           }
/* 075 */           project_value_0 = project_unsafeArrayData_0;
/* 076 */
/* 077 */         }
```
### Non-primitive types
```
val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */
/* 044 */         ArrayData project_value_0 = null;
/* 045 */
/* 046 */         final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */         final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */         final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */         final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 051 */         for (int z = 0; z < project_numElements_0; z++) {
/* 052 */           project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)});
/* 053 */         }
/* 054 */         project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
```

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 23:14:03 +09:00
Wenchen Fan 000e25ae79 Revert "[SPARK-24250][SQL] support accessing SQLConf inside tasks"
This reverts commit dd37529a8d.
2018-05-20 16:13:42 +08:00
Wenchen Fan dd37529a8d [SPARK-24250][SQL] support accessing SQLConf inside tasks
## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21299 from cloud-fan/config.
2018-05-19 18:51:02 +08:00
Marcelo Vanzin ed7ba7db8f [SPARK-23850][SQL] Add separate config for SQL options redaction.
The old code was relying on a core configuration and extended its
default value to include things that redact desired things in the
app's environment. Instead, add a SQL-specific option for which
options to redact, and apply both the core and SQL-specific rules
when redacting the options in the save command.

This is a little sub-optimal since it adds another config, but it
retains the current default behavior.

While there I also fixed a typo and a couple of minor config API
usage issues in the related redaction option that SQL already had.

Tested with existing unit tests, plus checking the env page on
a shell UI.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21158 from vanzin/SPARK-23850.
2018-05-18 11:14:22 -07:00
jinxing 8a837bf4f3 [SPARK-24193] create TakeOrderedAndProjectExec only when the limit number is below spark.sql.execution.topKSortFallbackThreshold.
## What changes were proposed in this pull request?

Physical plan of `select colA from t order by colB limit M` is `TakeOrderedAndProject`;
Currently `TakeOrderedAndProject` sorts data in memory, see https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala#L158
We can add a config – if the number of limit (M) is too big, we can sort by disk. Thus memory issue can be resolved.

## How was this patch tested?

Test added

Author: jinxing <jinxing6042@126.com>

Closes #21252 from jinxing64/SPARK-24193.
2018-05-17 22:29:18 +08:00
Marco Gaido 69350aa2f0 [SPARK-23922][SQL] Add arrays_overlap function
## What changes were proposed in this pull request?

The PR adds the function `arrays_overlap`. This function returns `true` if the input arrays contain a non-null common element; if not, it returns `null` if any of the arrays contains a `null` element, `false` otherwise.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21028 from mgaido91/SPARK-23922.
2018-05-17 20:45:32 +08:00
Florent Pépin 3e66350c24 [SPARK-23925][SQL] Add array_repeat collection function
## What changes were proposed in this pull request?

The PR adds a new collection function, array_repeat. As there already was a function repeat with the same signature, with the only difference being the expected return type (String instead of Array), the new function is called array_repeat to distinguish.
The behaviour of the function is based on Presto's one.

The function creates an array containing a given element repeated the requested number of times.

## How was this patch tested?

New unit tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

Author: Florent Pépin <florentpepin.92@gmail.com>
Author: Florent Pépin <florent.pepin14@imperial.ac.uk>

Closes #21208 from pepinoflo/SPARK-23925.
2018-05-17 13:31:14 +09:00
Wenchen Fan 943493b165 Revert "[SPARK-22938][SQL][FOLLOWUP] Assert that SQLConf.get is acces…
…sed only on the driver"

This reverts commit a4206d58e0.

This is from https://github.com/apache/spark/pull/21299 and to ease the review of it.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21341 from cloud-fan/revert.
2018-05-16 22:01:24 +08:00
Jose Torres 3fabbc5762 [SPARK-24040][SS] Support single partition aggregates in continuous processing.
## What changes were proposed in this pull request?

Support aggregates with exactly 1 partition in continuous processing.

A few small tweaks are needed to make this work:

* Replace currentEpoch tracking with an ThreadLocal. This means that current epoch is scoped to a task rather than a node, but I think that's sustainable even once we add shuffle.
* Add a new testing-only flag to disable the UnsupportedOperationChecker whitelist of allowed continuous processing nodes. I think this is preferable to writing a pile of custom logic to enforce that there is in fact only 1 partition; we plan to support multi-partition aggregates before the next Spark release, so we'd just have to tear that logic back out.
* Restart continuous processing queries from the first available uncommitted epoch, rather than one that's guaranteed to be unused. This is required for stateful operators to overwrite partial state from the previous attempt at the epoch, and there was no specific motivation for the original strategy. In another PR before stabilizing the StreamWriter API, we'll need to narrow down and document more precise semantic guarantees for the epoch IDs.
* We need a single-partition ContinuousMemoryStream. The way MemoryStream is constructed means it can't be a text option like it is for rate source, unfortunately.

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21239 from jose-torres/withAggr.
2018-05-15 10:25:29 -07:00
maryannxue 80c6d35a3e [SPARK-24035][SQL] SQL syntax for Pivot - fix antlr warning
## What changes were proposed in this pull request?

1. Change antlr rule to fix the warning.
2. Add PIVOT/LATERAL check in AstBuilder with a more meaningful error message.

## How was this patch tested?

1. Add a counter case in `PlanParserSuite.test("lateral view")`

Author: maryannxue <maryann.xue@gmail.com>

Closes #21324 from maryannxue/spark-24035-fix.
2018-05-14 23:34:42 -07:00
Maxim Gekk 8cd83acf40 [SPARK-24027][SQL] Support MapType with StringType for keys as the root type by from_json
## What changes were proposed in this pull request?

Currently, the from_json function support StructType or ArrayType as the root type. The PR allows to specify MapType(StringType, DataType) as the root type additionally to mentioned types. For example:

```scala
import org.apache.spark.sql.types._
val schema = MapType(StringType, IntegerType)
val in = Seq("""{"a": 1, "b": 2, "c": 3}""").toDS()
in.select(from_json($"value", schema, Map[String, String]())).collect()
```
```
res1: Array[org.apache.spark.sql.Row] = Array([Map(a -> 1, b -> 2, c -> 3)])
```

## How was this patch tested?

It was checked by new tests for the map type with integer type and struct type as value types. Also roundtrip tests like from_json(to_json) and to_json(from_json) for MapType are added.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21108 from MaxGekk/from_json-map-type.
2018-05-14 14:05:42 -07:00
Shixiong Zhu c26f673252 [SPARK-24246][SQL] Improve AnalysisException by setting the cause when it's available
## What changes were proposed in this pull request?

If there is an exception, it's better to set it as the cause of AnalysisException since the exception may contain useful debug information.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #21297 from zsxwing/SPARK-24246.
2018-05-14 11:37:57 -07:00
Cody Allen 32acfa78c6 Improve implicitNotFound message for Encoder
The `implicitNotFound` message for `Encoder` doesn't mention the name of
the type for which it can't find an encoder. Furthermore, it covers up
the fact that `Encoder` is the name of the relevant type class.
Hopefully this new message provides a little more specific type detail
while still giving the general message about which types are supported.

## What changes were proposed in this pull request?

Augment the existing message to mention that it's looking for an `Encoder` and what the type of the encoder is.

For example instead of:

```
Unable to find encoder for type stored in a Dataset.  Primitive types (Int, String, etc) and Product types (case classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
```

return this message:

```
Unable to find encoder for type Exception. An implicit Encoder[Exception] is needed to store Exception instances in a Dataset. Primitive types (Int, String, etc) and Product types (ca
se classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
```

## How was this patch tested?

It was tested manually in the Scala REPL, since triggering this in a test would cause a compilation error.

```
scala> implicitly[Encoder[Exception]]
<console>:51: error: Unable to find encoder for type Exception. An implicit Encoder[Exception] is needed to store Exception instances in a Dataset. Primitive types (Int, String, etc) and Product types (ca
se classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
       implicitly[Encoder[Exception]]
                 ^
```

Author: Cody Allen <ceedubs@gmail.com>

Closes #20869 from ceedubs/encoder-implicit-msg.
2018-05-12 14:35:40 -05:00
aditkumar 92f6f52ff0 [MINOR][DOCS] Documenting months_between direction
## What changes were proposed in this pull request?

It's useful to know what relationship between date1 and date2 results in a positive number.

Author: aditkumar <aditkumar@gmail.com>
Author: Adit Kumar <aditkumar@gmail.com>

Closes #20787 from aditkumar/master.
2018-05-11 14:42:23 -05:00
Wenchen Fan a4206d58e0 [SPARK-22938][SQL][FOLLOWUP] Assert that SQLConf.get is accessed only on the driver
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/20136 . #20136 didn't really work because in the test, we are using local backend, which shares the driver side `SparkEnv`, so `SparkEnv.get.executorId == SparkContext.DRIVER_IDENTIFIER` doesn't work.

This PR changes the check to `TaskContext.get != null`, and move the check to `SQLConf.get`, and fix all the places that violate this check:
* `InMemoryTableScanExec#createAndDecompressColumn` is executed inside `rdd.map`, we can't access `conf.offHeapColumnVectorEnabled` there. https://github.com/apache/spark/pull/21223 merged
* `DataType#sameType` may be executed in executor side, for things like json schema inference, so we can't call `conf.caseSensitiveAnalysis` there. This contributes to most of the code changes, as we need to add `caseSensitive` parameter to a lot of methods.
* `ParquetFilters` is used in the file scan function, which is executed in executor side, so we can't can't call `conf.parquetFilterPushDownDate` there. https://github.com/apache/spark/pull/21224 merged
* `WindowExec#createBoundOrdering` is called on executor side, so we can't use `conf.sessionLocalTimezone` there. https://github.com/apache/spark/pull/21225 merged
* `JsonToStructs` can be serialized to executors and evaluate, we should not call `SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)` in the body. https://github.com/apache/spark/pull/21226 merged

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21190 from cloud-fan/minor.
2018-05-11 09:01:40 +08:00
Maxim Gekk f4fed05121 [SPARK-24171] Adding a note for non-deterministic functions
## What changes were proposed in this pull request?

I propose to add a clear statement for functions like `collect_list()` about non-deterministic behavior of such functions. The behavior must be taken into account by user while creating and running queries.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21228 from MaxGekk/deterministic-comments.
2018-05-10 09:44:49 -07:00
Marco Gaido 94d6714482 [SPARK-23907][SQL] Add regr_* functions
## What changes were proposed in this pull request?

The PR introduces regr_slope, regr_intercept, regr_r2, regr_sxx, regr_syy, regr_sxy, regr_avgx, regr_avgy, regr_count.

The implementation of this functions mirrors Hive's one in HIVE-15978.

## How was this patch tested?

added UT (values compared with Hive)

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21054 from mgaido91/SPARK-23907.
2018-05-10 20:38:52 +09:00
Ryan Blue cac9b1dea1 [SPARK-23972][BUILD][SQL] Update Parquet to 1.10.0.
## What changes were proposed in this pull request?

This updates Parquet to 1.10.0 and updates the vectorized path for buffer management changes. Parquet 1.10.0 uses ByteBufferInputStream instead of byte arrays in encoders. This allows Parquet to break allocations into smaller chunks that are better for garbage collection.

## How was this patch tested?

Existing Parquet tests. Running in production at Netflix for about 3 months.

Author: Ryan Blue <blue@apache.org>

Closes #21070 from rdblue/SPARK-23972-update-parquet-to-1.10.0.
2018-05-09 12:27:32 +08:00
Maxim Gekk e3de6ab30d [SPARK-24068] Propagating DataFrameReader's options to Text datasource on schema inferring
## What changes were proposed in this pull request?

While reading CSV or JSON files, DataFrameReader's options are converted to Hadoop's parameters, for example there:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L302

but the options are not propagated to Text datasource on schema inferring, for instance:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala#L184-L188

The PR proposes propagation of user's options to Text datasource on scheme inferring in similar way as user's options are converted to Hadoop parameters if schema is specified.

## How was this patch tested?
The changes were tested manually by using https://github.com/twitter/hadoop-lzo:

```
hadoop-lzo> mvn clean package
hadoop-lzo> ln -s ./target/hadoop-lzo-0.4.21-SNAPSHOT.jar ./hadoop-lzo.jar
```
Create 2 test files in JSON and CSV format and compress them:
```shell
$ cat test.csv
col1|col2
a|1
$ lzop test.csv
$ cat test.json
{"col1":"a","col2":1}
$ lzop test.json
```
Run `spark-shell` with hadoop-lzo:
```
bin/spark-shell --jars ~/hadoop-lzo/hadoop-lzo.jar
```
reading compressed CSV and JSON without schema:
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("inferSchema",true).option("header",true).option("sep","|").csv("test.csv.lzo").show()
+----+----+
|col1|col2|
+----+----+
|   a|   1|
+----+----+
```
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("multiLine", true).json("test.json.lzo").printSchema
root
 |-- col1: string (nullable = true)
 |-- col2: long (nullable = true)
```

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21182 from MaxGekk/text-options.
2018-05-09 08:32:20 +08:00
Yuming Wang 487faf17ab [SPARK-24117][SQL] Unified the getSizePerRow
## What changes were proposed in this pull request?

This pr unified the `getSizePerRow` because `getSizePerRow` is used in many places. For example:

1. [LocalRelation.scala#L80](f70f46d1e5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala (L80))
2. [SizeInBytesOnlyStatsPlanVisitor.scala#L36](76b8b840dd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala (L36))

## How was this patch tested?
Exist tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21189 from wangyum/SPARK-24117.
2018-05-08 23:43:02 +08:00
gatorsmile 2f6fe7d679 [SPARK-23094][SPARK-23723][SPARK-23724][SQL][FOLLOW-UP] Support custom encoding for json files
## What changes were proposed in this pull request?
This is to add a test case to check the behaviors when users write json in the specified UTF-16/UTF-32 encoding with multiline off.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21254 from gatorsmile/followupSPARK-23094.
2018-05-08 21:24:35 +08:00
Henry Robinson cd12c5c3ec [SPARK-24128][SQL] Mention configuration option in implicit CROSS JOIN error
## What changes were proposed in this pull request?

Mention `spark.sql.crossJoin.enabled` in error message when an implicit `CROSS JOIN` is detected.

## How was this patch tested?

`CartesianProductSuite` and `JoinSuite`.

Author: Henry Robinson <henry@apache.org>

Closes #21201 from henryr/spark-24128.
2018-05-08 12:21:33 +08:00
Bruce Robbins d83e963724 [SPARK-24043][SQL] Interpreted Predicate should initialize nondeterministic expressions
## What changes were proposed in this pull request?

When creating an InterpretedPredicate instance, initialize any Nondeterministic expressions in the expression tree to avoid java.lang.IllegalArgumentException on later call to eval().

## How was this patch tested?

- sbt SQL tests
- python SQL tests
- new unit test

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21144 from bersprockets/interpretedpredicate.
2018-05-07 17:54:39 +02:00
Herman van Hovell 4e861db5f1 [SPARK-16406][SQL] Improve performance of LogicalPlan.resolve
## What changes were proposed in this pull request?

`LogicalPlan.resolve(...)` uses linear searches to find an attribute matching a name. This is fine in normal cases, but gets problematic when you try to resolve a large number of columns on a plan with a large number of attributes.

This PR adds an indexing structure to `resolve(...)` in order to find potential matches quicker. This PR improves the reference resolution time for the following code by 4x (11.8s -> 2.4s):

``` scala
val n = 4000
val values = (1 to n).map(_.toString).mkString(", ")
val columns = (1 to n).map("column" + _).mkString(", ")
val query =
  s"""
     |SELECT $columns
     |FROM VALUES ($values) T($columns)
     |WHERE 1=2 AND 1 IN ($columns)
     |GROUP BY $columns
     |ORDER BY $columns
     |""".stripMargin

spark.time(sql(query))
```
## How was this patch tested?

Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14083 from hvanhovell/SPARK-16406.
2018-05-07 11:21:22 +02:00
Marco Gaido e35ad3cadd [SPARK-23930][SQL] Add slice function
## What changes were proposed in this pull request?

The PR add the `slice` function. The behavior of the function is based on Presto's one.

The function slices an array according to the requested start index and length.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21040 from mgaido91/SPARK-23930.
2018-05-07 16:57:37 +09:00
Kazuaki Ishizaki 7564a9a706 [SPARK-23921][SQL] Add array_sort function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_sort`. The behavior of the function is based on Presto's one.

The function sorts the input array in ascending order. The elements of the input array must be orderable. Null elements will be placed at the end of the returned array.

## How was this patch tested?

Added UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21021 from kiszk/SPARK-23921.
2018-05-07 15:22:23 +09:00
gatorsmile f38ea00e83 [SPARK-24017][SQL] Refactor ExternalCatalog to be an interface
## What changes were proposed in this pull request?
This refactors the external catalog to be an interface. It can be easier for the future work in the catalog federation. After the refactoring, `ExternalCatalog` is much cleaner without mixing the listener event generation logic.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21122 from gatorsmile/refactorExternalCatalog.
2018-05-06 20:41:32 -07:00
Tathagata Das 47b5b68528 [SPARK-24157][SS] Enabled no-data batches in MicroBatchExecution for streaming aggregation and deduplication.
## What changes were proposed in this pull request?

This PR enables the MicroBatchExecution to run no-data batches if some SparkPlan requires running another batch to output results based on updated watermark / processing time. In this PR, I have enabled streaming aggregations and streaming deduplicates to automatically run addition batch even if new data is available. See https://issues.apache.org/jira/browse/SPARK-24156 for more context.

Major changes/refactoring done in this PR.
- Refactoring MicroBatchExecution - A major point of confusion in MicroBatchExecution control flow was always (at least to me) was that `populateStartOffsets` internally called `constructNextBatch` which was not obvious from just the name "populateStartOffsets" and made the control flow from the main trigger execution loop very confusing (main loop in `runActivatedStream` called `constructNextBatch` but only if `populateStartOffsets` hadn't already called it). Instead, the refactoring makes it cleaner.
    - `populateStartOffsets` only the updates `availableOffsets` and `committedOffsets`. Does not call `constructNextBatch`.
    - Main loop in `runActivatedStream` calls `constructNextBatch` which returns true or false reflecting whether the next batch is ready for executing. This method is now idempotent; if a batch has already been constructed, then it will always return true until the batch has been executed.
    - If next batch is ready then we call `runBatch` or sleep.
    - That's it.

- Refactoring watermark management logic - This has been refactored out from `MicroBatchExecution` in a separate class to simplify `MicroBatchExecution`.

- New method `shouldRunAnotherBatch` in `IncrementalExecution` - This returns true if there is any stateful operation in the last execution plan that requires another batch for state cleanup, etc. This is used to decide whether to construct a batch or not in `constructNextBatch`.

- Changes to stream testing framework - Many tests used CheckLastBatch to validate answers. This assumed that there will be no more batches after the last set of input has been processed, so the last batch is the one that has output corresponding to the last input. This is not true anymore. To account for that, I made two changes.
    - `CheckNewAnswer` is a new test action that verifies the new rows generated since the last time the answer was checked by `CheckAnswer`, `CheckNewAnswer` or `CheckLastBatch`. This is agnostic to how many batches occurred between the last check and now. To do make this easier, I added a common trait between MemorySink and MemorySinkV2 to abstract out some common methods.
    - `assertNumStateRows` has been updated in the same way to be agnostic to batches while checking what the total rows and how many state rows were updated (sums up updates since the last check).

## How was this patch tested?
- Changes made to existing tests - Tests have been changed in one of the following patterns.
    - Tests where the last input was given again to force another batch to be executed and state cleaned up / output generated, they were simplified by removing the extra input.
    - Tests using aggregation+watermark where CheckLastBatch were replaced with CheckNewAnswer to make them batch agnostic.
- New tests added to check whether the flag works for streaming aggregation and deduplication

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #21220 from tdas/SPARK-24157.
2018-05-04 16:35:24 -07:00
maryannxue e3201e165e [SPARK-24035][SQL] SQL syntax for Pivot
## What changes were proposed in this pull request?

Add SQL support for Pivot according to Pivot grammar defined by Oracle (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_clause.htm) with some simplifications, based on our existing functionality and limitations for Pivot at the backend:
1. For pivot_for_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_for_clause.htm), the column list form is not supported, which means the pivot column can only be one single column.
2. For pivot_in_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_in_clause.htm), the sub-query form and "ANY" is not supported (this is only supported by Oracle for XML anyway).
3. For pivot_in_clause, aliases for the constant values are not supported.

The code changes are:
1. Add parser support for Pivot. Note that according to https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#i2076542, Pivot cannot be used together with lateral views in the from clause. This restriction has been implemented in the Parser rule.
2. Infer group-by expressions: group-by expressions are not explicitly specified in SQL Pivot clause and need to be deduced based on this rule: https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#CHDFAFIE, so we have to post-fix it at query analysis stage.
3. Override Pivot.resolved as "false": for the reason mentioned in [2] and the fact that output attributes change after Pivot being replaced by Project or Aggregate, we avoid resolving parent references until after Pivot has been resolved and replaced.
4. Verify aggregate expressions: only aggregate expressions with or without aliases can appear in the first part of the Pivot clause, and this check is performed as analysis stage.

## How was this patch tested?

A new test suite PivotSuite is added.

Author: maryannxue <maryann.xue@gmail.com>

Closes #21187 from maryannxue/spark-24035.
2018-05-03 17:05:02 -07:00
Wenchen Fan 96a50016bb [SPARK-24169][SQL] JsonToStructs should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from #21190 , to make it easier to backport.

`JsonToStructs` can be serialized to executors and evaluate, we should not call `SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)` in the body.

## How was this patch tested?

tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21226 from cloud-fan/minor4.
2018-05-03 23:36:09 +08:00
Wenchen Fan 417ad92502 [SPARK-23715][SQL] the input of to/from_utc_timestamp can not have timezone
## What changes were proposed in this pull request?

`from_utc_timestamp` assumes its input is in UTC timezone and shifts it to the specified timezone. When the timestamp contains timezone(e.g. `2018-03-13T06:18:23+00:00`), Spark breaks the semantic and respect the timezone in the string. This is not what user expects and the result is different from Hive/Impala. `to_utc_timestamp` has the same problem.

More details please refer to the JIRA ticket.

This PR fixes this by returning null if the input timestamp contains timezone.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21169 from cloud-fan/from_utc_timezone.
2018-05-03 19:27:01 +08:00
Kazuaki Ishizaki 5be8aab144 [SPARK-23923][SQL] Add cardinality function
## What changes were proposed in this pull request?

The PR adds the SQL function `cardinality`. The behavior of the function is based on Presto's one.

The function returns the length of the array or map stored in the column as `int` while the Presto version returns the value as `BigInt` (`long` in Spark). The discussions regarding the difference of return type are [here](https://github.com/apache/spark/pull/21031#issuecomment-381284638) and [there](https://github.com/apache/spark/pull/21031#discussion_r181622107).

## How was this patch tested?

Added UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21031 from kiszk/SPARK-23923.
2018-05-02 13:53:10 -07:00
Marco Gaido 504c9cfd21 [SPARK-24123][SQL] Fix precision issues in monthsBetween with more than 8 digits
## What changes were proposed in this pull request?

SPARK-23902 introduced the ability to retrieve more than 8 digits in `monthsBetween`. Unfortunately, current implementation can cause precision loss in such a case. This was causing also a flaky UT.

This PR mirrors Hive's implementation in order to avoid precision loss also when more than 8 digits are returned.

## How was this patch tested?

running 10000000 times the flaky UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21196 from mgaido91/SPARK-24123.
2018-05-02 13:49:15 -07:00
Marco Gaido 8dbf56c055 [SPARK-24013][SQL] Remove unneeded compress in ApproximatePercentile
## What changes were proposed in this pull request?

`ApproximatePercentile` contains a workaround logic to compress the samples since at the beginning `QuantileSummaries` was ignoring the compression threshold. This problem was fixed in SPARK-17439, but the workaround logic was not removed. So we are compressing the samples many more times than needed: this could lead to critical performance degradation.

This can create serious performance issues in queries like:
```
select approx_percentile(id, array(0.1)) from range(10000000)
```

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21133 from mgaido91/SPARK-24013.
2018-05-02 11:58:55 -07:00
wangyanlin01 7bbec0dced [SPARK-24061][SS] Add TypedFilter support for continuous processing
## What changes were proposed in this pull request?

Add TypedFilter support for continuous processing application.

## How was this patch tested?

unit tests

Author: wangyanlin01 <wangyanlin01@baidu.com>

Closes #21136 from yanlin-Lynn/SPARK-24061.
2018-05-01 16:22:52 +08:00
Maxim Gekk bd14da6fd5 [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support custom encoding for json files
## What changes were proposed in this pull request?

I propose new option for JSON datasource which allows to specify encoding (charset) of input and output files. Here is an example of using of the option:

```
spark.read.schema(schema)
  .option("multiline", "true")
  .option("encoding", "UTF-16LE")
  .json(fileName)
```

If the option is not specified, charset auto-detection mechanism is used by default.

The option can be used for saving datasets to jsons. Currently Spark is able to save datasets into json files in `UTF-8` charset only. The changes allow to save data in any supported charset. Here is the approximate list of supported charsets by Oracle Java SE: https://docs.oracle.com/javase/8/docs/technotes/guides/intl/encoding.doc.html . An user can specify the charset of output jsons via the charset option like `.option("charset", "UTF-16BE")`. By default the output charset is still `UTF-8` to keep backward compatibility.

The solution has the following restrictions for per-line mode (`multiline = false`):

- If charset is different from UTF-8, the lineSep option must be specified. The option required because Hadoop LineReader cannot detect the line separator correctly. Here is the ticket for solving the issue: https://issues.apache.org/jira/browse/SPARK-23725

- Encoding with [BOM](https://en.wikipedia.org/wiki/Byte_order_mark) are not supported. For example, the `UTF-16` and `UTF-32` encodings are blacklisted. The problem can be solved by https://github.com/MaxGekk/spark-1/pull/2

## How was this patch tested?

I added the following tests:
- reads an json file in `UTF-16LE` encoding with BOM in `multiline` mode
- read json file by using charset auto detection (`UTF-32BE` with BOM)
- read json file using of user's charset (`UTF-16LE`)
- saving in `UTF-32BE` and read the result by standard library (not by Spark)
- checking that default charset is `UTF-8`
- handling wrong (unsupported) charset

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #20937 from MaxGekk/json-encoding-line-sep.
2018-04-29 11:25:31 +08:00
Marco Gaido ad94e8592b [SPARK-23736][SQL][FOLLOWUP] Error message should contains SQL types
## What changes were proposed in this pull request?

In the error messages we should return the SQL types (like `string` rather than the internal types like `StringType`).

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21181 from mgaido91/SPARK-23736_followup.
2018-04-28 10:47:43 +08:00
gatorsmile ce2f919f8d [SPARK-23799][SQL][FOLLOW-UP] FilterEstimation.evaluateInSet produces wrong stats for STRING
## What changes were proposed in this pull request?
`colStat.min` AND `colStat.max` are empty for string type. Thus, `evaluateInSet` should not return zero when either `colStat.min` or `colStat.max`.

## How was this patch tested?
Added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21147 from gatorsmile/cached.
2018-04-26 19:07:13 +08:00
Marco Gaido cd10f9df82 [SPARK-23916][SQL] Add array_join function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_join`. The behavior of the function is based on Presto's one.

The function accepts an `array` of `string` which is to be joined, a `string` which is the delimiter to use between the items of the first argument and optionally a `string` which is used to replace `null` values.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21011 from mgaido91/SPARK-23916.
2018-04-26 13:37:13 +09:00
Marco Gaido 58c55cb4a6 [SPARK-23902][SQL] Add roundOff flag to months_between
## What changes were proposed in this pull request?

HIVE-15511 introduced the `roundOff` flag in order to disable the rounding to 8 digits which is performed in `months_between`. Since this can be a computational intensive operation, skipping it may improve performances when the rounding is not needed.

## How was this patch tested?

modified existing UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21008 from mgaido91/SPARK-23902.
2018-04-26 12:19:20 +09:00
Wenchen Fan ac4ca7c4dd [SPARK-24012][SQL][TEST][FOLLOWUP] add unit test
## What changes were proposed in this pull request?

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

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21154 from cloud-fan/test.
2018-04-25 13:42:44 -07:00
liutang123 64e8408e6f [SPARK-24012][SQL] Union of map and other compatible column
## What changes were proposed in this pull request?
Union of map and other compatible column result in unresolved operator 'Union; exception

Reproduction
`spark-sql>select map(1,2), 'str' union all select map(1,2,3,null), 1`
Output:
```
Error in query: unresolved operator 'Union;;
'Union
:- Project [map(1, 2) AS map(1, 2)#106, str AS str#107]
:  +- OneRowRelation$
+- Project [map(1, cast(2 as int), 3, cast(null as int)) AS map(1, CAST(2 AS INT), 3, CAST(NULL AS INT))#109, 1 AS 1#108]
   +- OneRowRelation$
```
So, we should cast part of columns to be compatible when appropriate.

## How was this patch tested?
Added a test (query union of map and other columns) to SQLQueryTestSuite's union.sql.

Author: liutang123 <liutang123@yeah.net>

Closes #21100 from liutang123/SPARK-24012.
2018-04-25 18:10:51 +08:00
mn-mikke 5fea17b3be [SPARK-23821][SQL] Collection function: flatten
## What changes were proposed in this pull request?

This PR adds a new collection function that transforms an array of arrays into a single array. The PR comprises:
- An expression for flattening array structure
- Flatten function
- A wrapper for PySpark

## How was this patch tested?

New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

## Codegen examples
### Primitive type
```
val df = Seq(
  Seq(Seq(1, 2), Seq(4, 5)),
  Seq(null, Seq(1))
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(flatten($"i")).debugCodegen
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         boolean filter_value = true;
/* 038 */
/* 039 */         if (!(!inputadapter_isNull)) {
/* 040 */           filter_value = inputadapter_isNull;
/* 041 */         }
/* 042 */         if (!filter_value) continue;
/* 043 */
/* 044 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */         boolean project_isNull = inputadapter_isNull;
/* 047 */         ArrayData project_value = null;
/* 048 */
/* 049 */         if (!inputadapter_isNull) {
/* 050 */           for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */             project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */           }
/* 053 */           if (!project_isNull) {
/* 054 */             long project_numElements = 0;
/* 055 */             for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */               project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */             }
/* 058 */             if (project_numElements > 2147483632) {
/* 059 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */                 project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */             }
/* 062 */
/* 063 */             long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 064 */               project_numElements,
/* 065 */               4);
/* 066 */             if (project_size > 2147483632) {
/* 067 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 068 */                 project_size + " bytes of data due to exceeding the limit 2147483632" +
/* 069 */                 " bytes for UnsafeArrayData.");
/* 070 */             }
/* 071 */
/* 072 */             byte[] project_array = new byte[(int)project_size];
/* 073 */             UnsafeArrayData project_tempArrayData = new UnsafeArrayData();
/* 074 */             Platform.putLong(project_array, 16, project_numElements);
/* 075 */             project_tempArrayData.pointTo(project_array, 16, (int)project_size);
/* 076 */             int project_counter = 0;
/* 077 */             for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 078 */               ArrayData arr = inputadapter_value.getArray(k);
/* 079 */               for (int l = 0; l < arr.numElements(); l++) {
/* 080 */                 if (arr.isNullAt(l)) {
/* 081 */                   project_tempArrayData.setNullAt(project_counter);
/* 082 */                 } else {
/* 083 */                   project_tempArrayData.setInt(
/* 084 */                     project_counter,
/* 085 */                     arr.getInt(l)
/* 086 */                   );
/* 087 */                 }
/* 088 */                 project_counter++;
/* 089 */               }
/* 090 */             }
/* 091 */             project_value = project_tempArrayData;
/* 092 */
/* 093 */           }
/* 094 */
/* 095 */         }
```
### Non-primitive type
```
val df = Seq(
  Seq(Seq("a", "b"), Seq(null, "d")),
  Seq(null, Seq("a"))
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(flatten($"s")).debugCodegen
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         boolean filter_value = true;
/* 038 */
/* 039 */         if (!(!inputadapter_isNull)) {
/* 040 */           filter_value = inputadapter_isNull;
/* 041 */         }
/* 042 */         if (!filter_value) continue;
/* 043 */
/* 044 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */         boolean project_isNull = inputadapter_isNull;
/* 047 */         ArrayData project_value = null;
/* 048 */
/* 049 */         if (!inputadapter_isNull) {
/* 050 */           for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */             project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */           }
/* 053 */           if (!project_isNull) {
/* 054 */             long project_numElements = 0;
/* 055 */             for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */               project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */             }
/* 058 */             if (project_numElements > 2147483632) {
/* 059 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */                 project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */             }
/* 062 */
/* 063 */             Object[] project_arrayObject = new Object[(int)project_numElements];
/* 064 */             int project_counter = 0;
/* 065 */             for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 066 */               ArrayData arr = inputadapter_value.getArray(k);
/* 067 */               for (int l = 0; l < arr.numElements(); l++) {
/* 068 */                 project_arrayObject[project_counter] = arr.getUTF8String(l);
/* 069 */                 project_counter++;
/* 070 */               }
/* 071 */             }
/* 072 */             project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObject);
/* 073 */
/* 074 */           }
/* 075 */
/* 076 */         }
```

Author: mn-mikke <mrkAha12346github>

Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-25 11:19:08 +09:00
Takeshi Yamamuro 4926a7c2f0 [SPARK-23589][SQL][FOLLOW-UP] Reuse InternalRow in ExternalMapToCatalyst eval
## What changes were proposed in this pull request?
This pr is a follow-up of #20980 and fixes code to reuse `InternalRow` for converting input keys/values in `ExternalMapToCatalyst` eval.

## How was this patch tested?
Existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21137 from maropu/SPARK-23589-FOLLOWUP.
2018-04-24 17:52:05 +02:00
Marco Gaido 281c1ca0dc [SPARK-23973][SQL] Remove consecutive Sorts
## What changes were proposed in this pull request?

In SPARK-23375 we introduced the ability of removing `Sort` operation during query optimization if the data is already sorted. In this follow-up we remove also a `Sort` which is followed by another `Sort`: in this case the first sort is not needed and can be safely removed.

The PR starts from henryr's comment: https://github.com/apache/spark/pull/20560#discussion_r180601594. So credit should be given to him.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21072 from mgaido91/SPARK-23973.
2018-04-24 10:11:09 +08:00
Takeshi Yamamuro afbdf42730 [SPARK-23589][SQL] ExternalMapToCatalyst should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `ExternalMapToCatalyst`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20980 from maropu/SPARK-23589.
2018-04-23 14:28:28 +02:00
Wenchen Fan d87d30e4fe [SPARK-23564][SQL] infer additional filters from constraints for join's children
## What changes were proposed in this pull request?

The existing query constraints framework has 2 steps:
1. propagate constraints bottom up.
2. use constraints to infer additional filters for better data pruning.

For step 2, it mostly helps with Join, because we can connect the constraints from children to the join condition and infer powerful filters to prune the data of the join sides. e.g., the left side has constraints `a = 1`, the join condition is `left.a = right.a`, then we can infer `right.a = 1` to the right side and prune the right side a lot.

However, the current logic of inferring filters from constraints for Join is pretty weak. It infers the filters from Join's constraints. Some joins like left semi/anti exclude output from right side and the right side constraints will be lost here.

This PR propose to check the left and right constraints individually, expand the constraints with join condition and add filters to children of join directly, instead of adding to the join condition.

This reverts https://github.com/apache/spark/pull/20670 , covers https://github.com/apache/spark/pull/20717 and https://github.com/apache/spark/pull/20816

This is inspired by the original PRs and the tests are all from these PRs. Thanks to the authors mgaido91 maryannxue KaiXinXiaoLei !

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21083 from cloud-fan/join.
2018-04-23 20:21:01 +08:00
Wenchen Fan f70f46d1e5 [SPARK-23877][SQL][FOLLOWUP] use PhysicalOperation to simplify the handling of Project and Filter over partitioned relation
## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/20988

`PhysicalOperation` can collect Project and Filters over a certain plan and substitute the alias with the original attributes in the bottom plan. We can use it in `OptimizeMetadataOnlyQuery` rule to handle the Project and Filter over partitioned relation.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21111 from cloud-fan/refactor.
2018-04-23 20:18:50 +08:00
Mykhailo Shtelma c48085aa91 [SPARK-23799][SQL] FilterEstimation.evaluateInSet produces devision by zero in a case of empty table with analyzed statistics
>What changes were proposed in this pull request?

During evaluation of IN conditions, if the source data frame, is represented by a plan, that uses hive table with columns, which were previously analysed, and the plan has conditions for these fields, that cannot be satisfied (which leads us to an empty data frame), FilterEstimation.evaluateInSet method produces NumberFormatException and ClassCastException.
In order to fix this bug, method FilterEstimation.evaluateInSet at first checks, if distinct count is not zero, and also checks if colStat.min and colStat.max  are defined, and only in this case proceeds with the calculation. If at least one of the conditions is not satisfied, zero is returned.

>How was this patch tested?

In order to test the PR two tests were implemented: one in FilterEstimationSuite, that tests the plan with the statistics that violates the conditions mentioned above,  and another one in StatisticsCollectionSuite, that test the whole process of analysis/optimisation of the query, that leads to the problems, mentioned in the first section.

Author: Mykhailo Shtelma <mykhailo.shtelma@bearingpoint.com>
Author: smikesh <mshtelma@gmail.com>

Closes #21052 from mshtelma/filter_estimation_evaluateInSet_Bugs.
2018-04-21 23:33:57 -07:00
gatorsmile 7bc853d089 [SPARK-24033][SQL] Fix Mismatched of Window Frame specifiedwindowframe(RowFrame, -1, -1)
## What changes were proposed in this pull request?

When the OffsetWindowFunction's frame is `UnaryMinus(Literal(1))` but the specified window frame has been simplified to `Literal(-1)` by some optimizer rules e.g., `ConstantFolding`. Thus, they do not match and cause the following error:
```
org.apache.spark.sql.AnalysisException: Window Frame specifiedwindowframe(RowFrame, -1, -1) must match the required frame specifiedwindowframe(RowFrame, -1, -1);
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:41)
at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:91)
at
```
## How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21115 from gatorsmile/fixLag.
2018-04-21 10:45:12 -07:00
Takeshi Yamamuro 0dd97f6ea4 [SPARK-23595][SQL] ValidateExternalType should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `ValidateExternalType`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20757 from maropu/SPARK-23595.
2018-04-20 15:02:27 +02:00
Takeshi Yamamuro 074a7f9053 [SPARK-23588][SQL][FOLLOW-UP] Resolve a map builder method per execution in CatalystToExternalMap
## What changes were proposed in this pull request?
This pr is a follow-up pr of #20979 and fixes code to resolve a map builder method per execution instead of per row in `CatalystToExternalMap`.

## How was this patch tested?
Existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21112 from maropu/SPARK-23588-FOLLOWUP.
2018-04-20 14:43:47 +02:00
mn-mikke e6b466084c [SPARK-23736][SQL] Extending the concat function to support array columns
## What changes were proposed in this pull request?
The PR adds a logic for easy concatenation of multiple array columns and covers:
- Concat expression has been extended to support array columns
- A Python wrapper

## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
- typeCoercion/native/concat.sql

## Codegen examples
### Primitive-type elements
```
val df = Seq(
  (Seq(1 ,2), Seq(3, 4)),
  (Seq(1, 2, 3), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 070 */               project_numElements,
/* 071 */               4);
/* 072 */             if (project_size > 2147483632) {
/* 073 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size +
/* 074 */                 " bytes of data due to exceeding the limit 2147483632 bytes" +
/* 075 */                 " for UnsafeArrayData.");
/* 076 */             }
/* 077 */
/* 078 */             byte[] project_array = new byte[(int)project_size];
/* 079 */             UnsafeArrayData project_arrayData = new UnsafeArrayData();
/* 080 */             Platform.putLong(project_array, 16, project_numElements);
/* 081 */             project_arrayData.pointTo(project_array, 16, (int)project_size);
/* 082 */             int project_counter = 0;
/* 083 */             for (int y = 0; y < 2; y++) {
/* 084 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 085 */                 if (args[y].isNullAt(z)) {
/* 086 */                   project_arrayData.setNullAt(project_counter);
/* 087 */                 } else {
/* 088 */                   project_arrayData.setInt(
/* 089 */                     project_counter,
/* 090 */                     args[y].getInt(z)
/* 091 */                   );
/* 092 */                 }
/* 093 */                 project_counter++;
/* 094 */               }
/* 095 */             }
/* 096 */             return project_arrayData;
/* 097 */           }
/* 098 */         }.concat(project_args);
/* 099 */         boolean project_isNull = project_value == null;
```

### Non-primitive-type elements
```
val df = Seq(
  (Seq("aa" ,"bb"), Seq("ccc", "ddd")),
  (Seq("x", "y"), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             Object[] project_arrayObjects = new Object[(int)project_numElements];
/* 070 */             int project_counter = 0;
/* 071 */             for (int y = 0; y < 2; y++) {
/* 072 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 073 */                 project_arrayObjects[project_counter] = args[y].getUTF8String(z);
/* 074 */                 project_counter++;
/* 075 */               }
/* 076 */             }
/* 077 */             return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects);
/* 078 */           }
/* 079 */         }.concat(project_args);
/* 080 */         boolean project_isNull = project_value == null;
```

Author: mn-mikke <mrkAha12346github>

Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 14:58:11 +09:00
Xingbo Jiang d96c3e33cc [SPARK-21811][SQL] Fix the inconsistency behavior when finding the widest common type
## What changes were proposed in this pull request?

Currently we find the wider common type by comparing the two types from left to right, this can be a problem when you have two data types which don't have a common type but each can be promoted to StringType.

For instance, if you have a table with the schema:
[c1: date, c2: string, c3: int]

The following succeeds:
SELECT coalesce(c1, c2, c3) FROM table

While the following produces an exception:
SELECT coalesce(c1, c3, c2) FROM table

This is only a issue when the seq of dataTypes contains `StringType` and all the types can do string promotion.

close #19033

## How was this patch tested?

Add test in `TypeCoercionSuite`

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21074 from jiangxb1987/typeCoercion.
2018-04-19 21:21:22 +08:00
Takeshi Yamamuro e13416502f [SPARK-23588][SQL] CatalystToExternalMap should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `CatalystToExternalMap`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20979 from maropu/SPARK-23588.
2018-04-19 14:42:50 +02:00
Takeshi Yamamuro 1b08c4393c [SPARK-23584][SQL] NewInstance should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `NewInstance`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20778 from maropu/SPARK-23584.
2018-04-19 14:38:26 +02:00
Kazuaki Ishizaki 46bb2b5129 [SPARK-23924][SQL] Add element_at function
## What changes were proposed in this pull request?

The PR adds the SQL function `element_at`. The behavior of the function is based on Presto's one.

This function returns element of array at given index in value if column is array, or returns value for the given key in value if column is map.

## How was this patch tested?

Added UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21053 from kiszk/SPARK-23924.
2018-04-19 21:00:10 +09:00
Kazuaki Ishizaki d5bec48b9c [SPARK-23919][SQL] Add array_position function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_position`. The behavior of the function is based on Presto's one.

The function returns the position of the first occurrence of the element in array x (or 0 if not found) using 1-based index as BigInt.

## How was this patch tested?

Added UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21037 from kiszk/SPARK-23919.
2018-04-19 11:59:17 +09:00
Liang-Chi Hsieh a9066478f6 [SPARK-23875][SQL][FOLLOWUP] Add IndexedSeq wrapper for ArrayData
## What changes were proposed in this pull request?

Use specified accessor in `ArrayData.foreach` and `toArray`.

## How was this patch tested?

Existing tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21099 from viirya/SPARK-23875-followup.
2018-04-19 00:05:47 +02:00
Takuya UESHIN f09a9e9418 [SPARK-24007][SQL] EqualNullSafe for FloatType and DoubleType might generate a wrong result by codegen.
## What changes were proposed in this pull request?

`EqualNullSafe` for `FloatType` and `DoubleType` might generate a wrong result by codegen.

```scala
scala> val df = Seq((Some(-1.0d), None), (None, Some(-1.0d))).toDF()
df: org.apache.spark.sql.DataFrame = [_1: double, _2: double]

scala> df.show()
+----+----+
|  _1|  _2|
+----+----+
|-1.0|null|
|null|-1.0|
+----+----+

scala> df.filter("_1 <=> _2").show()
+----+----+
|  _1|  _2|
+----+----+
|-1.0|null|
|null|-1.0|
+----+----+
```

The result should be empty but the result remains two rows.

## How was this patch tested?

Added a test.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21094 from ueshin/issues/SPARK-24007/equalnullsafe.
2018-04-18 08:22:05 -07:00
mn-mikke f81fa478ff [SPARK-23926][SQL] Extending reverse function to support ArrayType arguments
## What changes were proposed in this pull request?

This PR extends `reverse` functions to be able to operate over array columns and covers:
- Introduction of `Reverse` expression that represents logic for reversing arrays and also strings
- Removal of `StringReverse` expression
- A wrapper for PySpark

## How was this patch tested?

New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

## Codegen examples
### Primitive type
```
val df = Seq(
  Seq(1, 3, 4, 2),
  null
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen
```
Result:
```
/* 032 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */         null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */         boolean filter_value = true;
/* 037 */
/* 038 */         if (!(!inputadapter_isNull)) {
/* 039 */           filter_value = inputadapter_isNull;
/* 040 */         }
/* 041 */         if (!filter_value) continue;
/* 042 */
/* 043 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */         boolean project_isNull = inputadapter_isNull;
/* 046 */         ArrayData project_value = null;
/* 047 */
/* 048 */         if (!inputadapter_isNull) {
/* 049 */           final int project_length = inputadapter_value.numElements();
/* 050 */           project_value = inputadapter_value.copy();
/* 051 */           for(int k = 0; k < project_length / 2; k++) {
/* 052 */             int l = project_length - k - 1;
/* 053 */             boolean isNullAtK = project_value.isNullAt(k);
/* 054 */             boolean isNullAtL = project_value.isNullAt(l);
/* 055 */             if(!isNullAtK) {
/* 056 */               int el = project_value.getInt(k);
/* 057 */               if(!isNullAtL) {
/* 058 */                 project_value.setInt(k, project_value.getInt(l));
/* 059 */               } else {
/* 060 */                 project_value.setNullAt(k);
/* 061 */               }
/* 062 */               project_value.setInt(l, el);
/* 063 */             } else if (!isNullAtL) {
/* 064 */               project_value.setInt(k, project_value.getInt(l));
/* 065 */               project_value.setNullAt(l);
/* 066 */             }
/* 067 */           }
/* 068 */
/* 069 */         }
```
### Non-primitive type
```
val df = Seq(
  Seq("a", "c", "d", "b"),
  null
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen
```
Result:
```
/* 032 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */         null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */         boolean filter_value = true;
/* 037 */
/* 038 */         if (!(!inputadapter_isNull)) {
/* 039 */           filter_value = inputadapter_isNull;
/* 040 */         }
/* 041 */         if (!filter_value) continue;
/* 042 */
/* 043 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */         boolean project_isNull = inputadapter_isNull;
/* 046 */         ArrayData project_value = null;
/* 047 */
/* 048 */         if (!inputadapter_isNull) {
/* 049 */           final int project_length = inputadapter_value.numElements();
/* 050 */           project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]);
/* 051 */           for(int k = 0; k < project_length; k++) {
/* 052 */             int l = project_length - k - 1;
/* 053 */             project_value.update(k, inputadapter_value.getUTF8String(l));
/* 054 */           }
/* 055 */
/* 056 */         }
```

Author: mn-mikke <mrkAha12346github>

Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 18:41:55 +09:00
maryannxue 1e3b8762a8 [SPARK-21479][SQL] Outer join filter pushdown in null supplying table when condition is on one of the joined columns
## What changes were proposed in this pull request?

Added `TransitPredicateInOuterJoin` optimization rule that transits constraints from the preserved side of an outer join to the null-supplying side. The constraints of the join operator will remain unchanged.

## How was this patch tested?

Added 3 tests in `InferFiltersFromConstraintsSuite`.

Author: maryannxue <maryann.xue@gmail.com>

Closes #20816 from maryannxue/spark-21479.
2018-04-18 10:36:41 +08:00
Marco Gaido f39e82ce15 [SPARK-23986][SQL] freshName can generate non-unique names
## What changes were proposed in this pull request?

We are using `CodegenContext.freshName` to get a unique name for any new variable we are adding. Unfortunately, this method currently fails to create a unique name when we request more than one instance of variables with starting name `name1` and an instance with starting name `name11`.

The PR changes the way a new name is generated by `CodegenContext.freshName` so that we generate unique names in this scenario too.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21080 from mgaido91/SPARK-23986.
2018-04-18 00:35:44 +08:00
jinxing ed4101d29f [SPARK-22676] Avoid iterating all partition paths when spark.sql.hive.verifyPartitionPath=true
## What changes were proposed in this pull request?

In current code, it will scanning all partition paths when spark.sql.hive.verifyPartitionPath=true.
e.g. table like below:
```
CREATE TABLE `test`(
`id` int,
`age` int,
`name` string)
PARTITIONED BY (
`A` string,
`B` string)
load data local inpath '/tmp/data0' into table test partition(A='00', B='00')
load data local inpath '/tmp/data1' into table test partition(A='01', B='01')
load data local inpath '/tmp/data2' into table test partition(A='10', B='10')
load data local inpath '/tmp/data3' into table test partition(A='11', B='11')
```
If I query with SQL – "select * from test where A='00' and B='01'  ", current code will scan all partition paths including '/data/A=00/B=00', '/data/A=00/B=00', '/data/A=01/B=01', '/data/A=10/B=10', '/data/A=11/B=11'. It costs much time and memory cost.

This pr proposes to avoid iterating all partition paths. Add a config `spark.files.ignoreMissingFiles` and ignore the `file not found` when `getPartitions/compute`(for hive table scan). This is much like the logic brought by
`spark.sql.files.ignoreMissingFiles`(which is for datasource scan).

## How was this patch tested?
UT

Author: jinxing <jinxing6042@126.com>

Closes #19868 from jinxing64/SPARK-22676.
2018-04-17 21:52:33 +08:00
Marco Gaido 0a9172a05e [SPARK-23835][SQL] Add not-null check to Tuples' arguments deserialization
## What changes were proposed in this pull request?

There was no check on nullability for arguments of `Tuple`s. This could lead to have weird behavior when a null value had to be deserialized into a non-nullable Scala object: in those cases, the `null` got silently transformed in a valid value (like `-1` for `Int`), corresponding to the default value we are using in the SQL codebase. This situation was very likely to happen when deserializing to a Tuple of primitive Scala types (like Double, Int, ...).

The PR adds the `AssertNotNull` to arguments of tuples which have been asked to be converted to non-nullable types.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20976 from mgaido91/SPARK-23835.
2018-04-17 21:45:20 +08:00
Liang-Chi Hsieh 30ffb53cad [SPARK-23875][SQL] Add IndexedSeq wrapper for ArrayData
## What changes were proposed in this pull request?

We don't have a good way to sequentially access `UnsafeArrayData` with a common interface such as `Seq`. An example is `MapObject` where we need to access several sequence collection types together. But `UnsafeArrayData` doesn't implement `ArrayData.array`. Calling `toArray` will copy the entire array. We can provide an `IndexedSeq` wrapper for `ArrayData`, so we can avoid copying the entire array.

## How was this patch tested?

Added test.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #20984 from viirya/SPARK-23875.
2018-04-17 15:09:36 +02:00
Marco Gaido 14844a62c0 [SPARK-23918][SQL] Add array_min function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_min`. It takes an array as argument and returns the minimum value in it.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21025 from mgaido91/SPARK-23918.
2018-04-17 17:55:35 +09:00
Liang-Chi Hsieh fd990a908b [SPARK-23873][SQL] Use accessors in interpreted LambdaVariable
## What changes were proposed in this pull request?

Currently, interpreted execution of `LambdaVariable` just uses `InternalRow.get` to access element. We should use specified accessors if possible.

## How was this patch tested?

Added test.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #20981 from viirya/SPARK-23873.
2018-04-16 22:45:57 +02:00
Marco Gaido 6931022031 [SPARK-23917][SQL] Add array_max function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_max`. It takes an array as argument and returns the maximum value in it.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21024 from mgaido91/SPARK-23917.
2018-04-15 21:45:55 -07:00
Liang-Chi Hsieh 73f28530d6 [SPARK-23979][SQL] MultiAlias should not be a CodegenFallback
## What changes were proposed in this pull request?

Just found `MultiAlias` is a `CodegenFallback`. It should not be as looks like `MultiAlias` won't be evaluated.

## How was this patch tested?

Existing tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #21065 from viirya/multialias-without-codegenfallback.
2018-04-14 08:59:04 +08:00
Tathagata Das cbb41a0c5b [SPARK-23966][SS] Refactoring all checkpoint file writing logic in a common CheckpointFileManager interface
## What changes were proposed in this pull request?

Checkpoint files (offset log files, state store files) in Structured Streaming must be written atomically such that no partial files are generated (would break fault-tolerance guarantees). Currently, there are 3 locations which try to do this individually, and in some cases, incorrectly.

1. HDFSOffsetMetadataLog - This uses a FileManager interface to use any implementation of `FileSystem` or `FileContext` APIs. It preferably loads `FileContext` implementation as FileContext of HDFS has atomic renames.
1. HDFSBackedStateStore (aka in-memory state store)
  - Writing a version.delta file - This uses FileSystem APIs only to perform a rename. This is incorrect as rename is not atomic in HDFS FileSystem implementation.
  - Writing a snapshot file - Same as above.

#### Current problems:
1. State Store behavior is incorrect - HDFS FileSystem implementation does not have atomic rename.
1. Inflexible - Some file systems provide mechanisms other than write-to-temp-file-and-rename for writing atomically and more efficiently. For example, with S3 you can write directly to the final file and it will be made visible only when the entire file is written and closed correctly. Any failure can be made to terminate the writing without making any partial files visible in S3. The current code does not abstract out this mechanism enough that it can be customized.

#### Solution:

1. Introduce a common interface that all 3 cases above can use to write checkpoint files atomically.
2. This interface must provide the necessary interfaces that allow customization of the write-and-rename mechanism.

This PR does that by introducing the interface `CheckpointFileManager` and modifying `HDFSMetadataLog` and `HDFSBackedStateStore` to use the interface. Similar to earlier `FileManager`, there are implementations based on `FileSystem` and `FileContext` APIs, and the latter implementation is preferred to make it work correctly with HDFS.

The key method this interface has is `createAtomic(path, overwrite)` which returns a `CancellableFSDataOutputStream` that has the method `cancel()`. All users of this method need to either call `close()` to successfully write the file, or `cancel()` in case of an error.

## How was this patch tested?
New tests in `CheckpointFileManagerSuite` and slightly modified existing tests.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #21048 from tdas/SPARK-23966.
2018-04-13 16:31:39 -07:00
Marco Gaido 25892f3cc9 [SPARK-23375][SQL] Eliminate unneeded Sort in Optimizer
## What changes were proposed in this pull request?

Added a new rule to remove Sort operation when its child is already sorted.
For instance, this simple code:
```
spark.sparkContext.parallelize(Seq(("a", "b"))).toDF("a", "b").registerTempTable("table1")
val df = sql(s"""SELECT b
                | FROM (
                |     SELECT a, b
                |     FROM table1
                |     ORDER BY a
                | ) t
                | ORDER BY a""".stripMargin)
df.explain(true)
```
before the PR produces this plan:
```
== Parsed Logical Plan ==
'Sort ['a ASC NULLS FIRST], true
+- 'Project ['b]
   +- 'SubqueryAlias t
      +- 'Sort ['a ASC NULLS FIRST], true
         +- 'Project ['a, 'b]
            +- 'UnresolvedRelation `table1`

== Analyzed Logical Plan ==
b: string
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- SubqueryAlias t
         +- Sort [a#6 ASC NULLS FIRST], true
            +- Project [a#6, b#7]
               +- SubqueryAlias table1
                  +- Project [_1#3 AS a#6, _2#4 AS b#7]
                     +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#4]
                        +- ExternalRDD [obj#2]

== Optimized Logical Plan ==
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- Sort [a#6 ASC NULLS FIRST], true
         +- Project [_1#3 AS a#6, _2#4 AS b#7]
            +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
               +- ExternalRDD [obj#2]

== Physical Plan ==
*(3) Project [b#7]
+- *(3) Sort [a#6 ASC NULLS FIRST], true, 0
   +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 200)
      +- *(2) Project [b#7, a#6]
         +- *(2) Sort [a#6 ASC NULLS FIRST], true, 0
            +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 200)
               +- *(1) Project [_1#3 AS a#6, _2#4 AS b#7]
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
                     +- Scan ExternalRDDScan[obj#2]
```

while after the PR produces:

```
== Parsed Logical Plan ==
'Sort ['a ASC NULLS FIRST], true
+- 'Project ['b]
   +- 'SubqueryAlias t
      +- 'Sort ['a ASC NULLS FIRST], true
         +- 'Project ['a, 'b]
            +- 'UnresolvedRelation `table1`

== Analyzed Logical Plan ==
b: string
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- SubqueryAlias t
         +- Sort [a#6 ASC NULLS FIRST], true
            +- Project [a#6, b#7]
               +- SubqueryAlias table1
                  +- Project [_1#3 AS a#6, _2#4 AS b#7]
                     +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#4]
                        +- ExternalRDD [obj#2]

== Optimized Logical Plan ==
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [_1#3 AS a#6, _2#4 AS b#7]
      +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
         +- ExternalRDD [obj#2]

== Physical Plan ==
*(2) Project [b#7]
+- *(2) Sort [a#6 ASC NULLS FIRST], true, 0
   +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 5)
      +- *(1) Project [_1#3 AS a#6, _2#4 AS b#7]
         +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
            +- Scan ExternalRDDScan[obj#2]
```

this means that an unnecessary sort operation is not performed after the PR.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20560 from mgaido91/SPARK-23375.
2018-04-14 01:01:00 +08:00