Commit graph

6882 commits

Author SHA1 Message Date
Dongjoon Hyun f5850e7892 [SPARK-23457][SQL] Register task completion listeners first in ParquetFileFormat
## What changes were proposed in this pull request?

ParquetFileFormat leaks opened files in some cases. This PR prevents that by registering task completion listers first before initialization.

- [spark-branch-2.3-test-sbt-hadoop-2.7](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.3-test-sbt-hadoop-2.7/205/testReport/org.apache.spark.sql/FileBasedDataSourceSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/)
- [spark-master-test-sbt-hadoop-2.6](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4228/testReport/junit/org.apache.spark.sql.execution.datasources.parquet/ParquetQuerySuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/)

```
Caused by: sbt.ForkMain$ForkError: java.lang.Throwable: null
	at org.apache.spark.DebugFilesystem$.addOpenStream(DebugFilesystem.scala:36)
	at org.apache.spark.DebugFilesystem.open(DebugFilesystem.scala:70)
	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:769)
	at org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:538)
	at org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.initialize(SpecificParquetRecordReaderBase.java:149)
	at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initialize(VectorizedParquetRecordReader.java:133)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.apply(ParquetFileFormat.scala:400)
	at
```

## How was this patch tested?

Manual. The following test case generates the same leakage.

```scala
  test("SPARK-23457 Register task completion listeners first in ParquetFileFormat") {
    withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE.key -> s"${Int.MaxValue}") {
      withTempDir { dir =>
        val basePath = dir.getCanonicalPath
        Seq(0).toDF("a").write.format("parquet").save(new Path(basePath, "first").toString)
        Seq(1).toDF("a").write.format("parquet").save(new Path(basePath, "second").toString)
        val df = spark.read.parquet(
          new Path(basePath, "first").toString,
          new Path(basePath, "second").toString)
        val e = intercept[SparkException] {
          df.collect()
        }
        assert(e.getCause.isInstanceOf[OutOfMemoryError])
      }
    }
  }
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20619 from dongjoon-hyun/SPARK-23390.
2018-02-20 13:33:03 +08:00
Dongjoon Hyun 3ee3b2ae1f [SPARK-23340][SQL] Upgrade Apache ORC to 1.4.3
## What changes were proposed in this pull request?

This PR updates Apache ORC dependencies to 1.4.3 released on February 9th. Apache ORC 1.4.2 release removes unnecessary dependencies and 1.4.3 has 5 more patches (https://s.apache.org/Fll8).

Especially, the following ORC-285 is fixed at 1.4.3.

```scala
scala> val df = Seq(Array.empty[Float]).toDF()

scala> df.write.format("orc").save("/tmp/floatarray")

scala> spark.read.orc("/tmp/floatarray")
res1: org.apache.spark.sql.DataFrame = [value: array<float>]

scala> spark.read.orc("/tmp/floatarray").show()
18/02/12 22:09:10 ERROR Executor: Exception in task 0.0 in stage 1.0 (TID 1)
java.io.IOException: Error reading file: file:/tmp/floatarray/part-00000-9c0b461b-4df1-4c23-aac1-3e4f349ac7d6-c000.snappy.orc
	at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1191)
	at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78)
...
Caused by: java.io.EOFException: Read past EOF for compressed stream Stream for column 2 kind DATA position: 0 length: 0 range: 0 offset: 0 limit: 0
```

## How was this patch tested?

Pass the Jenkins test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20511 from dongjoon-hyun/SPARK-23340.
2018-02-17 00:25:36 -08:00
Kris Mok 15ad4a7f10 [SPARK-23447][SQL] Cleanup codegen template for Literal
## What changes were proposed in this pull request?

Cleaned up the codegen templates for `Literal`s, to make sure that the `ExprCode` returned from `Literal.doGenCode()` has:
1. an empty `code` field;
2. an `isNull` field of either literal `true` or `false`;
3. a `value` field that is just a simple literal/constant.

Before this PR, there are a couple of paths that would return a non-trivial `code` and all of them are actually unnecessary. The `NaN` and `Infinity` constants for `double` and `float` can be accessed through constants directly available so there's no need to add a reference for them.

Also took the opportunity to add a new util method for ease of creating `ExprCode` for inline-able non-null values.

## How was this patch tested?

Existing tests.

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

Closes #20626 from rednaxelafx/codegen-literal.
2018-02-17 10:54:14 +08:00
Tathagata Das 0a73aa31f4 [SPARK-23362][SS] Migrate Kafka Microbatch source to v2
## What changes were proposed in this pull request?
Migrating KafkaSource (with data source v1) to KafkaMicroBatchReader (with data source v2).

Performance comparison:
In a unit test with in-process Kafka broker, I tested the read throughput of V1 and V2 using 20M records in a single partition. They were comparable.

## How was this patch tested?
Existing tests, few modified to be better tests than the existing ones.

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

Closes #20554 from tdas/SPARK-23362.
2018-02-16 14:30:19 -08:00
Dongjoon Hyun 6968c3cfd7 [MINOR][SQL] Fix an error message about inserting into bucketed tables
## What changes were proposed in this pull request?

This replaces `Sparkcurrently` to `Spark currently` in the following error message.

```scala
scala> sql("insert into t2 select * from v1")
org.apache.spark.sql.AnalysisException: Output Hive table `default`.`t2`
is bucketed but Sparkcurrently does NOT populate bucketed ...
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20617 from dongjoon-hyun/SPARK-ERROR-MSG.
2018-02-15 09:40:08 -08:00
Dongjoon Hyun 2f0498d1e8 [SPARK-23426][SQL] Use hive ORC impl and disable PPD for Spark 2.3.0
## What changes were proposed in this pull request?

To prevent any regressions, this PR changes ORC implementation to `hive` by default like Spark 2.2.X.
Users can enable `native` ORC. Also, ORC PPD is also restored to `false` like Spark 2.2.X.

![orc_section](https://user-images.githubusercontent.com/9700541/36221575-57a1d702-1173-11e8-89fe-dca5842f4ca7.png)

## How was this patch tested?

Pass all test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20610 from dongjoon-hyun/SPARK-ORC-DISABLE.
2018-02-15 08:55:39 -08:00
hyukjinkwon ed86476098 [SPARK-23359][SQL] Adds an alias 'names' of 'fieldNames' in Scala's StructType
## What changes were proposed in this pull request?

This PR proposes to add an alias 'names' of  'fieldNames' in Scala. Please see the discussion in [SPARK-20090](https://issues.apache.org/jira/browse/SPARK-20090).

## How was this patch tested?

Unit tests added in `DataTypeSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20545 from HyukjinKwon/SPARK-23359.
2018-02-15 17:13:05 +08:00
Wenchen Fan f38c760638 [SPARK-23419][SPARK-23416][SS] data source v2 write path should re-throw interruption exceptions directly
## What changes were proposed in this pull request?

Streaming execution has a list of exceptions that means interruption, and handle them specially. `WriteToDataSourceV2Exec` should also respect this list and not wrap them with `SparkException`.

## How was this patch tested?

existing test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20605 from cloud-fan/write.
2018-02-15 16:59:44 +08:00
gatorsmile 95e4b49160 [SPARK-23094] Revert [] Fix invalid character handling in JsonDataSource
## What changes were proposed in this pull request?
This PR is to revert the PR https://github.com/apache/spark/pull/20302, because it causes a regression.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20614 from gatorsmile/revertJsonFix.
2018-02-14 23:56:02 -08:00
gatorsmile a77ebb0921 [SPARK-23421][SPARK-22356][SQL] Document the behavior change in
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/19579 introduces a behavior change. We need to document it in the migration guide.

## How was this patch tested?
Also update the HiveExternalCatalogVersionsSuite to verify it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20606 from gatorsmile/addMigrationGuide.
2018-02-14 23:52:59 -08:00
Tathagata Das 658d9d9d78 [SPARK-23406][SS] Enable stream-stream self-joins
## What changes were proposed in this pull request?

Solved two bugs to enable stream-stream self joins.

### Incorrect analysis due to missing MultiInstanceRelation trait
Streaming leaf nodes did not extend MultiInstanceRelation, which is necessary for the catalyst analyzer to convert the self-join logical plan DAG into a tree (by creating new instances of the leaf relations). This was causing the error `Failure when resolving conflicting references in Join:` (see JIRA for details).

### Incorrect attribute rewrite when splicing batch plans in MicroBatchExecution
When splicing the source's batch plan into the streaming plan (by replacing the StreamingExecutionPlan), we were rewriting the attribute reference in the streaming plan with the new attribute references from the batch plan. This was incorrectly handling the scenario when multiple StreamingExecutionRelation point to the same source, and therefore eventually point to the same batch plan returned by the source. Here is an example query, and its corresponding plan transformations.
```
val df = input.toDF
val join =
      df.select('value % 5 as "key", 'value).join(
        df.select('value % 5 as "key", 'value), "key")
```
Streaming logical plan before splicing the batch plan
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- StreamingExecutionRelation Memory[#1], value#12  // two different leaves pointing to same source
```
Batch logical plan after splicing the batch plan and before rewriting
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- LocalRelation [value#66]           // replaces StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- LocalRelation [value#66]           // replaces StreamingExecutionRelation Memory[#1], value#12
```
Batch logical plan after rewriting the attributes. Specifically, for spliced, the new output attributes (value#66) replace the earlier output attributes (value#12, and value#1, one for each StreamingExecutionRelation).
```
Project [key#6, value#66, value#66]       // both value#1 and value#12 replaces by value#66
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9, value#66]
      +- LocalRelation [value#66]
```
This causes the optimizer to eliminate value#66 from one side of the join.
```
Project [key#6, value#66, value#66]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9]   // this does not generate value, incorrect join results
      +- LocalRelation [value#66]
```

**Solution**: Instead of rewriting attributes, use a Project to introduce aliases between the output attribute references and the new reference generated by the spliced plans. The analyzer and optimizer will take care of the rest.
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- Project [value#66 AS value#1]   // solution: project with aliases
   :     +- LocalRelation [value#66]
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- Project [value#66 AS value#12]    // solution: project with aliases
         +- LocalRelation [value#66]
```

## How was this patch tested?
New unit test

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

Closes #20598 from tdas/SPARK-23406.
2018-02-14 14:27:02 -08:00
gatorsmile 400a1d9e25 Revert "[SPARK-23249][SQL] Improved block merging logic for partitions"
This reverts commit 8c21170dec.
2018-02-14 10:57:12 -08:00
Dongjoon Hyun 357babde5a [SPARK-23399][SQL] Register a task completion listener first for OrcColumnarBatchReader
## What changes were proposed in this pull request?

This PR aims to resolve an open file leakage issue reported at [SPARK-23390](https://issues.apache.org/jira/browse/SPARK-23390) by moving the listener registration position. Currently, the sequence is like the following.

1. Create `batchReader`
2. `batchReader.initialize` opens a ORC file.
3. `batchReader.initBatch` may take a long time to alloc memory in some environment and cause errors.
4. `Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close()))`

This PR moves 4 before 2 and 3. To sum up, the new sequence is 1 -> 4 -> 2 -> 3.

## How was this patch tested?

Manual. The following test case makes OOM intentionally to cause leaked filesystem connection in the current code base. With this patch, leakage doesn't occurs.

```scala
  // This should be tested manually because it raises OOM intentionally
  // in order to cause `Leaked filesystem connection`.
  test("SPARK-23399 Register a task completion listener first for OrcColumnarBatchReader") {
    withSQLConf(SQLConf.ORC_VECTORIZED_READER_BATCH_SIZE.key -> s"${Int.MaxValue}") {
      withTempDir { dir =>
        val basePath = dir.getCanonicalPath
        Seq(0).toDF("a").write.format("orc").save(new Path(basePath, "first").toString)
        Seq(1).toDF("a").write.format("orc").save(new Path(basePath, "second").toString)
        val df = spark.read.orc(
          new Path(basePath, "first").toString,
          new Path(basePath, "second").toString)
        val e = intercept[SparkException] {
          df.collect()
        }
        assert(e.getCause.isInstanceOf[OutOfMemoryError])
      }
    }
  }
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20590 from dongjoon-hyun/SPARK-23399.
2018-02-14 10:55:24 +08:00
gatorsmile d6f5e172b4 Revert "[SPARK-23303][SQL] improve the explain result for data source v2 relations"
This reverts commit f17b936f0d.
2018-02-13 16:21:17 -08:00
gatorsmile 2ee76c22b6 [SPARK-23400][SQL] Add a constructors for ScalaUDF
## What changes were proposed in this pull request?

In this upcoming 2.3 release, we changed the interface of `ScalaUDF`. Unfortunately, some Spark packages (e.g., spark-deep-learning) are using our internal class `ScalaUDF`. In the release 2.3, we added new parameters into this class. The users hit the binary compatibility issues and got the exception:

```
> java.lang.NoSuchMethodError: org.apache.spark.sql.catalyst.expressions.ScalaUDF.&lt;init&gt;(Ljava/lang/Object;Lorg/apache/spark/sql/types/DataType;Lscala/collection/Seq;Lscala/collection/Seq;Lscala/Option;)V
```

This PR is to improve the backward compatibility. However, we definitely should not encourage the external packages to use our internal classes. This might make us hard to maintain/develop the codes in Spark.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20591 from gatorsmile/scalaUDF.
2018-02-13 11:56:49 -08:00
Bogdan Raducanu 05d051293f [SPARK-23316][SQL] AnalysisException after max iteration reached for IN query
## What changes were proposed in this pull request?
Added flag ignoreNullability to DataType.equalsStructurally.
The previous semantic is for ignoreNullability=false.
When ignoreNullability=true equalsStructurally ignores nullability of contained types (map key types, value types, array element types, structure field types).
In.checkInputTypes calls equalsStructurally to check if the children types match. They should match regardless of nullability (which is just a hint), so it is now called with ignoreNullability=true.

## How was this patch tested?
New test in SubquerySuite

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #20548 from bogdanrdc/SPARK-23316.
2018-02-13 09:49:52 -08:00
Wenchen Fan f17b936f0d [SPARK-23303][SQL] improve the explain result for data source v2 relations
## What changes were proposed in this pull request?

The current explain result for data source v2 relation is unreadable:
```
== Parsed Logical Plan ==
'Filter ('i > 6)
+- AnalysisBarrier
      +- Project [j#1]
         +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Filter (i#0 > 6)
   +- Project [j#1, i#0]
      +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Optimized Logical Plan ==
Project [j#1]
+- Filter isnotnull(i#0)
   +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Physical Plan ==
*(1) Project [j#1]
+- *(1) Filter isnotnull(i#0)
   +- *(1) DataSourceV2Scan [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940
```

after this PR
```
== Parsed Logical Plan ==
'Project [unresolvedalias('j, None)]
+- AnalysisBarrier
      +- Relation AdvancedDataSourceV2[i#0, j#1]

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Relation AdvancedDataSourceV2[i#0, j#1]

== Optimized Logical Plan ==
Relation AdvancedDataSourceV2[j#1]

== Physical Plan ==
*(1) Scan AdvancedDataSourceV2[j#1]
```
-------
```
== Analyzed Logical Plan ==
i: int, j: int
Filter (i#88 > 3)
+- Relation JavaAdvancedDataSourceV2[i#88, j#89]

== Optimized Logical Plan ==
Filter isnotnull(i#88)
+- Relation JavaAdvancedDataSourceV2[i#88, j#89] (PushedFilter: [GreaterThan(i,3)])

== Physical Plan ==
*(1) Filter isnotnull(i#88)
+- *(1) Scan JavaAdvancedDataSourceV2[i#88, j#89] (PushedFilter: [GreaterThan(i,3)])
```

an example for streaming query
```
== Parsed Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Analyzed Logical Plan ==
value: string, count(1): bigint
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Optimized Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject value#25.toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Physical Plan ==
*(4) HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#11L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *(3) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5]
         +- *(2) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *(1) HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#16L])
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *(1) MapElements <function1>, obj#5: java.lang.String
                        +- *(1) DeserializeToObject value#25.toString, obj#4: java.lang.String
                           +- *(1) Scan FakeDataSourceV2$[value#25]
```
## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20477 from cloud-fan/explain.
2018-02-12 21:12:22 -08:00
Feng Liu ed4e78bd60 [SPARK-23379][SQL] skip when setting the same current database in HiveClientImpl
## What changes were proposed in this pull request?

If the target database name is as same as the current database, we should be able to skip one metastore access.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20565 from liufengdb/remove-redundant.
2018-02-12 20:57:26 -08:00
Ryan Blue c1bcef876c [SPARK-23323][SQL] Support commit coordinator for DataSourceV2 writes
## What changes were proposed in this pull request?

DataSourceV2 batch writes should use the output commit coordinator if it is required by the data source. This adds a new method, `DataWriterFactory#useCommitCoordinator`, that determines whether the coordinator will be used. If the write factory returns true, `WriteToDataSourceV2` will use the coordinator for batch writes.

## How was this patch tested?

This relies on existing write tests, which now use the commit coordinator.

Author: Ryan Blue <blue@apache.org>

Closes #20490 from rdblue/SPARK-23323-add-commit-coordinator.
2018-02-13 11:40:34 +08:00
sychen 4104b68e95 [SPARK-23230][SQL] When hive.default.fileformat is other kinds of file types, create textfile table cause a serde error
When hive.default.fileformat is other kinds of file types, create textfile table cause a serde error.
We should take the default type of textfile and sequencefile both as org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.

```
set hive.default.fileformat=orc;
create table tbl( i string ) stored as textfile;
desc formatted tbl;

Serde Library org.apache.hadoop.hive.ql.io.orc.OrcSerde
InputFormat  org.apache.hadoop.mapred.TextInputFormat
OutputFormat  org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
```

Author: sychen <sychen@ctrip.com>

Closes #20406 from cxzl25/default_serde.
2018-02-12 16:00:47 -08:00
Feng Liu fba01b9a65 [SPARK-23378][SQL] move setCurrentDatabase from HiveExternalCatalog to HiveClientImpl
## What changes were proposed in this pull request?

This removes the special case that `alterPartitions` call from `HiveExternalCatalog` can reset the current database in the hive client as a side effect.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20564 from liufengdb/move.
2018-02-12 14:58:31 -08:00
Takuya UESHIN 0c66fe4f22 [SPARK-22002][SQL][FOLLOWUP][TEST] Add a test to check if the original schema doesn't have metadata.
## What changes were proposed in this pull request?

This is a follow-up pr of #19231 which modified the behavior to remove metadata from JDBC table schema.
This pr adds a test to check if the schema doesn't have metadata.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20585 from ueshin/issues/SPARK-22002/fup1.
2018-02-12 12:20:29 -08:00
James Thompson 5bb11411ae [SPARK-23388][SQL] Support for Parquet Binary DecimalType in VectorizedColumnReader
## What changes were proposed in this pull request?

Re-add support for parquet binary DecimalType in VectorizedColumnReader

## How was this patch tested?

Existing test suite

Author: James Thompson <jamesthomp@users.noreply.github.com>

Closes #20580 from jamesthomp/jt/add-back-binary-decimal.
2018-02-12 11:34:56 -08:00
liuxian 4a4dd4f36f [SPARK-23391][CORE] It may lead to overflow for some integer multiplication
## What changes were proposed in this pull request?
In the `getBlockData`,`blockId.reduceId` is the `Int` type, when it is greater than 2^28, `blockId.reduceId*8` will overflow
In the `decompress0`, `len` and  `unitSize` are  Int type, so `len * unitSize` may lead to  overflow
## How was this patch tested?
N/A

Author: liuxian <liu.xian3@zte.com.cn>

Closes #20581 from 10110346/overflow2.
2018-02-12 08:49:45 -06:00
Wenchen Fan 0e2c266de7 [SPARK-22977][SQL] fix web UI SQL tab for CTAS
## What changes were proposed in this pull request?

This is a regression in Spark 2.3.

In Spark 2.2, we have a fragile UI support for SQL data writing commands. We only track the input query plan of `FileFormatWriter` and display its metrics. This is not ideal because we don't know who triggered the writing(can be table insertion, CTAS, etc.), but it's still useful to see the metrics of the input query.

In Spark 2.3, we introduced a new mechanism: `DataWritigCommand`, to fix the UI issue entirely. Now these writing commands have real children, and we don't need to hack into the `FileFormatWriter` for the UI. This also helps with `explain`, now `explain` can show the physical plan of the input query, while in 2.2 the physical writing plan is simply `ExecutedCommandExec` and it has no child.

However there is a regression in CTAS. CTAS commands don't extend `DataWritigCommand`, and we don't have the UI hack in `FileFormatWriter` anymore, so the UI for CTAS is just an empty node. See https://issues.apache.org/jira/browse/SPARK-22977 for more information about this UI issue.

To fix it, we should apply the `DataWritigCommand` mechanism to CTAS commands.

TODO: In the future, we should refactor this part and create some physical layer code pieces for data writing, and reuse them in different writing commands. We should have different logical nodes for different operators, even some of them share some same logic, e.g. CTAS, CREATE TABLE, INSERT TABLE. Internally we can share the same physical logic.

## How was this patch tested?

manually tested.
For data source table
<img width="644" alt="1" src="https://user-images.githubusercontent.com/3182036/35874155-bdffab28-0ba6-11e8-94a8-e32e106ba069.png">
For hive table
<img width="666" alt="2" src="https://user-images.githubusercontent.com/3182036/35874161-c437e2a8-0ba6-11e8-98ed-7930f01432c5.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20521 from cloud-fan/UI.
2018-02-12 22:07:59 +08:00
caoxuewen caeb108e25 [MINOR][TEST] spark.testing` No effect on the SparkFunSuite unit test
## What changes were proposed in this pull request?

Currently, we use SBT and MAVN to spark unit test, are affected by the parameters of `spark.testing`. However, when using the IDE test tool, `spark.testing` support is not very good, sometimes need to be manually added to the beforeEach. example: HiveSparkSubmitSuite RPackageUtilsSuite SparkSubmitSuite. The PR unified `spark.testing` parameter extraction to SparkFunSuite, support IDE test tool, and the test code is more compact.

## How was this patch tested?

the existed test cases.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #20582 from heary-cao/sparktesting.
2018-02-12 22:05:27 +08:00
hyukjinkwon c338c8cf82 [SPARK-23352][PYTHON] Explicitly specify supported types in Pandas UDFs
## What changes were proposed in this pull request?

This PR targets to explicitly specify supported types in Pandas UDFs.
The main change here is to add a deduplicated and explicit type checking in `returnType` ahead with documenting this; however, it happened to fix multiple things.

1. Currently, we don't support `BinaryType` in Pandas UDFs, for example, see:

    ```python
    from pyspark.sql.functions import pandas_udf
    pudf = pandas_udf(lambda x: x, "binary")
    df = spark.createDataFrame([[bytearray(1)]])
    df.select(pudf("_1")).show()
    ```
    ```
    ...
    TypeError: Unsupported type in conversion to Arrow: BinaryType
    ```

    We can document this behaviour for its guide.

2. Also, the grouped aggregate Pandas UDF fails fast on `ArrayType` but seems we can support this case.

    ```python
    from pyspark.sql.functions import pandas_udf, PandasUDFType
    foo = pandas_udf(lambda v: v.mean(), 'array<double>', PandasUDFType.GROUPED_AGG)
    df = spark.range(100).selectExpr("id", "array(id) as value")
    df.groupBy("id").agg(foo("value")).show()
    ```

    ```
    ...
     NotImplementedError: ArrayType, StructType and MapType are not supported with PandasUDFType.GROUPED_AGG
    ```

3. Since we can check the return type ahead, we can fail fast before actual execution.

    ```python
    # we can fail fast at this stage because we know the schema ahead
    pandas_udf(lambda x: x, BinaryType())
    ```

## How was this patch tested?

Manually tested and unit tests for `BinaryType` and `ArrayType(...)` were added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20531 from HyukjinKwon/pudf-cleanup.
2018-02-12 20:49:36 +09:00
Wenchen Fan 6efd5d117e [SPARK-23390][SQL] Flaky Test Suite: FileBasedDataSourceSuite in Spark 2.3/hadoop 2.7
## What changes were proposed in this pull request?

This test only fails with sbt on Hadoop 2.7, I can't reproduce it locally, but here is my speculation by looking at the code:
1. FileSystem.delete doesn't delete the directory entirely, somehow we can still open the file as a 0-length empty file.(just speculation)
2. ORC intentionally allow empty files, and the reader fails during reading without closing the file stream.

This PR improves the test to make sure all files are deleted and can't be opened.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20584 from cloud-fan/flaky-test.
2018-02-11 23:46:23 -08:00
Wenchen Fan 4bbd7443eb [SPARK-23376][SQL] creating UnsafeKVExternalSorter with BytesToBytesMap may fail
## What changes were proposed in this pull request?

This is a long-standing bug in `UnsafeKVExternalSorter` and was reported in the dev list multiple times.

When creating `UnsafeKVExternalSorter` with `BytesToBytesMap`, we need to create a `UnsafeInMemorySorter` to sort the data in `BytesToBytesMap`. The data format of the sorter and the map is same, so no data movement is required. However, both the sorter and the map need a point array for some bookkeeping work.

There is an optimization in `UnsafeKVExternalSorter`: reuse the point array between the sorter and the map, to avoid an extra memory allocation. This sounds like a reasonable optimization, the length of the `BytesToBytesMap` point array is at least 4 times larger than the number of keys(to avoid hash collision, the hash table size should be at least 2 times larger than the number of keys, and each key occupies 2 slots). `UnsafeInMemorySorter` needs the pointer array size to be 4 times of the number of entries, so we are safe to reuse the point array.

However, the number of keys of the map doesn't equal to the number of entries in the map, because `BytesToBytesMap` supports duplicated keys. This breaks the assumption of the above optimization and we may run out of space when inserting data into the sorter, and hit error
```
java.lang.IllegalStateException: There is no space for new record
   at org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.insertRecord(UnsafeInMemorySorter.java:239)
   at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:149)
...
```

This PR fixes this bug by creating a new point array if the existing one is not big enough.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20561 from cloud-fan/bug.
2018-02-12 00:03:49 +08:00
Feng Liu 6d7c38330e [SPARK-23275][SQL] fix the thread leaking in hive/tests
## What changes were proposed in this pull request?

This is a follow up of https://github.com/apache/spark/pull/20441.

The two lines actually can trigger the hive metastore bug: https://issues.apache.org/jira/browse/HIVE-16844

The two configs are not in the default `ObjectStore` properties, so any run hive commands after these two lines will set the `propsChanged` flag in the `ObjectStore.setConf` and then cause thread leaks.

I don't think the two lines are very useful. They can be removed safely.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20562 from liufengdb/fix-omm.
2018-02-09 16:21:47 -08:00
Jacek Laskowski 557938e283 [MINOR][HIVE] Typo fixes
## What changes were proposed in this pull request?

Typo fixes (with expanding a Hive property)

## How was this patch tested?

local build. Awaiting Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #20550 from jaceklaskowski/hiveutils-typos.
2018-02-09 18:18:30 -06:00
Dongjoon Hyun 8cbcc33876 [SPARK-23186][SQL] Initialize DriverManager first before loading JDBC Drivers
## What changes were proposed in this pull request?

Since some JDBC Drivers have class initialization code to call `DriverManager`, we need to initialize `DriverManager` first in order to avoid potential executor-side **deadlock** situations like the following (or [STORM-2527](https://issues.apache.org/jira/browse/STORM-2527)).

```
Thread 9587: (state = BLOCKED)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame; information may be imprecise)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - java.util.ServiceLoader$LazyIterator.nextService() bci=119, line=380 (Interpreted frame)
 - java.util.ServiceLoader$LazyIterator.next() bci=11, line=404 (Interpreted frame)
 - java.util.ServiceLoader$1.next() bci=37, line=480 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=21, line=603 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=1, line=583 (Interpreted frame)
 - java.security.AccessController.doPrivileged(java.security.PrivilegedAction) bci=0 (Compiled frame)
 - java.sql.DriverManager.loadInitialDrivers() bci=27, line=583 (Interpreted frame)
 - java.sql.DriverManager.<clinit>() bci=32, line=101 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getConnection(java.lang.String, java.lang.Integer, java.lang.String, java.util.Properties) bci=12, line=98 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getInputConnection(org.apache.hadoop.conf.Configuration, java.util.Properties) bci=22, line=57 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.getQueryPlan(org.apache.hadoop.mapreduce.JobContext, org.apache.hadoop.conf.Configuration) bci=61, line=116 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.createRecordReader(org.apache.hadoop.mapreduce.InputSplit, org.apache.hadoop.mapreduce.TaskAttemptContext) bci=10, line=71 (Interpreted frame)
 - org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(org.apache.spark.rdd.NewHadoopRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=233, line=156 (Interpreted frame)

Thread 9170: (state = BLOCKED)
 - org.apache.phoenix.jdbc.PhoenixDriver.<clinit>() bci=35, line=125 (Interpreted frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - org.apache.spark.sql.execution.datasources.jdbc.DriverRegistry$.register(java.lang.String) bci=89, line=46 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=7, line=53 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=1, line=52 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD$$anon$1.<init>(org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=81, line=347 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD.compute(org.apache.spark.Partition, org.apache.spark.TaskContext) bci=7, line=339 (Interpreted frame)
```

## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20359 from dongjoon-hyun/SPARK-23186.
2018-02-09 12:54:57 +08:00
Wenchen Fan a75f927173 [SPARK-23268][SQL][FOLLOWUP] Reorganize packages in data source V2
## What changes were proposed in this pull request?

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

While reorganizing the packages for streaming data source v2, the top level stream read/write support interfaces should not be in the reader/writer package, but should be in the `sources.v2` package, to follow the `ReadSupport`, `WriteSupport`, etc.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20509 from cloud-fan/followup.
2018-02-08 19:20:11 +08:00
Wenchen Fan 7f5f5fb129 [SPARK-23348][SQL] append data using saveAsTable should adjust the data types
## What changes were proposed in this pull request?

For inserting/appending data to an existing table, Spark should adjust the data types of the input query according to the table schema, or fail fast if it's uncastable.

There are several ways to insert/append data: SQL API, `DataFrameWriter.insertInto`, `DataFrameWriter.saveAsTable`. The first 2 ways create `InsertIntoTable` plan, and the last way creates `CreateTable` plan. However, we only adjust input query data types for `InsertIntoTable`, and users may hit weird errors when appending data using `saveAsTable`. See the JIRA for the error case.

This PR fixes this bug by adjusting data types for `CreateTable` too.

## How was this patch tested?

new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20527 from cloud-fan/saveAsTable.
2018-02-08 00:08:54 -08:00
gatorsmile 3473fda6dc Revert [SPARK-22279][SQL] Turn on spark.sql.hive.convertMetastoreOrc by default
## What changes were proposed in this pull request?

This is to revert the changes made in https://github.com/apache/spark/pull/19499 , because this causes a regression. We should not ignore the table-specific compression conf when the Hive serde tables are converted to the data source tables.

## How was this patch tested?

The existing tests.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20536 from gatorsmile/revert22279.
2018-02-08 12:21:18 +08:00
Tathagata Das 30295bf5a6 [SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs
## What changes were proposed in this pull request?

This PR migrates the MemoryStream to DataSourceV2 APIs.

One additional change is in the reported keys in StreamingQueryProgress.durationMs. "getOffset" and "getBatch" replaced with "setOffsetRange" and "getEndOffset" as tracking these make more sense. Unit tests changed accordingly.

## How was this patch tested?
Existing unit tests, few updated unit tests.

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

Closes #20445 from tdas/SPARK-23092.
2018-02-07 15:22:53 -08:00
Liang-Chi Hsieh 9841ae0313 [SPARK-23345][SQL] Remove open stream record even closing it fails
## What changes were proposed in this pull request?

When `DebugFilesystem` closes opened stream, if any exception occurs, we still need to remove the open stream record from `DebugFilesystem`. Otherwise, it goes to report leaked filesystem connection.

## How was this patch tested?

Existing tests.

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

Closes #20524 from viirya/SPARK-23345.
2018-02-07 09:48:49 -08:00
gatorsmile 9775df67f9 [SPARK-23122][PYSPARK][FOLLOWUP] Replace registerTempTable by createOrReplaceTempView
## What changes were proposed in this pull request?
Replace `registerTempTable` by `createOrReplaceTempView`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20523 from gatorsmile/updateExamples.
2018-02-07 23:24:16 +09:00
gatorsmile c36fecc3b4 [SPARK-23327][SQL] Update the description and tests of three external API or functions
## What changes were proposed in this pull request?
Update the description and tests of three external API or functions `createFunction `, `length` and `repartitionByRange `

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20495 from gatorsmile/updateFunc.
2018-02-06 16:46:43 -08:00
Wenchen Fan b96a083b1c [SPARK-23315][SQL] failed to get output from canonicalized data source v2 related plans
## What changes were proposed in this pull request?

`DataSourceV2Relation`  keeps a `fullOutput` and resolves the real output on demand by column name lookup. i.e.
```
lazy val output: Seq[Attribute] = reader.readSchema().map(_.name).map { name =>
  fullOutput.find(_.name == name).get
}
```

This will be broken after we canonicalize the plan, because all attribute names become "None", see https://github.com/apache/spark/blob/v2.3.0-rc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala#L42

To fix this, `DataSourceV2Relation` should just keep `output`, and update the `output` when doing column pruning.

## How was this patch tested?

a new test case

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20485 from cloud-fan/canonicalize.
2018-02-06 12:43:45 -08:00
Wenchen Fan ac7454cac0 [SPARK-23312][SQL][FOLLOWUP] add a config to turn off vectorized cache reader
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/20483 tried to provide a way to turn off the new columnar cache reader, to restore the behavior in 2.2. However even we turn off that config, the behavior is still different than 2.2.

If the output data are rows, we still enable whole stage codegen for the scan node, which is different with 2.2, we should also fix it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20513 from cloud-fan/cache.
2018-02-06 12:27:37 -08:00
Xingbo Jiang c2766b07b4 [SPARK-23330][WEBUI] Spark UI SQL executions page throws NPE
## What changes were proposed in this pull request?

Spark SQL executions page throws the following error and the page crashes:
```
HTTP ERROR 500
Problem accessing /SQL/. Reason:

Server Error
Caused by:
java.lang.NullPointerException
at scala.collection.immutable.StringOps$.length$extension(StringOps.scala:47)
at scala.collection.immutable.StringOps.length(StringOps.scala:47)
at scala.collection.IndexedSeqOptimized$class.isEmpty(IndexedSeqOptimized.scala:27)
at scala.collection.immutable.StringOps.isEmpty(StringOps.scala:29)
at scala.collection.TraversableOnce$class.nonEmpty(TraversableOnce.scala:111)
at scala.collection.immutable.StringOps.nonEmpty(StringOps.scala:29)
at org.apache.spark.sql.execution.ui.ExecutionTable.descriptionCell(AllExecutionsPage.scala:182)
at org.apache.spark.sql.execution.ui.ExecutionTable.row(AllExecutionsPage.scala:155)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at org.apache.spark.ui.UIUtils$.listingTable(UIUtils.scala:339)
at org.apache.spark.sql.execution.ui.ExecutionTable.toNodeSeq(AllExecutionsPage.scala:203)
at org.apache.spark.sql.execution.ui.AllExecutionsPage.render(AllExecutionsPage.scala:67)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)
at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)
at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)
at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:213)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
at org.eclipse.jetty.server.Server.handle(Server.java:534)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
at java.lang.Thread.run(Thread.java:748)
```

One of the possible reason that this page fails may be the `SparkListenerSQLExecutionStart` event get dropped before processed, so the execution description and details don't get updated.
This was not a issue in 2.2 because it would ignore any job start event that arrives before the corresponding execution start event, which doesn't sound like a good decision.

We shall try to handle the null values in the front page side, that is, try to give a default value when `execution.details` or `execution.description` is null.
Another possible approach is not to spill the `LiveExecutionData` in `SQLAppStatusListener.update(exec: LiveExecutionData)` if `exec.details` is null. This is not ideal because this way you will not see the execution if `SparkListenerSQLExecutionStart` event is lost, because `AllExecutionsPage` only read executions from KVStore.

## How was this patch tested?

After the change, the page shows the following:
![image](https://user-images.githubusercontent.com/4784782/35775480-28cc5fde-093e-11e8-8ccc-f58c2ef4a514.png)

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

Closes #20502 from jiangxb1987/executionPage.
2018-02-05 14:17:11 -08:00
Shixiong Zhu a6bf3db207 [SPARK-23307][WEBUI] Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them
## What changes were proposed in this pull request?

Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them to make the behavior consistent with 2.2.

## How was this patch tested?

- Jenkins.
- Manually ran the following codes and checked the UI for jobs/stages/tasks/queries.

```
spark.ui.retainedJobs 10
spark.ui.retainedStages 10
spark.sql.ui.retainedExecutions 10
spark.ui.retainedTasks 10
```

```
new Thread() {
  override def run() {
    spark.range(1, 2).foreach { i =>
        Thread.sleep(10000)
    }
  }
}.start()

Thread.sleep(5000)

for (_ <- 1 to 20) {
    new Thread() {
      override def run() {
        spark.range(1, 2).foreach { i =>
        }
      }
    }.start()
}

Thread.sleep(15000)
  spark.range(1, 2).foreach { i =>
}

sc.makeRDD(1 to 100, 100).foreach { i =>
}
```

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20481 from zsxwing/SPARK-23307.
2018-02-05 18:41:49 +08:00
Yuming Wang 6fb3fd1536 [SPARK-22036][SQL][FOLLOWUP] Fix decimalArithmeticOperations.sql
## What changes were proposed in this pull request?

Fix decimalArithmeticOperations.sql test

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>
Author: wangyum <wgyumg@gmail.com>
Author: Yuming Wang <yumwang@ebay.com>

Closes #20498 from wangyum/SPARK-22036.
2018-02-04 09:15:48 -08:00
Dongjoon Hyun 522e0b1866 [SPARK-23305][SQL][TEST] Test spark.sql.files.ignoreMissingFiles for all file-based data sources
## What changes were proposed in this pull request?

Like Parquet, all file-based data source handles `spark.sql.files.ignoreMissingFiles` correctly. We had better have a test coverage for feature parity and in order to prevent future accidental regression for all data sources.

## How was this patch tested?

Pass Jenkins with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20479 from dongjoon-hyun/SPARK-23305.
2018-02-03 00:04:00 -08:00
caoxuewen 63b49fa2e5 [SPARK-23311][SQL][TEST] add FilterFunction test case for test CombineTypedFilters
## What changes were proposed in this pull request?

In the current test case for CombineTypedFilters, we lack the test of FilterFunction, so let's add it.
In addition, in TypedFilterOptimizationSuite's existing test cases, Let's extract a common LocalRelation.

## How was this patch tested?

add new test cases.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #20482 from heary-cao/TypedFilterOptimizationSuite.
2018-02-03 00:02:03 -08:00
Wenchen Fan fe73cb4b43 [SPARK-23317][SQL] rename ContinuousReader.setOffset to setStartOffset
## What changes were proposed in this pull request?

In the document of `ContinuousReader.setOffset`, we say this method is used to specify the start offset. We also have a `ContinuousReader.getStartOffset` to get the value back. I think it makes more sense to rename `ContinuousReader.setOffset` to `setStartOffset`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20486 from cloud-fan/rename.
2018-02-02 20:49:08 -08:00
Reynold Xin 3ff83ad43a [SQL] Minor doc update: Add an example in DataFrameReader.schema
## What changes were proposed in this pull request?
This patch adds a small example to the schema string definition of schema function. It isn't obvious how to use it, so an example would be useful.

## How was this patch tested?
N/A - doc only.

Author: Reynold Xin <rxin@databricks.com>

Closes #20491 from rxin/schema-doc.
2018-02-02 20:36:27 -08:00
Wenchen Fan b9503fcbb3 [SPARK-23312][SQL] add a config to turn off vectorized cache reader
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-23309 reported a performance regression about cached table in Spark 2.3. While the investigating is still going on, this PR adds a conf to turn off the vectorized cache reader, to unblock the 2.3 release.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20483 from cloud-fan/cache.
2018-02-02 22:43:28 +08:00
Wenchen Fan 19c7c7ebde [SPARK-23301][SQL] data source column pruning should work for arbitrary expressions
## What changes were proposed in this pull request?

This PR fixes a mistake in the `PushDownOperatorsToDataSource` rule, the column pruning logic is incorrect about `Project`.

## How was this patch tested?

a new test case for column pruning with arbitrary expressions, and improve the existing tests to make sure the `PushDownOperatorsToDataSource` really works.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20476 from cloud-fan/push-down.
2018-02-01 20:44:46 -08:00
Liang-Chi Hsieh 90848d5074 [SPARK-23284][SQL] Document the behavior of several ColumnVector's get APIs when accessing null slot
## What changes were proposed in this pull request?

For some ColumnVector get APIs such as getDecimal, getBinary, getStruct, getArray, getInterval, getUTF8String, we should clearly document their behaviors when accessing null slot. They should return null in this case. Then we can remove null checks from the places using above APIs.

For the APIs of primitive values like getInt, getInts, etc., this also documents their behaviors when accessing null slots. Their returning values are undefined and can be anything.

## How was this patch tested?

Added tests into `ColumnarBatchSuite`.

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

Closes #20455 from viirya/SPARK-23272-followup.
2018-02-02 10:18:32 +08:00
Wenchen Fan 73da3b6968 [SPARK-23293][SQL] fix data source v2 self join
## What changes were proposed in this pull request?

`DataSourceV2Relation` should extend `MultiInstanceRelation`, to take care of self-join.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20466 from cloud-fan/dsv2-selfjoin.
2018-02-01 10:48:34 -08:00
Yuming Wang f051f83403 [SPARK-13983][SQL] Fix HiveThriftServer2 can not get "--hiveconf" and ''--hivevar" variables since 2.0
## What changes were proposed in this pull request?

`--hiveconf` and `--hivevar` variables no longer work since Spark 2.0. The `spark-sql` client has fixed by [SPARK-15730](https://issues.apache.org/jira/browse/SPARK-15730) and [SPARK-18086](https://issues.apache.org/jira/browse/SPARK-18086). but `beeline`/[`Spark SQL HiveThriftServer2`](https://github.com/apache/spark/blob/v2.1.1/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala) is still broken. This pull request fix it.

This pull request works for both `JDBC client` and `beeline`.

## How was this patch tested?

unit tests for  `JDBC client`
manual tests for `beeline`:
```
git checkout origin/pr/17886

dev/make-distribution.sh --mvn mvn  --tgz -Phive -Phive-thriftserver -Phadoop-2.6 -DskipTests

tar -zxf spark-2.3.0-SNAPSHOT-bin-2.6.5.tgz && cd spark-2.3.0-SNAPSHOT-bin-2.6.5

sbin/start-thriftserver.sh
```
```
cat <<EOF > test.sql
select '\${a}', '\${b}';
EOF

beeline -u jdbc:hive2://localhost:10000 --hiveconf a=avalue --hivevar b=bvalue -f test.sql

```

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17886 from wangyum/SPARK-13983-dev.
2018-02-01 10:36:31 -08:00
Wang Gengliang ffbca84519 [SPARK-23202][SQL] Add new API in DataSourceWriter: onDataWriterCommit
## What changes were proposed in this pull request?

The current DataSourceWriter API makes it hard to implement `onTaskCommit(taskCommit: TaskCommitMessage)` in `FileCommitProtocol`.
In general, on receiving commit message, driver can start processing messages(e.g. persist messages into files) before all the messages are collected.

The proposal to add a new API:
`add(WriterCommitMessage message)`:  Handles a commit message on receiving from a successful data writer.

This should make the whole API of DataSourceWriter compatible with `FileCommitProtocol`, and more flexible.

There was another radical attempt in #20386.  This one should be more reasonable.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20454 from gengliangwang/write_api.
2018-02-01 20:39:15 +08:00
Takuya UESHIN 89e8d556b9 [SPARK-23280][SQL][FOLLOWUP] Enable MutableColumnarRow.getMap().
## What changes were proposed in this pull request?

This is a followup pr of #20450.
We should've enabled `MutableColumnarRow.getMap()` as well.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20471 from ueshin/issues/SPARK-23280/fup2.
2018-02-01 21:28:53 +09:00
Takuya UESHIN 8bb70b068e [SPARK-23280][SQL][FOLLOWUP] Fix Java style check issues.
## What changes were proposed in this pull request?

This is a follow-up of #20450 which broke lint-java checks.
This pr fixes the lint-java issues.

```
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java:[20,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java:[21,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
```

## How was this patch tested?

Checked manually in my local environment.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20468 from ueshin/issues/SPARK-23280/fup1.
2018-02-01 21:25:02 +09:00
Xingbo Jiang b6b50efc85 [SQL][MINOR] Inline SpecifiedWindowFrame.defaultWindowFrame().
## What changes were proposed in this pull request?

SpecifiedWindowFrame.defaultWindowFrame(hasOrderSpecification, acceptWindowFrame) was designed to handle the cases when some Window functions don't support setting a window frame (e.g. rank). However this param is never used.

We may inline the whole of this function to simplify the code.

## How was this patch tested?

Existing tests.

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

Closes #20463 from jiangxb1987/defaultWindowFrame.
2018-01-31 20:59:19 -08:00
Xingbo Jiang cc41245fa3 [SPARK-23188][SQL] Make vectorized columar reader batch size configurable
## What changes were proposed in this pull request?

This PR include the following changes:
- Make the capacity of `VectorizedParquetRecordReader` configurable;
- Make the capacity of `OrcColumnarBatchReader` configurable;
- Update the error message when required capacity in writable columnar vector cannot be fulfilled.

## How was this patch tested?

N/A

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

Closes #20361 from jiangxb1987/vectorCapacity.
2018-02-01 12:56:07 +08:00
Atallah Hezbor b2e7677f4d [SPARK-21396][SQL] Fixes MatchError when UDTs are passed through Hive Thriftserver
Signed-off-by: Atallah Hezbor <atallahhezborgmail.com>

## What changes were proposed in this pull request?

This PR proposes modifying the match statement that gets the columns of a row in HiveThriftServer. There was previously no case for `UserDefinedType`, so querying a table that contained them would throw a match error. The changes catch that case and return the string representation.

## How was this patch tested?

While I would have liked to add a unit test, I couldn't easily incorporate UDTs into the ``HiveThriftServer2Suites`` pipeline. With some guidance I would be happy to push a commit with tests.

Instead I did a manual test by loading a `DataFrame` with Point UDT in a spark shell with a HiveThriftServer. Then in beeline, connecting to the server and querying that table.

Here is the result before the change
```
0: jdbc:hive2://localhost:10000> select * from chicago;
Error: scala.MatchError: org.apache.spark.sql.PointUDT2d980dc3 (of class org.apache.spark.sql.PointUDT) (state=,code=0)

```

And after the change:
```
0: jdbc:hive2://localhost:10000> select * from chicago;
+---------------------------------------+--------------+------------------------+---------------------+--+
|                __fid__                | case_number  |          dtg           |        geom         |
+---------------------------------------+--------------+------------------------+---------------------+--+
| 109602f9-54f8-414b-8c6f-42b1a337643e  | 2            | 2016-01-01 19:00:00.0  | POINT (-77 38)      |
| 709602f9-fcff-4429-8027-55649b6fd7ed  | 1            | 2015-12-31 19:00:00.0  | POINT (-76.5 38.5)  |
| 009602f9-fcb5-45b1-a867-eb8ba10cab40  | 3            | 2016-01-02 19:00:00.0  | POINT (-78 39)      |
+---------------------------------------+--------------+------------------------+---------------------+--+
```

Author: Atallah Hezbor <atallahhezbor@gmail.com>

Closes #20385 from atallahhezbor/udts_over_hive.
2018-01-31 20:45:55 -08:00
Wang Gengliang 56ae32657e [SPARK-23268][SQL] Reorganize packages in data source V2
## What changes were proposed in this pull request?
1. create a new package for partitioning/distribution related classes.
    As Spark will add new concrete implementations of `Distribution` in new releases, it is good to
    have a new package for partitioning/distribution related classes.

2. move streaming related class to package `org.apache.spark.sql.sources.v2.reader/writer.streaming`, instead of `org.apache.spark.sql.sources.v2.streaming.reader/writer`.
So that the there won't be package reader/writer inside package streaming, which is quite confusing.
Before change:
```
v2
├── reader
├── streaming
│   ├── reader
│   └── writer
└── writer
```

After change:
```
v2
├── reader
│   └── streaming
└── writer
    └── streaming
```
## How was this patch tested?
Unit test.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20435 from gengliangwang/new_pkg.
2018-01-31 20:33:51 -08:00
caoxuewen 2ac895be90 [SPARK-23247][SQL] combines Unsafe operations and statistics operations in Scan Data Source
## What changes were proposed in this pull request?

Currently, we scan the execution plan of the data source, first the unsafe operation of each row of data, and then re traverse the data for the count of rows. In terms of performance, this is not necessary. this PR combines the two operations and makes statistics on the number of rows while performing the unsafe operation.

Before modified,

```
val unsafeRow = rdd.mapPartitionsWithIndexInternal { (index, iter) =>
val proj = UnsafeProjection.create(schema)
 proj.initialize(index)
iter.map(proj)
}

val numOutputRows = longMetric("numOutputRows")
unsafeRow.map { r =>
numOutputRows += 1
 r
}
```
After modified,

    val numOutputRows = longMetric("numOutputRows")

    rdd.mapPartitionsWithIndexInternal { (index, iter) =>
      val proj = UnsafeProjection.create(schema)
      proj.initialize(index)
      iter.map( r => {
        numOutputRows += 1
        proj(r)
      })
    }

## How was this patch tested?

the existed test cases.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #20415 from heary-cao/DataSourceScanExec.
2018-02-01 12:05:12 +08:00
Wenchen Fan 52e00f7066 [SPARK-23280][SQL] add map type support to ColumnVector
## What changes were proposed in this pull request?

Fill the last missing piece of `ColumnVector`: the map type support.

The idea is similar to the array type support. A map is basically 2 arrays: keys and values. We ask the implementations to provide a key array, a value array, and an offset and length to specify the range of this map in the key/value array.

In `WritableColumnVector`, we put the key array in first child vector, and value array in second child vector, and offsets and lengths in the current vector, which is very similar to how array type is implemented here.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20450 from cloud-fan/map.
2018-02-01 11:56:06 +08:00
Dilip Biswal 9ff1d96f01 [SPARK-23281][SQL] Query produces results in incorrect order when a composite order by clause refers to both original columns and aliases
## What changes were proposed in this pull request?
Here is the test snippet.
``` SQL
scala> Seq[(Integer, Integer)](
     |         (1, 1),
     |         (1, 3),
     |         (2, 3),
     |         (3, 3),
     |         (4, null),
     |         (5, null)
     |       ).toDF("key", "value").createOrReplaceTempView("src")

scala> sql(
     |         """
     |           |SELECT MAX(value) as value, key as col2
     |           |FROM src
     |           |GROUP BY key
     |           |ORDER BY value desc, key
     |         """.stripMargin).show
+-----+----+
|value|col2|
+-----+----+
|    3|   3|
|    3|   2|
|    3|   1|
| null|   5|
| null|   4|
+-----+----+
```SQL
Here is the explain output :

```SQL
== Parsed Logical Plan ==
'Sort ['value DESC NULLS LAST, 'key ASC NULLS FIRST], true
+- 'Aggregate ['key], ['MAX('value) AS value#9, 'key AS col2#10]
   +- 'UnresolvedRelation `src`

== Analyzed Logical Plan ==
value: int, col2: int
Project [value#9, col2#10]
+- Sort [value#9 DESC NULLS LAST, col2#10 DESC NULLS LAST], true
   +- Aggregate [key#5], [max(value#6) AS value#9, key#5 AS col2#10]
      +- SubqueryAlias src
         +- Project [_1#2 AS key#5, _2#3 AS value#6]
            +- LocalRelation [_1#2, _2#3]
``` SQL
The sort direction is being wrongly changed from ASC to DSC while resolving ```Sort``` in
resolveAggregateFunctions.

The above testcase models TPCDS-Q71 and thus we have the same issue in Q71 as well.

## How was this patch tested?
A few tests are added in SQLQuerySuite.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #20453 from dilipbiswal/local_spark.
2018-01-31 13:52:47 -08:00
Glen Takahashi 8c21170dec [SPARK-23249][SQL] Improved block merging logic for partitions
## What changes were proposed in this pull request?

Change DataSourceScanExec so that when grouping blocks together into partitions, also checks the end of the sorted list of splits to more efficiently fill out partitions.

## How was this patch tested?

Updated old test to reflect the new logic, which causes the # of partitions to drop from 4 -> 3
Also, a current test exists to test large non-splittable files at c575977a59/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala (L346)

## Rationale

The current bin-packing method of next-fit descending for blocks into partitions is sub-optimal in a lot of cases and will result in extra partitions, un-even distribution of block-counts across partitions, and un-even distribution of partition sizes.

As an example, 128 files ranging from 1MB, 2MB,...127MB,128MB. will result in 82 partitions with the current algorithm, but only 64 using this algorithm. Also in this example, the max # of blocks per partition in NFD is 13, while in this algorithm is is 2.

More generally, running a simulation of 1000 runs using 128MB blocksize, between 1-1000 normally distributed file sizes between 1-500Mb, you can see an improvement of approx 5% reduction of partition counts, and a large reduction in standard deviation of blocks per partition.

This algorithm also runs in O(n) time as NFD does, and in every case is strictly better results than NFD.

Overall, the more even distribution of blocks across partitions and therefore reduced partition counts should result in a small but significant performance increase across the board

Author: Glen Takahashi <gtakahashi@palantir.com>

Closes #20372 from glentakahashi/feature/improved-block-merging.
2018-02-01 01:14:01 +08:00
Wenchen Fan 48dd6a4c79 revert [SPARK-22785][SQL] remove ColumnVector.anyNullsSet
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19980 , we thought `anyNullsSet` can be simply implemented by `numNulls() > 0`. This is logically true, but may have performance problems.

`OrcColumnVector` is an example. It doesn't have the `numNulls` property, only has a `noNulls` property. We will lose a lot of performance if we use `numNulls() > 0` to check null.

This PR simply revert #19980, with a renaming to call it `hasNull`. Better name suggestions are welcome, e.g. `nullable`?

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20452 from cloud-fan/null.
2018-02-01 00:24:42 +08:00
Wenchen Fan 695f7146bc [SPARK-23272][SQL] add calendar interval type support to ColumnVector
## What changes were proposed in this pull request?

`ColumnVector` is aimed to support all the data types, but `CalendarIntervalType` is missing. Actually we do support interval type for inner fields, e.g. `ColumnarRow`, `ColumnarArray` both support interval type. It's weird if we don't support interval type at the top level.

This PR adds the interval type support.

This PR also makes `ColumnVector.getChild` protect. We need it public because `MutableColumnaRow.getInterval` needs it. Now the interval implementation is in `ColumnVector.getInterval`.

## How was this patch tested?

a new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20438 from cloud-fan/interval.
2018-01-31 15:13:15 +08:00
jerryshao 8c6a9c90a3 [SPARK-23279][SS] Avoid triggering distributed job for Console sink
## What changes were proposed in this pull request?

Console sink will redistribute collected local data and trigger a distributed job in each batch, this is not necessary, so here change to local job.

## How was this patch tested?

Existing UT and manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #20447 from jerryshao/console-minor.
2018-01-31 13:59:21 +08:00
gatorsmile ca04c3ff23 [SPARK-23274][SQL] Fix ReplaceExceptWithFilter when the right's Filter contains the references that are not in the left output
## What changes were proposed in this pull request?
This PR is to fix the `ReplaceExceptWithFilter` rule when the right's Filter contains the references that are not in the left output.

Before this PR, we got the error like
```
java.util.NoSuchElementException: key not found: a
  at scala.collection.MapLike$class.default(MapLike.scala:228)
  at scala.collection.AbstractMap.default(Map.scala:59)
  at scala.collection.MapLike$class.apply(MapLike.scala:141)
  at scala.collection.AbstractMap.apply(Map.scala:59)
```

After this PR, `ReplaceExceptWithFilter ` will not take an effect in this case.

## How was this patch tested?
Added tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20444 from gatorsmile/fixReplaceExceptWithFilter.
2018-01-30 20:05:57 -08:00
Dongjoon Hyun 7786616733 [SPARK-23276][SQL][TEST] Enable UDT tests in (Hive)OrcHadoopFsRelationSuite
## What changes were proposed in this pull request?

Like Parquet, ORC test suites should enable UDT tests.

## How was this patch tested?

Pass the Jenkins with newly enabled test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20440 from dongjoon-hyun/SPARK-23276.
2018-01-30 17:14:17 -08:00
Dilip Biswal 58fcb5a95e [SPARK-23275][SQL] hive/tests have been failing when run locally on the laptop (Mac) with OOM
## What changes were proposed in this pull request?
hive tests have been failing when they are run locally (Mac Os) after a recent change in the trunk. After running the tests for some time, the test fails with OOM with Error: unable to create new native thread.

I noticed the thread count goes all the way up to 2000+ after which we start getting these OOM errors. Most of the threads seem to be related to the connection pool in hive metastore (BoneCP-xxxxx-xxxx ). This behaviour change is happening after we made the following change to HiveClientImpl.reset()

``` SQL
 def reset(): Unit = withHiveState {
    try {
      // code
    } finally {
      runSqlHive("USE default")  ===> this is causing the issue
    }
```
I am proposing to temporarily back-out part of a fix made to address SPARK-23000 to resolve this issue while we work-out the exact reason for this sudden increase in thread counts.

## How was this patch tested?
Ran hive/test multiple times in different machines.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #20441 from dilipbiswal/hive_tests.
2018-01-30 14:11:06 -08:00
gatorsmile 31c00ad8b0 [SPARK-23267][SQL] Increase spark.sql.codegen.hugeMethodLimit to 65535
## What changes were proposed in this pull request?
Still saw the performance regression introduced by `spark.sql.codegen.hugeMethodLimit` in our internal workloads. There are two major issues in the current solution.
- The size of the complied byte code is not identical to the bytecode size of the method. The detection is still not accurate.
- The bytecode size of a single operator (e.g., `SerializeFromObject`) could still exceed 8K limit. We saw the performance regression in such scenario.

Since it is close to the release of 2.3, we decide to increase it to 64K for avoiding the perf regression.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20434 from gatorsmile/revertConf.
2018-01-30 11:33:30 -08:00
Liang-Chi Hsieh 84bcf9dc88 [SPARK-23222][SQL] Make DataFrameRangeSuite not flaky
## What changes were proposed in this pull request?

It is reported that the test `Cancelling stage in a query with Range` in `DataFrameRangeSuite` fails a few times in unrelated PRs. I personally also saw it too in my PR.

This test is not very flaky actually but only fails occasionally. Based on how the test works, I guess that is because `range` finishes before the listener calls `cancelStage`.

I increase the range number from `1000000000L` to `100000000000L` and count the range in one partition. I also reduce the `interval` of checking stage id. Hopefully it can make the test not flaky anymore.

## How was this patch tested?

The modified tests.

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

Closes #20431 from viirya/SPARK-23222.
2018-01-30 21:00:29 +08:00
gatorsmile 7a2ada223e [SPARK-23261][PYSPARK] Rename Pandas UDFs
## What changes were proposed in this pull request?
Rename the public APIs and names of pandas udfs.

- `PANDAS SCALAR UDF` -> `SCALAR PANDAS UDF`
- `PANDAS GROUP MAP UDF` -> `GROUPED MAP PANDAS UDF`
- `PANDAS GROUP AGG UDF` -> `GROUPED AGG PANDAS UDF`

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20428 from gatorsmile/renamePandasUDFs.
2018-01-30 21:55:55 +09:00
Wenchen Fan 0a9ac0248b [SPARK-23260][SPARK-23262][SQL] several data source v2 naming cleanup
## What changes were proposed in this pull request?

All other classes in the reader/writer package doesn't have `V2` in their names, and the streaming reader/writer don't have `V2` either. It's more consistent to remove `V2` from `DataSourceV2Reader` and `DataSourceVWriter`.

Also rename `DataSourceV2Option` to remote the `V2`, we should only have `V2` in the root interface: `DataSourceV2`.

This PR also fixes some places that the mix-in interface doesn't extend the interface it aimed to mix in.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20427 from cloud-fan/ds-v2.
2018-01-30 19:43:17 +08:00
Henry Robinson 8b983243e4 [SPARK-23157][SQL] Explain restriction on column expression in withColumn()
## What changes were proposed in this pull request?

It's not obvious from the comments that any added column must be a
function of the dataset that we are adding it to. Add a comment to
that effect to Scala, Python and R Data* methods.

Author: Henry Robinson <henry@cloudera.com>

Closes #20429 from henryr/SPARK-23157.
2018-01-29 22:19:59 -08:00
Xingbo Jiang b375397b16 [SPARK-23207][SQL][FOLLOW-UP] Don't perform local sort for DataFrame.repartition(1)
## What changes were proposed in this pull request?

In `ShuffleExchangeExec`, we don't need to insert extra local sort before round-robin partitioning, if the new partitioning has only 1 partition, because under that case all output rows go to the same partition.

## How was this patch tested?

The existing test cases.

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

Closes #20426 from jiangxb1987/repartition1.
2018-01-30 11:40:42 +08:00
Bryan Cutler f235df66a4 [SPARK-22221][SQL][FOLLOWUP] Externalize spark.sql.execution.arrow.maxRecordsPerBatch
## What changes were proposed in this pull request?

This is a followup to #19575 which added a section on setting max Arrow record batches and this will externalize the conf that was referenced in the docs.

## How was this patch tested?
NA

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20423 from BryanCutler/arrow-user-doc-externalize-maxRecordsPerBatch-SPARK-22221.
2018-01-29 17:37:55 -08:00
gatorsmile e30b34f7bd [SPARK-22916][SQL][FOLLOW-UP] Update the Description of Join Selection
## What changes were proposed in this pull request?
This PR is to update the description of the join algorithm changes.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20420 from gatorsmile/followUp22916.
2018-01-29 10:29:42 -08:00
Herman van Hovell 2d903cf9d3 [SPARK-23223][SQL] Make stacking dataset transforms more performant
## What changes were proposed in this pull request?
It is a common pattern to apply multiple transforms to a `Dataset` (using `Dataset.withColumn` for example. This is currently quite expensive because we run `CheckAnalysis` on the full plan and create an encoder for each intermediate `Dataset`.

This PR extends the usage of the `AnalysisBarrier` to include `CheckAnalysis`. By doing this we hide the already analyzed plan  from `CheckAnalysis` because barrier is a `LeafNode`. The `AnalysisBarrier` is in the `FinishAnalysis` phase of the optimizer.

We also make binding the `Dataset` encoder lazy. The bound encoder is only needed when we materialize the dataset.

## How was this patch tested?
Existing test should cover this.

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

Closes #20402 from hvanhovell/SPARK-23223.
2018-01-29 09:00:54 -08:00
xubo245 fbce2ed0fa [SPARK-23059][SQL][TEST] Correct some improper with view related method usage
## What changes were proposed in this pull request?

Correct some improper with view related method usage
Only change test cases

like:

```
 test("list global temp views") {
    try {
      sql("CREATE GLOBAL TEMP VIEW v1 AS SELECT 3, 4")
      sql("CREATE TEMP VIEW v2 AS SELECT 1, 2")

      checkAnswer(sql(s"SHOW TABLES IN $globalTempDB"),
        Row(globalTempDB, "v1", true) ::
        Row("", "v2", true) :: Nil)

      assert(spark.catalog.listTables(globalTempDB).collect().toSeq.map(_.name) == Seq("v1", "v2"))
    } finally {
      spark.catalog.dropTempView("v1")
      spark.catalog.dropGlobalTempView("v2")
    }
  }
```

other change please review the code.
## How was this patch tested?

See test case.

Author: xubo245 <601450868@qq.com>

Closes #20250 from xubo245/DropTempViewError.
2018-01-29 08:58:14 -08:00
caoxuewen 54dd7cf4ef [SPARK-23199][SQL] improved Removes repetition from group expressions in Aggregate
## What changes were proposed in this pull request?

Currently, all Aggregate operations will go into RemoveRepetitionFromGroupExpressions, but there is no group expression or there is no duplicate group expression in group expression, we not need copy for logic plan.

## How was this patch tested?

the existed test case.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #20375 from heary-cao/RepetitionGroupExpressions.
2018-01-29 08:56:42 -08:00
Wang Gengliang badf0d0e0d [SPARK-23219][SQL] Rename ReadTask to DataReaderFactory in data source v2
## What changes were proposed in this pull request?

Currently we have `ReadTask` in data source v2 reader, while in writer we have `DataWriterFactory`.
To make the naming consistent and better, renaming `ReadTask` to `DataReaderFactory`.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20397 from gengliangwang/rename.
2018-01-30 00:50:49 +08:00
hyukjinkwon 39d2c6b034 [SPARK-23238][SQL] Externalize SQLConf configurations exposed in documentation
## What changes were proposed in this pull request?

This PR proposes to expose few internal configurations found in the documentation.

Also it fixes the description for `spark.sql.execution.arrow.enabled`.
It's quite self-explanatory.

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20403 from HyukjinKwon/minor-doc-arrow.
2018-01-29 21:09:05 +09:00
Jose Torres 49b0207dc9 [SPARK-23196] Unify continuous and microbatch V2 sinks
## What changes were proposed in this pull request?

Replace streaming V2 sinks with a unified StreamWriteSupport interface, with a shim to use it with microbatch execution.

Add a new SQL config to use for disabling V2 sinks, falling back to the V1 sink implementation.

## How was this patch tested?

Existing tests, which in the case of Kafka (the only existing continuous V2 sink) now use V2 for microbatch.

Author: Jose Torres <jose@databricks.com>

Closes #20369 from jose-torres/streaming-sink.
2018-01-29 13:10:38 +08:00
CCInCharge 686a622c93 [SPARK-23250][DOCS] Typo in JavaDoc/ScalaDoc for DataFrameWriter
## What changes were proposed in this pull request?

Fix typo in ScalaDoc for DataFrameWriter - originally stated "This is applicable for all file-based data sources (e.g. Parquet, JSON) staring Spark 2.1.0", should be "starting with Spark 2.1.0".

## How was this patch tested?

Check of correct spelling in ScalaDoc

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

Author: CCInCharge <charles.l.chen.clc@gmail.com>

Closes #20417 from CCInCharge/master.
2018-01-28 14:55:43 -06:00
Jose Torres 6328868e52 [SPARK-23245][SS][TESTS] Don't access lastExecution.executedPlan in StreamTest
## What changes were proposed in this pull request?

`lastExecution.executedPlan` is lazy val so accessing it in StreamTest may need to acquire the lock of `lastExecution`. It may be waiting forever when the streaming thread is holding it and running a continuous Spark job.

This PR changes to check if `s.lastExecution` is null to avoid accessing `lastExecution.executedPlan`.

## How was this patch tested?

Jenkins

Author: Jose Torres <jose@databricks.com>

Closes #20413 from zsxwing/SPARK-23245.
2018-01-26 23:06:03 -08:00
Dongjoon Hyun e7bc9f0524 [MINOR][SS][DOC] Fix Trigger Scala/Java doc examples
## What changes were proposed in this pull request?

This PR fixes Scala/Java doc examples in `Trigger.java`.

## How was this patch tested?

N/A.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20401 from dongjoon-hyun/SPARK-TRIGGER.
2018-01-26 18:57:32 -06:00
Wenchen Fan 5b5447c68a [SPARK-23214][SQL] cached data should not carry extra hint info
## What changes were proposed in this pull request?

This is a regression introduced by https://github.com/apache/spark/pull/19864

When we lookup cache, we should not carry the hint info, as this cache entry might be added by a plan having hint info, while the input plan for this lookup may not have hint info, or have different hint info.

## How was this patch tested?

a new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20394 from cloud-fan/cache.
2018-01-26 16:46:51 -08:00
Xingbo Jiang 94c67a76ec [SPARK-23207][SQL] Shuffle+Repartition on a DataFrame could lead to incorrect answers
## What changes were proposed in this pull request?

Currently shuffle repartition uses RoundRobinPartitioning, the generated result is nondeterministic since the sequence of input rows are not determined.

The bug can be triggered when there is a repartition call following a shuffle (which would lead to non-deterministic row ordering), as the pattern shows below:
upstream stage -> repartition stage -> result stage
(-> indicate a shuffle)
When one of the executors process goes down, some tasks on the repartition stage will be retried and generate inconsistent ordering, and some tasks of the result stage will be retried generating different data.

The following code returns 931532, instead of 1000000:
```
import scala.sys.process._

import org.apache.spark.TaskContext
val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x =>
  x
}.repartition(200).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) {
    throw new Exception("pkill -f java".!!)
  }
  x
}
res.distinct().count()
```

In this PR, we propose a most straight-forward way to fix this problem by performing a local sort before partitioning, after we make the input row ordering deterministic, the function from rows to partitions is fully deterministic too.

The downside of the approach is that with extra local sort inserted, the performance of repartition() will go down, so we add a new config named `spark.sql.execution.sortBeforeRepartition` to control whether this patch is applied. The patch is default enabled to be safe-by-default, but user may choose to manually turn it off to avoid performance regression.

This patch also changes the output rows ordering of repartition(), that leads to a bunch of test cases failure because they are comparing the results directly.

## How was this patch tested?

Add unit test in ExchangeSuite.

With this patch(and `spark.sql.execution.sortBeforeRepartition` set to true), the following query returns 1000000:
```
import scala.sys.process._

import org.apache.spark.TaskContext

spark.conf.set("spark.sql.execution.sortBeforeRepartition", "true")

val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x =>
  x
}.repartition(200).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) {
    throw new Exception("pkill -f java".!!)
  }
  x
}
res.distinct().count()

res7: Long = 1000000
```

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

Closes #20393 from jiangxb1987/shuffle-repartition.
2018-01-26 15:01:03 -08:00
Wenchen Fan dd8e257d1c [SPARK-23218][SQL] simplify ColumnVector.getArray
## What changes were proposed in this pull request?

`ColumnVector` is very flexible about how to implement array type. As a result `ColumnVector` has 3 abstract methods for array type: `arrayData`, `getArrayOffset`, `getArrayLength`. For example, in `WritableColumnVector` we use the first child vector as the array data vector, and store offsets and lengths in 2 arrays in the parent vector. `ArrowColumnVector` has a different implementation.

This PR simplifies `ColumnVector` by using only one abstract method for array type: `getArray`.

## How was this patch tested?

existing tests.

rerun `ColumnarBatchBenchmark`, there is no performance regression.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20395 from cloud-fan/vector.
2018-01-26 09:17:05 -08:00
Kris Mok e57f394818 [SPARK-23032][SQL] Add a per-query codegenStageId to WholeStageCodegenExec
## What changes were proposed in this pull request?

**Proposal**

Add a per-query ID to the codegen stages as represented by `WholeStageCodegenExec` operators. This ID will be used in
-  the explain output of the physical plan, and in
- the generated class name.

Specifically, this ID will be stable within a query, counting up from 1 in depth-first post-order for all the `WholeStageCodegenExec` inserted into a plan.
The ID value 0 is reserved for "free-floating" `WholeStageCodegenExec` objects, which may have been created for one-off purposes, e.g. for fallback handling of codegen stages that failed to codegen the whole stage and wishes to codegen a subset of the children operators (as seen in `org.apache.spark.sql.execution.FileSourceScanExec#doExecute`).

Example: for the following query:
```scala
scala> spark.conf.set("spark.sql.autoBroadcastJoinThreshold", 1)

scala> val df1 = spark.range(10).select('id as 'x, 'id + 1 as 'y).orderBy('x).select('x + 1 as 'z, 'y)
df1: org.apache.spark.sql.DataFrame = [z: bigint, y: bigint]

scala> val df2 = spark.range(5)
df2: org.apache.spark.sql.Dataset[Long] = [id: bigint]

scala> val query = df1.join(df2, 'z === 'id)
query: org.apache.spark.sql.DataFrame = [z: bigint, y: bigint ... 1 more field]
```

The explain output before the change is:
```scala
scala> query.explain
== Physical Plan ==
*SortMergeJoin [z#9L], [id#13L], Inner
:- *Sort [z#9L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(z#9L, 200)
:     +- *Project [(x#3L + 1) AS z#9L, y#4L]
:        +- *Sort [x#3L ASC NULLS FIRST], true, 0
:           +- Exchange rangepartitioning(x#3L ASC NULLS FIRST, 200)
:              +- *Project [id#0L AS x#3L, (id#0L + 1) AS y#4L]
:                 +- *Range (0, 10, step=1, splits=8)
+- *Sort [id#13L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#13L, 200)
      +- *Range (0, 5, step=1, splits=8)
```
Note how codegen'd operators are annotated with a prefix `"*"`. See how the `SortMergeJoin` operator and its direct children `Sort` operators are adjacent and all annotated with the `"*"`, so it's hard to tell they're actually in separate codegen stages.

and after this change it'll be:
```scala
scala> query.explain
== Physical Plan ==
*(6) SortMergeJoin [z#9L], [id#13L], Inner
:- *(3) Sort [z#9L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(z#9L, 200)
:     +- *(2) Project [(x#3L + 1) AS z#9L, y#4L]
:        +- *(2) Sort [x#3L ASC NULLS FIRST], true, 0
:           +- Exchange rangepartitioning(x#3L ASC NULLS FIRST, 200)
:              +- *(1) Project [id#0L AS x#3L, (id#0L + 1) AS y#4L]
:                 +- *(1) Range (0, 10, step=1, splits=8)
+- *(5) Sort [id#13L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#13L, 200)
      +- *(4) Range (0, 5, step=1, splits=8)
```
Note that the annotated prefix becomes `"*(id) "`. See how the `SortMergeJoin` operator and its direct children `Sort` operators have different codegen stage IDs.

It'll also show up in the name of the generated class, as a suffix in the format of `GeneratedClass$GeneratedIterator$id`.

For example, note how `GeneratedClass$GeneratedIteratorForCodegenStage3` and `GeneratedClass$GeneratedIteratorForCodegenStage6` in the following stack trace corresponds to the IDs shown in the explain output above:
```
"Executor task launch worker for task 42412957" daemon prio=5 tid=0x58 nid=NA runnable
  java.lang.Thread.State: RUNNABLE
	  at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:109)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage3.sort_addToSorter$(generated.java:32)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage3.processNext(generated.java:41)
	  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$9$$anon$1.hasNext(WholeStageCodegenExec.scala:494)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage6.findNextInnerJoinRows$(generated.java:42)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage6.processNext(generated.java:101)
	  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$2.hasNext(WholeStageCodegenExec.scala:513)
	  at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:253)
	  at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
	  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:828)
	  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:828)
	  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	  at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	  at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	  at org.apache.spark.scheduler.Task.run(Task.scala:109)
	  at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
	  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	  at java.lang.Thread.run(Thread.java:748)
```

**Rationale**

Right now, the codegen from Spark SQL lacks the means to differentiate between a couple of things:

1. It's hard to tell which physical operators are in the same WholeStageCodegen stage. Note that this "stage" is a separate notion from Spark's RDD execution stages; this one is only to delineate codegen units.
There can be adjacent physical operators that are both codegen'd but are in separate codegen stages. Some of this is due to hacky implementation details, such as the case with `SortMergeJoin` and its `Sort` inputs -- they're hard coded to be split into separate stages although both are codegen'd.
When printing out the explain output of the physical plan, you'd only see the codegen'd physical operators annotated with a preceding star (`'*'`) but would have no way to figure out if they're in the same stage.

2. Performance/error diagnosis
The generated code has class/method names that are hard to differentiate between queries or even between codegen stages within the same query. If we use a Java-level profiler to collect profiles, or if we encounter a Java-level exception with a stack trace in it, it's really hard to tell which part of a query it's at.
By introducing a per-query codegen stage ID, we'd at least be able to know which codegen stage (and in turn, which group of physical operators) was a profile tick or an exception happened.

The reason why this proposal uses a per-query ID is because it's stable within a query, so that multiple runs of the same query will see the same resulting IDs. This both benefits understandability for users, and also it plays well with the codegen cache in Spark SQL which uses the generated source code as the key.

The downside to using per-query IDs as opposed to a per-session or globally incrementing ID is of course we can't tell apart different query runs with this ID alone. But for now I believe this is a good enough tradeoff.

## How was this patch tested?

Existing tests. This PR does not involve any runtime behavior changes other than some name changes.
The SQL query test suites that compares explain outputs have been updates to ignore the newly added `codegenStageId`.

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

Closes #20224 from rednaxelafx/wsc-codegenstageid.
2018-01-25 16:11:33 -08:00
Huaxin Gao 8480c0c576 [SPARK-23081][PYTHON] Add colRegex API to PySpark
## What changes were proposed in this pull request?

Add colRegex API to PySpark

## How was this patch tested?

add a test in sql/tests.py

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

Closes #20390 from huaxingao/spark-23081.
2018-01-26 07:50:48 +09:00
Liang-Chi Hsieh d20bbc2d87 [SPARK-21717][SQL] Decouple consume functions of physical operators in whole-stage codegen
## What changes were proposed in this pull request?

It has been observed in SPARK-21603 that whole-stage codegen suffers performance degradation, if the generated functions are too long to be optimized by JIT.

We basically produce a single function to incorporate generated codes from all physical operators in whole-stage. Thus, it is possibly to grow the size of generated function over a threshold that we can't have JIT optimization for it anymore.

This patch is trying to decouple the logic of consuming rows in physical operators to avoid a giant function processing rows.

## How was this patch tested?

Added tests.

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

Closes #18931 from viirya/SPARK-21717.
2018-01-25 19:49:58 +08:00
Herman van Hovell e29b08add9 [SPARK-23208][SQL] Fix code generation for complex create array (related) expressions
## What changes were proposed in this pull request?
The `GenArrayData.genCodeToCreateArrayData` produces illegal java code when code splitting is enabled. This is used in `CreateArray` and `CreateMap` expressions for complex object arrays.

This issue is caused by a typo.

## How was this patch tested?
Added a regression test in `complexTypesSuite`.

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

Closes #20391 from hvanhovell/SPARK-23208.
2018-01-25 16:40:41 +08:00
caoxuewen 6f0ba8472d [MINOR][SQL] add new unit test to LimitPushdown
## What changes were proposed in this pull request?

This PR is repaired as follows
1、update y -> x in "left outer join" test case ,maybe is mistake.
2、add a new test case:"left outer join and left sides are limited"
3、add a new test case:"left outer join and right sides are limited"
4、add a new test case: "right outer join and right sides are limited"
5、add a new test case: "right outer join and left sides are limited"
6、Remove annotations without code implementation

## How was this patch tested?

add new unit test case.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #20381 from heary-cao/LimitPushdownSuite.
2018-01-24 13:06:09 -08:00
zuotingbing bbb87b350d [SPARK-22837][SQL] Session timeout checker does not work in SessionManager.
## What changes were proposed in this pull request?

Currently we do not call the `super.init(hiveConf)` in `SparkSQLSessionManager.init`. So we do not load the config `HIVE_SERVER2_SESSION_CHECK_INTERVAL HIVE_SERVER2_IDLE_SESSION_TIMEOUT HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION` , which cause the session timeout checker does not work.

## How was this patch tested?

manual tests

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #20025 from zuotingbing/SPARK-22837.
2018-01-24 10:07:24 -08:00
Henry Robinson de36f65d3a [SPARK-23148][SQL] Allow pathnames with special characters for CSV / JSON / text
…JSON / text

## What changes were proposed in this pull request?

Fix for JSON and CSV data sources when file names include characters
that would be changed by URL encoding.

## How was this patch tested?

New unit tests for JSON, CSV and text suites

Author: Henry Robinson <henry@cloudera.com>

Closes #20355 from henryr/spark-23148.
2018-01-24 21:19:09 +09:00
gatorsmile 4e7b49041a Revert "[SPARK-23195][SQL] Keep the Hint of Cached Data"
This reverts commit 44cc4daf3a.
2018-01-23 22:38:20 -08:00
Liang-Chi Hsieh a3911cf896 [SPARK-23177][SQL][PYSPARK] Extract zero-parameter UDFs from aggregate
## What changes were proposed in this pull request?

We extract Python UDFs in logical aggregate which depends on aggregate expression or grouping key in ExtractPythonUDFFromAggregate rule. But Python UDFs which don't depend on above expressions should also be extracted to avoid the issue reported in the JIRA.

A small code snippet to reproduce that issue looks like:
```python
import pyspark.sql.functions as f

df = spark.createDataFrame([(1,2), (3,4)])
f_udf = f.udf(lambda: str("const_str"))
df2 = df.distinct().withColumn("a", f_udf())
df2.show()
```

Error exception is raised as:
```
: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#50
        at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:90)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
        at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:90)
        at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:514)
        at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:513)
```

This exception raises because `HashAggregateExec` tries to bind the aliased Python UDF expression (e.g., `pythonUDF0#50 AS a#44`) to grouping key.

## How was this patch tested?

Added test.

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

Closes #20360 from viirya/SPARK-23177.
2018-01-24 11:43:48 +09:00
gatorsmile 44cc4daf3a [SPARK-23195][SQL] Keep the Hint of Cached Data
## What changes were proposed in this pull request?
The broadcast hint of the cached plan is lost if we cache the plan. This PR is to correct it.

```Scala
  val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value")
  val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value")
  broadcast(df2).cache()
  df2.collect()
  val df3 = df1.join(df2, Seq("key"), "inner")
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20368 from gatorsmile/cachedBroadcastHint.
2018-01-23 16:17:09 -08:00
gatorsmile 613c290336 [SPARK-23192][SQL] Keep the Hint after Using Cached Data
## What changes were proposed in this pull request?

The hint of the plan segment is lost, if the plan segment is replaced by the cached data.

```Scala
      val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value")
      val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value")
      df2.cache()
      val df3 = df1.join(broadcast(df2), Seq("key"), "inner")
```

This PR is to fix it.

## How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20365 from gatorsmile/fixBroadcastHintloss.
2018-01-23 14:56:28 -08:00
Marcelo Vanzin dc4761fd8f [SPARK-17088][HIVE] Fix 'sharesHadoopClasses' option when creating client.
Because the call to the constructor of HiveClientImpl crosses class loader
boundaries, different versions of the same class (Configuration in this
case) were loaded, and that caused a runtime error when instantiating the
client. By using a safer type in the signature of the constructor, it's
possible to avoid the problem.

I considered removing 'sharesHadoopClasses', but it may still be desired
(even though there are 0 users of it since it was not working). When Spark
starts to support Hadoop 3, it may be necessary to use that option to
load clients for older Hive metastore versions that don't know about
Hadoop 3.

Tested with added unit test.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20169 from vanzin/SPARK-17088.
2018-01-23 12:51:40 -08:00
gatorsmile ee572ba8c1 [SPARK-20749][SQL][FOLLOW-UP] Override prettyName for bit_length and octet_length
## What changes were proposed in this pull request?
We need to override the prettyName for bit_length and octet_length for getting the expected auto-generated alias name.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20358 from gatorsmile/test2.3More.
2018-01-23 21:36:20 +09:00
Li Jin b2ce17b4c9 [SPARK-22274][PYTHON][SQL] User-defined aggregation functions with pandas udf (full shuffle)
## What changes were proposed in this pull request?

Add support for using pandas UDFs with groupby().agg().

This PR introduces a new type of pandas UDF - group aggregate pandas UDF. This type of UDF defines a transformation of multiple pandas Series -> a scalar value. Group aggregate pandas UDFs can be used with groupby().agg(). Note group aggregate pandas UDF doesn't support partial aggregation, i.e., a full shuffle is required.

This PR doesn't support group aggregate pandas UDFs that return ArrayType, StructType or MapType. Support for these types is left for future PR.

## How was this patch tested?

GroupbyAggPandasUDFTests

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

Closes #19872 from icexelloss/SPARK-22274-groupby-agg.
2018-01-23 14:11:30 +09:00
Wenchen Fan 51eb750263 [SPARK-22389][SQL] data source v2 partitioning reporting interface
## What changes were proposed in this pull request?

a new interface which allows data source to report partitioning and avoid shuffle at Spark side.

The design is pretty like the internal distribution/partitioing framework. Spark defines a `Distribution` interfaces and several concrete implementations, and ask the data source to report a `Partitioning`, the `Partitioning` should tell Spark if it can satisfy a `Distribution` or not.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20201 from cloud-fan/partition-reporting.
2018-01-22 15:21:09 -08:00
Jacek Laskowski 76b8b840dd [MINOR] Typo fixes
## What changes were proposed in this pull request?

Typo fixes

## How was this patch tested?

Local build / Doc-only changes

Author: Jacek Laskowski <jacek@japila.pl>

Closes #20344 from jaceklaskowski/typo-fixes.
2018-01-22 13:55:14 -06:00
Wenchen Fan 5d680cae48 [SPARK-23090][SQL] polish ColumnVector
## What changes were proposed in this pull request?

Several improvements:
* provide a default implementation for the batch get methods
* rename `getChildColumn` to `getChild`, which is more concise
* remove `getStruct(int, int)`, it's only used to simplify the codegen, which is an internal thing, we should not add a public API for this purpose.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20277 from cloud-fan/column-vector.
2018-01-22 20:56:38 +08:00
gatorsmile 896e45af5f [MINOR][SQL][TEST] Test case cleanups for recent PRs
## What changes were proposed in this pull request?
Revert the unneeded test case changes we made in SPARK-23000

Also fixes the test suites that do not call `super.afterAll()` in the local `afterAll`. The `afterAll()` of `TestHiveSingleton` actually reset the environments.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20341 from gatorsmile/testRelated.
2018-01-22 04:32:59 -08:00
gatorsmile 78801881c4 [SPARK-23170][SQL] Dump the statistics of effective runs of analyzer and optimizer rules
## What changes were proposed in this pull request?

Dump the statistics of effective runs of analyzer and optimizer rules.

## How was this patch tested?

Do a manual run of TPCDSQuerySuite

```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 175899
Total time: 25.486559948 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              1603280450 / 2868461549                         761 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$CTESubstitution                                    2045860009 / 2056602674                         37 / 788
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions                          440719059 / 1693110949                          38 / 1982
org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries                               1429834919 / 1446016225                         39 / 285
org.apache.spark.sql.catalyst.optimizer.PruneFilters                                               33273083 / 1389586938                           3 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  821183615 / 1266668754                          616 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderJoin                                                775837028 / 866238225                           132 / 1592
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            550683593 / 748854507                           211 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubquery                                    513075345 / 634370596                           49 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$FixNullability                                     33475731 / 606406532                            12 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              193144298 / 545403925                           86 / 1982
org.apache.spark.sql.catalyst.optimizer.BooleanSimplification                                      18651497 / 495725004                            7 / 1592
org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughJoin                                   369257217 / 489934378                           709 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantAliases                                     3707000 / 468291609                             9 / 1592
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints                                410155900 / 435254175                           192 / 285
org.apache.spark.sql.execution.datasources.FindDataSourceTable                                     348885539 / 371855866                           233 / 1982
org.apache.spark.sql.catalyst.optimizer.NullPropagation                                            11307645 / 307531225                            26 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions                                   120324545 / 304948785                           294 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     92323199 / 286695007                            38 / 1982
org.apache.spark.sql.catalyst.optimizer.PushDownPredicate                                          230084193 / 265845972                           785 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 45938401 / 265144009                            40 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   14888776 / 261499450                            1 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$CaseWhenCoercion                               113796384 / 244913861                           29 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantFolding                                            65008069 / 236548480                            126 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 226338929                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTimeZone                                             98134906 / 221323770                            417 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderAssociativeOperator                                 0 / 208421703                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.OptimizeIn                                                 8762534 / 199351958                             16 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$DateTimeOperations                             11980016 / 190779046                            27 / 1982
org.apache.spark.sql.catalyst.optimizer.SimplifyBinaryComparison                                   0 / 188887385                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyConditionals                                       0 / 186812106                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCaseConversionExpressions                          0 / 183885230                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCasts                                              17128295 / 182901910                            69 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$Division                                       14579110 / 180309340                            8 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$BooleanEquality                                0 / 176740516                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$IfCoercion                                     0 / 170781986                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.LikeSimplification                                         771605 / 164136736                              1 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveDispensableExpressions                               0 / 155958962                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.ResolveCreateNamedStruct                                    0 / 151222943                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowOrder                                 7534632 / 146596355                             14 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$EltCoercion                                    0 / 144488654                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ConcatCoercion                                 0 / 142403338                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowFrame                                 12067635 / 141500665                            21 / 1982
org.apache.spark.sql.catalyst.analysis.TimeWindowing                                               0 / 140431958                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WindowFrameCoercion                            0 / 125471960                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         14226972 / 124922019                            11 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$StackCoercion                                  0 / 123613887                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery                            8491071 / 121179056                             7 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGroupingAnalytics                           55526073 / 120290529                            11 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantPropagation                                        0 / 113886790                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveDeserializer                                52383759 / 107160222                            148 / 1982
org.apache.spark.sql.catalyst.analysis.CleanupAliases                                              52543524 / 102091518                            344 / 1086
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject                                     40682895 / 94403652                             342 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractWindowExpressions                           38473816 / 89740578                             23 / 1982
org.apache.spark.sql.catalyst.optimizer.CollapseProject                                            46806090 / 83315506                             281 / 1877
org.apache.spark.sql.catalyst.optimizer.FoldablePropagation                                        0 / 78750087                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases                                     13742765 / 77227258                             47 / 1982
org.apache.spark.sql.catalyst.optimizer.CombineFilters                                             53386729 / 76960344                             448 / 1592
org.apache.spark.sql.execution.datasources.DataSourceAnalysis                                      68034341 / 75724186                             24 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$LookupFunctions                                    0 / 71151084                                    0 / 750
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveMissingReferences                           12139848 / 67599140                             8 / 1982
org.apache.spark.sql.catalyst.optimizer.PullupCorrelatedPredicates                                 45017938 / 65968777                             23 / 285
org.apache.spark.sql.execution.datasources.v2.PushDownOperatorsToDataSource                        0 / 60937767                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseRepartition                                        0 / 59897237                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.PushProjectionThroughUnion                                 8547262 / 53941370                              10 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$HandleNullInputsForUDF                             0 / 52735976                                    0 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WidenSetOperationTypes                         9797713 / 52401665                              9 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$PullOutNondeterministic                            0 / 51741500                                    0 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   28614911 / 51061186                             233 / 1990
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               0 / 50621510                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineUnions                                              2777800 / 50262112                              17 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$GlobalAggregates                                   1640641 / 49633909                              46 / 1982
org.apache.spark.sql.catalyst.optimizer.DecimalAggregates                                          20198374 / 48488419                             100 / 385
org.apache.spark.sql.catalyst.optimizer.LimitPushDown                                              0 / 45052523                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineLimits                                              0 / 44719443                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSorts                                             0 / 44216930                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery                                   36235699 / 44165786                             148 / 285
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNewInstance                                 0 / 42750307                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast                                      0 / 41811748                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveOrdinalInOrderByAndGroupBy                  3819476 / 41776562                              4 / 1982
org.apache.spark.sql.catalyst.optimizer.ComputeCurrentTime                                         0 / 40527808                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseWindow                                             0 / 36832538                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSerialization                                     0 / 36120667                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggAliasInGroupBy                           0 / 32435826                                    0 / 1982
org.apache.spark.sql.execution.datasources.PreprocessTableCreation                                 0 / 32145218                                    0 / 742
org.apache.spark.sql.execution.datasources.ResolveSQLOnFile                                        0 / 30295614                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolvePivot                                       0 / 30111655                                    0 / 1982
org.apache.spark.sql.catalyst.expressions.codegen.package$ExpressionCanonicalizer$CleanExpressions 59930 / 28038201                                26 / 8280
org.apache.spark.sql.catalyst.analysis.ResolveInlineTables                                         0 / 27808108                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubqueryColumnAliases                       0 / 27066690                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate                                    0 / 26660210                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNaturalAndUsingJoin                         0 / 25255184                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTableValuedFunctions                                 0 / 24663088                                    0 / 1990
org.apache.spark.sql.catalyst.analysis.SubstituteUnresolvedOrdinals                                9709079 / 24450670                              4 / 788
org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveBroadcastHints                          0 / 23776535                                    0 / 750
org.apache.spark.sql.catalyst.optimizer.ReplaceExpressions                                         0 / 22697895                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CheckCartesianProducts                                     0 / 22523798                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.ReplaceDistinctWithAggregate                               988593 / 21535410                               15 / 300
org.apache.spark.sql.catalyst.optimizer.EliminateMapObjects                                        0 / 20269996                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates                                  0 / 19388592                                    0 / 285
org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases                                    17675532 / 18971185                             215 / 285
org.apache.spark.sql.catalyst.optimizer.GetCurrentDatabase                                         0 / 18271152                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.PropagateEmptyRelation                                     2077097 / 17190855                              3 / 288
org.apache.spark.sql.catalyst.analysis.EliminateBarriers                                           0 / 16736359                                    0 / 1086
org.apache.spark.sql.execution.OptimizeMetadataOnlyQuery                                           0 / 16669341                                    0 / 285
org.apache.spark.sql.catalyst.analysis.UpdateOuterReferences                                       0 / 14470235                                    0 / 742
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithAntiJoin                                  6715625 / 12190561                              1 / 300
org.apache.spark.sql.catalyst.optimizer.ReplaceIntersectWithSemiJoin                               3451793 / 11431432                              7 / 300
org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate                                0 / 10810568                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveRepetitionFromGroupExpressions                       344198 / 10475276                               1 / 286
org.apache.spark.sql.catalyst.analysis.Analyzer$WindowsSubstitution                                0 / 10386630                                    0 / 788
org.apache.spark.sql.catalyst.analysis.EliminateUnions                                             0 / 10096526                                    0 / 788
org.apache.spark.sql.catalyst.analysis.AliasViewChild                                              0 / 9991706                                     0 / 742
org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation                                     0 / 9649334                                     0 / 288
org.apache.spark.sql.catalyst.analysis.ResolveHints$RemoveAllHints                                 0 / 8739109                                     0 / 750
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion                                0 / 8420889                                     0 / 742
org.apache.spark.sql.catalyst.analysis.EliminateView                                               0 / 8319134                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveLiteralFromGroupExpressions                          0 / 7392627                                     0 / 286
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithFilter                                    0 / 7170516                                     0 / 300
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateArrayOps                                     0 / 7109643                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateStructOps                                    0 / 6837590                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateMapOps                                       0 / 6617848                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineConcats                                             0 / 5768406                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.ReplaceDeduplicateWithAggregate                            0 / 5349831                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineTypedFilters                                        0 / 5186642                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.EliminateDistinct                                          0 / 2427686                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder                                       0 / 2420436                                     0 / 285

```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20342 from gatorsmile/reportExecution.
2018-01-22 04:31:24 -08:00
Dongjoon Hyun 8142a3b883 [MINOR][SQL] Fix wrong comments on org.apache.spark.sql.parquet.row.attributes
## What changes were proposed in this pull request?

This PR fixes the wrong comment on `org.apache.spark.sql.parquet.row.attributes`
which is useful for UDTs like Vector/Matrix. Please see [SPARK-22320](https://issues.apache.org/jira/browse/SPARK-22320) for the usage.

Originally, [SPARK-19411](bf493686eb (diff-ee26d4c4be21e92e92a02e9f16dbc285L314)) left this behind during removing optional column metadatas. In the same PR, the same comment was removed at line 310-311.

## How was this patch tested?

N/A (This is about comments).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20346 from dongjoon-hyun/minor_comment_parquet.
2018-01-22 15:18:57 +09:00
Marco Gaido 121dc96f08 [SPARK-23087][SQL] CheckCartesianProduct too restrictive when condition is false/null
## What changes were proposed in this pull request?

CheckCartesianProduct raises an AnalysisException also when the join condition is always false/null. In this case, we shouldn't raise it, since the result will not be a cartesian product.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20333 from mgaido91/SPARK-23087.
2018-01-20 22:39:49 -08:00
fjh100456 00d169156d [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing
[SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing

What changes were proposed in this pull request?

Pass ‘spark.sql.parquet.compression.codec’ value to ‘parquet.compression’.
Pass ‘spark.sql.orc.compression.codec’ value to ‘orc.compress’.

How was this patch tested?

Add test.

Note:
This is the same issue mentioned in #19218 . That branch was deleted mistakenly, so make a new pr instead.

gatorsmile maropu dongjoon-hyun discipleforteen

Author: fjh100456 <fu.jinhua6@zte.com.cn>
Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Wenchen Fan <wenchen@databricks.com>
Author: gatorsmile <gatorsmile@gmail.com>
Author: Yinan Li <liyinan926@gmail.com>
Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Juliusz Sompolski <julek@databricks.com>
Author: Felix Cheung <felixcheung_m@hotmail.com>
Author: jerryshao <sshao@hortonworks.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Gera Shegalov <gera@apache.org>
Author: chetkhatri <ckhatrimanjal@gmail.com>
Author: Joseph K. Bradley <joseph@databricks.com>
Author: Bago Amirbekian <bago@databricks.com>
Author: Xianjin YE <advancedxy@gmail.com>
Author: Bruce Robbins <bersprockets@gmail.com>
Author: zuotingbing <zuo.tingbing9@zte.com.cn>
Author: Kent Yao <yaooqinn@hotmail.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Adrian Ionescu <adrian@databricks.com>

Closes #20087 from fjh100456/HiveTableWriting.
2018-01-20 14:49:49 -08:00
Sean Owen 396cdfbea4 [SPARK-23091][ML] Incorrect unit test for approxQuantile
## What changes were proposed in this pull request?

Narrow bound on approx quantile test to epsilon from 2*epsilon to match paper

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #20324 from srowen/SPARK-23091.
2018-01-19 22:46:34 -08:00
Kent Yao 793841c6b8 [SPARK-21771][SQL] remove useless hive client in SparkSQLEnv
## What changes were proposed in this pull request?

Once a meta hive client is created, it generates its SessionState which creates a lot of session related directories, some deleteOnExit, some does not. if a hive client is useless we may not create it at the very start.

## How was this patch tested?
N/A

cc hvanhovell cloud-fan

Author: Kent Yao <11215016@zju.edu.cn>

Closes #18983 from yaooqinn/patch-1.
2018-01-19 15:49:29 -08:00
Wenchen Fan d8aaa771e2 [SPARK-23149][SQL] polish ColumnarBatch
## What changes were proposed in this pull request?

Several cleanups in `ColumnarBatch`
* remove `schema`. The `ColumnVector`s inside `ColumnarBatch` already have the data type information, we don't need this `schema`.
* remove `capacity`. `ColumnarBatch` is just a wrapper of `ColumnVector`s, not builders, it doesn't need a capacity property.
* remove `DEFAULT_BATCH_SIZE`. As a wrapper, `ColumnarBatch` can't decide the batch size, it should be decided by the reader, e.g. parquet reader, orc reader, cached table reader. The default batch size should also be defined by the reader.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20316 from cloud-fan/columnar-batch.
2018-01-19 08:58:21 -08:00
gatorsmile 6c39654efc [SPARK-23000][TEST] Keep Derby DB Location Unchanged After Session Cloning
## What changes were proposed in this pull request?
After session cloning in `TestHive`, the conf of the singleton SparkContext for derby DB location is changed to a new directory. The new directory is created in `HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false)`.

This PR is to keep the conf value of `ConfVars.METASTORECONNECTURLKEY.varname` unchanged during the session clone.

## How was this patch tested?
The issue can be reproduced by the command:
> build/sbt -Phive "hive/test-only org.apache.spark.sql.hive.HiveSessionStateSuite org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite"

Also added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20328 from gatorsmile/fixTestFailure.
2018-01-19 22:47:18 +08:00
Marco Gaido e41400c3c8 [SPARK-23089][STS] Recreate session log directory if it doesn't exist
## What changes were proposed in this pull request?

When creating a session directory, Thrift should create the parent directory (i.e. /tmp/base_session_log_dir) if it is not present. It is common that many tools delete empty directories, so the directory may be deleted. This can cause the session log to be disabled.

This was fixed in HIVE-12262: this PR brings it in Spark too.

## How was this patch tested?

manual tests

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20281 from mgaido91/SPARK-23089.
2018-01-19 19:46:48 +08:00
Sameer Agarwal 9c4b99861c [BUILD][MINOR] Fix java style check issues
## What changes were proposed in this pull request?

This patch fixes a few recently introduced java style check errors in master and release branch.

As an aside, given that [java linting currently fails](https://github.com/apache/spark/pull/10763
) on machines with a clean maven cache, it'd be great to find another workaround to [re-enable the java style checks](3a07eff5af/dev/run-tests.py (L577)) as part of Spark PRB.

/cc zsxwing JoshRosen srowen for any suggestions

## How was this patch tested?

Manual Check

Author: Sameer Agarwal <sameerag@apache.org>

Closes #20323 from sameeragarwal/java.
2018-01-19 01:38:08 -08:00
Takuya UESHIN 568055da93 [SPARK-23054][SQL][PYSPARK][FOLLOWUP] Use sqlType casting when casting PythonUserDefinedType to String.
## What changes were proposed in this pull request?

This is a follow-up of #20246.

If a UDT in Python doesn't have its corresponding Scala UDT, cast to string will be the raw string of the internal value, e.g. `"org.apache.spark.sql.catalyst.expressions.UnsafeArrayDataxxxxxxxx"` if the internal type is `ArrayType`.

This pr fixes it by using its `sqlType` casting.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20306 from ueshin/issues/SPARK-23054/fup1.
2018-01-19 11:37:08 +08:00
Burak Yavuz e01919e834 [SPARK-23094] Fix invalid character handling in JsonDataSource
## What changes were proposed in this pull request?

There were two related fixes regarding `from_json`, `get_json_object` and `json_tuple` ([Fix #1](c8803c0685),
 [Fix #2](86174ea89b)), but they weren't comprehensive it seems. I wanted to extend those fixes to all the parsers, and add tests for each case.

## How was this patch tested?

Regression tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #20302 from brkyvz/json-invfix.
2018-01-19 07:36:06 +09:00
Tathagata Das bf34d665b9 [SPARK-23144][SS] Added console sink for continuous processing
## What changes were proposed in this pull request?
Refactored ConsoleWriter into ConsoleMicrobatchWriter and ConsoleContinuousWriter.

## How was this patch tested?
new unit test

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

Closes #20311 from tdas/SPARK-23144.
2018-01-18 12:33:39 -08:00
Marco Gaido e28eb43114 [SPARK-22036][SQL] Decimal multiplication with high precision/scale often returns NULL
## What changes were proposed in this pull request?

When there is an operation between Decimals and the result is a number which is not representable exactly with the result's precision and scale, Spark is returning `NULL`. This was done to reflect Hive's behavior, but it is against SQL ANSI 2011, which states that "If the result cannot be represented exactly in the result type, then whether it is rounded or truncated is implementation-defined". Moreover, Hive now changed its behavior in order to respect the standard, thanks to HIVE-15331.

Therefore, the PR propose to:
 - update the rules to determine the result precision and scale according to the new Hive's ones introduces in HIVE-15331;
 - round the result of the operations, when it is not representable exactly with the result's precision and scale, instead of returning `NULL`
 - introduce a new config `spark.sql.decimalOperations.allowPrecisionLoss` which default to `true` (ie. the new behavior) in order to allow users to switch back to the previous one.

Hive behavior reflects SQLServer's one. The only difference is that the precision and scale are adjusted for all the arithmetic operations in Hive, while SQL Server is said to do so only for multiplications and divisions in the documentation. This PR follows Hive's behavior.

A more detailed explanation is available here: https://mail-archives.apache.org/mod_mbox/spark-dev/201712.mbox/%3CCAEorWNAJ4TxJR9NBcgSFMD_VxTg8qVxusjP%2BAJP-x%2BJV9zH-yA%40mail.gmail.com%3E.

## How was this patch tested?

modified and added UTs. Comparisons with results of Hive and SQLServer.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20023 from mgaido91/SPARK-22036.
2018-01-18 21:24:39 +08:00
jerryshao 7a22483413 [SPARK-23140][SQL] Add DataSourceV2Strategy to Hive Session state's planner
## What changes were proposed in this pull request?

`DataSourceV2Strategy` is missing in `HiveSessionStateBuilder`'s planner, which will throw exception as described in [SPARK-23140](https://issues.apache.org/jira/browse/SPARK-23140).

## How was this patch tested?

Manual test.

Author: jerryshao <sshao@hortonworks.com>

Closes #20305 from jerryshao/SPARK-23140.
2018-01-18 19:18:55 +08:00
Jose Torres 1c76a91e5f [SPARK-23052][SS] Migrate ConsoleSink to data source V2 api.
## What changes were proposed in this pull request?

Migrate ConsoleSink to data source V2 api.

Note that this includes a missing piece in DataStreamWriter required to specify a data source V2 writer.

Note also that I've removed the "Rerun batch" part of the sink, because as far as I can tell this would never have actually happened. A MicroBatchExecution object will only commit each batch once for its lifetime, and a new MicroBatchExecution object would have a new ConsoleSink object which doesn't know it's retrying a batch. So I think this represents an anti-feature rather than a weakness in the V2 API.

## How was this patch tested?

new unit test

Author: Jose Torres <jose@databricks.com>

Closes #20243 from jose-torres/console-sink.
2018-01-17 22:36:29 -08:00
Xiayun Sun 0219470206 [SPARK-21996][SQL] read files with space in name for streaming
## What changes were proposed in this pull request?

Structured streaming is now able to read files with space in file name (previously it would skip the file and output a warning)

## How was this patch tested?

Added new unit test.

Author: Xiayun Sun <xiayunsun@gmail.com>

Closes #19247 from xysun/SPARK-21996.
2018-01-17 16:42:38 -08:00
Tathagata Das bac0d661af [SPARK-23119][SS] Minor fixes to V2 streaming APIs
## What changes were proposed in this pull request?

- Added `InterfaceStability.Evolving` annotations
- Improved docs.

## How was this patch tested?
Existing tests.

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

Closes #20286 from tdas/SPARK-23119.
2018-01-17 16:40:02 -08:00
Li Jin 4e6f8fb150 [SPARK-23047][PYTHON][SQL] Change MapVector to NullableMapVector in ArrowColumnVector
## What changes were proposed in this pull request?
This PR changes usage of `MapVector` in Spark codebase to use `NullableMapVector`.

`MapVector` is an internal Arrow class that is not supposed to be used directly. We should use `NullableMapVector` instead.

## How was this patch tested?

Existing test.

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

Closes #20239 from icexelloss/arrow-map-vector.
2018-01-18 07:26:43 +09:00
Jose Torres e946c63dd5 [SPARK-23093][SS] Don't change run id when reconfiguring a continuous processing query.
## What changes were proposed in this pull request?

Keep the run ID static, using a different ID for the epoch coordinator to avoid cross-execution message contamination.

## How was this patch tested?

new and existing unit tests

Author: Jose Torres <jose@databricks.com>

Closes #20282 from jose-torres/fix-runid.
2018-01-17 13:58:44 -08:00
Jose Torres 86a8450318 [SPARK-23033][SS] Don't use task level retry for continuous processing
## What changes were proposed in this pull request?

Continuous processing tasks will fail on any attempt number greater than 0. ContinuousExecution will catch these failures and restart globally from the last recorded checkpoints.
## How was this patch tested?
unit test

Author: Jose Torres <jose@databricks.com>

Closes #20225 from jose-torres/no-retry.
2018-01-17 13:52:51 -08:00
Wang Gengliang 8598a982b4 [SPARK-23079][SQL] Fix query constraints propagation with aliases
## What changes were proposed in this pull request?

Previously, PR #19201 fix the problem of non-converging constraints.
After that PR #19149 improve the loop and constraints is inferred only once.
So the problem of non-converging constraints is gone.

However, the case below will fail.

```

spark.range(5).write.saveAsTable("t")
val t = spark.read.table("t")
val left = t.withColumn("xid", $"id" + lit(1)).as("x")
val right = t.withColumnRenamed("id", "xid").as("y")
val df = left.join(right, "xid").filter("id = 3").toDF()
checkAnswer(df, Row(4, 3))

```

Because `aliasMap` replace all the aliased child. See the test case in PR for details.

This PR is to fix this bug by removing useless code for preventing non-converging constraints.
It can be also fixed with #20270, but this is much simpler and clean up the code.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20278 from gengliangwang/FixConstraintSimple.
2018-01-18 00:05:26 +08:00
Dongjoon Hyun 0f8a28617a [SPARK-21783][SQL] Turn on ORC filter push-down by default
## What changes were proposed in this pull request?

ORC filter push-down is disabled by default from the beginning, [SPARK-2883](aa31e431fc (diff-41ef65b9ef5b518f77e2a03559893f4dR149)
).

Now, Apache Spark starts to depend on Apache ORC 1.4.1. For Apache Spark 2.3, this PR turns on ORC filter push-down by default like Parquet ([SPARK-9207](https://issues.apache.org/jira/browse/SPARK-21783)) as a part of [SPARK-20901](https://issues.apache.org/jira/browse/SPARK-20901), "Feature parity for ORC with Parquet".

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20265 from dongjoon-hyun/SPARK-21783.
2018-01-17 21:53:36 +08:00
Henry Robinson 1f3d933e0b [SPARK-23062][SQL] Improve EXCEPT documentation
## What changes were proposed in this pull request?

Make the default behavior of EXCEPT (i.e. EXCEPT DISTINCT) more
explicit in the documentation, and call out the change in behavior
from 1.x.

Author: Henry Robinson <henry@cloudera.com>

Closes #20254 from henryr/spark-23062.
2018-01-17 16:01:41 +08:00
Dongjoon Hyun a0aedb0ded [SPARK-23072][SQL][TEST] Add a Unicode schema test for file-based data sources
## What changes were proposed in this pull request?

After [SPARK-20682](https://github.com/apache/spark/pull/19651), Apache Spark 2.3 is able to read ORC files with Unicode schema. Previously, it raises `org.apache.spark.sql.catalyst.parser.ParseException`.

This PR adds a Unicode schema test for CSV/JSON/ORC/Parquet file-based data sources. Note that TEXT data source only has [a single column with a fixed name 'value'](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala#L71).

## How was this patch tested?

Pass the newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20266 from dongjoon-hyun/SPARK-23072.
2018-01-17 14:32:18 +08:00
Jose Torres 1667057851 [SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to continuous Kafka data reader
## What changes were proposed in this pull request?

The Kafka reader is now interruptible and can close itself.
## How was this patch tested?

I locally ran one of the ContinuousKafkaSourceSuite tests in a tight loop. Before the fix, my machine ran out of open file descriptors a few iterations in; now it works fine.

Author: Jose Torres <jose@databricks.com>

Closes #20253 from jose-torres/fix-data-reader.
2018-01-16 18:11:27 -08:00
Gabor Somogyi a9b845ebb5 [SPARK-22361][SQL][TEST] Add unit test for Window Frames
## What changes were proposed in this pull request?

There are already quite a few integration tests using window frames, but the unit tests coverage is not ideal.

In this PR the already existing tests are reorganized, extended and where gaps found additional cases added.

## How was this patch tested?

Automated: Pass the Jenkins.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20019 from gaborgsomogyi/SPARK-22361.
2018-01-17 10:03:25 +08:00
Dilip Biswal 0c2ba427bc [SPARK-23095][SQL] Decorrelation of scalar subquery fails with java.util.NoSuchElementException
## What changes were proposed in this pull request?
The following SQL involving scalar correlated query returns a map exception.
``` SQL
SELECT t1a
FROM   t1
WHERE  t1a = (SELECT   count(*)
              FROM     t2
              WHERE    t2c = t1c
              HAVING   count(*) >= 1)
```
``` SQL
key not found: ExprId(278,786682bb-41f9-4bd5-a397-928272cc8e4e)
java.util.NoSuchElementException: key not found: ExprId(278,786682bb-41f9-4bd5-a397-928272cc8e4e)
        at scala.collection.MapLike$class.default(MapLike.scala:228)
        at scala.collection.AbstractMap.default(Map.scala:59)
        at scala.collection.MapLike$class.apply(MapLike.scala:141)
        at scala.collection.AbstractMap.apply(Map.scala:59)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$.org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$evalSubqueryOnZeroTups(subquery.scala:378)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$$anonfun$org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$constructLeftJoins$1.apply(subquery.scala:430)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$$anonfun$org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$constructLeftJoins$1.apply(subquery.scala:426)
```

In this case, after evaluating the HAVING clause "count(*) > 1" statically
against the binding of aggregtation result on empty input, we determine
that this query will not have a the count bug. We should simply return
the evalSubqueryOnZeroTups with empty value.
(Please fill in changes proposed in this fix)

## How was this patch tested?
A new test was added in the Subquery bucket.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #20283 from dilipbiswal/scalar-count-defect.
2018-01-17 09:57:30 +08:00
Gabor Somogyi 12db365b4f [SPARK-16139][TEST] Add logging functionality for leaked threads in tests
## What changes were proposed in this pull request?

Lots of our tests don't properly shutdown everything they create, and end up leaking lots of threads. For example, `TaskSetManagerSuite` doesn't stop the extra `TaskScheduler` and `DAGScheduler` it creates. There are a couple more instances, eg. in `DAGSchedulerSuite`.

This PR adds the possibility to print out the not properly stopped thread list after a test suite executed. The format is the following:

```
===== FINISHED o.a.s.scheduler.DAGSchedulerSuite: 'task end event should have updated accumulators (SPARK-20342)' =====

...

===== Global thread whitelist loaded with name /thread_whitelist from classpath: rpc-client.*, rpc-server.*, shuffle-client.*, shuffle-server.*' =====

ScalaTest-run:

===== THREADS NOT STOPPED PROPERLY =====

ScalaTest-run: dag-scheduler-event-loop
ScalaTest-run: globalEventExecutor-2-5
ScalaTest-run:

===== END OF THREAD DUMP =====

ScalaTest-run:

===== EITHER PUT THREAD NAME INTO THE WHITELIST FILE OR SHUT IT DOWN PROPERLY =====
```

With the help of this leaking threads has been identified in TaskSetManagerSuite. My intention is to hunt down and fix such bugs in later PRs.

## How was this patch tested?

Manual: TaskSetManagerSuite test executed and found out where are the leaking threads.
Automated: Pass the Jenkins.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #19893 from gaborgsomogyi/SPARK-16139.
2018-01-16 11:41:08 -08:00
Wenchen Fan 75db14864d [SPARK-22392][SQL] data source v2 columnar batch reader
## What changes were proposed in this pull request?

a new Data Source V2 interface to allow the data source to return `ColumnarBatch` during the scan.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20153 from cloud-fan/columnar-reader.
2018-01-16 22:41:30 +08:00
Yuanjian Li 07ae39d0ec [SPARK-22956][SS] Bug fix for 2 streams union failover scenario
## What changes were proposed in this pull request?

This problem reported by yanlin-Lynn ivoson and LiangchangZ. Thanks!

When we union 2 streams from kafka or other sources, while one of them have no continues data coming and in the same time task restart, this will cause an `IllegalStateException`. This mainly cause because the code in [MicroBatchExecution](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala#L190) , while one stream has no continues data, its comittedOffset same with availableOffset during `populateStartOffsets`, and `currentPartitionOffsets` not properly handled in KafkaSource. Also, maybe we should also consider this scenario in other Source.

## How was this patch tested?

Add a UT in KafkaSourceSuite.scala

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #20150 from xuanyuanking/SPARK-22956.
2018-01-15 22:01:14 -08:00
Sameer Agarwal c7572b79da [SPARK-23000] Use fully qualified table names in HiveMetastoreCatalogSuite
## What changes were proposed in this pull request?

In another attempt to fix DataSourceWithHiveMetastoreCatalogSuite, this patch uses qualified table names (`default.t`) in the individual tests.

## How was this patch tested?

N/A (Test Only Change)

Author: Sameer Agarwal <sameerag@apache.org>

Closes #20273 from sameeragarwal/flaky-test.
2018-01-16 11:20:18 +08:00
Marco Gaido 8ab2d7ea99 [SPARK-23080][SQL] Improve error message for built-in functions
## What changes were proposed in this pull request?

When a user puts the wrong number of parameters in a function, an AnalysisException is thrown. If the function is a UDF, he user is told how many parameters the function expected and how many he/she put. If the function, instead, is a built-in one, no information about the number of parameters expected and the actual one is provided. This can help in some cases, to debug the errors (eg. bad quotes escaping may lead to a different number of parameters than expected, etc. etc.)

The PR adds the information about the number of parameters passed and the expected one, analogously to what happens for UDF.

## How was this patch tested?

modified existing UT + manual test

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20271 from mgaido91/SPARK-23080.
2018-01-16 11:47:42 +09:00
xubo245 6c81fe227a [SPARK-23035][SQL] Fix improper information of TempTableAlreadyExistsException
## What changes were proposed in this pull request?

Problem: it throw TempTableAlreadyExistsException and output "Temporary table '$table' already exists" when we create temp view by using org.apache.spark.sql.catalyst.catalog.GlobalTempViewManager#create, it's improper.

So fix improper information about TempTableAlreadyExistsException when create temp view:

change "Temporary table"  to  "Temporary view"

## How was this patch tested?

test("rename temporary view - destination table already exists, with: CREATE TEMPORARY view")

test("rename temporary view - destination table with database name,with:CREATE TEMPORARY view")

Author: xubo245 <601450868@qq.com>

Closes #20227 from xubo245/fixDeprecated.
2018-01-15 23:13:15 +08:00
Yuming Wang a38c887ac0 [SPARK-19550][BUILD][FOLLOW-UP] Remove MaxPermSize for sql module
## What changes were proposed in this pull request?

Remove `MaxPermSize` for `sql` module

## How was this patch tested?

Manually tested.

Author: Yuming Wang <yumwang@ebay.com>

Closes #20268 from wangyum/SPARK-19550-MaxPermSize.
2018-01-15 07:49:34 -06:00
Takeshi Yamamuro b59808385c [SPARK-23023][SQL] Cast field data to strings in showString
## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------------------------------------------+
|a                                                           |
+------------------------------------------------------------+
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
+------------------------------------------------------------+
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------+
|a                       |
+------------------------+
|[[1, 2], [3], [4, 5, 6]]|
+------------------------+
```
So, this pr fixed code in `showString` to cast field data to strings before printing.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20214 from maropu/SPARK-23023.
2018-01-15 16:26:52 +08:00
Dongjoon Hyun 9a96bfc8bf [SPARK-23049][SQL] spark.sql.files.ignoreCorruptFiles should work for ORC files
## What changes were proposed in this pull request?

When `spark.sql.files.ignoreCorruptFiles=true`, we should ignore corrupted ORC files.

## How was this patch tested?

Pass the Jenkins with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20240 from dongjoon-hyun/SPARK-23049.
2018-01-15 12:06:56 +08:00
Takeshi Yamamuro b98ffa4d6d [SPARK-23054][SQL] Fix incorrect results of casting UserDefinedType to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting `UserDefinedType`s into strings;
```
>>> from pyspark.ml.classification import MultilayerPerceptronClassifier
>>> from pyspark.ml.linalg import Vectors
>>> df = spark.createDataFrame([(0.0, Vectors.dense([0.0, 0.0])), (1.0, Vectors.dense([0.0, 1.0]))], ["label", "features"])
>>> df.selectExpr("CAST(features AS STRING)").show(truncate = False)
+-------------------------------------------+
|features                                   |
+-------------------------------------------+
|[6,1,0,0,2800000020,2,0,0,0]               |
|[6,1,0,0,2800000020,2,0,0,3ff0000000000000]|
+-------------------------------------------+
```
The root cause is that `Cast` handles input data as `UserDefinedType.sqlType`(this is underlying storage type), so we should pass data into `UserDefinedType.deserialize` then `toString`.
This pr modified the result into;
```
+---------+
|features |
+---------+
|[0.0,0.0]|
|[0.0,1.0]|
+---------+
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20246 from maropu/SPARK-23054.
2018-01-15 10:55:21 +08:00
guoxiaolong 42a1a15d73 [SPARK-22999][SQL] show databases like command' can remove the like keyword
## What changes were proposed in this pull request?

SHOW DATABASES (LIKE pattern = STRING)? Can be like the back increase?
When using this command, LIKE keyword can be removed.
You can refer to the SHOW TABLES command, SHOW TABLES 'test *' and SHOW TABELS like 'test *' can be used.
Similarly SHOW DATABASES 'test *' and SHOW DATABASES like 'test *' can be used.

## How was this patch tested?
unit tests   manual tests
Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #20194 from guoxiaolongzte/SPARK-22999.
2018-01-15 02:02:49 +08:00
Takeshi Yamamuro 990f05c803 [SPARK-23021][SQL] AnalysisBarrier should override innerChildren to print correct explain output
## What changes were proposed in this pull request?
`AnalysisBarrier` in the current master cuts off explain results for parsed logical plans;
```
scala> Seq((1, 1)).toDF("a", "b").groupBy("a").count().sample(0.1).explain(true)
== Parsed Logical Plan ==
Sample 0.0, 0.1, false, -7661439431999668039
+- AnalysisBarrier Aggregate [a#5], [a#5, count(1) AS count#14L]
```
To fix this, `AnalysisBarrier` needs to override `innerChildren` and this pr changed the output to;
```
== Parsed Logical Plan ==
Sample 0.0, 0.1, false, -5086223488015741426
+- AnalysisBarrier
      +- Aggregate [a#5], [a#5, count(1) AS count#14L]
         +- Project [_1#2 AS a#5, _2#3 AS b#6]
            +- LocalRelation [_1#2, _2#3]
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20247 from maropu/SPARK-23021-2.
2018-01-14 22:26:21 +08:00
Takeshi Yamamuro 0066d6f6fa [SPARK-21213][SQL][FOLLOWUP] Use compatible types for comparisons in compareAndGetNewStats
## What changes were proposed in this pull request?
This pr fixed code to compare values in `compareAndGetNewStats`.
The test below fails in the current master;
```
    val oldStats2 = CatalogStatistics(sizeInBytes = BigInt(Long.MaxValue) * 2)
    val newStats5 = CommandUtils.compareAndGetNewStats(
      Some(oldStats2), newTotalSize = BigInt(Long.MaxValue) * 2, None)
    assert(newStats5.isEmpty)
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20245 from maropu/SPARK-21213-FOLLOWUP.
2018-01-14 05:39:38 +08:00
CodingCat ba891ec993 [SPARK-22790][SQL] add a configurable factor to describe HadoopFsRelation's size
## What changes were proposed in this pull request?

as per discussion in https://github.com/apache/spark/pull/19864#discussion_r156847927

the current HadoopFsRelation is purely based on the underlying file size which is not accurate and makes the execution vulnerable to errors like OOM

Users can enable CBO with the functionalities in https://github.com/apache/spark/pull/19864 to avoid this issue

This JIRA proposes to add a configurable factor to sizeInBytes method in HadoopFsRelation class so that users can mitigate this problem without CBO

## How was this patch tested?

Existing tests

Author: CodingCat <zhunansjtu@gmail.com>
Author: Nan Zhu <nanzhu@uber.com>

Closes #20072 from CodingCat/SPARK-22790.
2018-01-14 02:36:32 +08:00
xubo245 bd4a21b482 [SPARK-23036][SQL][TEST] Add withGlobalTempView for testing
## What changes were proposed in this pull request?

Add withGlobalTempView when create global temp view, like withTempView and withView.
And correct some improper usage.
Please see jira.
There are other similar place like that. I will fix it if community need. Please confirm it.
## How was this patch tested?

no new test.

Author: xubo245 <601450868@qq.com>

Closes #20228 from xubo245/DropTempView.
2018-01-14 02:28:57 +08:00
Sameer Agarwal 55dbfbca37 Revert "[SPARK-22908] Add kafka source and sink for continuous processing."
This reverts commit 6f7aaed805.
2018-01-12 15:00:00 -08:00
Marco Gaido 54277398af [SPARK-22975][SS] MetricsReporter should not throw exception when there was no progress reported
## What changes were proposed in this pull request?

`MetricsReporter ` assumes that there has been some progress for the query, ie. `lastProgress` is not null. If this is not true, as it might happen in particular conditions, a `NullPointerException` can be thrown.

The PR checks whether there is a `lastProgress` and if this is not true, it returns a default value for the metrics.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20189 from mgaido91/SPARK-22975.
2018-01-12 11:25:37 -08:00
Dongjoon Hyun 7bd14cfd40 [MINOR][BUILD] Fix Java linter errors
## What changes were proposed in this pull request?

This PR cleans up the java-lint errors (for v2.3.0-rc1 tag). Hopefully, this will be the final one.

```
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java:[85] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/launcher/InProcessAppHandle.java:[20,8] (imports) UnusedImports: Unused import - java.io.IOException.
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java:[41,9] (modifier) ModifierOrder: 'private' modifier out of order with the JLS suggestions.
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java:[464] (sizes) LineLength: Line is longer than 100 characters (found 102).
```

## How was this patch tested?

Manual.

```
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20242 from dongjoon-hyun/fix_lint_java_2.3_rc1.
2018-01-12 10:18:42 -08:00
gatorsmile 651f76153f [SPARK-23028] Bump master branch version to 2.4.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20222 from gatorsmile/bump24.
2018-01-13 00:37:59 +08:00
Marco Gaido 5050868069 [SPARK-23025][SQL] Support Null type in scala reflection
## What changes were proposed in this pull request?

Add support for `Null` type in the `schemaFor` method for Scala reflection.

## How was this patch tested?

Added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20219 from mgaido91/SPARK-23025.
2018-01-12 18:04:44 +08:00
Jose Torres 6f7aaed805 [SPARK-22908] Add kafka source and sink for continuous processing.
## What changes were proposed in this pull request?

Add kafka source and sink for continuous processing. This involves two small changes to the execution engine:

* Bring data reader close() into the normal data reader thread to avoid thread safety issues.
* Fix up the semantics of the RECONFIGURING StreamExecution state. State updates are now atomic, and we don't have to deal with swallowing an exception.

## How was this patch tested?

new unit tests

Author: Jose Torres <jose@databricks.com>

Closes #20096 from jose-torres/continuous-kafka.
2018-01-11 10:52:12 -08:00
gatorsmile 76892bcf2c [SPARK-23000][TEST-HADOOP2.6] Fix Flaky test suite DataSourceWithHiveMetastoreCatalogSuite
## What changes were proposed in this pull request?
The Spark 2.3 branch still failed due to the flaky test suite `DataSourceWithHiveMetastoreCatalogSuite `. https://amplab.cs.berkeley.edu/jenkins/job/spark-branch-2.3-test-sbt-hadoop-2.6/

Although https://github.com/apache/spark/pull/20207 is unable to reproduce it in Spark 2.3, it sounds like the current DB of Spark's Catalog is changed based on the following stacktrace. Thus, we just need to reset it.

```
[info] DataSourceWithHiveMetastoreCatalogSuite:
02:40:39.486 ERROR org.apache.hadoop.hive.ql.parse.CalcitePlanner: org.apache.hadoop.hive.ql.parse.SemanticException: Line 1:14 Table not found 't'
	at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.getMetaData(SemanticAnalyzer.java:1594)
	at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.getMetaData(SemanticAnalyzer.java:1545)
	at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genResolvedParseTree(SemanticAnalyzer.java:10077)
	at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:10128)
	at org.apache.hadoop.hive.ql.parse.CalcitePlanner.analyzeInternal(CalcitePlanner.java:209)
	at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:227)
	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:424)
	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:308)
	at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1122)
	at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1170)
	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1059)
	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1049)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:694)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:683)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:272)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:210)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:209)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:255)
	at org.apache.spark.sql.hive.client.HiveClientImpl.runHive(HiveClientImpl.scala:683)
	at org.apache.spark.sql.hive.client.HiveClientImpl.runSqlHive(HiveClientImpl.scala:673)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$9$$anonfun$apply$1$$anonfun$apply$mcV$sp$3.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:185)
	at org.apache.spark.sql.test.SQLTestUtilsBase$class.withTable(SQLTestUtils.scala:273)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.withTable(HiveMetastoreCatalogSuite.scala:139)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$9$$anonfun$apply$1.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:163)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$9$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:163)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$9$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:163)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:183)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:196)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:196)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:289)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:196)
	at org.scalatest.FunSuite.runTest(FunSuite.scala:1560)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:229)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:229)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:396)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:384)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:384)
	at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:379)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:461)
	at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:229)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
	at org.scalatest.Suite$class.run(Suite.scala:1147)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:233)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:233)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:521)
	at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:233)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:31)
	at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:213)
	at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:210)
	at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:31)
	at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:314)
	at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:480)
	at sbt.ForkMain$Run$2.call(ForkMain.java:296)
	at sbt.ForkMain$Run$2.call(ForkMain.java:286)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20218 from gatorsmile/testFixAgain.
2018-01-11 21:32:36 +08:00
wuyi5 0552c36e02 [SPARK-22967][TESTS] Fix VersionSuite's unit tests by change Windows path into URI path
## What changes were proposed in this pull request?

Two unit test will fail due to Windows format path:

1.test(s"$version: read avro file containing decimal")
```
org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

2.test(s"$version: SPARK-17920: Insert into/overwrite avro table")
```
Unable to infer the schema. The schema specification is required to create the table `default`.`tab2`.;
org.apache.spark.sql.AnalysisException: Unable to infer the schema. The schema specification is required to create the table `default`.`tab2`.;
```

This pr fix these two unit test by change Windows path into URI path.

## How was this patch tested?
Existed.

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

Author: wuyi5 <ngone_5451@163.com>

Closes #20199 from Ngone51/SPARK-22967.
2018-01-11 22:17:15 +09:00
gatorsmile 87c98de8b2 [SPARK-23001][SQL] Fix NullPointerException when DESC a database with NULL description
## What changes were proposed in this pull request?
When users' DB description is NULL, users might hit `NullPointerException`. This PR is to fix the issue.

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20215 from gatorsmile/SPARK-23001.
2018-01-11 18:17:34 +08:00
Feng Liu 9b33dfc408 [SPARK-22951][SQL] fix aggregation after dropDuplicates on empty data frames
## What changes were proposed in this pull request?

(courtesy of liancheng)

Spark SQL supports both global aggregation and grouping aggregation. Global aggregation always return a single row with the initial aggregation state as the output, even there are zero input rows. Spark implements this by simply checking the number of grouping keys and treats an aggregation as a global aggregation if it has zero grouping keys.

However, this simple principle drops the ball in the following case:

```scala
spark.emptyDataFrame.dropDuplicates().agg(count($"*") as "c").show()
// +---+
// | c |
// +---+
// | 1 |
// +---+
```

The reason is that:

1. `df.dropDuplicates()` is roughly translated into something equivalent to:

```scala
val allColumns = df.columns.map { col }
df.groupBy(allColumns: _*).agg(allColumns.head, allColumns.tail: _*)
```

This translation is implemented in the rule `ReplaceDeduplicateWithAggregate`.

2. `spark.emptyDataFrame` contains zero columns and zero rows.

Therefore, rule `ReplaceDeduplicateWithAggregate` makes a confusing transformation roughly equivalent to the following one:

```scala
spark.emptyDataFrame.dropDuplicates()
=> spark.emptyDataFrame.groupBy().agg(Map.empty[String, String])
```

The above transformation is confusing because the resulting aggregate operator contains no grouping keys (because `emptyDataFrame` contains no columns), and gets recognized as a global aggregation. As a result, Spark SQL allocates a single row filled by the initial aggregation state and uses it as the output, and returns a wrong result.

To fix this issue, this PR tweaks `ReplaceDeduplicateWithAggregate` by appending a literal `1` to the grouping key list of the resulting `Aggregate` operator when the input plan contains zero output columns. In this way, `spark.emptyDataFrame.dropDuplicates()` is now translated into a grouping aggregation, roughly depicted as:

```scala
spark.emptyDataFrame.dropDuplicates()
=> spark.emptyDataFrame.groupBy(lit(1)).agg(Map.empty[String, String])
```

Which is now properly treated as a grouping aggregation and returns the correct answer.

## How was this patch tested?

New unit tests added

Author: Feng Liu <fengliu@databricks.com>

Closes #20174 from liufengdb/fix-duplicate.
2018-01-10 14:25:04 -08:00
Wenchen Fan eaac60a1e2 [SPARK-16060][SQL][FOLLOW-UP] add a wrapper solution for vectorized orc reader
## What changes were proposed in this pull request?

This is mostly from https://github.com/apache/spark/pull/13775

The wrapper solution is pretty good for string/binary type, as the ORC column vector doesn't keep bytes in a continuous memory region, and has a significant overhead when copying the data to Spark columnar batch. For other cases, the wrapper solution is almost same with the current solution.

I think we can treat the wrapper solution as a baseline and keep improving the writing to Spark solution.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20205 from cloud-fan/orc.
2018-01-10 15:16:27 +08:00
Wenchen Fan 6f169ca9e1 [MINOR] fix a typo in BroadcastJoinSuite
## What changes were proposed in this pull request?

`BroadcastNestedLoopJoinExec` should be `BroadcastHashJoinExec`

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20202 from cloud-fan/typo.
2018-01-10 10:20:34 +08:00
Wang Gengliang 96ba217a06 [SPARK-23005][CORE] Improve RDD.take on small number of partitions
## What changes were proposed in this pull request?
In current implementation of RDD.take, we overestimate the number of partitions we need to try by 50%:
`(1.5 * num * partsScanned / buf.size).toInt`
However, when the number is small, the result of `.toInt` is not what we want.
E.g, 2.9 will become 2, which should be 3.
Use Math.ceil to fix the problem.

Also clean up the code in RDD.scala.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20200 from gengliangwang/Take.
2018-01-10 10:15:27 +08:00
Takeshi Yamamuro 2250cb75b9 [SPARK-22981][SQL] Fix incorrect results of Casting Struct to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting structs into strings;
```
scala> val df = Seq(((1, "a"), 0), ((2, "b"), 0)).toDF("a", "b")
scala> df.write.saveAsTable("t")
scala> sql("SELECT CAST(a AS STRING) FROM t").show
+-------------------+
|                  a|
+-------------------+
|[0,1,1800000001,61]|
|[0,2,1800000001,62]|
+-------------------+
```
This pr modified the result into;
```
+------+
|     a|
+------+
|[1, a]|
|[2, b]|
+------+
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20176 from maropu/SPARK-22981.
2018-01-09 21:58:55 +08:00
Dongjoon Hyun f44ba910f5 [SPARK-16060][SQL] Support Vectorized ORC Reader
## What changes were proposed in this pull request?

This PR adds an ORC columnar-batch reader to native `OrcFileFormat`. Since both Spark `ColumnarBatch` and ORC `RowBatch` are used together, it is faster than the current Spark implementation. This replaces the prior PR, #17924.

Also, this PR adds `OrcReadBenchmark` to show the performance improvement.

## How was this patch tested?

Pass the existing test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19943 from dongjoon-hyun/SPARK-16060.
2018-01-09 21:48:14 +08:00
gatorsmile 0959aa581a [SPARK-23000] Fix Flaky test suite DataSourceWithHiveMetastoreCatalogSuite in Spark 2.3
## What changes were proposed in this pull request?
https://amplab.cs.berkeley.edu/jenkins/job/spark-branch-2.3-test-sbt-hadoop-2.6/

The test suite DataSourceWithHiveMetastoreCatalogSuite of Branch 2.3 always failed in hadoop 2.6

The table `t` exists in `default`, but `runSQLHive` reported the table does not exist. Obviously, Hive client's default database is different. The fix is to clean the environment and use `DEFAULT` as the database.

```
org.apache.spark.sql.execution.QueryExecutionException: FAILED: SemanticException [Error 10001]: Line 1:14 Table not found 't'
Stacktrace

sbt.ForkMain$ForkError: org.apache.spark.sql.execution.QueryExecutionException: FAILED: SemanticException [Error 10001]: Line 1:14 Table not found 't'
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:699)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:683)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:272)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:210)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:209)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:255)
	at org.apache.spark.sql.hive.client.HiveClientImpl.runHive(HiveClientImpl.scala:683)
	at org.apache.spark.sql.hive.client.HiveClientImpl.runSqlHive(HiveClientImpl.scala:673)
```

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20196 from gatorsmile/testFix.
2018-01-09 16:31:20 +08:00
Josh Rosen f20131dd35 [SPARK-22984] Fix incorrect bitmap copying and offset adjustment in GenerateUnsafeRowJoiner
## What changes were proposed in this pull request?

This PR fixes a longstanding correctness bug in `GenerateUnsafeRowJoiner`. This class was introduced in https://github.com/apache/spark/pull/7821 (July 2015 / Spark 1.5.0+) and is used to combine pairs of UnsafeRows in TungstenAggregationIterator, CartesianProductExec, and AppendColumns.

### Bugs fixed by this patch

1. **Incorrect combining of null-tracking bitmaps**: when concatenating two UnsafeRows, the implementation "Concatenate the two bitsets together into a single one, taking padding into account". If one row has no columns then it has a bitset size of 0, but the code was incorrectly assuming that if the left row had a non-zero number of fields then the right row would also have at least one field, so it was copying invalid bytes and and treating them as part of the bitset. I'm not sure whether this bug was also present in the original implementation or whether it was introduced in https://github.com/apache/spark/pull/7892 (which fixed another bug in this code).
2. **Incorrect updating of data offsets for null variable-length fields**: after updating the bitsets and copying fixed-length and variable-length data, we need to perform adjustments to the offsets pointing the start of variable length fields's data. The existing code was _conditionally_ adding a fixed offset to correct for the new length of the combined row, but it is unsafe to do this if the variable-length field has a null value: we always represent nulls by storing `0` in the fixed-length slot, but this code was incorrectly incrementing those values. This bug was present since the original version of `GenerateUnsafeRowJoiner`.

### Why this bug remained latent for so long

The PR which introduced `GenerateUnsafeRowJoiner` features several randomized tests, including tests of the cases where one side of the join has no fields and where string-valued fields are null. However, the existing assertions were too weak to uncover this bug:

- If a null field has a non-zero value in its fixed-length data slot then this will not cause problems for field accesses because the null-tracking bitmap should still be correct and we will not try to use the incorrect offset for anything.
- If the null tracking bitmap is corrupted by joining against a row with no fields then the corruption occurs in field numbers past the actual field numbers contained in the row. Thus valid `isNullAt()` calls will not read the incorrectly-set bits.

The existing `GenerateUnsafeRowJoinerSuite` tests only exercised `.get()` and `isNullAt()`, but didn't actually check the UnsafeRows for bit-for-bit equality, preventing these bugs from failing assertions. It turns out that there was even a [GenerateUnsafeRowJoinerBitsetSuite](03377d2522/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerBitsetSuite.scala) but it looks like it also didn't catch this problem because it only tested the bitsets in an end-to-end fashion by accessing them through the `UnsafeRow` interface instead of actually comparing the bitsets' bytes.

### Impact of these bugs

- This bug will cause `equals()` and `hashCode()` to be incorrect for these rows, which will be problematic in case`GenerateUnsafeRowJoiner`'s results are used as join or grouping keys.
- Chained / repeated invocations of `GenerateUnsafeRowJoiner` may result in reads from invalid null bitmap positions causing fields to incorrectly become NULL (see the end-to-end example below).
  - It looks like this generally only happens in `CartesianProductExec`, which our query optimizer often avoids executing (usually we try to plan a `BroadcastNestedLoopJoin` instead).

### End-to-end test case demonstrating the problem

The following query demonstrates how this bug may result in incorrect query results:

```sql
set spark.sql.autoBroadcastJoinThreshold=-1; -- Needed to trigger CartesianProductExec

create table a as select * from values 1;
create table b as select * from values 2;

SELECT
  t3.col1,
  t1.col1
FROM a t1
CROSS JOIN b t2
CROSS JOIN b t3
```

This should return `(2, 1)` but instead was returning `(null, 1)`.

Column pruning ends up trimming off all columns from `t2`, so when `t2` joins with another table this triggers the bitmap-copying bug. This incorrect bitmap is subsequently copied again when performing the final join, causing the final output to have an incorrectly-set null bit for the first field.

## How was this patch tested?

Strengthened the assertions in existing tests in GenerateUnsafeRowJoinerSuite. Also verified that the end-to-end test case which uncovered this now passes.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #20181 from JoshRosen/SPARK-22984-fix-generate-unsaferow-joiner-bitmap-bugs.
2018-01-09 11:49:10 +08:00
xubo245 68ce792b58 [SPARK-22972] Couldn't find corresponding Hive SerDe for data source provider org.apache.spark.sql.hive.orc
## What changes were proposed in this pull request?
Fix the warning: Couldn't find corresponding Hive SerDe for data source provider org.apache.spark.sql.hive.orc.

## How was this patch tested?
 test("SPARK-22972: hive orc source")
    assert(HiveSerDe.sourceToSerDe("org.apache.spark.sql.hive.orc")
      .equals(HiveSerDe.sourceToSerDe("orc")))

Author: xubo245 <601450868@qq.com>

Closes #20165 from xubo245/HiveSerDe.
2018-01-09 10:15:01 +08:00
Jose Torres 4f7e758834 [SPARK-22912] v2 data source support in MicroBatchExecution
## What changes were proposed in this pull request?

Support for v2 data sources in microbatch streaming.

## How was this patch tested?

A very basic new unit test on the toy v2 implementation of rate source. Once we have a v1 source fully migrated to v2, we'll need to do more detailed compatibility testing.

Author: Jose Torres <jose@databricks.com>

Closes #20097 from jose-torres/v2-impl.
2018-01-08 13:24:08 -08:00
Xianjin YE 40b983c3b4 [SPARK-22952][CORE] Deprecate stageAttemptId in favour of stageAttemptNumber
## What changes were proposed in this pull request?
1.  Deprecate attemptId in StageInfo and add `def attemptNumber() = attemptId`
2. Replace usage of stageAttemptId with stageAttemptNumber

## How was this patch tested?
I manually checked the compiler warning info

Author: Xianjin YE <advancedxy@gmail.com>

Closes #20178 from advancedxy/SPARK-22952.
2018-01-08 23:49:07 +08:00
Wenchen Fan eb45b52e82 [SPARK-21865][SQL] simplify the distribution semantic of Spark SQL
## What changes were proposed in this pull request?

**The current shuffle planning logic**

1. Each operator specifies the distribution requirements for its children, via the `Distribution` interface.
2. Each operator specifies its output partitioning, via the `Partitioning` interface.
3. `Partitioning.satisfy` determines whether a `Partitioning` can satisfy a `Distribution`.
4. For each operator, check each child of it, add a shuffle node above the child if the child partitioning can not satisfy the required distribution.
5. For each operator, check if its children's output partitionings are compatible with each other, via the `Partitioning.compatibleWith`.
6. If the check in 5 failed, add a shuffle above each child.
7. try to eliminate the shuffles added in 6, via `Partitioning.guarantees`.

This design has a major problem with the definition of "compatible".

`Partitioning.compatibleWith` is not well defined, ideally a `Partitioning` can't know if it's compatible with other `Partitioning`, without more information from the operator. For example, `t1 join t2 on t1.a = t2.b`, `HashPartitioning(a, 10)` should be compatible with `HashPartitioning(b, 10)` under this case, but the partitioning itself doesn't know it.

As a result, currently `Partitioning.compatibleWith` always return false except for literals, which make it almost useless. This also means, if an operator has distribution requirements for multiple children, Spark always add shuffle nodes to all the children(although some of them can be eliminated). However, there is no guarantee that the children's output partitionings are compatible with each other after adding these shuffles, we just assume that the operator will only specify `ClusteredDistribution` for multiple children.

I think it's very hard to guarantee children co-partition for all kinds of operators, and we can not even give a clear definition about co-partition between distributions like `ClusteredDistribution(a,b)` and `ClusteredDistribution(c)`.

I think we should drop the "compatible" concept in the distribution model, and let the operator achieve the co-partition requirement by special distribution requirements.

**Proposed shuffle planning logic after this PR**
(The first 4 are same as before)
1. Each operator specifies the distribution requirements for its children, via the `Distribution` interface.
2. Each operator specifies its output partitioning, via the `Partitioning` interface.
3. `Partitioning.satisfy` determines whether a `Partitioning` can satisfy a `Distribution`.
4. For each operator, check each child of it, add a shuffle node above the child if the child partitioning can not satisfy the required distribution.
5. For each operator, check if its children's output partitionings have the same number of partitions.
6. If the check in 5 failed, pick the max number of partitions from children's output partitionings, and add shuffle to child whose number of partitions doesn't equal to the max one.

The new distribution model is very simple, we only have one kind of relationship, which is `Partitioning.satisfy`. For multiple children, Spark only guarantees they have the same number of partitions, and it's the operator's responsibility to leverage this guarantee to achieve more complicated requirements. For example, non-broadcast joins can use the newly added `HashPartitionedDistribution` to achieve co-partition.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19080 from cloud-fan/exchange.
2018-01-08 19:41:41 +08:00
Josh Rosen 2c73d2a948 [SPARK-22983] Don't push filters beneath aggregates with empty grouping expressions
## What changes were proposed in this pull request?

The following SQL query should return zero rows, but in Spark it actually returns one row:

```
SELECT 1 from (
  SELECT 1 AS z,
  MIN(a.x)
  FROM (select 1 as x) a
  WHERE false
) b
where b.z != b.z
```

The problem stems from the `PushDownPredicate` rule: when this rule encounters a filter on top of an Aggregate operator, e.g. `Filter(Agg(...))`, it removes the original filter and adds a new filter onto Aggregate's child, e.g. `Agg(Filter(...))`. This is sometimes okay, but the case above is a counterexample: because there is no explicit `GROUP BY`, we are implicitly computing a global aggregate over the entire table so the original filter was not acting like a `HAVING` clause filtering the number of groups: if we push this filter then it fails to actually reduce the cardinality of the Aggregate output, leading to the wrong answer.

In 2016 I fixed a similar problem involving invalid pushdowns of data-independent filters (filters which reference no columns of the filtered relation). There was additional discussion after my fix was merged which pointed out that my patch was an incomplete fix (see #15289), but it looks I must have either misunderstood the comment or forgot to follow up on the additional points raised there.

This patch fixes the problem by choosing to never push down filters in cases where there are no grouping expressions. Since there are no grouping keys, the only columns are aggregate columns and we can't push filters defined over aggregate results, so this change won't cause us to miss out on any legitimate pushdown opportunities.

## How was this patch tested?

New regression tests in `SQLQueryTestSuite` and `FilterPushdownSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #20180 from JoshRosen/SPARK-22983-dont-push-filters-beneath-aggs-with-empty-grouping-expressions.
2018-01-08 16:04:03 +08:00
hyukjinkwon 8fdeb4b994 [SPARK-22979][PYTHON][SQL] Avoid per-record type dispatch in Python data conversion (EvaluatePython.fromJava)
## What changes were proposed in this pull request?

Seems we can avoid type dispatch for each value when Java objection (from Pyrolite) -> Spark's internal data format because we know the schema ahead.

I manually performed the benchmark as below:

```scala
  test("EvaluatePython.fromJava / EvaluatePython.makeFromJava") {
    val numRows = 1000 * 1000
    val numFields = 30

    val random = new Random(System.nanoTime())
    val types = Array(
      BooleanType, ByteType, FloatType, DoubleType, IntegerType, LongType, ShortType,
      DecimalType.ShortDecimal, DecimalType.IntDecimal, DecimalType.ByteDecimal,
      DecimalType.FloatDecimal, DecimalType.LongDecimal, new DecimalType(5, 2),
      new DecimalType(12, 2), new DecimalType(30, 10), CalendarIntervalType)
    val schema = RandomDataGenerator.randomSchema(random, numFields, types)
    val rows = mutable.ArrayBuffer.empty[Array[Any]]
    var i = 0
    while (i < numRows) {
      val row = RandomDataGenerator.randomRow(random, schema)
      rows += row.toSeq.toArray
      i += 1
    }

    val benchmark = new Benchmark("EvaluatePython.fromJava / EvaluatePython.makeFromJava", numRows)
    benchmark.addCase("Before - EvaluatePython.fromJava", 3) { _ =>
      var i = 0
      while (i < numRows) {
        EvaluatePython.fromJava(rows(i), schema)
        i += 1
      }
    }

    benchmark.addCase("After - EvaluatePython.makeFromJava", 3) { _ =>
      val fromJava = EvaluatePython.makeFromJava(schema)
      var i = 0
      while (i < numRows) {
        fromJava(rows(i))
        i += 1
      }
    }

    benchmark.run()
  }
```

```
EvaluatePython.fromJava / EvaluatePython.makeFromJava: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Before - EvaluatePython.fromJava              1265 / 1346          0.8        1264.8       1.0X
After - EvaluatePython.makeFromJava            571 /  649          1.8         570.8       2.2X
```

If the structure is nested, I think the advantage should be larger than this.

## How was this patch tested?

Existing tests should cover this. Also, I manually checked if the values from before / after are actually same via `assert` when performing the benchmarks.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20172 from HyukjinKwon/type-dispatch-python-eval.
2018-01-08 13:59:08 +08:00
Josh Rosen 71d65a3215 [SPARK-22985] Fix argument escaping bug in from_utc_timestamp / to_utc_timestamp codegen
## What changes were proposed in this pull request?

This patch adds additional escaping in `from_utc_timestamp` / `to_utc_timestamp` expression codegen in order to a bug where invalid timezones which contain special characters could cause generated code to fail to compile.

## How was this patch tested?

New regression tests in `DateExpressionsSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #20182 from JoshRosen/SPARK-22985-fix-utc-timezone-function-escaping-bugs.
2018-01-08 11:39:45 +08:00
Takeshi Yamamuro 18e9414999 [SPARK-22973][SQL] Fix incorrect results of Casting Map to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting maps into strings;
```
scala> Seq(Map(1 -> "a", 2 -> "b")).toDF("a").write.saveAsTable("t")
scala> sql("SELECT cast(a as String) FROM t").show(false)
+----------------------------------------------------------------+
|a                                                               |
+----------------------------------------------------------------+
|org.apache.spark.sql.catalyst.expressions.UnsafeMapData38bdd75d|
+----------------------------------------------------------------+
```
This pr modified the result into;
```
+----------------+
|a               |
+----------------+
|[1 -> a, 2 -> b]|
+----------------+
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20166 from maropu/SPARK-22973.
2018-01-07 13:42:01 +08:00
gatorsmile 9a7048b288 [HOTFIX] Fix style checking failure
## What changes were proposed in this pull request?
This PR is to fix the  style checking failure.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20175 from gatorsmile/stylefix.
2018-01-07 00:19:21 +08:00
fjh100456 7b78041423 [SPARK-21786][SQL] When acquiring 'compressionCodecClassName' in 'ParquetOptions', parquet.compression needs to be considered.
[SPARK-21786][SQL] When acquiring 'compressionCodecClassName' in 'ParquetOptions', `parquet.compression` needs to be considered.

## What changes were proposed in this pull request?
Since Hive 1.1, Hive allows users to set parquet compression codec via table-level properties parquet.compression. See the JIRA: https://issues.apache.org/jira/browse/HIVE-7858 . We do support orc.compression for ORC. Thus, for external users, it is more straightforward to support both. See the stackflow question: https://stackoverflow.com/questions/36941122/spark-sql-ignores-parquet-compression-propertie-specified-in-tblproperties
In Spark side, our table-level compression conf compression was added by #11464 since Spark 2.0.
We need to support both table-level conf. Users might also use session-level conf spark.sql.parquet.compression.codec. The priority rule will be like
If other compression codec configuration was found through hive or parquet, the precedence would be compression, parquet.compression, spark.sql.parquet.compression.codec. Acceptable values include: none, uncompressed, snappy, gzip, lzo.
The rule for Parquet is consistent with the ORC after the change.

Changes:
1.Increased acquiring 'compressionCodecClassName' from `parquet.compression`,and the precedence order is `compression`,`parquet.compression`,`spark.sql.parquet.compression.codec`, just like what we do in `OrcOptions`.

2.Change `spark.sql.parquet.compression.codec` to support "none".Actually in `ParquetOptions`,we do support "none" as equivalent to "uncompressed", but it does not allowed to configured to "none".

3.Change `compressionCode` to `compressionCodecClassName`.

## How was this patch tested?
Add test.

Author: fjh100456 <fu.jinhua6@zte.com.cn>

Closes #20076 from fjh100456/ParquetOptionIssue.
2018-01-06 18:19:57 +08:00
zuotingbing be9a804f2e [SPARK-22793][SQL] Memory leak in Spark Thrift Server
# What changes were proposed in this pull request?
1. Start HiveThriftServer2.
2. Connect to thriftserver through beeline.
3. Close the beeline.
4. repeat step2 and step 3 for many times.
we found there are many directories never be dropped under the path `hive.exec.local.scratchdir` and `hive.exec.scratchdir`, as we know the scratchdir has been added to deleteOnExit when it be created. So it means that the cache size of FileSystem `deleteOnExit` will keep increasing until JVM terminated.

In addition, we use `jmap -histo:live [PID]`
to printout the size of objects in HiveThriftServer2 Process, we can find the object `org.apache.spark.sql.hive.client.HiveClientImpl` and `org.apache.hadoop.hive.ql.session.SessionState` keep increasing even though we closed all the beeline connections, which may caused the leak of Memory.

# How was this patch tested?
manual tests

This PR follw-up the https://github.com/apache/spark/pull/19989

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #20029 from zuotingbing/SPARK-22793.
2018-01-06 18:07:45 +08:00
Takeshi Yamamuro e8af7e8aec [SPARK-22937][SQL] SQL elt output binary for binary inputs
## What changes were proposed in this pull request?
This pr modified `elt` to output binary for binary inputs.
`elt` in the current master always output data as a string. But, in some databases (e.g., MySQL), if all inputs are binary, `elt` also outputs binary (Also, this might be a small surprise).
This pr is related to #19977.

## How was this patch tested?
Added tests in `SQLQueryTestSuite` and `TypeCoercionSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20135 from maropu/SPARK-22937.
2018-01-06 09:26:03 +08:00
Bruce Robbins c0b7424eca [SPARK-22940][SQL] HiveExternalCatalogVersionsSuite should succeed on platforms that don't have wget
## What changes were proposed in this pull request?

Modified HiveExternalCatalogVersionsSuite.scala to use Utils.doFetchFile to download different versions of Spark binaries rather than launching wget as an external process.

On platforms that don't have wget installed, this suite fails with an error.

cloud-fan : would you like to check this change?

## How was this patch tested?

1) test-only of HiveExternalCatalogVersionsSuite on several platforms. Tested bad mirror, read timeout, and redirects.
2) ./dev/run-tests

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #20147 from bersprockets/SPARK-22940-alt.
2018-01-05 09:58:28 -08:00
Adrian Ionescu 51c33bd0d4 [SPARK-22961][REGRESSION] Constant columns should generate QueryPlanConstraints
## What changes were proposed in this pull request?

#19201 introduced the following regression: given something like `df.withColumn("c", lit(2))`, we're no longer picking up `c === 2` as a constraint and infer filters from it when joins are involved, which may lead to noticeable performance degradation.

This patch re-enables this optimization by picking up Aliases of Literals in Projection lists as constraints and making sure they're not treated as aliased columns.

## How was this patch tested?

Unit test was added.

Author: Adrian Ionescu <adrian@databricks.com>

Closes #20155 from adrian-ionescu/constant_constraints.
2018-01-05 21:32:39 +08:00
Takeshi Yamamuro 52fc5c17d9 [SPARK-22825][SQL] Fix incorrect results of Casting Array to String
## What changes were proposed in this pull request?
This pr fixed the issue when casting arrays into strings;
```
scala> val df = spark.range(10).select('id.cast("integer")).agg(collect_list('id).as('ids))
scala> df.write.saveAsTable("t")
scala> sql("SELECT cast(ids as String) FROM t").show(false)
+------------------------------------------------------------------+
|ids                                                               |
+------------------------------------------------------------------+
|org.apache.spark.sql.catalyst.expressions.UnsafeArrayData8bc285df|
+------------------------------------------------------------------+
```

This pr modified the result into;
```
+------------------------------+
|ids                           |
+------------------------------+
|[0, 1, 2, 3, 4, 5, 6, 7, 8, 9]|
+------------------------------+
```

## How was this patch tested?
Added tests in `CastSuite` and `SQLQuerySuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20024 from maropu/SPARK-22825.
2018-01-05 14:02:21 +08:00
Juliusz Sompolski df7fc3ef38 [SPARK-22957] ApproxQuantile breaks if the number of rows exceeds MaxInt
## What changes were proposed in this pull request?

32bit Int was used for row rank.
That overflowed in a dataframe with more than 2B rows.

## How was this patch tested?

Added test, but ignored, as it takes 4 minutes.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20152 from juliuszsompolski/SPARK-22957.
2018-01-05 10:16:34 +08:00
Takeshi Yamamuro 6f68316e98 [SPARK-22771][SQL] Add a missing return statement in Concat.checkInputDataTypes
## What changes were proposed in this pull request?
This pr is a follow-up to fix a bug left in #19977.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20149 from maropu/SPARK-22771-FOLLOWUP.
2018-01-04 21:15:10 +08:00
Wenchen Fan d5861aba9d [SPARK-22945][SQL] add java UDF APIs in the functions object
## What changes were proposed in this pull request?

Currently Scala users can use UDF like
```
val foo = udf((i: Int) => Math.random() + i).asNondeterministic
df.select(foo('a))
```
Python users can also do it with similar APIs. However Java users can't do it, we should add Java UDF APIs in the functions object.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20141 from cloud-fan/udf.
2018-01-04 19:17:22 +08:00
Kent Yao 9fa703e893 [SPARK-22950][SQL] Handle ChildFirstURLClassLoader's parent
## What changes were proposed in this pull request?

ChildFirstClassLoader's parent is set to null, so we can't get jars from its parent. This will cause ClassNotFoundException during HiveClient initialization with builtin hive jars, where we may should use spark context loader instead.

## How was this patch tested?

add new ut
cc cloud-fan gatorsmile

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #20145 from yaooqinn/SPARK-22950.
2018-01-04 19:10:10 +08:00
Felix Cheung df95a908ba [SPARK-22933][SPARKR] R Structured Streaming API for withWatermark, trigger, partitionBy
## What changes were proposed in this pull request?

R Structured Streaming API for withWatermark, trigger, partitionBy

## How was this patch tested?

manual, unit tests

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #20129 from felixcheung/rwater.
2018-01-03 21:43:14 -08:00
Wenchen Fan 7d045c5f00 [SPARK-22944][SQL] improve FoldablePropagation
## What changes were proposed in this pull request?

`FoldablePropagation` is a little tricky as it needs to handle attributes that are miss-derived from children, e.g. outer join outputs. This rule does a kind of stop-able tree transform, to skip to apply this rule when hit a node which may have miss-derived attributes.

Logically we should be able to apply this rule above the unsupported nodes, by just treating the unsupported nodes as leaf nodes. This PR improves this rule to not stop the tree transformation, but reduce the foldable expressions that we want to propagate.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20139 from cloud-fan/foldable.
2018-01-04 13:14:52 +08:00
Wenchen Fan b297029130 [SPARK-20960][SQL] make ColumnVector public
## What changes were proposed in this pull request?

move `ColumnVector` and related classes to `org.apache.spark.sql.vectorized`, and improve the document.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20116 from cloud-fan/column-vector.
2018-01-04 07:28:53 +08:00
Wenchen Fan a66fe36cee [SPARK-20236][SQL] dynamic partition overwrite
## What changes were proposed in this pull request?

When overwriting a partitioned table with dynamic partition columns, the behavior is different between data source and hive tables.

data source table: delete all partition directories that match the static partition values provided in the insert statement.

hive table: only delete partition directories which have data written into it

This PR adds a new config to make users be able to choose hive's behavior.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18714 from cloud-fan/overwrite-partition.
2018-01-03 22:18:13 +08:00
gatorsmile 1a87a1609c [SPARK-22934][SQL] Make optional clauses order insensitive for CREATE TABLE SQL statement
## What changes were proposed in this pull request?
Currently, our CREATE TABLE syntax require the EXACT order of clauses. It is pretty hard to remember the exact order. Thus, this PR is to make optional clauses order insensitive for `CREATE TABLE` SQL statement.

```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
    [(col_name1 col_type1 [COMMENT col_comment1], ...)]
    USING datasource
    [OPTIONS (key1=val1, key2=val2, ...)]
    [PARTITIONED BY (col_name1, col_name2, ...)]
    [CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
    [LOCATION path]
    [COMMENT table_comment]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
    [AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
    [OPTIONS (key1=val1, key2=val2, ...)]
    [PARTITIONED BY (col_name1, col_name2, ...)]
    [CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
    [LOCATION path]
    [COMMENT table_comment]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

The same idea is also applicable to Create Hive Table.
```
CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name
    [(col_name1[:] col_type1 [COMMENT col_comment1], ...)]
    [COMMENT table_comment]
    [PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
    [ROW FORMAT row_format]
    [STORED AS file_format]
    [LOCATION path]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
    [AS select_statement]
```

The proposal is to make the following clauses order insensitive.
```
    [COMMENT table_comment]
    [PARTITIONED BY (col_name2[:] col_type2 [COMMENT col_comment2], ...)]
    [ROW FORMAT row_format]
    [STORED AS file_format]
    [LOCATION path]
    [TBLPROPERTIES (key1=val1, key2=val2, ...)]
```

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20133 from gatorsmile/createDataSourceTableDDL.
2018-01-03 22:09:30 +08:00
Juliusz Sompolski 247a08939d [SPARK-22938] Assert that SQLConf.get is accessed only on the driver.
## What changes were proposed in this pull request?

Assert if code tries to access SQLConf.get on executor.
This can lead to hard to detect bugs, where the executor will read fallbackConf, falling back to default config values, ignoring potentially changed non-default configs.
If a config is to be passed to executor code, it needs to be read on the driver, and passed explicitly.

## How was this patch tested?

Check in existing tests.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20136 from juliuszsompolski/SPARK-22938.
2018-01-03 21:40:51 +08:00
Xianjin YE a6fc300e91 [SPARK-22897][CORE] Expose stageAttemptId in TaskContext
## What changes were proposed in this pull request?
stageAttemptId added in TaskContext and corresponding construction modification

## How was this patch tested?
Added a new test in TaskContextSuite, two cases are tested:
1. Normal case without failure
2. Exception case with resubmitted stages

Link to [SPARK-22897](https://issues.apache.org/jira/browse/SPARK-22897)

Author: Xianjin YE <advancedxy@gmail.com>

Closes #20082 from advancedxy/SPARK-22897.
2018-01-02 23:30:38 +08:00
gatorsmile e0c090f227 [SPARK-22932][SQL] Refactor AnalysisContext
## What changes were proposed in this pull request?
Add a `reset` function to ensure the state in `AnalysisContext ` is per-query.

## How was this patch tested?
The existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20127 from gatorsmile/refactorAnalysisContext.
2018-01-02 09:19:18 +08:00
Bryan Cutler 1c9f95cb77 [SPARK-22530][PYTHON][SQL] Adding Arrow support for ArrayType
## What changes were proposed in this pull request?

This change adds `ArrayType` support for working with Arrow in pyspark when creating a DataFrame, calling `toPandas()`, and using vectorized `pandas_udf`.

## How was this patch tested?

Added new Python unit tests using Array data.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20114 from BryanCutler/arrow-ArrayType-support-SPARK-22530.
2018-01-02 07:13:27 +09:00
Sean Owen c284c4e1f6 [MINOR] Fix a bunch of typos 2018-01-02 07:10:19 +09:00
gatorsmile cfbe11e816 [SPARK-22895][SQL] Push down the deterministic predicates that are after the first non-deterministic
## What changes were proposed in this pull request?
Currently, we do not guarantee an order evaluation of conjuncts in either Filter or Join operator. This is also true to the mainstream RDBMS vendors like DB2 and MS SQL Server. Thus, we should also push down the deterministic predicates that are after the first non-deterministic, if possible.

## How was this patch tested?
Updated the existing test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20069 from gatorsmile/morePushDown.
2017-12-31 15:06:54 +08:00
Gabor Somogyi ee3af15fea [SPARK-22363][SQL][TEST] Add unit test for Window spilling
## What changes were proposed in this pull request?

There is already test using window spilling, but the test coverage is not ideal.

In this PR the already existing test was fixed and additional cases added.

## How was this patch tested?

Automated: Pass the Jenkins.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20022 from gaborgsomogyi/SPARK-22363.
2017-12-31 14:47:23 +08:00
Zhenhua Wang 234d9435d4 [TEST][MINOR] remove redundant EliminateSubqueryAliases in test code
## What changes were proposed in this pull request?

The `analyze` method in `implicit class DslLogicalPlan` already includes `EliminateSubqueryAliases`. So there's no need to call `EliminateSubqueryAliases` again after calling `analyze` in some test code.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #20122 from wzhfy/redundant_code.
2017-12-30 20:48:39 +08:00
Takeshi Yamamuro f2b3525c17 [SPARK-22771][SQL] Concatenate binary inputs into a binary output
## What changes were proposed in this pull request?
This pr modified `concat` to concat binary inputs into a single binary output.
`concat` in the current master always output data as a string. But, in some databases (e.g., PostgreSQL), if all inputs are binary, `concat` also outputs binary.

## How was this patch tested?
Added tests in `SQLQueryTestSuite` and `TypeCoercionSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19977 from maropu/SPARK-22771.
2017-12-30 14:09:56 +08:00
WeichenXu 2ea17afb63 [SPARK-22881][ML][TEST] ML regression package testsuite add StructuredStreaming test
## What changes were proposed in this pull request?

ML regression package testsuite add StructuredStreaming test

In order to make testsuite easier to modify, new helper function added in `MLTest`:
```
def testTransformerByGlobalCheckFunc[A : Encoder](
      dataframe: DataFrame,
      transformer: Transformer,
      firstResultCol: String,
      otherResultCols: String*)
      (globalCheckFunction: Seq[Row] => Unit): Unit
```

## How was this patch tested?

N/A

Author: WeichenXu <weichen.xu@databricks.com>
Author: Bago Amirbekian <bago@databricks.com>

Closes #19979 from WeichenXu123/ml_stream_test.
2017-12-29 20:06:56 -08:00
oraviv fcf66a3276 [SPARK-21657][SQL] optimize explode quadratic memory consumpation
## What changes were proposed in this pull request?

The issue has been raised in two Jira tickets: [SPARK-21657](https://issues.apache.org/jira/browse/SPARK-21657), [SPARK-16998](https://issues.apache.org/jira/browse/SPARK-16998). Basically, what happens is that in collection generators like explode/inline we create many rows from each row. Currently each exploded row contains also the column on which it was created. This causes, for example, if we have a 10k array in one row that this array will get copy 10k times - to each of the row. this results a qudratic memory consumption. However, it is a common case that the original column gets projected out after the explode, so we can avoid duplicating it.
In this solution we propose to identify this situation in the optimizer and turn on a flag for omitting the original column in the generation process.

## How was this patch tested?

1. We added a benchmark test to MiscBenchmark that shows x16 improvement in runtimes.
2. We ran some of the other tests in MiscBenchmark and they show 15% improvements.
3. We ran this code on a specific case from our production data with rows containing arrays of size ~200k and it reduced the runtime from 6 hours to 3 mins.

Author: oraviv <oraviv@paypal.com>
Author: uzadude <ohad.raviv@gmail.com>
Author: uzadude <15645757+uzadude@users.noreply.github.com>

Closes #19683 from uzadude/optimize_explode.
2017-12-29 21:08:34 +08:00
Feng Liu cc30ef8009 [SPARK-22916][SQL] shouldn't bias towards build right if user does not specify
## What changes were proposed in this pull request?

When there are no broadcast hints, the current spark strategies will prefer to building the right side, without considering the sizes of the two tables. This patch added the logic to consider the sizes of the two tables for the build side. To make the logic clear, the build side is determined by two steps:

1. If there are broadcast hints, the build side is determined by `broadcastSideByHints`;
2. If there are no broadcast hints, the build side is determined by `broadcastSideBySizes`;
3. If the broadcast is disabled by the config, it falls back to the next cases.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20099 from liufengdb/fix-spark-strategies.
2017-12-29 18:48:47 +08:00
Zhenhua Wang 224375c55f [SPARK-22892][SQL] Simplify some estimation logic by using double instead of decimal
## What changes were proposed in this pull request?

Simplify some estimation logic by using double instead of decimal.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #20062 from wzhfy/simplify_by_double.
2017-12-29 15:39:56 +08:00
Wang Gengliang d4f0b1d2c5 [SPARK-22834][SQL] Make insertion commands have real children to fix UI issues
## What changes were proposed in this pull request?

With #19474,  children of insertion commands are missing in UI.
To fix it:
1. Create a new physical plan `DataWritingCommandExec` to exec `DataWritingCommand` with children.  So that the other commands won't be affected.
2. On creation of `DataWritingCommand`, a new field `allColumns` must be specified, which is the output of analyzed plan.
3. In `FileFormatWriter`, the output schema will use `allColumns` instead of the output of optimized plan.

Before code changes:
![2017-12-19 10 27 10](https://user-images.githubusercontent.com/1097932/34161850-d2fd0acc-e50c-11e7-898a-177154fe7d8e.png)

After code changes:
![2017-12-19 10 27 04](https://user-images.githubusercontent.com/1097932/34161865-de23de26-e50c-11e7-9131-0c32f7b7b749.png)

## How was this patch tested?
Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20020 from gengliangwang/insert.
2017-12-29 15:28:33 +08:00
Feng Liu 67ea11ea0f [SPARK-22891][SQL] Make hive client creation thread safe
## What changes were proposed in this pull request?

This is to walk around the hive issue: https://issues.apache.org/jira/browse/HIVE-11935

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20109 from liufengdb/synchronized.
2017-12-29 15:05:47 +08:00
soonmok-kwon ffe6fd77a4 [SPARK-22818][SQL] csv escape of quote escape
## What changes were proposed in this pull request?

Escape of escape should be considered when using the UniVocity csv encoding/decoding library.

Ref: https://github.com/uniVocity/univocity-parsers#escaping-quote-escape-characters

One option is added for reading and writing CSV: `escapeQuoteEscaping`

## How was this patch tested?

Unit test added.

Author: soonmok-kwon <soonmok.kwon@navercorp.com>

Closes #20004 from ep1804/SPARK-22818.
2017-12-29 07:30:06 +08:00
Yuming Wang 613b71a123 [SPARK-22890][TEST] Basic tests for DateTimeOperations
## What changes were proposed in this pull request?

Test Coverage for `DateTimeOperations`, this is a Sub-tasks for [SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20061 from wangyum/SPARK-22890.
2017-12-29 06:58:38 +08:00
Dongjoon Hyun 5536f3181c [MINOR][BUILD] Fix Java linter errors
## What changes were proposed in this pull request?

This PR cleans up a few Java linter errors for Apache Spark 2.3 release.

## How was this patch tested?

```bash
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```

We can see the result from [Travis CI](https://travis-ci.org/dongjoon-hyun/spark/builds/322470787), too.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20101 from dongjoon-hyun/fix-java-lint.
2017-12-28 09:43:50 -06:00
Zhenhua Wang 2877817420 [SPARK-22917][SQL] Should not try to generate histogram for empty/null columns
## What changes were proposed in this pull request?

For empty/null column, the result of `ApproximatePercentile` is null. Then in `ApproxCountDistinctForIntervals`, a `MatchError` (for `endpoints`) will be thrown if we try to generate histogram for that column. Besides, there is no need to generate histogram for such column. In this patch, we exclude such column when generating histogram.

## How was this patch tested?

Enhanced test cases for empty/null columns.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #20102 from wzhfy/no_record_hgm_bug.
2017-12-28 21:49:37 +08:00
Wenchen Fan 755f2f5189 [SPARK-20392][SQL][FOLLOWUP] should not add extra AnalysisBarrier
## What changes were proposed in this pull request?

I found this problem while auditing the analyzer code. It's dangerous to introduce extra `AnalysisBarrer` during analysis, as the plan inside it will bypass all analysis afterward, which may not be expected. We should only preserve `AnalysisBarrer` but not introduce new ones.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20094 from cloud-fan/barrier.
2017-12-28 21:33:03 +08:00
Shixiong Zhu 32ec269d08 [SPARK-22909][SS] Move Structured Streaming v2 APIs to streaming folder
## What changes were proposed in this pull request?

This PR moves Structured Streaming v2 APIs to streaming folder as following:
```
sql/core/src/main/java/org/apache/spark/sql/sources/v2/streaming
├── ContinuousReadSupport.java
├── ContinuousWriteSupport.java
├── MicroBatchReadSupport.java
├── MicroBatchWriteSupport.java
├── reader
│   ├── ContinuousDataReader.java
│   ├── ContinuousReader.java
│   ├── MicroBatchReader.java
│   ├── Offset.java
│   └── PartitionOffset.java
└── writer
    └── ContinuousWriter.java
```

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20093 from zsxwing/move.
2017-12-28 12:35:17 +08:00
Kazuaki Ishizaki 5683984520 [SPARK-18016][SQL][FOLLOW-UP] Code Generation: Constant Pool Limit - reduce entries for mutable state
## What changes were proposed in this pull request?

This PR addresses additional review comments in #19811

## How was this patch tested?

Existing test suites

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

Closes #20036 from kiszk/SPARK-18066-followup.
2017-12-28 12:28:19 +08:00
Marco Gaido 774715d5c7 [SPARK-22904][SQL] Add tests for decimal operations and string casts
## What changes were proposed in this pull request?

Test coverage for arithmetic operations leading to:

 1. Precision loss
 2. Overflow

Moreover, tests for casting bad string to other input types and for using bad string as operators of some functions.

## How was this patch tested?

added tests

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20084 from mgaido91/SPARK-22904.
2017-12-27 23:53:10 +08:00
xu.wenchun 6674acd1ed [SPARK-22846][SQL] Fix table owner is null when creating table through spark sql or thriftserver
## What changes were proposed in this pull request?
fix table owner is null when create new table through spark sql

## How was this patch tested?
manual test.
1、first create a table
2、then select the table properties from mysql which connected to hive metastore

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

Author: xu.wenchun <xu.wenchun@immomo.com>

Closes #20034 from BruceXu1991/SPARK-22846.
2017-12-27 10:08:32 +08:00
Yuming Wang 91d1b300d4 [SPARK-22894][SQL] DateTimeOperations should accept SQL like string type
## What changes were proposed in this pull request?

`DateTimeOperations` accept [`StringType`](ae998ec2b5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala (L669)),  but:

```
spark-sql> SELECT '2017-12-24' + interval 2 months 2 seconds;
Error in query: cannot resolve '(CAST('2017-12-24' AS DOUBLE) + interval 2 months 2 seconds)' due to data type mismatch: differing types in '(CAST('2017-12-24' AS DOUBLE) + interval 2 months 2 seconds)' (double and calendarinterval).; line 1 pos 7;
'Project [unresolvedalias((cast(2017-12-24 as double) + interval 2 months 2 seconds), None)]
+- OneRowRelation
spark-sql>
```

After this PR:
```
spark-sql> SELECT '2017-12-24' + interval 2 months 2 seconds;
2018-02-24 00:00:02
Time taken: 0.2 seconds, Fetched 1 row(s)

```

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20067 from wangyum/SPARK-22894.
2017-12-26 09:40:41 -08:00
Wenchen Fan 9348e68420 [SPARK-22833][EXAMPLE] Improvement SparkHive Scala Examples
## What changes were proposed in this pull request?
Some improvements:
1. Point out we are using both Spark SQ native syntax and HQL syntax in the example
2. Avoid using the same table name with temp view, to not confuse users.
3. Create the external hive table with a directory that already has data, which is a more common use case.
4. Remove the usage of `spark.sql.parquet.writeLegacyFormat`. This config was introduced by https://github.com/apache/spark/pull/8566 and has nothing to do with Hive.
5. Remove `repartition` and `coalesce` example. These 2 are not Hive specific, we should put them in a different example file. BTW they can't accurately control the number of output files, `spark.sql.files.maxRecordsPerFile` also controls it.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20081 from cloud-fan/minor.
2017-12-26 09:37:39 -08:00
Marco Gaido ff48b1b338 [SPARK-22901][PYTHON] Add deterministic flag to pyspark UDF
## What changes were proposed in this pull request?

In SPARK-20586 the flag `deterministic` was added to Scala UDF, but it is not available for python UDF. This flag is useful for cases when the UDF's code can return different result with the same input. Due to optimization, duplicate invocations may be eliminated or the function may even be invoked more times than it is present in the query. This can lead to unexpected behavior.

This PR adds the deterministic flag, via the `asNondeterministic` method, to let the user mark the function as non-deterministic and therefore avoid the optimizations which might lead to strange behaviors.

## How was this patch tested?

Manual tests:
```
>>> from pyspark.sql.functions import *
>>> from pyspark.sql.types import *
>>> df_br = spark.createDataFrame([{'name': 'hello'}])
>>> import random
>>> udf_random_col =  udf(lambda: int(100*random.random()), IntegerType()).asNondeterministic()
>>> df_br = df_br.withColumn('RAND', udf_random_col())
>>> random.seed(1234)
>>> udf_add_ten =  udf(lambda rand: rand + 10, IntegerType())
>>> df_br.withColumn('RAND_PLUS_TEN', udf_add_ten('RAND')).show()
+-----+----+-------------+
| name|RAND|RAND_PLUS_TEN|
+-----+----+-------------+
|hello|   3|           13|
+-----+----+-------------+

```

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

Closes #19929 from mgaido91/SPARK-22629.
2017-12-26 06:39:40 -08:00
Takuya UESHIN eb386be1ed [SPARK-21552][SQL] Add DecimalType support to ArrowWriter.
## What changes were proposed in this pull request?

Decimal type is not yet supported in `ArrowWriter`.
This is adding the decimal type support.

## How was this patch tested?

Added a test to `ArrowConvertersSuite`.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18754 from ueshin/issues/SPARK-21552.
2017-12-26 21:37:25 +09:00
Dongjoon Hyun be03d3ad79 [SPARK-22893][SQL][HOTFIX] Fix a error message of VersionsSuite
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/20064 breaks Jenkins tests because it missed to update one error message for Hive 0.12 and Hive 0.13. This PR fixes that.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/3924/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/3977/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/4226/
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.6/4260/

## How was this patch tested?

Pass the Jenkins without failure.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20079 from dongjoon-hyun/SPARK-22893.
2017-12-25 16:17:39 -08:00
Yuming Wang 33ae2437ba [SPARK-22893][SQL] Unified the data type mismatch message
## What changes were proposed in this pull request?

We should use `dataType.simpleString` to unified the data type mismatch message:
Before:
```
spark-sql> select cast(1 as binary);
Error in query: cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast IntegerType to BinaryType; line 1 pos 7;
```
After:
```
park-sql> select cast(1 as binary);
Error in query: cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7;
```

## How was this patch tested?

Exist test.

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20064 from wangyum/SPARK-22893.
2017-12-25 01:14:09 -08:00
Jose Torres 8941a4abca [SPARK-22789] Map-only continuous processing execution
## What changes were proposed in this pull request?

Basic continuous execution, supporting map/flatMap/filter, with commits and advancement through RPC.

## How was this patch tested?

new unit-ish tests (exercising execution end to end)

Author: Jose Torres <jose@databricks.com>

Closes #19984 from jose-torres/continuous-impl.
2017-12-22 23:05:03 -08:00
Michael Armbrust 8df1da396f [SPARK-22862] Docs on lazy elimination of columns missing from an encoder
This behavior has confused some users, so lets clarify it.

Author: Michael Armbrust <michael@databricks.com>

Closes #20048 from marmbrus/datasetAsDocs.
2017-12-21 21:38:16 -08:00
Marco Gaido c6f01caded [SPARK-22750][SQL] Reuse mutable states when possible
## What changes were proposed in this pull request?

The PR introduces a new method `addImmutableStateIfNotExists ` to `CodeGenerator` to allow reusing and sharing the same global variable between different Expressions. This helps reducing the number of global variables needed, which is important to limit the impact on the constant pool.

## How was this patch tested?

added UTs

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

Closes #19940 from mgaido91/SPARK-22750.
2017-12-22 10:13:26 +08:00
Imran Rashid 7beb375bf4 [SPARK-22861][SQL] SQLAppStatusListener handles multi-job executions.
When one execution has multiple jobs, we need to append to the set of
stages, not replace them on every job.

Added unit test and ran existing tests on jenkins

Author: Imran Rashid <irashid@cloudera.com>

Closes #20047 from squito/SPARK-22861.
2017-12-21 15:37:55 -08:00
Tejas Patil fe65361b05 [SPARK-22042][FOLLOW-UP][SQL] ReorderJoinPredicates can break when child's partitioning is not decided
## What changes were proposed in this pull request?

This is a followup PR of https://github.com/apache/spark/pull/19257 where gatorsmile had left couple comments wrt code style.

## How was this patch tested?

Doesn't change any functionality. Will depend on build to see if no checkstyle rules are violated.

Author: Tejas Patil <tejasp@fb.com>

Closes #20041 from tejasapatil/followup_19257.
2017-12-21 09:22:08 -08:00
Yuming Wang 4e107fdb74 [SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPrecision
## What changes were proposed in this pull request?

Test Coverage for `WindowFrameCoercion` and `DecimalPrecision`, this is a Sub-tasks for [SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20008 from wangyum/SPARK-22822.
2017-12-21 09:18:27 -08:00
Wenchen Fan d3a1d9527b [SPARK-22786][SQL] only use AppStatusPlugin in history server
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19681 we introduced a new interface called `AppStatusPlugin`, to register listeners and set up the UI for both live and history UI.

However I think it's an overkill for live UI. For example, we should not register `SQLListener` if users are not using SQL functions. Previously we register the `SQLListener` and set up SQL tab when `SparkSession` is firstly created, which indicates users are going to use SQL functions. But in #19681 , we register the SQL functions during `SparkContext` creation. The same thing should apply to streaming too.

I think we should keep the previous behavior, and only use this new interface for history server.

To reflect this change, I also rename the new interface to `SparkHistoryUIPlugin`

This PR also refines the tests for sql listener.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19981 from cloud-fan/listener.
2017-12-22 01:08:13 +08:00
Wenchen Fan 8a0ed5a5ee [SPARK-22668][SQL] Ensure no global variables in arguments of method split by CodegenContext.splitExpressions()
## What changes were proposed in this pull request?

Passing global variables to the split method is dangerous, as any mutating to it is ignored and may lead to unexpected behavior.

To prevent this, one approach is to make sure no expression would output global variables: Localizing lifetime of mutable states in expressions.

Another approach is, when calling `ctx.splitExpression`, make sure we don't use children's output as parameter names.

Approach 1 is actually hard to do, as we need to check all expressions and operators that support whole-stage codegen. Approach 2 is easier as the callers of `ctx.splitExpressions` are not too many.

Besides, approach 2 is more flexible, as children's output may be other stuff that can't be parameter name: literal, inlined statement(a + 1), etc.

close https://github.com/apache/spark/pull/19865
close https://github.com/apache/spark/pull/19938

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20021 from cloud-fan/codegen.
2017-12-22 00:21:27 +08:00
Bryan Cutler 59d52631eb [SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0
## What changes were proposed in this pull request?

Upgrade Spark to Arrow 0.8.0 for Java and Python.  Also includes an upgrade of Netty to 4.1.17 to resolve dependency requirements.

The highlights that pertain to Spark for the update from Arrow versoin 0.4.1 to 0.8.0 include:

* Java refactoring for more simple API
* Java reduced heap usage and streamlined hot code paths
* Type support for DecimalType, ArrayType
* Improved type casting support in Python
* Simplified type checking in Python

## How was this patch tested?

Existing tests

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19884 from BryanCutler/arrow-upgrade-080-SPARK-22324.
2017-12-21 20:43:56 +09:00
Kazuaki Ishizaki cb9fc8d9b6 [SPARK-22848][SQL] Eliminate mutable state from Stack
## What changes were proposed in this pull request?

This PR eliminates mutable states from the generated code for `Stack`.

## How was this patch tested?

Existing test suites

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

Closes #20035 from kiszk/SPARK-22848.
2017-12-21 14:54:38 +08:00
Xingbo Jiang 9c289a5cb4 [SPARK-22387][SQL] Propagate session configs to data source read/write options
## What changes were proposed in this pull request?

Introduce a new interface `SessionConfigSupport` for `DataSourceV2`, it can help to propagate session configs with the specified key-prefix to all data source operations in this session.

## How was this patch tested?

Add new test suite `DataSourceV2UtilsSuite`.

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

Closes #19861 from jiangxb1987/datasource-configs.
2017-12-21 10:02:30 +08:00
Jose Torres 7798c9e6ef [SPARK-22824] Restore old offset for binary compatibility
## What changes were proposed in this pull request?

Some users depend on source compatibility with the org.apache.spark.sql.execution.streaming.Offset class. Although this is not a stable interface, we can keep it in place for now to simplify upgrades to 2.3.

Author: Jose Torres <jose@databricks.com>

Closes #20012 from joseph-torres/binary-compat.
2017-12-20 10:43:10 -08:00
Dongjoon Hyun 9962390af7 [SPARK-22781][SS] Support creating streaming dataset with ORC files
## What changes were proposed in this pull request?

Like `Parquet`, users can use `ORC` with Apache Spark structured streaming. This PR adds `orc()` to `DataStreamReader`(Scala/Python) in order to support creating streaming dataset with ORC file format more easily like the other file formats. Also, this adds a test coverage for ORC data source and updates the document.

**BEFORE**

```scala
scala> spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start()
<console>:24: error: value orc is not a member of org.apache.spark.sql.streaming.DataStreamReader
       spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start()
```

**AFTER**
```scala
scala> spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start()
res0: org.apache.spark.sql.streaming.StreamingQuery = org.apache.spark.sql.execution.streaming.StreamingQueryWrapper678b3746

scala>
-------------------------------------------
Batch: 0
-------------------------------------------
+---+
|  a|
+---+
|  1|
+---+
```

## How was this patch tested?

Pass the newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19975 from dongjoon-hyun/SPARK-22781.
2017-12-19 23:50:06 -08:00
Fernando Pereira 13268a58f8 [SPARK-22649][PYTHON][SQL] Adding localCheckpoint to Dataset API
## What changes were proposed in this pull request?

This change adds local checkpoint support to datasets and respective bind from Python Dataframe API.

If reliability requirements can be lowered to favor performance, as in cases of further quick transformations followed by a reliable save, localCheckpoints() fit very well.
Furthermore, at the moment Reliable checkpoints still incur double computation (see #9428)
In general it makes the API more complete as well.

## How was this patch tested?

Python land quick use case:

```python
>>> from time import sleep
>>> from pyspark.sql import types as T
>>> from pyspark.sql import functions as F

>>> def f(x):
    sleep(1)
    return x*2
   ...:

>>> df1 = spark.range(30, numPartitions=6)
>>> df2 = df1.select(F.udf(f, T.LongType())("id"))

>>> %time _ = df2.collect()
CPU times: user 7.79 ms, sys: 5.84 ms, total: 13.6 ms
Wall time: 12.2 s

>>> %time df3 = df2.localCheckpoint()
CPU times: user 2.38 ms, sys: 2.3 ms, total: 4.68 ms
Wall time: 10.3 s

>>> %time _ = df3.collect()
CPU times: user 5.09 ms, sys: 410 µs, total: 5.5 ms
Wall time: 148 ms

>>> sc.setCheckpointDir(".")
>>> %time df3 = df2.checkpoint()
CPU times: user 4.04 ms, sys: 1.63 ms, total: 5.67 ms
Wall time: 20.3 s
```

Author: Fernando Pereira <fernando.pereira@epfl.ch>

Closes #19805 from ferdonline/feature_dataset_localCheckpoint.
2017-12-19 20:47:12 -08:00
Youngbin Kim 6e36d8d562 [SPARK-22829] Add new built-in function date_trunc()
## What changes were proposed in this pull request?

Adding date_trunc() as a built-in function.
`date_trunc` is common in other databases, but Spark or Hive does not have support for this. `date_trunc` is commonly used by data scientists and business intelligence application such as Superset (https://github.com/apache/incubator-superset).
We do have `trunc` but this only works with 'MONTH' and 'YEAR' level on the DateType input.

date_trunc() in other databases:
AWS Redshift: http://docs.aws.amazon.com/redshift/latest/dg/r_DATE_TRUNC.html
PostgreSQL: https://www.postgresql.org/docs/9.1/static/functions-datetime.html
Presto: https://prestodb.io/docs/current/functions/datetime.html

## How was this patch tested?

Unit tests

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

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

Author: Youngbin Kim <ykim828@hotmail.com>

Closes #20015 from youngbink/date_trunc.
2017-12-19 20:22:33 -08:00
Sital Kedia 3a7494dfee [SPARK-22827][CORE] Avoid throwing OutOfMemoryError in case of exception in spill
## What changes were proposed in this pull request?
Currently, the task memory manager throws an OutofMemory error when there is an IO exception happens in spill() - https://github.com/apache/spark/blob/master/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java#L194. Similarly there any many other places in code when if a task is not able to acquire memory due to an exception we throw an OutofMemory error which kills the entire executor and hence failing all the tasks that are running on that executor instead of just failing one single task.

## How was this patch tested?

Unit tests

Author: Sital Kedia <skedia@fb.com>

Closes #20014 from sitalkedia/skedia/upstream_SPARK-22827.
2017-12-20 12:21:00 +08:00
Yuming Wang 6129ffa11e [SPARK-22821][TEST] Basic tests for WidenSetOperationTypes, BooleanEquality, StackCoercion and Division
## What changes were proposed in this pull request?

Test Coverage for `WidenSetOperationTypes`, `BooleanEquality`, `StackCoercion`  and `Division`, this is a Sub-tasks for [SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?
N/A

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20006 from wangyum/SPARK-22821.
2017-12-19 11:56:22 -08:00
gatorsmile ef10f452e6 [SPARK-21652][SQL][FOLLOW-UP] Fix rule conflict caused by InferFiltersFromConstraints
## What changes were proposed in this pull request?
The optimizer rule `InferFiltersFromConstraints` could trigger our batch  `Operator Optimizations` exceeds the max iteration limit (i.e., 100) so that the final plan might not be properly optimized. The rule `InferFiltersFromConstraints` could conflict with the other Filter/Join predicate reduction rules. Thus, we need to separate `InferFiltersFromConstraints` from the other rules.

This PR is to separate `InferFiltersFromConstraints ` from the main batch `Operator Optimizations` .

## How was this patch tested?
The existing test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19149 from gatorsmile/inferFilterRule.
2017-12-19 09:05:47 -08:00
Kazuaki Ishizaki ee56fc3432 [SPARK-18016][SQL] Code Generation: Constant Pool Limit - reduce entries for mutable state
## What changes were proposed in this pull request?

This PR is follow-on of #19518. This PR tries to reduce the number of constant pool entries used for accessing mutable state.
There are two directions:
1. Primitive type variables should be allocated at the outer class due to better performance. Otherwise, this PR allocates an array.
2. The length of allocated array is up to 32768 due to avoiding usage of constant pool entry at access (e.g. `mutableStateArray[32767]`).

Here are some discussions to determine these directions.
1. [[1]](https://github.com/apache/spark/pull/19518#issuecomment-346690464), [[2]](https://github.com/apache/spark/pull/19518#issuecomment-346690642), [[3]](https://github.com/apache/spark/pull/19518#issuecomment-346828180), [[4]](https://github.com/apache/spark/pull/19518#issuecomment-346831544), [[5]](https://github.com/apache/spark/pull/19518#issuecomment-346857340)
2. [[6]](https://github.com/apache/spark/pull/19518#issuecomment-346729172), [[7]](https://github.com/apache/spark/pull/19518#issuecomment-346798358), [[8]](https://github.com/apache/spark/pull/19518#issuecomment-346870408)

This PR modifies `addMutableState` function in the `CodeGenerator` to check if the declared state can be easily initialized compacted into an array. We identify three types of states that cannot compacted:

- Primitive type state (ints, booleans, etc) if the number of them does not exceed threshold
- Multiple-dimensional array type
- `inline = true`

When `useFreshName = false`, the given name is used.

Many codes were ported from #19518. Many efforts were put here. I think this PR should credit to bdrillard

With this PR, the following code is generated:
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private boolean isNull_0;
/* 010 */   private boolean isNull_1;
/* 011 */   private boolean isNull_2;
/* 012 */   private int value_2;
/* 013 */   private boolean isNull_3;
...
/* 10006 */   private int value_4999;
/* 10007 */   private boolean isNull_5000;
/* 10008 */   private int value_5000;
/* 10009 */   private InternalRow[] mutableStateArray = new InternalRow[2];
/* 10010 */   private boolean[] mutableStateArray1 = new boolean[7001];
/* 10011 */   private int[] mutableStateArray2 = new int[1001];
/* 10012 */   private UTF8String[] mutableStateArray3 = new UTF8String[6000];
/* 10013 */
...
/* 107956 */     private void init_176() {
/* 107957 */       isNull_4986 = true;
/* 107958 */       value_4986 = -1;
...
/* 108004 */     }
...
```

## How was this patch tested?

Added a new test case to `GeneratedProjectionSuite`

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

Closes #19811 from kiszk/SPARK-18016.
2017-12-20 00:10:54 +08:00
gatorsmile b779c93518 [SPARK-22815][SQL] Keep PromotePrecision in Optimized Plans
## What changes were proposed in this pull request?
We could get incorrect results by running DecimalPrecision twice. This PR resolves the original found in https://github.com/apache/spark/pull/15048 and https://github.com/apache/spark/pull/14797. After this PR, it becomes easier to change it back using `children` instead of using `innerChildren`.

## How was this patch tested?
The existing test.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20000 from gatorsmile/keepPromotePrecision.
2017-12-19 22:17:04 +08:00
gatorsmile 28315714dd [SPARK-22791][SQL][SS] Redact Output of Explain
## What changes were proposed in this pull request?

When calling explain on a query, the output can contain sensitive information. We should provide an admin/user to redact such information.

Before this PR, the plan of SS is like this
```
== Physical Plan ==
*HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#12L])
+- StateStoreSave [value#6], state info [ checkpoint = file:/private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-91c6fac0-609f-4bc8-ad57-52c189f06797/state, runId = 05a4b3af-f02c-40f8-9ff9-a3e18bae496f, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#18L])
      +- StateStoreRestore [value#6], state info [ checkpoint = file:/private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-91c6fac0-609f-4bc8-ad57-52c189f06797/state, runId = 05a4b3af-f02c-40f8-9ff9-a3e18bae496f, opId = 0, ver = 0, numPartitions = 5]
         +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#18L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#18L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *MapElements <function1>, obj#5: java.lang.String
                        +- *DeserializeToObject value#30.toString, obj#4: java.lang.String
                           +- LocalTableScan [value#30]
```

After this PR, we can get the following output if users set `spark.redaction.string.regex` to `file:/[\\w_]+`
```
== Physical Plan ==
*HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#12L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-e7da9b7d-3ec0-474d-8b8c-927f7d12ed72/state, runId = 8a9c3761-93d5-4896-ab82-14c06240dcea, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#32L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-e7da9b7d-3ec0-474d-8b8c-927f7d12ed72/state, runId = 8a9c3761-93d5-4896-ab82-14c06240dcea, opId = 0, ver = 0, numPartitions = 5]
         +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#32L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#32L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *MapElements <function1>, obj#5: java.lang.String
                        +- *DeserializeToObject value#27.toString, obj#4: java.lang.String
                           +- LocalTableScan [value#27]
```
## How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19985 from gatorsmile/redactPlan.
2017-12-19 22:12:23 +08:00
Zhenhua Wang 571aa27554 [SPARK-21984][SQL] Join estimation based on equi-height histogram
## What changes were proposed in this pull request?

Equi-height histogram is one of the state-of-the-art statistics for cardinality estimation, which can provide better estimation accuracy, and good at cases with skew data.

This PR is to improve join estimation based on equi-height histogram. The difference from basic estimation (based on ndv) is the logic for computing join cardinality and the new ndv after join.

The main idea is as follows:
1. find overlapped ranges between two histograms from two join keys;
2. apply the formula `T(A IJ B) = T(A) * T(B) / max(V(A.k1), V(B.k1))` in each overlapped range.

## How was this patch tested?
Added new test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19594 from wzhfy/join_estimation_histogram.
2017-12-19 21:55:21 +08:00
CodingCat ab7346f20c [SPARK-22673][SQL] InMemoryRelation should utilize existing stats whenever possible
## What changes were proposed in this pull request?

The current implementation of InMemoryRelation always uses the most expensive execution plan when writing cache
With CBO enabled, we can actually have a more exact estimation of the underlying table size...

## How was this patch tested?

existing test

Author: CodingCat <zhunansjtu@gmail.com>
Author: Nan Zhu <CodingCat@users.noreply.github.com>
Author: Nan Zhu <nanzhu@uber.com>

Closes #19864 from CodingCat/SPARK-22673.
2017-12-19 21:51:56 +08:00
gatorsmile d4e69595dd [MINOR][SQL] Remove Useless zipWithIndex from ResolveAliases
## What changes were proposed in this pull request?
Remove useless `zipWithIndex` from `ResolveAliases `.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20009 from gatorsmile/try22.
2017-12-19 09:48:31 +08:00
hyukjinkwon fbfa9be7e0 Revert "Revert "[SPARK-22496][SQL] thrift server adds operation logs""
This reverts commit e58f275678.
2017-12-19 07:30:29 +09:00
Marcelo Vanzin 772e4648d9 [SPARK-20653][CORE] Add cleaning of old elements from the status store.
This change restores the functionality that keeps a limited number of
different types (jobs, stages, etc) depending on configuration, to avoid
the store growing indefinitely over time.

The feature is implemented by creating a new type (ElementTrackingStore)
that wraps a KVStore and allows triggers to be set up for when elements
of a certain type meet a certain threshold. Triggers don't need to
necessarily only delete elements, but the current API is set up in a way
that makes that use case easier.

The new store also has a trigger for the "close" call, which makes it
easier for listeners to register code for cleaning things up and flushing
partial state to the store.

The old configurations for cleaning up the stored elements from the core
and SQL UIs are now active again, and the old unit tests are re-enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19751 from vanzin/SPARK-20653.
2017-12-18 14:08:48 -06:00
Yuming Wang 7f6d10a737 [SPARK-22816][TEST] Basic tests for PromoteStrings and InConversion
## What changes were proposed in this pull request?

Test Coverage for `PromoteStrings` and `InConversion`, this is a Sub-tasks for [SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722).

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20001 from wangyum/SPARK-22816.
2017-12-17 09:15:10 -08:00
Yuming Wang 46776234a4 [SPARK-22762][TEST] Basic tests for IfCoercion and CaseWhenCoercion
## What changes were proposed in this pull request?

Basic tests for IfCoercion and CaseWhenCoercion

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>

Closes #19949 from wangyum/SPARK-22762.
2017-12-15 09:58:31 -08:00
Takeshi Yamamuro 9fafa8209c [SPARK-22800][TEST][SQL] Add a SSB query suite
## What changes were proposed in this pull request?
Add a test suite to ensure all the [SSB (Star Schema Benchmark)](https://www.cs.umb.edu/~poneil/StarSchemaB.PDF) queries can be successfully analyzed, optimized and compiled without hitting the max iteration threshold.

## How was this patch tested?
Added `SSBQuerySuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19990 from maropu/SPARK-22800.
2017-12-15 09:56:22 -08:00
gatorsmile e58f275678 Revert "[SPARK-22496][SQL] thrift server adds operation logs"
This reverts commit 0ea2d8c12e.
2017-12-15 09:46:15 -08:00
Yuanjian Li 3775dd31ee [SPARK-22753][SQL] Get rid of dataSource.writeAndRead
## What changes were proposed in this pull request?

As the discussion in https://github.com/apache/spark/pull/16481 and https://github.com/apache/spark/pull/18975#discussion_r155454606
Currently the BaseRelation returned by `dataSource.writeAndRead` only used in `CreateDataSourceTableAsSelect`, planForWriting and writeAndRead has some common code paths.
In this patch I removed the writeAndRead function and added the getRelation function which only use in `CreateDataSourceTableAsSelectCommand` while saving data to non-existing table.

## How was this patch tested?

Existing UT

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #19941 from xuanyuanking/SPARK-22753.
2017-12-14 23:11:13 -08:00
gatorsmile 3fea5c4f19 [SPARK-22787][TEST][SQL] Add a TPC-H query suite
## What changes were proposed in this pull request?
Add a test suite to ensure all the TPC-H queries can be successfully analyzed, optimized and compiled without hitting the max iteration threshold.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19982 from gatorsmile/testTPCH.
2017-12-14 22:56:57 -08:00
zouchenjun 0ea2d8c12e [SPARK-22496][SQL] thrift server adds operation logs
## What changes were proposed in this pull request?
since hive 2.0+  upgrades log4j to log4j2,a lot of [changes](https://issues.apache.org/jira/browse/HIVE-11304) are made working on it.
as spark is not to ready to update its inner hive version(1.2.1) , so I manage to make little changes.
the function registerCurrentOperationLog  is moved from SQLOperstion to its parent class ExecuteStatementOperation so spark can use it.

## How was this patch tested?
manual test

Closes #19721 from ChenjunZou/operation-log.

Author: zouchenjun <zouchenjun@youzan.com>

Closes #19961 from ChenjunZou/spark-22496.
2017-12-14 15:37:26 -08:00
Jose Torres 59daf91b7c [SPARK-22733] Split StreamExecution into MicroBatchExecution and StreamExecution.
## What changes were proposed in this pull request?

StreamExecution is now an abstract base class, which MicroBatchExecution (the current StreamExecution) inherits. When continuous processing is implemented, we'll have a new ContinuousExecution implementation of StreamExecution.

A few fields are also renamed to make them less microbatch-specific.

## How was this patch tested?

refactoring only

Author: Jose Torres <jose@databricks.com>

Closes #19926 from joseph-torres/continuous-refactor.
2017-12-14 14:31:21 -08:00
Prashant Sharma 40de176c93 [SPARK-16496][SQL] Add wholetext as option for reading text in SQL.
## What changes were proposed in this pull request?

In multiple text analysis problems, it is not often desirable for the rows to be split by "\n". There exists a wholeText reader for RDD API, and this JIRA just adds the same support for Dataset API.
## How was this patch tested?

Added relevant new tests for both scala and Java APIs

Author: Prashant Sharma <prashsh1@in.ibm.com>
Author: Prashant Sharma <prashant@apache.org>

Closes #14151 from ScrapCodes/SPARK-16496/wholetext.
2017-12-14 11:19:34 -08:00
Kazuaki Ishizaki 606ae491e4 [SPARK-22774][SQL][TEST] Add compilation check into TPCDSQuerySuite
## What changes were proposed in this pull request?

This PR adds check whether Java code generated by Catalyst can be compiled by `janino` correctly or not into `TPCDSQuerySuite`. Before this PR, this suite only checks whether analysis can be performed correctly or not.

This check will be able to avoid unexpected performance degrade by interpreter execution due to a Java compilation error.

## How was this patch tested?

Existing a test case, but updated it.

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

Closes #19971 from kiszk/SPARK-22774.
2017-12-15 02:14:08 +08:00
Wenchen Fan d095795439 [SPARK-22785][SQL] remove ColumnVector.anyNullsSet
## What changes were proposed in this pull request?
`ColumnVector.anyNullsSet` is not called anywhere except tests, and we can easily replace it with `ColumnVector.numNulls > 0`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19980 from cloud-fan/minor.
2017-12-15 00:29:44 +08:00
Wenchen Fan 7d8e2ca7f8 [SPARK-22775][SQL] move dictionary related APIs from ColumnVector to WritableColumnVector
## What changes were proposed in this pull request?

These dictionary related APIs are special to `WritableColumnVector` and should not be in `ColumnVector`, which will be public soon.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19970 from cloud-fan/final.
2017-12-14 19:33:54 +08:00
Marcelo Vanzin c3dd2a26de [SPARK-22779][SQL] Resolve default values for fallback configs.
SQLConf allows some callers to define a custom default value for
configs, and that complicates a little bit the handling of fallback
config entries, since most of the default value resolution is
hidden by the config code.

This change peaks into the internals of these fallback configs
to figure out the correct default value, and also returns the
current human-readable default when showing the default value
(e.g. through "set -v").

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19974 from vanzin/SPARK-22779.
2017-12-13 22:46:20 -08:00
Jose Torres f8c7c1f21a [SPARK-22732] Add Structured Streaming APIs to DataSourceV2
## What changes were proposed in this pull request?

This PR provides DataSourceV2 API support for structured streaming, including new pieces needed to support continuous processing [SPARK-20928]. High level summary:

- DataSourceV2 includes new mixins to support micro-batch and continuous reads and writes. For reads, we accept an optional user specified schema rather than using the ReadSupportWithSchema model, because doing so would severely complicate the interface.

- DataSourceV2Reader includes new interfaces to read a specific microbatch or read continuously from a given offset. These follow the same setter pattern as the existing Supports* mixins so that they can work with SupportsScanUnsafeRow.

- DataReader (the per-partition reader) has a new subinterface ContinuousDataReader only for continuous processing. This reader has a special method to check progress, and next() blocks for new input rather than returning false.

- Offset, an abstract representation of position in a streaming query, is ported to the public API. (Each type of reader will define its own Offset implementation.)

- DataSourceV2Writer has a new subinterface ContinuousWriter only for continuous processing. Commits to this interface come tagged with an epoch number, as the execution engine will continue to produce new epoch commits as the task continues indefinitely.

Note that this PR does not propose to change the existing DataSourceV2 batch API, or deprecate the existing streaming source/sink internal APIs in spark.sql.execution.streaming.

## How was this patch tested?

Toy implementations of the new interfaces with unit tests.

Author: Jose Torres <jose@databricks.com>

Closes #19925 from joseph-torres/continuous-api.
2017-12-13 22:31:39 -08:00
Wenchen Fan 2a29a60da3 Revert "[SPARK-22600][SQL] Fix 64kb limit for deeply nested expressions under wholestage codegen"
This reverts commit c7d0148615.
2017-12-14 11:22:23 +08:00
Wenchen Fan bc7e4a90c0 Revert "[SPARK-22600][SQL][FOLLOW-UP] Fix a compilation error in TPCDS q75/q77"
This reverts commit ef92999653.
2017-12-14 11:21:34 +08:00
Takeshi Yamamuro ef92999653 [SPARK-22600][SQL][FOLLOW-UP] Fix a compilation error in TPCDS q75/q77
## What changes were proposed in this pull request?
This pr fixed a compilation error of TPCDS `q75`/`q77`  caused by #19813;
```
  java.util.concurrent.ExecutionException: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 371, Column 16: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 371, Column 16: Expression "bhj_matched" is not an rvalue
  at com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:306)
  at com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:293)
  at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)
  at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:135)
```

## How was this patch tested?
Manually checked `q75`/`q77` can be properly compiled

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19969 from maropu/SPARK-22600-FOLLOWUP.
2017-12-13 15:55:16 -08:00
Liang-Chi Hsieh ba0e79f57c [SPARK-22772][SQL] Use splitExpressionsWithCurrentInputs to split codes in elt
## What changes were proposed in this pull request?

In SPARK-22550 which fixes 64KB JVM bytecode limit problem with elt, `buildCodeBlocks` is used to split codes. However, we should use `splitExpressionsWithCurrentInputs` because it considers both normal and wholestage codgen (it is not supported yet, so it simply doesn't split the codes).

## How was this patch tested?

Existing tests.

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

Closes #19964 from viirya/SPARK-22772.
2017-12-13 13:54:16 -08:00
gatorsmile c5a4701acc Revert "[SPARK-21417][SQL] Infer join conditions using propagated constraints"
This reverts commit 6ac57fd0d1.
2017-12-13 11:50:04 -08:00
Wenchen Fan f6bcd3e53f [SPARK-22767][SQL] use ctx.addReferenceObj in InSet and ScalaUDF
## What changes were proposed in this pull request?

We should not operate on `references` directly in `Expression.doGenCode`, instead we should use the high-level API `addReferenceObj`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19962 from cloud-fan/codegen.
2017-12-14 01:16:44 +08:00
Zhenhua Wang 7453ab0243 [SPARK-22745][SQL] read partition stats from Hive
## What changes were proposed in this pull request?

Currently Spark can read table stats (e.g. `totalSize, numRows`) from Hive, we can also support to read partition stats from Hive using the same logic.

## How was this patch tested?

Added a new test case and modified an existing test case.

Author: Zhenhua Wang <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19932 from wzhfy/read_hive_partition_stats.
2017-12-13 16:27:29 +08:00
Tejas Patil 682eb4f2ea [SPARK-22042][SQL] ReorderJoinPredicates can break when child's partitioning is not decided
## What changes were proposed in this pull request?

See jira description for the bug : https://issues.apache.org/jira/browse/SPARK-22042

Fix done in this PR is:  In `EnsureRequirements`, apply `ReorderJoinPredicates` over the input tree before doing its core logic. Since the tree is transformed bottom-up, we can assure that the children are resolved before doing `ReorderJoinPredicates`.

Theoretically this will guarantee to cover all such cases while keeping the code simple. My small grudge is for cosmetic reasons. This PR will look weird given that we don't call rules from other rules (not to my knowledge). I could have moved all the logic for `ReorderJoinPredicates` into `EnsureRequirements` but that will make it a but crowded. I am happy to discuss if there are better options.

## How was this patch tested?

Added a new test case

Author: Tejas Patil <tejasp@fb.com>

Closes #19257 from tejasapatil/SPARK-22042_ReorderJoinPredicates.
2017-12-12 23:30:06 -08:00
Wenchen Fan bdb5e55c2a [SPARK-21322][SQL][FOLLOWUP] support histogram in filter cardinality estimation
## What changes were proposed in this pull request?

some code cleanup/refactor and naming improvement.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19952 from cloud-fan/minor.
2017-12-13 14:49:15 +08:00
gatorsmile 13e489b675 [SPARK-22759][SQL] Filters can be combined iff both are deterministic
## What changes were proposed in this pull request?
The query execution/optimization does not guarantee the expressions are evaluated in order. We only can combine them if and only if both are deterministic. We need to update the optimizer rule: CombineFilters.

## How was this patch tested?
Updated the existing tests.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19947 from gatorsmile/combineFilters.
2017-12-12 22:48:31 -08:00
Dongjoon Hyun 6b80ce4fb2 [SPARK-19809][SQL][TEST][FOLLOWUP] Move the test case to HiveOrcQuerySuite
## What changes were proposed in this pull request?

As a follow-up of #19948 , this PR moves the test case and adds comments.

## How was this patch tested?

Pass the Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19960 from dongjoon-hyun/SPARK-19809-2.
2017-12-12 22:41:38 -08:00
WeichenXu 0e36ba6212 [SPARK-22644][ML][TEST] Make ML testsuite support StructuredStreaming test
## What changes were proposed in this pull request?

We need to add some helper code to make testing ML transformers & models easier with streaming data. These tests might help us catch any remaining issues and we could encourage future PRs to use these tests to prevent new Models & Transformers from having issues.

I add a `MLTest` trait which extends `StreamTest` trait, and override `createSparkSession`. So ML testsuite can only extend `MLTest`, to use both ML & Stream test util functions.

I only modify one testcase in `LinearRegressionSuite`, for first pass review.

Link to #19746

## How was this patch tested?

`MLTestSuite` added.

Author: WeichenXu <weichen.xu@databricks.com>

Closes #19843 from WeichenXu123/ml_stream_test_helper.
2017-12-12 21:28:24 -08:00
Liang-Chi Hsieh c7d0148615 [SPARK-22600][SQL] Fix 64kb limit for deeply nested expressions under wholestage codegen
## What changes were proposed in this pull request?

SPARK-22543 fixes the 64kb compile error for deeply nested expression for non-wholestage codegen. This PR extends it to support wholestage codegen.

This patch brings some util methods in to extract necessary parameters for an expression if it is split to a function.

The util methods are put in object `ExpressionCodegen` under `codegen`. The main entry is `getExpressionInputParams` which returns all necessary parameters to evaluate the given expression in a split function.

This util methods can be used to split expressions too. This is a TODO item later.

## How was this patch tested?

Added test.

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

Closes #19813 from viirya/reduce-expr-code-for-wholestage.
2017-12-13 10:40:05 +08:00
Marco Gaido 4117786a87 [SPARK-22716][SQL] Avoid the creation of mutable states in addReferenceObj
## What changes were proposed in this pull request?

We have two methods to reference an object `addReferenceMinorObj` and `addReferenceObj `. The latter creates a new global variable, which means new entries in the constant pool.

The PR unifies the two method in a single `addReferenceObj` which returns the code to access the object in the `references` array and doesn't add new mutable states.

## How was this patch tested?

added UTs.

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19916 from mgaido91/SPARK-22716.
2017-12-13 10:29:14 +08:00
Dongjoon Hyun 17cdabb887 [SPARK-19809][SQL][TEST] NullPointerException on zero-size ORC file
## What changes were proposed in this pull request?

Until 2.2.1, Spark raises `NullPointerException` on zero-size ORC files. Usually, these zero-size ORC files are generated by 3rd-party apps like Flume.

```scala
scala> sql("create table empty_orc(a int) stored as orc location '/tmp/empty_orc'")

$ touch /tmp/empty_orc/zero.orc

scala> sql("select * from empty_orc").show
java.lang.RuntimeException: serious problem at
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1021)
...
Caused by: java.lang.NullPointerException at
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$BISplitStrategy.getSplits(OrcInputFormat.java:560)
```

After [SPARK-22279](https://github.com/apache/spark/pull/19499), Apache Spark with the default configuration doesn't have this bug. Although Hive 1.2.1 library code path still has the problem, we had better have a test coverage on what we have now in order to prevent future regression on it.

## How was this patch tested?

Pass a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19948 from dongjoon-hyun/SPARK-19809-EMPTY-FILE.
2017-12-13 07:42:24 +09:00
Daniel van der Ende e6dc5f2807 [SPARK-22729][SQL] Add getTruncateQuery to JdbcDialect
In order to enable truncate for PostgreSQL databases in Spark JDBC, a change is needed to the query used for truncating a PostgreSQL table. By default, PostgreSQL will automatically truncate any descendant tables if a TRUNCATE query is executed. As this may result in (unwanted) side-effects, the query used for the truncate should be specified separately for PostgreSQL, specifying only to TRUNCATE a single table.

## What changes were proposed in this pull request?

Add `getTruncateQuery` function to `JdbcDialect.scala`, with default query. Overridden this function for PostgreSQL to only truncate a single table. Also sets `isCascadingTruncateTable` to false, as this will allow truncates for PostgreSQL.

## How was this patch tested?

Existing tests all pass. Added test for `getTruncateQuery`

Author: Daniel van der Ende <daniel.vanderende@gmail.com>

Closes #19911 from danielvdende/SPARK-22717.
2017-12-12 10:41:37 -08:00
Ron Hu ecc179ecaa [SPARK-21322][SQL] support histogram in filter cardinality estimation
## What changes were proposed in this pull request?

Histogram is effective in dealing with skewed distribution. After we generate histogram information for column statistics, we need to adjust filter estimation based on histogram data structure.

## How was this patch tested?

We revised all the unit test cases by including histogram data structure.

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

Author: Ron Hu <ron.hu@huawei.com>

Closes #19783 from ron8hu/supportHistogram.
2017-12-12 15:04:49 +08:00
gatorsmile a4002651a3 [SPARK-20557][SQL] Only support TIMESTAMP WITH TIME ZONE for Oracle Dialect
## What changes were proposed in this pull request?
In the previous PRs, https://github.com/apache/spark/pull/17832 and https://github.com/apache/spark/pull/17835 , we convert `TIMESTAMP WITH TIME ZONE` and `TIME WITH TIME ZONE` to `TIMESTAMP` for all the JDBC sources. However, this conversion could be risky since it does not respect our SQL configuration `spark.sql.session.timeZone`.

In addition, each vendor might have different semantics for these two types. For example, Postgres simply returns `TIMESTAMP` types for `TIMESTAMP WITH TIME ZONE`. For such supports, we should do it case by case. This PR reverts the general support of `TIMESTAMP WITH TIME ZONE` and `TIME WITH TIME ZONE` for JDBC sources, except ORACLE Dialect.

When supporting the ORACLE's `TIMESTAMP WITH TIME ZONE`, we only support it when the JVM default timezone is the same as the user-specified configuration `spark.sql.session.timeZone` (whose default is the JVM default timezone). Now, we still treat `TIMESTAMP WITH TIME ZONE` as `TIMESTAMP` when fetching the values via the Oracle JDBC connector, whose client converts the timestamp values with time zone to the timestamp values using the local JVM default timezone (a test case is added to `OracleIntegrationSuite.scala` in this PR for showing the behavior). Thus, to avoid any future behavior change, we will not support it if JVM default timezone is different from `spark.sql.session.timeZone`

No regression because the previous two PRs were just merged to be unreleased master branch.

## How was this patch tested?
Added the test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19939 from gatorsmile/timezoneUpdate.
2017-12-11 16:33:06 -08:00
gatorsmile 3d82f6eb78 [SPARK-22726][TEST] Basic tests for Binary Comparison and ImplicitTypeCasts
## What changes were proposed in this pull request?
Before we deliver the Hive compatibility mode, we plan to write a set of test cases that can be easily run in both Spark and Hive sides. We can easily compare whether they are the same or not. When new typeCoercion rules are added, we also can easily track the changes. These test cases can also be backported to the previous Spark versions for determining the changes we made.

This PR is the first attempt for improving the test coverage for type coercion compatibility. We generate these test cases for our binary comparison and ImplicitTypeCasts based on the Apache Derby test cases in https://github.com/apache/derby/blob/10.14/java/testing/org/apache/derbyTesting/functionTests/tests/lang/implicitConversions.sql

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19918 from gatorsmile/typeCoercionTests.
2017-12-11 15:55:23 -08:00
Kazuaki Ishizaki c235b5f977 [SPARK-22746][SQL] Avoid the generation of useless mutable states by SortMergeJoin
## What changes were proposed in this pull request?

This PR reduce the number of global mutable variables in generated code of `SortMergeJoin`.

Before this PR, global mutable variables are used to extend lifetime of variables in the nested loop. This can be achieved by declaring variable at the outer most loop level where the variables are used.
In the following example, `smj_value8`, `smj_value8`, and `smj_value9` are declared as local variable at lines 145-147 in `With this PR`.

This PR fixes potential assertion error by #19865. Without this PR, a global mutable variable is potentially passed to arguments in generated code of split function.

Without this PR
```
/* 010 */   int smj_value8;
/* 011 */   boolean smj_value8;
/* 012 */   int smj_value9;
..
/* 143 */   protected void processNext() throws java.io.IOException {
/* 144 */     while (findNextInnerJoinRows(smj_leftInput, smj_rightInput)) {
/* 145 */       boolean smj_loaded = false;
/* 146 */       smj_isNull6 = smj_leftRow.isNullAt(1);
/* 147 */       smj_value9 = smj_isNull6 ? -1 : (smj_leftRow.getInt(1));
/* 148 */       scala.collection.Iterator<UnsafeRow> smj_iterator = smj_matches.generateIterator();
/* 149 */       while (smj_iterator.hasNext()) {
/* 150 */         InternalRow smj_rightRow1 = (InternalRow) smj_iterator.next();
/* 151 */         boolean smj_isNull8 = smj_rightRow1.isNullAt(1);
/* 152 */         int smj_value11 = smj_isNull8 ? -1 : (smj_rightRow1.getInt(1));
/* 153 */
/* 154 */         boolean smj_value12 = (smj_isNull6 && smj_isNull8) ||
/* 155 */         (!smj_isNull6 && !smj_isNull8 && smj_value9 == smj_value11);
/* 156 */         if (false || !smj_value12) continue;
/* 157 */         if (!smj_loaded) {
/* 158 */           smj_loaded = true;
/* 159 */           smj_value8 = smj_leftRow.getInt(0);
/* 160 */         }
/* 161 */         int smj_value10 = smj_rightRow1.getInt(0);
/* 162 */         smj_numOutputRows.add(1);
/* 163 */
/* 164 */         smj_rowWriter.zeroOutNullBytes();
/* 165 */
/* 166 */         smj_rowWriter.write(0, smj_value8);
/* 167 */
/* 168 */         if (smj_isNull6) {
/* 169 */           smj_rowWriter.setNullAt(1);
/* 170 */         } else {
/* 171 */           smj_rowWriter.write(1, smj_value9);
/* 172 */         }
/* 173 */
/* 174 */         smj_rowWriter.write(2, smj_value10);
/* 175 */
/* 176 */         if (smj_isNull8) {
/* 177 */           smj_rowWriter.setNullAt(3);
/* 178 */         } else {
/* 179 */           smj_rowWriter.write(3, smj_value11);
/* 180 */         }
/* 181 */         append(smj_result.copy());
/* 182 */
/* 183 */       }
/* 184 */       if (shouldStop()) return;
/* 185 */     }
/* 186 */   }
```

With this PR
```
/* 143 */   protected void processNext() throws java.io.IOException {
/* 144 */     while (findNextInnerJoinRows(smj_leftInput, smj_rightInput)) {
/* 145 */       int smj_value8 = -1;
/* 146 */       boolean smj_isNull6 = false;
/* 147 */       int smj_value9 = -1;
/* 148 */       boolean smj_loaded = false;
/* 149 */       smj_isNull6 = smj_leftRow.isNullAt(1);
/* 150 */       smj_value9 = smj_isNull6 ? -1 : (smj_leftRow.getInt(1));
/* 151 */       scala.collection.Iterator<UnsafeRow> smj_iterator = smj_matches.generateIterator();
/* 152 */       while (smj_iterator.hasNext()) {
/* 153 */         InternalRow smj_rightRow1 = (InternalRow) smj_iterator.next();
/* 154 */         boolean smj_isNull8 = smj_rightRow1.isNullAt(1);
/* 155 */         int smj_value11 = smj_isNull8 ? -1 : (smj_rightRow1.getInt(1));
/* 156 */
/* 157 */         boolean smj_value12 = (smj_isNull6 && smj_isNull8) ||
/* 158 */         (!smj_isNull6 && !smj_isNull8 && smj_value9 == smj_value11);
/* 159 */         if (false || !smj_value12) continue;
/* 160 */         if (!smj_loaded) {
/* 161 */           smj_loaded = true;
/* 162 */           smj_value8 = smj_leftRow.getInt(0);
/* 163 */         }
/* 164 */         int smj_value10 = smj_rightRow1.getInt(0);
/* 165 */         smj_numOutputRows.add(1);
/* 166 */
/* 167 */         smj_rowWriter.zeroOutNullBytes();
/* 168 */
/* 169 */         smj_rowWriter.write(0, smj_value8);
/* 170 */
/* 171 */         if (smj_isNull6) {
/* 172 */           smj_rowWriter.setNullAt(1);
/* 173 */         } else {
/* 174 */           smj_rowWriter.write(1, smj_value9);
/* 175 */         }
/* 176 */
/* 177 */         smj_rowWriter.write(2, smj_value10);
/* 178 */
/* 179 */         if (smj_isNull8) {
/* 180 */           smj_rowWriter.setNullAt(3);
/* 181 */         } else {
/* 182 */           smj_rowWriter.write(3, smj_value11);
/* 183 */         }
/* 184 */         append(smj_result.copy());
/* 185 */
/* 186 */       }
/* 187 */       if (shouldStop()) return;
/* 188 */     }
/* 189 */   }
```

## How was this patch tested?

Existing test cases

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

Closes #19937 from kiszk/SPARK-22746.
2017-12-11 13:15:45 -08:00
gatorsmile a04f2bea67 Revert "[SPARK-22496][SQL] thrift server adds operation logs"
This reverts commit 4289ac9d8d.
2017-12-11 13:08:42 -08:00
zuotingbing bf20abb2dc [SPARK-22642][SQL] the createdTempDir will not be deleted if an exception occurs, should delete it with try-finally.
## What changes were proposed in this pull request?

We found staging directories will not be dropped sometimes in our production environment.
The createdTempDir will not be deleted if an exception occurs, we should delete createdTempDir with try-finally.

This PR is follow-up SPARK-18703.

## How was this patch tested?

exist tests

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #19841 from zuotingbing/SPARK-stagedir.
2017-12-11 13:36:15 -06:00
Dongjoon Hyun 6cc7021a40 [SPARK-22267][SQL][TEST] Spark SQL incorrectly reads ORC files when column order is different
## What changes were proposed in this pull request?

Until 2.2.1, with the default configuration, Apache Spark returns incorrect results when ORC file schema is different from metastore schema order. This is due to Hive 1.2.1 library and some issues on `convertMetastoreOrc` option.

```scala
scala> Seq(1 -> 2).toDF("c1", "c2").write.format("orc").mode("overwrite").save("/tmp/o")
scala> sql("CREATE EXTERNAL TABLE o(c2 INT, c1 INT) STORED AS orc LOCATION '/tmp/o'")
scala> spark.table("o").show    // This is wrong.
+---+---+
| c2| c1|
+---+---+
|  1|  2|
+---+---+
scala> spark.read.orc("/tmp/o").show  // This is correct.
+---+---+
| c1| c2|
+---+---+
|  1|  2|
+---+---+
```

After [SPARK-22279](https://github.com/apache/spark/pull/19499), the default configuration doesn't have this bug. Although Hive 1.2.1 library code path still has the problem, we had better have a test coverage on what we have now in order to prevent future regression on it.

## How was this patch tested?

Pass the Jenkins with a newly added test test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19928 from dongjoon-hyun/SPARK-22267.
2017-12-11 21:52:57 +08:00
zouchenjun 4289ac9d8d [SPARK-22496][SQL] thrift server adds operation logs
## What changes were proposed in this pull request?
since hive 2.0+  upgrades log4j to log4j2,a lot of [changes](https://issues.apache.org/jira/browse/HIVE-11304) are made working on it.
as spark is not to ready to update its inner hive version(1.2.1) , so I manage to make little changes.
the function registerCurrentOperationLog  is moved from SQLOperstion to its parent class ExecuteStatementOperation so spark can use it.

## How was this patch tested?
manual test

Author: zouchenjun <zouchenjun@youzan.com>

Closes #19721 from ChenjunZou/operation-log.
2017-12-10 20:36:14 -08:00
Dongjoon Hyun 251b2c03b4 [SPARK-22672][SQL][TEST][FOLLOWUP] Fix to use spark.conf
## What changes were proposed in this pull request?

During https://github.com/apache/spark/pull/19882, `conf` is mistakenly used to switch ORC implementation between `native` and `hive`. To affect `OrcTest` correctly, `spark.conf` should be used.

## How was this patch tested?

Pass the tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19931 from dongjoon-hyun/SPARK-22672-2.
2017-12-09 20:20:28 +09:00
Imran Rashid acf7ef3154 [SPARK-12297][SQL] Adjust timezone for int96 data from impala
## What changes were proposed in this pull request?

Int96 data written by impala vs data written by hive & spark is stored slightly differently -- they use a different offset for the timezone.  This adds an option "spark.sql.parquet.int96TimestampConversion" (false by default) to adjust timestamps if and only if the writer is impala (or more precisely, if the parquet file's "createdBy" metadata does not start with "parquet-mr").  This matches the existing behavior in hive from HIVE-9482.

## How was this patch tested?

Unit test added, existing tests run via jenkins.

Author: Imran Rashid <irashid@cloudera.com>
Author: Henry Robinson <henry@apache.org>

Closes #19769 from squito/SPARK-12297_skip_conversion.
2017-12-09 11:53:15 +09:00
Sunitha Kambhampati f88a67bf08 [SPARK-22452][SQL] Add getDouble to DataSourceV2Options
- Implemented getDouble method in DataSourceV2Options
- Add unit test

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #19921 from skambha/ds2.
2017-12-08 14:48:19 +08:00
Tathagata Das b11869bc3b [SPARK-22187][SS][REVERT] Revert change in state row format for mapGroupsWithState
## What changes were proposed in this pull request?

#19416 changed the format in which rows were encoded in the state store. However, this can break existing streaming queries with the old format in unpredictable ways (potentially crashing the JVM). Hence I am reverting this for now. This will be re-applied in the future after we start saving more metadata in checkpoints to signify which version of state row format the existing streaming query is running. Then we can decode old and new formats accordingly.

## How was this patch tested?
Existing tests.

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

Closes #19924 from tdas/SPARK-22187-1.
2017-12-07 22:02:51 -08:00
Dongjoon Hyun 0ba8f4b211 [SPARK-21787][SQL] Support for pushing down filters for DateType in native OrcFileFormat
## What changes were proposed in this pull request?

This PR support for pushing down filters for DateType in ORC

## How was this patch tested?

Pass the Jenkins with newly add and updated test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18995 from dongjoon-hyun/SPARK-21787.
2017-12-08 09:52:16 +08:00
Dongjoon Hyun aa1764ba1a [SPARK-22279][SQL] Turn on spark.sql.hive.convertMetastoreOrc by default
## What changes were proposed in this pull request?

Like Parquet, this PR aims to turn on `spark.sql.hive.convertMetastoreOrc` by default.

## How was this patch tested?

Pass all the existing test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19499 from dongjoon-hyun/SPARK-22279.
2017-12-07 15:45:23 -08:00
Wang Gengliang 18b75d465b [SPARK-22719][SQL] Refactor ConstantPropagation
## What changes were proposed in this pull request?

The current time complexity of ConstantPropagation is O(n^2), which can be slow when the query is complex.
Refactor the implementation with O( n ) time complexity, and some pruning to avoid traversing the whole `Condition`

## How was this patch tested?

Unit test.

Also simple benchmark test in ConstantPropagationSuite
```
  val condition = (1 to 500).map{_ => Rand(0) === Rand(0)}.reduce(And)
  val query = testRelation
    .select(columnA)
    .where(condition)
  val start = System.currentTimeMillis()
  (1 to 40).foreach { _ =>
    Optimize.execute(query.analyze)
  }
  val end = System.currentTimeMillis()
  println(end - start)
```
Run time before changes: 18989ms (474ms per loop)
Run time after changes: 1275 ms (32ms per loop)

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19912 from gengliangwang/ConstantPropagation.
2017-12-07 10:24:49 -08:00
kellyzly f41c0a93fd [SPARK-22660][BUILD] Use position() and limit() to fix ambiguity issue in scala-2.12
…a-2.12 and JDK9

## What changes were proposed in this pull request?
Some compile error after upgrading to scala-2.12
```javascript
spark_source/core/src/main/scala/org/apache/spark/executor/Executor.scala:455: ambiguous reference to overloaded definition, method limit in class ByteBuffer of type (x$1: Int)java.nio.ByteBuffer
method limit in class Buffer of type ()Int
match expected type ?
     val resultSize = serializedDirectResult.limit
error
```
The limit method was moved from ByteBuffer to the superclass Buffer and it can no longer be called without (). The same reason for position method.

```javascript
/home/zly/prj/oss/jdk9_HOS_SOURCE/spark_source/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala:427: ambiguous reference to overloaded definition, [error] both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit [error] and  method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit [error] match argument types (java.util.Map[String,String])
 [error]       props.putAll(outputSerdeProps.toMap.asJava)
 [error]             ^
 ```
This is because the key type is Object instead of String which is unsafe.

## How was this patch tested?

running tests

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

Author: kellyzly <kellyzly@126.com>

Closes #19854 from kellyzly/SPARK-22660.
2017-12-07 10:04:04 -06:00
Marco Gaido b79071910e [SPARK-22696][SQL] objects functions should not use unneeded global variables
## What changes were proposed in this pull request?

Some objects functions are using global variables which are not needed. This can generate some unneeded entries in the constant pool.

The PR replaces the unneeded global variables with local variables.

## How was this patch tested?

added UTs

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

Closes #19908 from mgaido91/SPARK-22696.
2017-12-07 21:24:36 +08:00
Marco Gaido fc29446300 [SPARK-22699][SQL] GenerateSafeProjection should not use global variables for struct
## What changes were proposed in this pull request?

GenerateSafeProjection is defining a mutable state for each struct, which is not needed. This is bad for the well known issues related to constant pool limits.
The PR replace the global variable with a local one.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19914 from mgaido91/SPARK-22699.
2017-12-07 21:18:27 +08:00
Dongjoon Hyun dd59a4be36 [SPARK-22712][SQL] Use buildReaderWithPartitionValues in native OrcFileFormat
## What changes were proposed in this pull request?

To support vectorization in native OrcFileFormat later, we need to use `buildReaderWithPartitionValues` instead of `buildReader` like ParquetFileFormat. This PR replaces `buildReader` with `buildReaderWithPartitionValues`.

## How was this patch tested?

Pass the Jenkins with the existing test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19907 from dongjoon-hyun/SPARK-ORC-BUILD-READER.
2017-12-07 21:08:15 +08:00
Sunitha Kambhampati 2be448260d [SPARK-22452][SQL] Add getInt, getLong, getBoolean to DataSourceV2Options
- Implemented methods getInt, getLong, getBoolean for DataSourceV2Options
- Added new unit tests to exercise these methods

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #19902 from skambha/spark22452.
2017-12-07 20:59:47 +08:00
Kazuaki Ishizaki ea2fbf4197 [SPARK-22705][SQL] Case, Coalesce, and In use less global variables
## What changes were proposed in this pull request?

This PR accomplishes the following two items.

1. Reduce # of global variables from two to one for generated code of `Case` and `Coalesce` and remove global variables for generated code of `In`.
2. Make lifetime of global variable local within an operation

Item 1. reduces # of constant pool entries in a Java class. Item 2. ensures that an variable is not passed to arguments in a method split by `CodegenContext.splitExpressions()`, which is addressed by #19865.

## How was this patch tested?

Added new tests into `PredicateSuite`, `NullExpressionsSuite`, and `ConditionalExpressionSuite`.

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

Closes #19901 from kiszk/SPARK-22705.
2017-12-07 20:55:35 +08:00
Wenchen Fan e103adf45a [SPARK-22703][SQL] make ColumnarRow an immutable view
## What changes were proposed in this pull request?

Similar to https://github.com/apache/spark/pull/19842 , we should also make `ColumnarRow` an immutable view, and move forward to make `ColumnVector` public.

## How was this patch tested?

Existing tests.

The performance concern should be same as https://github.com/apache/spark/pull/19842 .

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19898 from cloud-fan/row-id.
2017-12-07 20:45:11 +08:00
Dongjoon Hyun c1e5688d1a [SPARK-22672][SQL][TEST] Refactor ORC Tests
## What changes were proposed in this pull request?

Since SPARK-20682, we have two `OrcFileFormat`s. This PR refactors ORC tests with three principles (with a few exceptions)
1. Move test suite into `sql/core`.
2. Create `HiveXXX` test suite in `sql/hive` by reusing `sql/core` test suite.
3. `OrcTest` will provide common helper functions and `val orcImp: String`.

**Test Suites**

*Native OrcFileFormat*
- org.apache.spark.sql.hive.orc
  - OrcFilterSuite
  - OrcPartitionDiscoverySuite
  - OrcQuerySuite
  - OrcSourceSuite
- o.a.s.sql.hive.orc
  - OrcHadoopFsRelationSuite

*Hive built-in OrcFileFormat*

- o.a.s.sql.hive.orc
  - HiveOrcFilterSuite
  - HiveOrcPartitionDiscoverySuite
  - HiveOrcQuerySuite
  - HiveOrcSourceSuite
  - HiveOrcHadoopFsRelationSuite

**Hierarchy**
```
OrcTest
    -> OrcSuite
        -> OrcSourceSuite
    -> OrcQueryTest
        -> OrcQuerySuite
    -> OrcPartitionDiscoveryTest
        -> OrcPartitionDiscoverySuite
    -> OrcFilterSuite

HadoopFsRelationTest
    -> OrcHadoopFsRelationSuite
        -> HiveOrcHadoopFsRelationSuite
```

Please note the followings.
- Unlike the other test suites, `OrcHadoopFsRelationSuite` doesn't inherit `OrcTest`. It is inside `sql/hive` like `ParquetHadoopFsRelationSuite` due to the dependencies and follows the existing convention to use `val dataSourceName: String`
- `OrcFilterSuite`s cannot reuse test cases due to the different function signatures using Hive 1.2.1 ORC classes and Apache ORC 1.4.1 classes.

## How was this patch tested?

Pass the Jenkins tests with reorganized test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19882 from dongjoon-hyun/SPARK-22672.
2017-12-07 20:42:46 +08:00
Kazuaki Ishizaki 8ae004b460 [SPARK-22688][SQL] Upgrade Janino version to 3.0.8
## What changes were proposed in this pull request?

This PR upgrade Janino version to 3.0.8. [Janino 3.0.8](https://janino-compiler.github.io/janino/changelog.html) includes an important fix to reduce the number of constant pool entries by using 'sipush' java bytecode.

* SIPUSH bytecode is not used for short integer constant [#33](https://github.com/janino-compiler/janino/issues/33).

Please see detail in [this discussion thread](https://github.com/apache/spark/pull/19518#issuecomment-346674976).

## How was this patch tested?

Existing tests

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

Closes #19890 from kiszk/SPARK-22688.
2017-12-06 16:15:25 -08:00
Marco Gaido f110a7f884 [SPARK-22693][SQL] CreateNamedStruct and InSet should not use global variables
## What changes were proposed in this pull request?

CreateNamedStruct and InSet are using a global variable which is not needed. This can generate some unneeded entries in the constant pool.

The PR removes the unnecessary mutable states and makes them local variables.

## How was this patch tested?

added UT

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

Closes #19896 from mgaido91/SPARK-22693.
2017-12-06 14:12:16 -08:00
smurakozi 9948b860ac [SPARK-22516][SQL] Bump up Univocity version to 2.5.9
## What changes were proposed in this pull request?

There was a bug in Univocity Parser that causes the issue in SPARK-22516. This was fixed by upgrading from 2.5.4 to 2.5.9 version of the library :

**Executing**
```
spark.read.option("header","true").option("inferSchema", "true").option("multiLine", "true").option("comment", "g").csv("test_file_without_eof_char.csv").show()
```
**Before**
```
ERROR Executor: Exception in task 0.0 in stage 6.0 (TID 6)
com.univocity.parsers.common.TextParsingException: java.lang.IllegalArgumentException - Unable to skip 1 lines from line 2. End of input reached
...
Internal state when error was thrown: line=3, column=0, record=2, charIndex=31
	at com.univocity.parsers.common.AbstractParser.handleException(AbstractParser.java:339)
	at com.univocity.parsers.common.AbstractParser.parseNext(AbstractParser.java:475)
	at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anon$1.next(UnivocityParser.scala:281)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
```
**After**
```
+-------+-------+
|column1|column2|
+-------+-------+
|    abc|    def|
+-------+-------+
```

## How was this patch tested?
The already existing `CSVSuite.commented lines in CSV data` test was extended to parse the file also in multiline mode. The test input file was modified to also include a comment in the last line.

Author: smurakozi <smurakozi@gmail.com>

Closes #19906 from smurakozi/SPARK-22516.
2017-12-06 13:22:08 -08:00
gatorsmile effca9868e [SPARK-22720][SS] Make EventTimeWatermark Extend UnaryNode
## What changes were proposed in this pull request?
Our Analyzer and Optimizer have multiple rules for `UnaryNode`. After making `EventTimeWatermark` extend `UnaryNode`, we do not need a special handling for `EventTimeWatermark`.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19913 from gatorsmile/eventtimewatermark.
2017-12-06 13:11:38 -08:00
Marco Gaido e98f9647f4 [SPARK-22695][SQL] ScalaUDF should not use global variables
## What changes were proposed in this pull request?

ScalaUDF is using global variables which are not needed. This can generate some unneeded entries in the constant pool.

The PR replaces the unneeded global variables with local variables.

## How was this patch tested?

added UT

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

Closes #19900 from mgaido91/SPARK-22695.
2017-12-07 00:50:49 +08:00
Kazuaki Ishizaki 813c0f945d [SPARK-22704][SQL] Least and Greatest use less global variables
## What changes were proposed in this pull request?

This PR accomplishes the following two items.

1. Reduce # of global variables from two to one
2. Make lifetime of global variable local within an operation

Item 1. reduces # of constant pool entries in a Java class. Item 2. ensures that an variable is not passed to arguments in a method split by `CodegenContext.splitExpressions()`, which is addressed by #19865.

## How was this patch tested?

Added new test into `ArithmeticExpressionSuite`

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

Closes #19899 from kiszk/SPARK-22704.
2017-12-07 00:45:51 +08:00
Dongjoon Hyun fb6a922751 [SPARK-20728][SQL][FOLLOWUP] Use an actionable exception message
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/19871 to improve an exception message.

## How was this patch tested?

Pass the Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19903 from dongjoon-hyun/orc_exception.
2017-12-06 20:20:20 +09:00
Liang-Chi Hsieh 00d176d2fe [SPARK-20392][SQL] Set barrier to prevent re-entering a tree
## What changes were proposed in this pull request?

The SQL `Analyzer` goes through a whole query plan even most part of it is analyzed. This increases the time spent on query analysis for long pipelines in ML, especially.

This patch adds a logical node called `AnalysisBarrier` that wraps an analyzed logical plan to prevent it from analysis again. The barrier is applied to the analyzed logical plan in `Dataset`. It won't change the output of wrapped logical plan and just acts as a wrapper to hide it from analyzer. New operations on the dataset will be put on the barrier, so only the new nodes created will be analyzed.

This analysis barrier will be removed at the end of analysis stage.

## How was this patch tested?

Added tests.

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

Closes #19873 from viirya/SPARK-20392-reopen.
2017-12-05 21:43:41 -08:00
Dongjoon Hyun 82183f7b57 [SPARK-22686][SQL] DROP TABLE IF EXISTS should not show AnalysisException
## What changes were proposed in this pull request?

During [SPARK-22488](https://github.com/apache/spark/pull/19713) to fix view resolution issue, there occurs a regression at `2.2.1` and `master` branch like the following. This PR fixes that.

```scala
scala> spark.version
res2: String = 2.2.1

scala> sql("DROP TABLE IF EXISTS t").show
17/12/04 21:01:06 WARN DropTableCommand: org.apache.spark.sql.AnalysisException:
Table or view not found: t;
org.apache.spark.sql.AnalysisException: Table or view not found: t;
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19888 from dongjoon-hyun/SPARK-22686.
2017-12-06 10:52:29 +08:00
Zhenhua Wang 1e17ab83de [SPARK-22662][SQL] Failed to prune columns after rewriting predicate subquery
## What changes were proposed in this pull request?

As a simple example:
```
spark-sql> create table base (a int, b int) using parquet;
Time taken: 0.066 seconds
spark-sql> create table relInSubq ( x int, y int, z int) using parquet;
Time taken: 0.042 seconds
spark-sql> explain select a from base where a in (select x from relInSubq);
== Physical Plan ==
*Project [a#83]
+- *BroadcastHashJoin [a#83], [x#85], LeftSemi, BuildRight
   :- *FileScan parquet default.base[a#83,b#84] Batched: true, Format: Parquet, Location: InMemoryFileIndex[hdfs://100.0.0.4:9000/wzh/base], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:int,b:int>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)))
      +- *Project [x#85]
         +- *FileScan parquet default.relinsubq[x#85] Batched: true, Format: Parquet, Location: InMemoryFileIndex[hdfs://100.0.0.4:9000/wzh/relinsubq], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<x:int>
```
We only need column `a` in table `base`, but all columns (`a`, `b`) are fetched.

The reason is that, in "Operator Optimizations" batch, `ColumnPruning` first produces a `Project` on table `base`, but then it's removed by `removeProjectBeforeFilter`. Because at that time, the predicate subquery is in filter form. Then, in "Rewrite Subquery" batch, `RewritePredicateSubquery` converts the subquery into a LeftSemi join, but this batch doesn't have the `ColumnPruning` rule. This results in reading all columns for the `base` table.

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

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19855 from wzhfy/column_pruning_subquery.
2017-12-05 15:15:32 -08:00
Wenchen Fan 132a3f4708 [SPARK-22500][SQL][FOLLOWUP] cast for struct can split code even with whole stage codegen
## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/19730, we can split the code for casting struct even with whole stage codegen.

This PR also has some renaming to make the code easier to read.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19891 from cloud-fan/cast.
2017-12-05 11:40:13 -08:00
Wenchen Fan ced6ccf0d6 [SPARK-22701][SQL] add ctx.splitExpressionsWithCurrentInputs
## What changes were proposed in this pull request?

This pattern appears many times in the codebase:
```
if (ctx.INPUT_ROW == null || ctx.currentVars != null) {
  exprs.mkString("\n")
} else {
  ctx.splitExpressions(...)
}
```

This PR adds a `ctx.splitExpressionsWithCurrentInputs` for this pattern

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19895 from cloud-fan/splitExpression.
2017-12-05 10:15:15 -08:00
Dongjoon Hyun 326f1d6728 [SPARK-20728][SQL] Make OrcFileFormat configurable between sql/hive and sql/core
## What changes were proposed in this pull request?

This PR aims to provide a configuration to choose the default `OrcFileFormat` from legacy `sql/hive` module or new `sql/core` module.

For example, this configuration will affects the following operations.
```scala
spark.read.orc(...)
```
```sql
CREATE TABLE t
USING ORC
...
```

## How was this patch tested?

Pass the Jenkins with new test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19871 from dongjoon-hyun/spark-sql-orc-enabled.
2017-12-05 20:46:35 +08:00
gatorsmile 53e5251bb3 [SPARK-22675][SQL] Refactoring PropagateTypes in TypeCoercion
## What changes were proposed in this pull request?
PropagateTypes are called twice in TypeCoercion. We do not need to call it twice. Instead, we should call it after each change on the types.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19874 from gatorsmile/deduplicatePropagateTypes.
2017-12-05 20:43:02 +08:00
Wenchen Fan a8af4da12c [SPARK-22682][SQL] HashExpression does not need to create global variables
## What changes were proposed in this pull request?

It turns out that `HashExpression` can pass around some values via parameter when splitting codes into methods, to save some global variable slots.

This can also prevent a weird case that global variable appears in parameter list, which is discovered by https://github.com/apache/spark/pull/19865

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19878 from cloud-fan/minor.
2017-12-05 12:43:05 +08:00
Wenchen Fan 295df746ec [SPARK-22677][SQL] cleanup whole stage codegen for hash aggregate
## What changes were proposed in this pull request?

The `HashAggregateExec` whole stage codegen path is a little messy and hard to understand, this code cleans it up a little bit, especially for the fast hash map part.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19869 from cloud-fan/hash-agg.
2017-12-05 12:38:26 +08:00
Marco Gaido 3887b7eef7 [SPARK-22665][SQL] Avoid repartitioning with empty list of expressions
## What changes were proposed in this pull request?

Repartitioning by empty set of expressions is currently possible, even though it is a case which is not handled properly. Indeed, in `HashExpression` there is a check to avoid to run it on an empty set, but this check is not performed while repartitioning.
Thus, the PR adds a check to avoid this wrong situation.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19870 from mgaido91/SPARK-22665.
2017-12-04 17:08:56 -08:00
Zhenhua Wang 1d5597b408 [SPARK-22626][SQL][FOLLOWUP] improve documentation and simplify test case
## What changes were proposed in this pull request?

This PR improves documentation for not using zero `numRows` statistics and simplifies the test case.

The reason why some Hive tables have zero `numRows` is that, in Hive, when stats gathering is disabled, `numRows` is always zero after INSERT command:
```
hive> create table src (key int, value string) stored as orc;
hive> desc formatted src;
Table Parameters:
	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
	numFiles            	0
	numRows             	0
	rawDataSize         	0
	totalSize           	0
	transient_lastDdlTime	1512399590

hive> set hive.stats.autogather=false;
hive> insert into src select 1, 'a';
hive> desc formatted src;
Table Parameters:
	numFiles            	1
	numRows             	0
	rawDataSize         	0
	totalSize           	275
	transient_lastDdlTime	1512399647

hive> insert into src select 1, 'b';
hive> desc formatted src;
Table Parameters:
	numFiles            	2
	numRows             	0
	rawDataSize         	0
	totalSize           	550
	transient_lastDdlTime	1512399687
```

## How was this patch tested?

Modified existing test.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19880 from wzhfy/doc_zero_rowCount.
2017-12-04 15:08:07 -08:00
Marco Gaido 3927bb9b46 [SPARK-22473][FOLLOWUP][TEST] Remove deprecated Date functions
## What changes were proposed in this pull request?

#19696 replaced the deprecated usages for `Date` and `Waiter`, but a few methods were missed. The PR fixes the forgotten deprecated usages.

## How was this patch tested?

existing UTs

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19875 from mgaido91/SPARK-22473_FOLLOWUP.
2017-12-04 11:07:27 -06:00
Yuming Wang dff440f1ec [SPARK-22626][SQL] deals with wrong Hive's statistics (zero rowCount)
This pr to ensure that the Hive's statistics `totalSize` (or `rawDataSize`) > 0, `rowCount` also must be > 0. Otherwise may cause OOM when CBO is enabled.

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #19831 from wangyum/SPARK-22626.
2017-12-03 23:05:39 +08:00
Marco Gaido 2c16267f7c [SPARK-22669][SQL] Avoid unnecessary function calls in code generation
## What changes were proposed in this pull request?

In many parts of the codebase for code generation, we are splitting the code to avoid exceptions due to the 64KB method size limit. This is generating a lot of methods which are called every time, even though sometime this is not needed. As pointed out here: https://github.com/apache/spark/pull/19752#discussion_r153081547, this is a not negligible overhead which can be avoided.

The PR applies the same approach used in #19752 also to the other places where this was feasible.

## How was this patch tested?

existing UTs.

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19860 from mgaido91/SPARK-22669.
2017-12-03 22:56:03 +08:00
Dongjoon Hyun f23dddf105 [SPARK-20682][SPARK-15474][SPARK-21791] Add new ORCFileFormat based on ORC 1.4.1
## What changes were proposed in this pull request?

Since [SPARK-2883](https://issues.apache.org/jira/browse/SPARK-2883), Apache Spark supports Apache ORC inside `sql/hive` module with Hive dependency. This PR aims to add a new ORC data source inside `sql/core` and to replace the old ORC data source eventually. This PR resolves the following three issues.

- [SPARK-20682](https://issues.apache.org/jira/browse/SPARK-20682): Add new ORCFileFormat based on Apache ORC 1.4.1
- [SPARK-15474](https://issues.apache.org/jira/browse/SPARK-15474): ORC data source fails to write and read back empty dataframe
- [SPARK-21791](https://issues.apache.org/jira/browse/SPARK-21791): ORC should support column names with dot

## How was this patch tested?

Pass the Jenkins with the existing all tests and new tests for SPARK-15474 and SPARK-21791.

Author: Dongjoon Hyun <dongjoon@apache.org>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #19651 from dongjoon-hyun/SPARK-20682.
2017-12-03 22:21:44 +08:00
Shixiong Zhu ee10ca7ec6 [SPARK-22638][SS] Use a separate queue for StreamingQueryListenerBus
## What changes were proposed in this pull request?

Use a separate Spark event queue for StreamingQueryListenerBus so that if there are many non-streaming events, streaming query listeners don't need to wait for other Spark listeners and can catch up.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19838 from zsxwing/SPARK-22638.
2017-12-01 13:02:03 -08:00
sujith71955 16adaf634b [SPARK-22601][SQL] Data load is getting displayed successful on providing non existing nonlocal file path
## What changes were proposed in this pull request?
When user tries to load data with a non existing hdfs file path system is not validating it and the load command operation is getting successful.
This is misleading to the user. already there is a validation in the scenario of none existing local file path. This PR has added validation in the scenario of nonexisting hdfs file path
## How was this patch tested?
UT has been added for verifying the issue, also snapshots has been added after the verification in a spark yarn cluster

Author: sujith71955 <sujithchacko.2010@gmail.com>

Closes #19823 from sujith71955/master_LoadComand_Issue.
2017-11-30 20:45:30 -08:00
Adrian Ionescu f5f8e84d9d [SPARK-22614] Dataset API: repartitionByRange(...)
## What changes were proposed in this pull request?

This PR introduces a way to explicitly range-partition a Dataset. So far, only round-robin and hash partitioning were possible via `df.repartition(...)`, but sometimes range partitioning might be desirable: e.g. when writing to disk, for better compression without the cost of global sort.

The current implementation piggybacks on the existing `RepartitionByExpression` `LogicalPlan` and simply adds the following logic: If its expressions are of type `SortOrder`, then it will do `RangePartitioning`; otherwise `HashPartitioning`. This was by far the least intrusive solution I could come up with.

## How was this patch tested?
Unit test for `RepartitionByExpression` changes, a test to ensure we're not changing the behavior of existing `.repartition()` and a few end-to-end tests in `DataFrameSuite`.

Author: Adrian Ionescu <adrian@databricks.com>

Closes #19828 from adrian-ionescu/repartitionByRange.
2017-11-30 15:41:34 -08:00
Yuming Wang bcceab6495 [SPARK-22489][SQL] Shouldn't change broadcast join buildSide if user clearly specified
## What changes were proposed in this pull request?

How to reproduce:
```scala
import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec

spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value").createTempView("table1")
spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value").createTempView("table2")

val bl = sql("SELECT /*+ MAPJOIN(t1) */ * FROM table1 t1 JOIN table2 t2 ON t1.key = t2.key").queryExecution.executedPlan

println(bl.children.head.asInstanceOf[BroadcastHashJoinExec].buildSide)
```
The result is `BuildRight`, but should be `BuildLeft`. This PR fix this issue.
## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #19714 from wangyum/SPARK-22489.
2017-11-30 15:36:26 -08:00
aokolnychyi 6ac57fd0d1 [SPARK-21417][SQL] Infer join conditions using propagated constraints
## What changes were proposed in this pull request?

This PR adds an optimization rule that infers join conditions using propagated constraints.

For instance, if there is a join, where the left relation has 'a = 1' and the right relation has 'b = 1', then the rule infers 'a = b' as a join predicate. Only semantically new predicates are appended to the existing join condition.

Refer to the corresponding ticket and tests for more details.

## How was this patch tested?

This patch comes with a new test suite to cover the implemented logic.

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

Closes #18692 from aokolnychyi/spark-21417.
2017-11-30 14:25:10 -08:00
Kazuaki Ishizaki 999ec137a9 [SPARK-22570][SQL] Avoid to create a lot of global variables by using a local variable with allocation of an object in generated code
## What changes were proposed in this pull request?

This PR reduces # of global variables in generated code by replacing a global variable with a local variable with an allocation of an object every time. When a lot of global variables were generated, the generated code may meet 64K constant pool limit.
This PR reduces # of generated global variables in the following three operations:
* `Cast` with String to primitive byte/short/int/long
* `RegExpReplace`
* `CreateArray`

I intentionally leave [this part](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L595-L603). This is because this variable keeps a class that is dynamically generated. In other word, it is not possible to reuse one class.

## How was this patch tested?

Added test cases

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

Closes #19797 from kiszk/SPARK-22570.
2017-12-01 02:28:24 +08:00
Marco Gaido 932bd09c80 [SPARK-22635][SQL][ORC] FileNotFoundException while reading ORC files containing special characters
## What changes were proposed in this pull request?

SPARK-22146 fix the FileNotFoundException issue only for the `inferSchema` method, ie. only for the schema inference, but it doesn't fix the problem when actually reading the data. Thus nearly the same exception happens when someone tries to use the data. This PR covers fixing the problem also there.

## How was this patch tested?

enhanced UT

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19844 from mgaido91/SPARK-22635.
2017-12-01 01:24:15 +09:00
Sean Owen 6eb203fae7 [SPARK-22654][TESTS] Retry Spark tarball download if failed in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

Adds a simple loop to retry download of Spark tarballs from different mirrors if the download fails.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19851 from srowen/SPARK-22654.
2017-12-01 01:21:52 +09:00
Wenchen Fan 9c29c55763 [SPARK-22643][SQL] ColumnarArray should be an immutable view
## What changes were proposed in this pull request?

To make `ColumnVector` public, `ColumnarArray` need to be public too, and we should not have mutable public fields in a public class. This PR proposes to make `ColumnarArray` an immutable view of the data, and always create a new instance of `ColumnarArray` in `ColumnVector#getArray`

## How was this patch tested?

new benchmark in `ColumnarBatchBenchmark`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19842 from cloud-fan/column-vector.
2017-11-30 18:34:38 +08:00
Wenchen Fan 444a2bbb67 [SPARK-22652][SQL] remove set methods in ColumnarRow
## What changes were proposed in this pull request?

As a step to make `ColumnVector` public, the `ColumnarRow` returned by `ColumnVector#getStruct` should be immutable.

However we do need the mutability of `ColumnaRow` for the fast vectorized hashmap in hash aggregate. To solve this, this PR introduces a `MutableColumnarRow` for this use case.

## How was this patch tested?

existing test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19847 from cloud-fan/mutable-row.
2017-11-30 18:28:58 +08:00
Kazuaki Ishizaki 284836862b [SPARK-22608][SQL] add new API to CodeGeneration.splitExpressions()
## What changes were proposed in this pull request?

This PR adds a new API to ` CodeGenenerator.splitExpression` since since several ` CodeGenenerator.splitExpression` are used with `ctx.INPUT_ROW` to avoid code duplication.

## How was this patch tested?

Used existing test suits

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

Closes #19821 from kiszk/SPARK-22608.
2017-11-30 01:19:37 +08:00
Wang Gengliang 57687280d4 [SPARK-22615][SQL] Handle more cases in PropagateEmptyRelation
## What changes were proposed in this pull request?

Currently, in the optimize rule `PropagateEmptyRelation`, the following cases is not handled:
1.  empty relation as right child in left outer join
2. empty relation as left child in right outer join
3. empty relation as right child  in left semi join
4. empty relation as right child  in left anti join
5. only one empty relation in full outer join

case 1 / 2 / 5 can be treated as **Cartesian product** and cause exception. See the new test cases.

## How was this patch tested?
Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19825 from gengliangwang/SPARK-22615.
2017-11-29 09:17:39 -08:00
Wenchen Fan 20b239845b [SPARK-22605][SQL] SQL write job should also set Spark task output metrics
## What changes were proposed in this pull request?

For SQL write jobs, we only set metrics for the SQL listener and display them in the SQL plan UI. We should also set metrics for Spark task output metrics, which will be shown in spark job UI.

## How was this patch tested?

test it manually. For a simple write job
```
spark.range(1000).write.parquet("/tmp/p1")
```
now the spark job UI looks like
![ui](https://user-images.githubusercontent.com/3182036/33326478-05a25b7c-d490-11e7-96ef-806117774356.jpg)

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19833 from cloud-fan/ui.
2017-11-29 19:18:47 +08:00
Herman van Hovell 475a29f11e [SPARK-22637][SQL] Only refresh a logical plan once.
## What changes were proposed in this pull request?
`CatalogImpl.refreshTable` uses `foreach(..)` to refresh all tables in a view. This traverses all nodes in the subtree and calls `LogicalPlan.refresh()` on these nodes. However `LogicalPlan.refresh()` is also refreshing its children, as a result refreshing a large view can be quite expensive.

This PR just calls `LogicalPlan.refresh()` on the top node.

## How was this patch tested?
Existing tests.

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

Closes #19837 from hvanhovell/SPARK-22637.
2017-11-28 16:03:47 -08:00
Sunitha Kambhampati a10b328dbc [SPARK-22431][SQL] Ensure that the datatype in the schema for the table/view metadata is parseable by Spark before persisting it
## What changes were proposed in this pull request?
* JIRA:  [SPARK-22431](https://issues.apache.org/jira/browse/SPARK-22431)  : Creating Permanent view with illegal type

**Description:**
- It is possible in Spark SQL to create a permanent view that uses an nested field with an illegal name.
- For example if we create the following view:
```create view x as select struct('a' as `$q`, 1 as b) q```
- A simple select fails with the following exception:

```
select * from x;

org.apache.spark.SparkException: Cannot recognize hive type string: struct<$q:string,b:int>
  at org.apache.spark.sql.hive.client.HiveClientImpl$.fromHiveColumn(HiveClientImpl.scala:812)
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11$$anonfun$7.apply(HiveClientImpl.scala:378)
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11$$anonfun$7.apply(HiveClientImpl.scala:378)
...
```
**Issue/Analysis**: Right now, we can create a view with a schema that cannot be read back by Spark from the Hive metastore.  For more details, please see the discussion about the analysis and proposed fix options in comment 1 and comment 2 in the [SPARK-22431](https://issues.apache.org/jira/browse/SPARK-22431)

**Proposed changes**:
 - Fix the hive table/view codepath to check whether the schema datatype is parseable by Spark before persisting it in the metastore. This change is localized to HiveClientImpl to do the check similar to the check in FromHiveColumn. This is fail-fast and we will avoid the scenario where we write something to the metastore that we are unable to read it back.
- Added new unit tests
- Ran the sql related unit test suites ( hive/test, sql/test, catalyst/test) OK

With the fix:
```
create view x as select struct('a' as `$q`, 1 as b) q;
17/11/28 10:44:55 ERROR SparkSQLDriver: Failed in [create view x as select struct('a' as `$q`, 1 as b) q]
org.apache.spark.SparkException: Cannot recognize hive type string: struct<$q:string,b:int>
	at org.apache.spark.sql.hive.client.HiveClientImpl$.org$apache$spark$sql$hive$client$HiveClientImpl$$getSparkSQLDataType(HiveClientImpl.scala:884)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$org$apache$spark$sql$hive$client$HiveClientImpl$$verifyColumnDataType$1.apply(HiveClientImpl.scala:906)
	at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$org$apache$spark$sql$hive$client$HiveClientImpl$$verifyColumnDataType$1.apply(HiveClientImpl.scala:906)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
...
```
## How was this patch tested?
- New unit tests have been added.

hvanhovell, Please review and share your thoughts/comments.  Thank you so much.

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #19747 from skambha/spark22431.
2017-11-28 22:01:01 +01:00
Zhenhua Wang da35574297 [SPARK-22515][SQL] Estimation relation size based on numRows * rowSize
## What changes were proposed in this pull request?

Currently, relation size is computed as the sum of file size, which is error-prone because storage format like parquet may have a much smaller file size compared to in-memory size. When we choose broadcast join based on file size, there's a risk of OOM. But if the number of rows is available in statistics, we can get a better estimation by `numRows * rowSize`, which helps to alleviate this problem.

## How was this patch tested?

Added a new test case for data source table and hive table.

Author: Zhenhua Wang <wzh_zju@163.com>
Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19743 from wzhfy/better_leaf_size.
2017-11-28 11:43:21 -08:00
Wenchen Fan b70e483cb3 [SPARK-22617][SQL] make splitExpressions extract current input of the context
## What changes were proposed in this pull request?

Mostly when we call `CodegenContext.splitExpressions`, we want to split the code into methods and pass the current inputs of the codegen context to these methods so that the code in these methods can still be evaluated.

This PR makes the expectation clear, while still keep the advanced version of `splitExpressions` to customize the inputs to pass to generated methods.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19827 from cloud-fan/codegen.
2017-11-28 22:57:30 +08:00
Wenchen Fan 1e07fff248 [SPARK-22520][SQL][FOLLOWUP] remove outer if for case when codegen
## What changes were proposed in this pull request?

a minor cleanup for https://github.com/apache/spark/pull/19752 . Remove the outer if as the code is inside `do while`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19830 from cloud-fan/minor.
2017-11-28 22:43:24 +08:00
Takuya UESHIN 64817c423c [SPARK-22395][SQL][PYTHON] Fix the behavior of timestamp values for Pandas to respect session timezone
## What changes were proposed in this pull request?

When converting Pandas DataFrame/Series from/to Spark DataFrame using `toPandas()` or pandas udfs, timestamp values behave to respect Python system timezone instead of session timezone.

For example, let's say we use `"America/Los_Angeles"` as session timezone and have a timestamp value `"1970-01-01 00:00:01"` in the timezone. Btw, I'm in Japan so Python timezone would be `"Asia/Tokyo"`.

The timestamp value from current `toPandas()` will be the following:

```
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
>>> df = spark.createDataFrame([28801], "long").selectExpr("timestamp(value) as ts")
>>> df.show()
+-------------------+
|                 ts|
+-------------------+
|1970-01-01 00:00:01|
+-------------------+

>>> df.toPandas()
                   ts
0 1970-01-01 17:00:01
```

As you can see, the value becomes `"1970-01-01 17:00:01"` because it respects Python timezone.
As we discussed in #18664, we consider this behavior is a bug and the value should be `"1970-01-01 00:00:01"`.

## How was this patch tested?

Added tests and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19607 from ueshin/issues/SPARK-22395.
2017-11-28 16:45:22 +08:00
gaborgsomogyi 33d43bf1b6 [SPARK-22484][DOC] Document PySpark DataFrame csv writer behavior whe…
## What changes were proposed in this pull request?

In PySpark API Document, DataFrame.write.csv() says that setting the quote parameter to an empty string should turn off quoting. Instead, it uses the [null character](https://en.wikipedia.org/wiki/Null_character) as the quote.

This PR fixes the doc.

## How was this patch tested?

Manual.

```
cd python/docs
make html
open _build/html/pyspark.sql.html
```

Author: gaborgsomogyi <gabor.g.somogyi@gmail.com>

Closes #19814 from gaborgsomogyi/SPARK-22484.
2017-11-28 10:14:35 +09:00
Marco Gaido 087879a77a [SPARK-22520][SQL] Support code generation for large CaseWhen
## What changes were proposed in this pull request?

Code generation is disabled for CaseWhen when the number of branches is higher than `spark.sql.codegen.maxCaseBranches` (which defaults to 20). This was done to prevent the well known 64KB method limit exception.
This PR proposes to support code generation also in those cases (without causing exceptions of course). As a side effect, we could get rid of the `spark.sql.codegen.maxCaseBranches` configuration.

## How was this patch tested?

existing UTs

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

Closes #19752 from mgaido91/SPARK-22520.
2017-11-28 07:46:18 +08:00
Zhenhua Wang 1ff4a77be4 [SPARK-22529][SQL] Relation stats should be consistent with other plans based on cbo config
## What changes were proposed in this pull request?

Currently, relation stats is the same whether cbo is enabled or not. While relation (`LogicalRelation` or `HiveTableRelation`) is a `LogicalPlan`, its behavior is inconsistent with other plans. This can cause confusion when user runs EXPLAIN COST commands. Besides, when CBO is disabled, we apply the size-only estimation strategy, so there's no need to propagate other catalog statistics to relation.

## How was this patch tested?

Enhanced existing tests case and added a test case.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19757 from wzhfy/catalog_stats_conversion.
2017-11-28 01:13:44 +08:00
Kazuaki Ishizaki 2dbe275b2d [SPARK-22603][SQL] Fix 64KB JVM bytecode limit problem with FormatString
## What changes were proposed in this pull request?

This PR changes `FormatString` code generation to place generated code for expressions for arguments into separated methods if these size could be large.
This PR passes variable arguments by using an `Object` array.

## How was this patch tested?

Added new test cases into `StringExpressionSuite`

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

Closes #19817 from kiszk/SPARK-22603.
2017-11-27 20:32:01 +08:00
Wenchen Fan 5a02e3a2ac [SPARK-22602][SQL] remove ColumnVector#loadBytes
## What changes were proposed in this pull request?

`ColumnVector#loadBytes` is only used as an optimization for reading UTF8String in `WritableColumnVector`, this PR moves this optimization to `WritableColumnVector` and simplified it.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19815 from cloud-fan/load-bytes.
2017-11-26 21:49:09 -08:00
Sean Owen fba63c1a7b [SPARK-22607][BUILD] Set large stack size consistently for tests to avoid StackOverflowError
## What changes were proposed in this pull request?

Set `-ea` and `-Xss4m` consistently for tests, to fix in particular:

```
OrderingSuite:
...
- GenerateOrdering with ShortType
*** RUN ABORTED ***
java.lang.StackOverflowError:
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:370)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
at org.codehaus.janino.CodeContext.flowAnalysis(CodeContext.java:541)
...
```

## How was this patch tested?

Existing tests. Manually verified it resolves the StackOverflowError this intends to resolve.

Author: Sean Owen <sowen@cloudera.com>

Closes #19820 from srowen/SPARK-22607.
2017-11-26 07:42:44 -06:00
Wenchen Fan e3fd93f149 [SPARK-22604][SQL] remove the get address methods from ColumnVector
## What changes were proposed in this pull request?

`nullsNativeAddress` and `valuesNativeAddress` are only used in tests and benchmark, no need to be top class API.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19818 from cloud-fan/minor.
2017-11-24 22:43:47 -08:00
Wenchen Fan 70221903f5 [SPARK-22596][SQL] set ctx.currentVars in CodegenSupport.consume
## What changes were proposed in this pull request?

`ctx.currentVars` means the input variables for the current operator, which is already decided in `CodegenSupport`, we can set it there instead of `doConsume`.

also add more comments to help people understand the codegen framework.

After this PR, we now have a principle about setting `ctx.currentVars` and `ctx.INPUT_ROW`:
1. for non-whole-stage-codegen path, never set them. (permit some special cases like generating ordering)
2. for whole-stage-codegen `produce` path, mostly we don't need to set them, but blocking operators may need to set them for expressions that produce data from data source, sort buffer, aggregate buffer, etc.
3. for whole-stage-codegen `consume` path, mostly we don't need to set them because `currentVars` is automatically set to child input variables and `INPUT_ROW` is mostly not used. A few plans need to tweak them as they may have different inputs, or they use the input row.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19803 from cloud-fan/codegen.
2017-11-24 21:50:30 -08:00
Kazuaki Ishizaki 554adc77d2 [SPARK-22595][SQL] fix flaky test: CastSuite.SPARK-22500: cast for struct should not generate codes beyond 64KB
## What changes were proposed in this pull request?

This PR reduces the number of fields in the test case of `CastSuite` to fix an issue that is pointed at [here](https://github.com/apache/spark/pull/19800#issuecomment-346634950).

```
java.lang.OutOfMemoryError: GC overhead limit exceeded
java.lang.OutOfMemoryError: GC overhead limit exceeded
	at org.codehaus.janino.UnitCompiler.findClass(UnitCompiler.java:10971)
	at org.codehaus.janino.UnitCompiler.findTypeByName(UnitCompiler.java:7607)
	at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:5758)
	at org.codehaus.janino.UnitCompiler.getType2(UnitCompiler.java:5732)
	at org.codehaus.janino.UnitCompiler.access$13200(UnitCompiler.java:206)
	at org.codehaus.janino.UnitCompiler$18.visitReferenceType(UnitCompiler.java:5668)
	at org.codehaus.janino.UnitCompiler$18.visitReferenceType(UnitCompiler.java:5660)
	at org.codehaus.janino.Java$ReferenceType.accept(Java.java:3356)
	at org.codehaus.janino.UnitCompiler.getType(UnitCompiler.java:5660)
	at org.codehaus.janino.UnitCompiler.buildLocalVariableMap(UnitCompiler.java:2892)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:2764)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1262)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMethods(UnitCompiler.java:1234)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:538)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:890)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:894)
	at org.codehaus.janino.UnitCompiler.access$600(UnitCompiler.java:206)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:377)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:369)
	at org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1128)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:369)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1209)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:564)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:890)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:894)
	at org.codehaus.janino.UnitCompiler.access$600(UnitCompiler.java:206)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:377)
	at org.codehaus.janino.UnitCompiler$2.visitMemberClassDeclaration(UnitCompiler.java:369)
	at org.codehaus.janino.Java$MemberClassDeclaration.accept(Java.java:1128)
	at org.codehaus.janino.UnitCompiler.compile(UnitCompiler.java:369)
	at org.codehaus.janino.UnitCompiler.compileDeclaredMemberTypes(UnitCompiler.java:1209)
	at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:564)
...
```

## How was this patch tested?

Used existing test case

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

Closes #19806 from kiszk/SPARK-22595.
2017-11-24 12:08:49 +01:00
Liang-Chi Hsieh 62a826f17c [SPARK-22591][SQL] GenerateOrdering shouldn't change CodegenContext.INPUT_ROW
## What changes were proposed in this pull request?

When I played with codegen in developing another PR, I found the value of `CodegenContext.INPUT_ROW` is not reliable. Under wholestage codegen, it is assigned to null first and then suddenly changed to `i`.

The reason is `GenerateOrdering` changes `CodegenContext.INPUT_ROW` but doesn't restore it back.

## How was this patch tested?

Added test.

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

Closes #19800 from viirya/SPARK-22591.
2017-11-24 11:46:58 +01:00
Wenchen Fan c1217565e2 [SPARK-22592][SQL] cleanup filter converting for hive
## What changes were proposed in this pull request?

We have 2 different methods to convert filters for hive, regarding a config. This introduces duplicated and inconsistent code(e.g. one use helper objects for pattern match and one doesn't).

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19801 from cloud-fan/cleanup.
2017-11-23 15:33:26 -08:00
Wenchen Fan 42f83d7c40 [SPARK-17920][FOLLOWUP] simplify the schema file creation in test
## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/19779 , to simplify the file creation.

## How was this patch tested?

test only change

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19799 from cloud-fan/minor.
2017-11-23 18:20:16 +01:00
Wenchen Fan 0605ad7614 [SPARK-22543][SQL] fix java 64kb compile error for deeply nested expressions
## What changes were proposed in this pull request?

A frequently reported issue of Spark is the Java 64kb compile error. This is because Spark generates a very big method and it's usually caused by 3 reasons:

1. a deep expression tree, e.g. a very complex filter condition
2. many individual expressions, e.g. expressions can have many children, operators can have many expressions.
3. a deep query plan tree (with whole stage codegen)

This PR focuses on 1. There are already several patches(#15620  #18972 #18641) trying to fix this issue and some of them are already merged. However this is an endless job as every non-leaf expression has this issue.

This PR proposes to fix this issue in `Expression.genCode`, to make sure the code for a single expression won't grow too big.

According to maropu 's benchmark, no regression is found with TPCDS (thanks maropu !): https://docs.google.com/spreadsheets/d/1K3_7lX05-ZgxDXi9X_GleNnDjcnJIfoSlSCDZcL4gdg/edit?usp=sharing

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@gmail.com>

Closes #19767 from cloud-fan/codegen.
2017-11-22 10:05:46 -08:00
Kazuaki Ishizaki 572af5027e [SPARK-20101][SQL][FOLLOW-UP] use correct config name "spark.sql.columnVector.offheap.enabled"
## What changes were proposed in this pull request?

This PR addresses [the spelling miss](https://github.com/apache/spark/pull/17436#discussion_r152189670) of the config name `spark.sql.columnVector.offheap.enabled`.
We should use `spark.sql.columnVector.offheap.enabled`.

## How was this patch tested?

Existing tests

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

Closes #19794 from kiszk/SPARK-20101-follow.
2017-11-22 13:27:20 +01:00
Takeshi Yamamuro 2c0fe818a6 [SPARK-22445][SQL][FOLLOW-UP] Respect stream-side child's needCopyResult in BroadcastHashJoin
## What changes were proposed in this pull request?
I found #19656 causes some bugs, for example, it changed the result set of `q6` in tpcds (I keep tracking TPCDS results daily [here](https://github.com/maropu/spark-tpcds-datagen/tree/master/reports/tests)):
- w/o pr19658
```
+-----+---+
|state|cnt|
+-----+---+
|   MA| 10|
|   AK| 10|
|   AZ| 11|
|   ME| 13|
|   VT| 14|
|   NV| 15|
|   NH| 16|
|   UT| 17|
|   NJ| 21|
|   MD| 22|
|   WY| 25|
|   NM| 26|
|   OR| 31|
|   WA| 36|
|   ND| 38|
|   ID| 39|
|   SC| 45|
|   WV| 50|
|   FL| 51|
|   OK| 53|
|   MT| 53|
|   CO| 57|
|   AR| 58|
|   NY| 58|
|   PA| 62|
|   AL| 63|
|   LA| 63|
|   SD| 70|
|   WI| 80|
| null| 81|
|   MI| 82|
|   NC| 82|
|   MS| 83|
|   CA| 84|
|   MN| 85|
|   MO| 88|
|   IL| 95|
|   IA|102|
|   TN|102|
|   IN|103|
|   KY|104|
|   NE|113|
|   OH|114|
|   VA|130|
|   KS|139|
|   GA|168|
|   TX|216|
+-----+---+
```
- w/   pr19658
```
+-----+---+
|state|cnt|
+-----+---+
|   RI| 14|
|   AK| 16|
|   FL| 20|
|   NJ| 21|
|   NM| 21|
|   NV| 22|
|   MA| 22|
|   MD| 22|
|   UT| 22|
|   AZ| 25|
|   SC| 28|
|   AL| 36|
|   MT| 36|
|   WA| 39|
|   ND| 41|
|   MI| 44|
|   AR| 45|
|   OR| 47|
|   OK| 52|
|   PA| 53|
|   LA| 55|
|   CO| 55|
|   NY| 64|
|   WV| 66|
|   SD| 72|
|   MS| 73|
|   NC| 79|
|   IN| 82|
| null| 85|
|   ID| 88|
|   MN| 91|
|   WI| 95|
|   IL| 96|
|   MO| 97|
|   CA|109|
|   CA|109|
|   TN|114|
|   NE|115|
|   KY|128|
|   OH|131|
|   IA|156|
|   TX|160|
|   VA|182|
|   KS|211|
|   GA|230|
+-----+---+
```
This pr is to keep the original logic of `CodegenContext.copyResult` in `BroadcastHashJoinExec`.

## How was this patch tested?
Existing tests

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19781 from maropu/SPARK-22445-bugfix.
2017-11-22 09:09:50 +01:00
vinodkc e0d7665cec [SPARK-17920][SPARK-19580][SPARK-19878][SQL] Support writing to Hive table which uses Avro schema url 'avro.schema.url'
## What changes were proposed in this pull request?
SPARK-19580 Support for avro.schema.url while writing to hive table
SPARK-19878 Add hive configuration when initialize hive serde in InsertIntoHiveTable.scala
SPARK-17920 HiveWriterContainer passes null configuration to serde.initialize, causing NullPointerException in AvroSerde when using avro.schema.url

Support writing to Hive table which uses Avro schema url 'avro.schema.url'
For ex:
create external table avro_in (a string) stored as avro location '/avro-in/' tblproperties ('avro.schema.url'='/avro-schema/avro.avsc');

create external table avro_out (a string) stored as avro location '/avro-out/' tblproperties ('avro.schema.url'='/avro-schema/avro.avsc');

 insert overwrite table avro_out select * from avro_in;  // fails with java.lang.NullPointerException

 WARN AvroSerDe: Encountered exception determining schema. Returning signal schema to indicate problem
java.lang.NullPointerException
	at org.apache.hadoop.fs.FileSystem.getDefaultUri(FileSystem.java:182)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:174)

## Changes proposed in this fix
Currently 'null' value is passed to serializer, which causes NPE during insert operation, instead pass Hadoop configuration object
## How was this patch tested?
Added new test case in VersionsSuite

Author: vinodkc <vinod.kc.in@gmail.com>

Closes #19779 from vinodkc/br_Fix_SPARK-17920.
2017-11-21 22:31:46 -08:00
Jia Li 881c5c8073 [SPARK-22548][SQL] Incorrect nested AND expression pushed down to JDBC data source
## What changes were proposed in this pull request?

Let’s say I have a nested AND expression shown below and p2 can not be pushed down,

(p1 AND p2) OR p3

In current Spark code, during data source filter translation, (p1 AND p2) is returned as p1 only and p2 is simply lost. This issue occurs with JDBC data source and is similar to [SPARK-12218](https://github.com/apache/spark/pull/10362) for Parquet. When we have AND nested below another expression, we should either push both legs or nothing.

Note that:
- The current Spark code will always split conjunctive predicate before it determines if a predicate can be pushed down or not
- If I have (p1 AND p2) AND p3, it will be split into p1, p2, p3. There won't be nested AND expression.
- The current Spark code logic for OR is OK. It either pushes both legs or nothing.

The same translation method is also called by Data Source V2.

## How was this patch tested?

Added new unit test cases to JDBCSuite

gatorsmile

Author: Jia Li <jiali@us.ibm.com>

Closes #19776 from jliwork/spark-22548.
2017-11-21 17:30:02 -08:00
Kazuaki Ishizaki ac10171bea [SPARK-22500][SQL] Fix 64KB JVM bytecode limit problem with cast
## What changes were proposed in this pull request?

This PR changes `cast` code generation to place generated code for expression for fields of a structure into separated methods if these size could be large.

## How was this patch tested?

Added new test cases into `CastSuite`

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

Closes #19730 from kiszk/SPARK-22500.
2017-11-21 22:24:43 +01:00
Marco Gaido b96f61b6b2 [SPARK-22475][SQL] show histogram in DESC COLUMN command
## What changes were proposed in this pull request?

Added the histogram representation to the output of the `DESCRIBE EXTENDED table_name column_name` command.

## How was this patch tested?

Modified SQL UT and checked output

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

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19774 from mgaido91/SPARK-22475.
2017-11-21 20:55:24 +01:00
hyukjinkwon 6d7ebf2f9f [SPARK-22165][SQL] Fixes type conflicts between double, long, decimals, dates and timestamps in partition column
## What changes were proposed in this pull request?

This PR proposes to add a rule that re-uses `TypeCoercion.findWiderCommonType` when resolving type conflicts in partition values.

Currently, this uses numeric precedence-like comparison; therefore, it looks introducing failures for type conflicts between timestamps, dates and decimals, please see:

```scala
private val upCastingOrder: Seq[DataType] =
  Seq(NullType, IntegerType, LongType, FloatType, DoubleType, StringType)
...
literals.map(_.dataType).maxBy(upCastingOrder.indexOf(_))
```

The codes below:

```scala
val df = Seq((1, "2015-01-01"), (2, "2016-01-01 00:00:00")).toDF("i", "ts")
df.write.format("parquet").partitionBy("ts").save("/tmp/foo")
spark.read.load("/tmp/foo").printSchema()

val df = Seq((1, "1"), (2, "1" * 30)).toDF("i", "decimal")
df.write.format("parquet").partitionBy("decimal").save("/tmp/bar")
spark.read.load("/tmp/bar").printSchema()
```

produces output as below:

**Before**

```
root
 |-- i: integer (nullable = true)
 |-- ts: date (nullable = true)

root
 |-- i: integer (nullable = true)
 |-- decimal: integer (nullable = true)
```

**After**

```
root
 |-- i: integer (nullable = true)
 |-- ts: timestamp (nullable = true)

root
 |-- i: integer (nullable = true)
 |-- decimal: decimal(30,0) (nullable = true)
```

### Type coercion table:

This PR proposes the type conflict resolusion as below:

**Before**

|InputA \ InputB|`NullType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DateType`|`TimestampType`|`StringType`|
|------------------------|----------|----------|----------|----------|----------|----------|----------|----------|
|**`NullType`**|`StringType`|`IntegerType`|`LongType`|`StringType`|`DoubleType`|`StringType`|`StringType`|`StringType`|
|**`IntegerType`**|`IntegerType`|`IntegerType`|`LongType`|`IntegerType`|`DoubleType`|`IntegerType`|`IntegerType`|`StringType`|
|**`LongType`**|`LongType`|`LongType`|`LongType`|`LongType`|`DoubleType`|`LongType`|`LongType`|`StringType`|
|**`DecimalType(38,0)`**|`StringType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DecimalType(38,0)`|`DecimalType(38,0)`|`StringType`|
|**`DoubleType`**|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`DoubleType`|`StringType`|
|**`DateType`**|`StringType`|`IntegerType`|`LongType`|`DateType`|`DoubleType`|`DateType`|`DateType`|`StringType`|
|**`TimestampType`**|`StringType`|`IntegerType`|`LongType`|`TimestampType`|`DoubleType`|`TimestampType`|`TimestampType`|`StringType`|
|**`StringType`**|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|

**After**

|InputA \ InputB|`NullType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DateType`|`TimestampType`|`StringType`|
|------------------------|----------|----------|----------|----------|----------|----------|----------|----------|
|**`NullType`**|`NullType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`DateType`|`TimestampType`|`StringType`|
|**`IntegerType`**|`IntegerType`|`IntegerType`|`LongType`|`DecimalType(38,0)`|`DoubleType`|`StringType`|`StringType`|`StringType`|
|**`LongType`**|`LongType`|`LongType`|`LongType`|`DecimalType(38,0)`|`StringType`|`StringType`|`StringType`|`StringType`|
|**`DecimalType(38,0)`**|`DecimalType(38,0)`|`DecimalType(38,0)`|`DecimalType(38,0)`|`DecimalType(38,0)`|`StringType`|`StringType`|`StringType`|`StringType`|
|**`DoubleType`**|`DoubleType`|`DoubleType`|`StringType`|`StringType`|`DoubleType`|`StringType`|`StringType`|`StringType`|
|**`DateType`**|`DateType`|`StringType`|`StringType`|`StringType`|`StringType`|`DateType`|`TimestampType`|`StringType`|
|**`TimestampType`**|`TimestampType`|`StringType`|`StringType`|`StringType`|`StringType`|`TimestampType`|`TimestampType`|`StringType`|
|**`StringType`**|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|`StringType`|

This was produced by:

```scala
  test("Print out chart") {
    val supportedTypes: Seq[DataType] = Seq(
      NullType, IntegerType, LongType, DecimalType(38, 0), DoubleType,
      DateType, TimestampType, StringType)

    // Old type conflict resolution:
    val upCastingOrder: Seq[DataType] =
      Seq(NullType, IntegerType, LongType, FloatType, DoubleType, StringType)
    def oldResolveTypeConflicts(dataTypes: Seq[DataType]): DataType = {
      val topType = dataTypes.maxBy(upCastingOrder.indexOf(_))
      if (topType == NullType) StringType else topType
    }
    println(s"|InputA \\ InputB|${supportedTypes.map(dt => s"`${dt.toString}`").mkString("|")}|")
    println(s"|------------------------|${supportedTypes.map(_ => "----------").mkString("|")}|")
    supportedTypes.foreach { inputA =>
      val types = supportedTypes.map(inputB => oldResolveTypeConflicts(Seq(inputA, inputB)))
      println(s"|**`$inputA`**|${types.map(dt => s"`${dt.toString}`").mkString("|")}|")
    }

    // New type conflict resolution:
    def newResolveTypeConflicts(dataTypes: Seq[DataType]): DataType = {
      dataTypes.fold[DataType](NullType)(findWiderTypeForPartitionColumn)
    }
    println(s"|InputA \\ InputB|${supportedTypes.map(dt => s"`${dt.toString}`").mkString("|")}|")
    println(s"|------------------------|${supportedTypes.map(_ => "----------").mkString("|")}|")
    supportedTypes.foreach { inputA =>
      val types = supportedTypes.map(inputB => newResolveTypeConflicts(Seq(inputA, inputB)))
      println(s"|**`$inputA`**|${types.map(dt => s"`${dt.toString}`").mkString("|")}|")
    }
  }
```

## How was this patch tested?

Unit tests added in `ParquetPartitionDiscoverySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19389 from HyukjinKwon/partition-type-coercion.
2017-11-21 20:53:38 +01:00
gatorsmile 96e947ed6c [SPARK-22569][SQL] Clean usage of addMutableState and splitExpressions
## What changes were proposed in this pull request?
This PR is to clean the usage of addMutableState and splitExpressions

1. replace hardcoded type string to ctx.JAVA_BOOLEAN etc.
2. create a default value of the initCode for ctx.addMutableStats
3. Use named arguments when calling `splitExpressions `

## How was this patch tested?
The existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19790 from gatorsmile/codeClean.
2017-11-21 13:48:09 +01:00
Kazuaki Ishizaki 9bdff0bcd8 [SPARK-22550][SQL] Fix 64KB JVM bytecode limit problem with elt
## What changes were proposed in this pull request?

This PR changes `elt` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved the case of `elt` with a lot of argument

## How was this patch tested?

Added new test cases into `StringExpressionsSuite`

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

Closes #19778 from kiszk/SPARK-22550.
2017-11-21 12:19:11 +01:00
Kazuaki Ishizaki c957714806 [SPARK-22508][SQL] Fix 64KB JVM bytecode limit problem with GenerateUnsafeRowJoiner.create()
## What changes were proposed in this pull request?

This PR changes `GenerateUnsafeRowJoiner.create()` code generation to place generated code for statements to operate bitmap and offset into separated methods if these size could be large.

## How was this patch tested?

Added a new test case into `GenerateUnsafeRowJoinerSuite`

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

Closes #19737 from kiszk/SPARK-22508.
2017-11-21 12:16:54 +01:00
Kazuaki Ishizaki 41c6f36018 [SPARK-22549][SQL] Fix 64KB JVM bytecode limit problem with concat_ws
## What changes were proposed in this pull request?

This PR changes `concat_ws` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved the case of `concat_ws` with a lot of argument

## How was this patch tested?

Added new test cases into `StringExpressionsSuite`

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

Closes #19777 from kiszk/SPARK-22549.
2017-11-21 01:42:05 +01:00
Kazuaki Ishizaki 3c3eebc873 [SPARK-20101][SQL] Use OffHeapColumnVector when "spark.sql.columnVector.offheap.enable" is set to "true"
This PR enables to use ``OffHeapColumnVector`` when ``spark.sql.columnVector.offheap.enable`` is set to ``true``. While ``ColumnVector`` has two implementations ``OnHeapColumnVector`` and ``OffHeapColumnVector``, only ``OnHeapColumnVector`` is always used.

This PR implements the followings
- Pass ``OffHeapColumnVector`` to ``ColumnarBatch.allocate()`` when ``spark.sql.columnVector.offheap.enable`` is set to ``true``
- Free all of off-heap memory regions by ``OffHeapColumnVector.close()``
- Ensure to call ``OffHeapColumnVector.close()``

Use existing tests

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

Closes #17436 from kiszk/SPARK-20101.
2017-11-20 12:40:26 +01:00
Dongjoon Hyun b10837ab1a [SPARK-22557][TEST] Use ThreadSignaler explicitly
## What changes were proposed in this pull request?

ScalaTest 3.0 uses an implicit `Signaler`. This PR makes it sure all Spark tests uses `ThreadSignaler` explicitly which has the same default behavior of interrupting a thread on the JVM like ScalaTest 2.2.x. This will reduce potential flakiness.

## How was this patch tested?

This is testsuite-only update. This should passes the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19784 from dongjoon-hyun/use_thread_signaler.
2017-11-20 13:32:01 +09:00
Kazuaki Ishizaki d54bfec2e0 [SPARK-22498][SQL] Fix 64KB JVM bytecode limit problem with concat
## What changes were proposed in this pull request?

This PR changes `concat` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved the case of `concat` with a lot of argument

## How was this patch tested?

Added new test cases into `StringExpressionsSuite`

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

Closes #19728 from kiszk/SPARK-22498.
2017-11-18 19:40:06 +01:00
Shixiong Zhu bf0c0ae2dc [SPARK-22544][SS] FileStreamSource should use its own hadoop conf to call globPathIfNecessary
## What changes were proposed in this pull request?

Pass the FileSystem created using the correct Hadoop conf into `globPathIfNecessary` so that it can pick up user's hadoop configurations, such as credentials.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19771 from zsxwing/fix-file-stream-conf.
2017-11-17 15:35:24 -08:00
Li Jin 7d039e0c0a [SPARK-22409] Introduce function type argument in pandas_udf
## What changes were proposed in this pull request?

* Add a "function type" argument to pandas_udf.
* Add a new public enum class `PandasUdfType` in pyspark.sql.functions
* Refactor udf related code from pyspark.sql.functions to pyspark.sql.udf
* Merge "PythonUdfType" and "PythonEvalType" into a single enum class "PythonEvalType"

Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType

pandas_udf('double', PandasUDFType.SCALAR):
def plus_one(v):
    return v + 1
```

## Design doc
https://docs.google.com/document/d/1KlLaa-xJ3oz28xlEJqXyCAHU3dwFYkFs_ixcUXrJNTc/edit

## How was this patch tested?

Added PandasUDFTests

## TODO:
* [x] Implement proper enum type for `PandasUDFType`
* [x] Update documentation
* [x] Add more tests in PandasUDFTests

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

Closes #19630 from icexelloss/spark-22409-pandas-udf-type.
2017-11-17 16:43:08 +01:00
Wenchen Fan b9dcbe5e1b [SPARK-22542][SQL] remove unused features in ColumnarBatch
## What changes were proposed in this pull request?

`ColumnarBatch` provides features to do fast filter and project in a columnar fashion, however this feature is never used by Spark, as Spark uses whole stage codegen and processes the data in a row fashion. This PR proposes to remove these unused features as we won't switch to columnar execution in the near future. Even we do, I think this part needs a proper redesign.

This is also a step to make `ColumnVector` public, as we don't wanna expose these features to users.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19766 from cloud-fan/vector.
2017-11-16 18:23:00 -08:00
Kazuaki Ishizaki 7f2e62ee6b [SPARK-22501][SQL] Fix 64KB JVM bytecode limit problem with in
## What changes were proposed in this pull request?

This PR changes `In` code generation to place generated code for expression for expressions for arguments into separated methods if these size could be large.

## How was this patch tested?

Added new test cases into `PredicateSuite`

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

Closes #19733 from kiszk/SPARK-22501.
2017-11-16 18:24:49 +01:00
Marco Gaido 4e7f07e255 [SPARK-22494][SQL] Fix 64KB limit exception with Coalesce and AtleastNNonNulls
## What changes were proposed in this pull request?

Both `Coalesce` and `AtLeastNNonNulls` can cause the 64KB limit exception when used with a lot of arguments and/or complex expressions.
This PR splits their expressions in order to avoid the issue.

## How was this patch tested?

Added UTs

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

Closes #19720 from mgaido91/SPARK-22494.
2017-11-16 18:19:13 +01:00
Kazuaki Ishizaki ed885e7a65 [SPARK-22499][SQL] Fix 64KB JVM bytecode limit problem with least and greatest
## What changes were proposed in this pull request?

This PR changes `least` and `greatest` code generation to place generated code for expression for arguments into separated methods if these size could be large.
This PR resolved two cases:

* `least` with a lot of argument
* `greatest` with a lot of argument

## How was this patch tested?

Added a new test case into `ArithmeticExpressionsSuite`

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

Closes #19729 from kiszk/SPARK-22499.
2017-11-16 17:56:21 +01:00
osatici 2014e7a789 [SPARK-22479][SQL] Exclude credentials from SaveintoDataSourceCommand.simpleString
## What changes were proposed in this pull request?

Do not include jdbc properties which may contain credentials in logging a logical plan with `SaveIntoDataSourceCommand` in it.

## How was this patch tested?

building locally and trying to reproduce (per the steps in https://issues.apache.org/jira/browse/SPARK-22479):
```
== Parsed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Analyzed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Optimized Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Physical Plan ==
Execute SaveIntoDataSourceCommand
   +- SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
         +- Range (0, 100, step=1, splits=Some(8))
```

Author: osatici <osatici@palantir.com>

Closes #19708 from onursatici/os/redact-jdbc-creds.
2017-11-15 14:08:51 -08:00
liutang123 bc0848b4c1 [SPARK-22469][SQL] Accuracy problem in comparison with string and numeric
## What changes were proposed in this pull request?
This fixes a problem caused by #15880
`select '1.5' > 0.5; // Result is NULL in Spark but is true in Hive.
`
When compare string and numeric, cast them as double like Hive.

Author: liutang123 <liutang123@yeah.net>

Closes #19692 from liutang123/SPARK-22469.
2017-11-15 09:02:54 -08:00
Wenchen Fan dce1610ae3 [SPARK-22514][SQL] move ColumnVector.Array and ColumnarBatch.Row to individual files
## What changes were proposed in this pull request?

Logically the `Array` doesn't belong to `ColumnVector`, and `Row` doesn't belong to `ColumnarBatch`. e.g. `ColumnVector` needs to return `Array` for `getArray`, and `Row` for `getStruct`. `Array` and `Row` can return each other with the `getArray`/`getStruct` methods.

This is also a step to make `ColumnVector` public, it's cleaner to have `Array` and `Row` as top-level classes.

This PR is just code moving around, with 2 renaming: `Array` -> `VectorBasedArray`, `Row` -> `VectorBasedRow`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19740 from cloud-fan/vector.
2017-11-15 14:42:37 +01:00
Marcelo Vanzin 0ffa7c488f [SPARK-20652][SQL] Store SQL UI data in the new app status store.
This change replaces the SQLListener with a new implementation that
saves the data to the same store used by the SparkContext's status
store. For that, the types used by the old SQLListener had to be
updated a bit so that they're more serialization-friendly.

The interface for getting data from the store was abstracted into
a new class, SQLAppStatusStore (following the convention used in
core).

Another change is the way that the SQL UI hooks up into the core
UI or the SHS. The old "SparkHistoryListenerFactory" was replaced
with a new "AppStatePlugin" that more explicitly differentiates
between the two use cases: processing events, and showing the UI.
Both live apps and the SHS use this new API (previously, it was
restricted to the SHS).

Note on the above: this causes a slight change of behavior for
live apps; the SQL tab will only show up after the first execution
is started.

The metrics gathering code was re-worked a bit so that the types
used are less memory hungry and more serialization-friendly. This
reduces memory usage when using in-memory stores, and reduces load
times when using disk stores.

Tested with existing and added unit tests. Note one unit test was
disabled because it depends on SPARK-20653, which isn't in yet.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19681 from vanzin/SPARK-20652.
2017-11-14 15:28:22 -06:00
Zhenhua Wang 11b60af737 [SPARK-17074][SQL] Generate equi-height histogram in column statistics
## What changes were proposed in this pull request?

Equi-height histogram is effective in cardinality estimation, and more accurate than basic column stats (min, max, ndv, etc) especially in skew distribution. So we need to support it.

For equi-height histogram, all buckets (intervals) have the same height (frequency).
In this PR, we use a two-step method to generate an equi-height histogram:
1. use `ApproximatePercentile` to get percentiles `p(0), p(1/n), p(2/n) ... p((n-1)/n), p(1)`;
2. construct range values of buckets, e.g. `[p(0), p(1/n)], [p(1/n), p(2/n)] ... [p((n-1)/n), p(1)]`, and use `ApproxCountDistinctForIntervals` to count ndv in each bucket. Each bucket is of the form: `(lowerBound, higherBound, ndv)`.

## How was this patch tested?

Added new test cases and modified some existing test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19479 from wzhfy/generate_histogram.
2017-11-14 16:41:43 +01:00
hyukjinkwon 673c670465 [SPARK-17310][SQL] Add an option to disable record-level filter in Parquet-side
## What changes were proposed in this pull request?

There is a concern that Spark-side codegen row-by-row filtering might be faster than Parquet's one in general due to type-boxing and additional fuction calls which Spark's one tries to avoid.

So, this PR adds an option to disable/enable record-by-record filtering in Parquet side.

It sets the default to `false` to take the advantage of the improvement.

This was also discussed in https://github.com/apache/spark/pull/14671.
## How was this patch tested?

Manually benchmarks were performed. I generated a billion (1,000,000,000) records and tested equality comparison concatenated with `OR`. This filter combinations were made from 5 to 30.

It seem indeed Spark-filtering is faster in the test case and the gap increased as the filter tree becomes larger.

The details are as below:

**Code**

``` scala
test("Parquet-side filter vs Spark-side filter - record by record") {
  withTempPath { path =>
    val N = 1000 * 1000 * 1000
    val df = spark.range(N).toDF("a")
    df.write.parquet(path.getAbsolutePath)

    val benchmark = new Benchmark("Parquet-side vs Spark-side", N)
    Seq(5, 10, 20, 30).foreach { num =>
      val filterExpr = (0 to num).map(i => s"a = $i").mkString(" OR ")

      benchmark.addCase(s"Parquet-side filter - number of filters [$num]", 3) { _ =>
        withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> false.toString,
          SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> true.toString) {

          // We should strip Spark-side filter to compare correctly.
          stripSparkFilter(
            spark.read.parquet(path.getAbsolutePath).filter(filterExpr)).count()
        }
      }

      benchmark.addCase(s"Spark-side filter - number of filters [$num]", 3) { _ =>
        withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> false.toString,
          SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> false.toString) {

          spark.read.parquet(path.getAbsolutePath).filter(filterExpr).count()
        }
      }
    }

    benchmark.run()
  }
}
```

**Result**

```
Parquet-side vs Spark-side:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Parquet-side filter - number of filters [5]      4268 / 4367        234.3           4.3       0.8X
Spark-side filter - number of filters [5]      3709 / 3741        269.6           3.7       0.9X
Parquet-side filter - number of filters [10]      5673 / 5727        176.3           5.7       0.6X
Spark-side filter - number of filters [10]      3588 / 3632        278.7           3.6       0.9X
Parquet-side filter - number of filters [20]      8024 / 8440        124.6           8.0       0.4X
Spark-side filter - number of filters [20]      3912 / 3946        255.6           3.9       0.8X
Parquet-side filter - number of filters [30]    11936 / 12041         83.8          11.9       0.3X
Spark-side filter - number of filters [30]      3929 / 3978        254.5           3.9       0.8X
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15049 from HyukjinKwon/SPARK-17310.
2017-11-14 12:34:21 +01:00
Wenchen Fan f7534b37ee [SPARK-22487][SQL][FOLLOWUP] still keep spark.sql.hive.version
## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/19712 , adds back the `spark.sql.hive.version`, so that if users try to read this config, they can still get a default value instead of null.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19719 from cloud-fan/minor.
2017-11-13 13:10:13 -08:00
Bryan Cutler 209b9361ac [SPARK-20791][PYSPARK] Use Arrow to create Spark DataFrame from Pandas
## What changes were proposed in this pull request?

This change uses Arrow to optimize the creation of a Spark DataFrame from a Pandas DataFrame. The input df is sliced according to the default parallelism. The optimization is enabled with the existing conf "spark.sql.execution.arrow.enabled" and is disabled by default.

## How was this patch tested?

Added new unit test to create DataFrame with and without the optimization enabled, then compare results.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19459 from BryanCutler/arrow-createDataFrame-from_pandas-SPARK-20791.
2017-11-13 13:16:01 +09:00
Kazuaki Ishizaki 9bf696dbec [SPARK-21720][SQL] Fix 64KB JVM bytecode limit problem with AND or OR
## What changes were proposed in this pull request?

This PR changes `AND` or `OR` code generation to place condition and then expressions' generated code into separated methods if these size could be large. When the method is newly generated, variables for `isNull` and `value` are declared as an instance variable to pass these values (e.g. `isNull1409` and `value1409`) to the callers of the generated method.

This PR resolved two cases:

* large code size of left expression
* large code size of right expression

## How was this patch tested?

Added a new test case into `CodeGenerationSuite`

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

Closes #18972 from kiszk/SPARK-21720.
2017-11-12 22:44:47 +01:00
Wenchen Fan 21a7bfd5c3 [SPARK-10365][SQL] Support Parquet logical type TIMESTAMP_MICROS
## What changes were proposed in this pull request?

This PR makes Spark to be able to read Parquet TIMESTAMP_MICROS values, and add a new config to allow Spark to write timestamp values to parquet as TIMESTAMP_MICROS type.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19702 from cloud-fan/parquet.
2017-11-11 22:40:26 +01:00
gatorsmile d6ee69e776 [SPARK-22488][SQL] Fix the view resolution issue in the SparkSession internal table() API
## What changes were proposed in this pull request?
The current internal `table()` API of `SparkSession` bypasses the Analyzer and directly calls `sessionState.catalog.lookupRelation` API. This skips the view resolution logics in our Analyzer rule `ResolveRelations`. This internal API is widely used by various DDL commands, public and internal APIs.

Users might get the strange error caused by view resolution when the default database is different.
```
Table or view not found: t1; line 1 pos 14
org.apache.spark.sql.AnalysisException: Table or view not found: t1; line 1 pos 14
	at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
```

This PR is to fix it by enforcing it to use `ResolveRelations` to resolve the table.

## How was this patch tested?
Added a test case and modified the existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19713 from gatorsmile/viewResolution.
2017-11-11 18:20:11 +01:00
Liang-Chi Hsieh 154351e6db [SPARK-22462][SQL] Make rdd-based actions in Dataset trackable in SQL UI
## What changes were proposed in this pull request?

For the few Dataset actions such as `foreach`, currently no SQL metrics are visible in the SQL tab of SparkUI. It is because it binds wrongly to Dataset's `QueryExecution`. As the actions directly evaluate on the RDD which has individual `QueryExecution`, to show correct SQL metrics on UI, we should bind to RDD's `QueryExecution`.

## How was this patch tested?

Manually test. Screenshot is attached in the PR.

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

Closes #19689 from viirya/SPARK-22462.
2017-11-11 12:34:30 +01:00
Rekha Joshi 808e886b96 [SPARK-21667][STREAMING] ConsoleSink should not fail streaming query with checkpointLocation option
## What changes were proposed in this pull request?
Fix to allow recovery on console , avoid checkpoint exception

## How was this patch tested?
existing tests
manual tests [ Replicating error and seeing no checkpoint error after fix]

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: rjoshi2 <rekhajoshm@gmail.com>

Closes #19407 from rekhajoshm/SPARK-21667.
2017-11-10 15:18:11 -08:00
Kazuaki Ishizaki f2da738c76 [SPARK-22284][SQL] Fix 64KB JVM bytecode limit problem in calculating hash for nested structs
## What changes were proposed in this pull request?

This PR avoids to generate a huge method for calculating a murmur3 hash for nested structs. This PR splits a huge method (e.g. `apply_4`) into multiple smaller methods.

Sample program
```
  val structOfString = new StructType().add("str", StringType)
  var inner = new StructType()
  for (_ <- 0 until 800) {
    inner = inner1.add("structOfString", structOfString)
  }
  var schema = new StructType()
  for (_ <- 0 until 50) {
    schema = schema.add("structOfStructOfStrings", inner)
  }
  GenerateMutableProjection.generate(Seq(Murmur3Hash(exprs, 42)))
```

Without this PR
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private int value;
/* 010 */   private int value_0;
...
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     value = 42;
/* 040 */     apply_0(i);
/* 041 */     apply_1(i);
/* 042 */     apply_2(i);
/* 043 */     apply_3(i);
/* 044 */     apply_4(i);
/* 045 */     nestedClassInstance.apply_5(i);
...
/* 089 */     nestedClassInstance8.apply_49(i);
/* 090 */     value_0 = value;
/* 091 */
/* 092 */     // copy all the results into MutableRow
/* 093 */     mutableRow.setInt(0, value_0);
/* 094 */     return mutableRow;
/* 095 */   }
/* 096 */
/* 097 */
/* 098 */   private void apply_4(InternalRow i) {
/* 099 */
/* 100 */     boolean isNull5 = i.isNullAt(4);
/* 101 */     InternalRow value5 = isNull5 ? null : (i.getStruct(4, 800));
/* 102 */     if (!isNull5) {
/* 103 */
/* 104 */       if (!value5.isNullAt(0)) {
/* 105 */
/* 106 */         final InternalRow element6400 = value5.getStruct(0, 1);
/* 107 */
/* 108 */         if (!element6400.isNullAt(0)) {
/* 109 */
/* 110 */           final UTF8String element6401 = element6400.getUTF8String(0);
/* 111 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6401.getBaseObject(), element6401.getBaseOffset(), element6401.numBytes(), value);
/* 112 */
/* 113 */         }
/* 114 */
/* 115 */
/* 116 */       }
/* 117 */
/* 118 */
/* 119 */       if (!value5.isNullAt(1)) {
/* 120 */
/* 121 */         final InternalRow element6402 = value5.getStruct(1, 1);
/* 122 */
/* 123 */         if (!element6402.isNullAt(0)) {
/* 124 */
/* 125 */           final UTF8String element6403 = element6402.getUTF8String(0);
/* 126 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6403.getBaseObject(), element6403.getBaseOffset(), element6403.numBytes(), value);
/* 127 */
/* 128 */         }
/* 128 */         }
/* 129 */
/* 130 */
/* 131 */       }
/* 132 */
/* 133 */
/* 134 */       if (!value5.isNullAt(2)) {
/* 135 */
/* 136 */         final InternalRow element6404 = value5.getStruct(2, 1);
/* 137 */
/* 138 */         if (!element6404.isNullAt(0)) {
/* 139 */
/* 140 */           final UTF8String element6405 = element6404.getUTF8String(0);
/* 141 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6405.getBaseObject(), element6405.getBaseOffset(), element6405.numBytes(), value);
/* 142 */
/* 143 */         }
/* 144 */
/* 145 */
/* 146 */       }
/* 147 */
...
/* 12074 */       if (!value5.isNullAt(798)) {
/* 12075 */
/* 12076 */         final InternalRow element7996 = value5.getStruct(798, 1);
/* 12077 */
/* 12078 */         if (!element7996.isNullAt(0)) {
/* 12079 */
/* 12080 */           final UTF8String element7997 = element7996.getUTF8String(0);
/* 12083 */         }
/* 12084 */
/* 12085 */
/* 12086 */       }
/* 12087 */
/* 12088 */
/* 12089 */       if (!value5.isNullAt(799)) {
/* 12090 */
/* 12091 */         final InternalRow element7998 = value5.getStruct(799, 1);
/* 12092 */
/* 12093 */         if (!element7998.isNullAt(0)) {
/* 12094 */
/* 12095 */           final UTF8String element7999 = element7998.getUTF8String(0);
/* 12096 */           value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element7999.getBaseObject(), element7999.getBaseOffset(), element7999.numBytes(), value);
/* 12097 */
/* 12098 */         }
/* 12099 */
/* 12100 */
/* 12101 */       }
/* 12102 */
/* 12103 */     }
/* 12104 */
/* 12105 */   }
/* 12106 */
/* 12106 */
/* 12107 */
/* 12108 */   private void apply_1(InternalRow i) {
...
```

With this PR
```
/* 005 */ class SpecificMutableProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseMutableProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private InternalRow mutableRow;
/* 009 */   private int value;
/* 010 */   private int value_0;
/* 011 */
...
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     value = 42;
/* 040 */     nestedClassInstance11.apply50_0(i);
/* 041 */     nestedClassInstance11.apply50_1(i);
...
/* 088 */     nestedClassInstance11.apply50_48(i);
/* 089 */     nestedClassInstance11.apply50_49(i);
/* 090 */     value_0 = value;
/* 091 */
/* 092 */     // copy all the results into MutableRow
/* 093 */     mutableRow.setInt(0, value_0);
/* 094 */     return mutableRow;
/* 095 */   }
/* 096 */
...
/* 37717 */   private void apply4_0(InternalRow value5, InternalRow i) {
/* 37718 */
/* 37719 */     if (!value5.isNullAt(0)) {
/* 37720 */
/* 37721 */       final InternalRow element6400 = value5.getStruct(0, 1);
/* 37722 */
/* 37723 */       if (!element6400.isNullAt(0)) {
/* 37724 */
/* 37725 */         final UTF8String element6401 = element6400.getUTF8String(0);
/* 37726 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6401.getBaseObject(), element6401.getBaseOffset(), element6401.numBytes(), value);
/* 37727 */
/* 37728 */       }
/* 37729 */
/* 37730 */
/* 37731 */     }
/* 37732 */
/* 37733 */     if (!value5.isNullAt(1)) {
/* 37734 */
/* 37735 */       final InternalRow element6402 = value5.getStruct(1, 1);
/* 37736 */
/* 37737 */       if (!element6402.isNullAt(0)) {
/* 37738 */
/* 37739 */         final UTF8String element6403 = element6402.getUTF8String(0);
/* 37740 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6403.getBaseObject(), element6403.getBaseOffset(), element6403.numBytes(), value);
/* 37741 */
/* 37742 */       }
/* 37743 */
/* 37744 */
/* 37745 */     }
/* 37746 */
/* 37747 */     if (!value5.isNullAt(2)) {
/* 37748 */
/* 37749 */       final InternalRow element6404 = value5.getStruct(2, 1);
/* 37750 */
/* 37751 */       if (!element6404.isNullAt(0)) {
/* 37752 */
/* 37753 */         final UTF8String element6405 = element6404.getUTF8String(0);
/* 37754 */         value = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(element6405.getBaseObject(), element6405.getBaseOffset(), element6405.numBytes(), value);
/* 37755 */
/* 37756 */       }
/* 37757 */
/* 37758 */
/* 37759 */     }
/* 37760 */
/* 37761 */   }
...
/* 218470 */
/* 218471 */     private void apply50_4(InternalRow i) {
/* 218472 */
/* 218473 */       boolean isNull5 = i.isNullAt(4);
/* 218474 */       InternalRow value5 = isNull5 ? null : (i.getStruct(4, 800));
/* 218475 */       if (!isNull5) {
/* 218476 */         apply4_0(value5, i);
/* 218477 */         apply4_1(value5, i);
/* 218478 */         apply4_2(value5, i);
...
/* 218742 */         nestedClassInstance.apply4_266(value5, i);
/* 218743 */       }
/* 218744 */
/* 218745 */     }
```

## How was this patch tested?

Added new test to `HashExpressionsSuite`

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

Closes #19563 from kiszk/SPARK-22284.
2017-11-10 21:17:49 +01:00
Shixiong Zhu 24ea781cd3 [SPARK-19644][SQL] Clean up Scala reflection garbage after creating Encoder
## What changes were proposed in this pull request?

Because of the memory leak issue in `scala.reflect.api.Types.TypeApi.<:<` (https://github.com/scala/bug/issues/8302), creating an encoder may leak memory.

This PR adds `cleanUpReflectionObjects` to clean up these leaking objects for methods calling `scala.reflect.api.Types.TypeApi.<:<`.

## How was this patch tested?

The updated unit tests.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19687 from zsxwing/SPARK-19644.
2017-11-10 11:27:28 -08:00
Marco Gaido 5b41cbf13b [SPARK-22473][TEST] Replace deprecated AsyncAssertions.Waiter and methods of java.sql.Date
## What changes were proposed in this pull request?

In `spark-sql` module tests there are deprecations warnings caused by the usage of deprecated methods of `java.sql.Date` and the usage of the deprecated `AsyncAssertions.Waiter` class.
This PR replace the deprecated methods of `java.sql.Date` with non-deprecated ones (using `Calendar` where needed). It replaces also the deprecated `org.scalatest.concurrent.AsyncAssertions.Waiter` with `org.scalatest.concurrent.Waiters._`.

## How was this patch tested?

existing UTs

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19696 from mgaido91/SPARK-22473.
2017-11-10 11:24:24 -06:00
Kent Yao 28ab5bf597 [SPARK-22487][SQL][HIVE] Remove the unused HIVE_EXECUTION_VERSION property
## What changes were proposed in this pull request?

At the beginning https://github.com/apache/spark/pull/2843 added `spark.sql.hive.version` to reveal underlying hive version for jdbc connections. For some time afterwards, it was used as a version identifier for the execution hive client.

Actually there is no hive client for executions in spark now and there are no usages of HIVE_EXECUTION_VERSION found in whole spark project. HIVE_EXECUTION_VERSION is set by `spark.sql.hive.version`, which is still set internally in some places or by users, this may confuse developers and users with HIVE_METASTORE_VERSION(spark.sql.hive.metastore.version).

It might better to be removed.

## How was this patch tested?

modify some existing ut

cc cloud-fan gatorsmile

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19712 from yaooqinn/SPARK-22487.
2017-11-10 12:01:02 +01:00
Wenchen Fan 0025ddeb1d [SPARK-22472][SQL] add null check for top-level primitive values
## What changes were proposed in this pull request?

One powerful feature of `Dataset` is, we can easily map SQL rows to Scala/Java objects and do runtime null check automatically.

For example, let's say we have a parquet file with schema `<a: int, b: string>`, and we have a `case class Data(a: Int, b: String)`. Users can easily read this parquet file into `Data` objects, and Spark will throw NPE if column `a` has null values.

However the null checking is left behind for top-level primitive values. For example, let's say we have a parquet file with schema `<a: Int>`, and we read it into Scala `Int`. If column `a` has null values, we will get some weird results.
```
scala> val ds = spark.read.parquet(...).as[Int]

scala> ds.show()
+----+
|v   |
+----+
|null|
|1   |
+----+

scala> ds.collect
res0: Array[Long] = Array(0, 1)

scala> ds.map(_ * 2).show
+-----+
|value|
+-----+
|-2   |
|2    |
+-----+
```

This is because internally Spark use some special default values for primitive types, but never expect users to see/operate these default value directly.

This PR adds null check for top-level primitive values

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19707 from cloud-fan/bug.
2017-11-09 21:56:20 -08:00
Nathan Kronenfeld b57ed2245c [SPARK-22308][TEST-MAVEN] Support alternative unit testing styles in external applications
Continuation of PR#19528 (https://github.com/apache/spark/pull/19529#issuecomment-340252119)

The problem with the maven build in the previous PR was the new tests.... the creation of a spark session outside the tests meant there was more than one spark session around at a time.
I was using the spark session outside the tests so that the tests could share data; I've changed it so that each test creates the data anew.

Author: Nathan Kronenfeld <nicole.oresme@gmail.com>
Author: Nathan Kronenfeld <nkronenfeld@uncharted.software>

Closes #19705 from nkronenfeld/alternative-style-tests-2.
2017-11-09 19:11:30 -08:00
Liang-Chi Hsieh 77f74539ec [SPARK-20542][ML][SQL] Add an API to Bucketizer that can bin multiple columns
## What changes were proposed in this pull request?

Current ML's Bucketizer can only bin a column of continuous features. If a dataset has thousands of of continuous columns needed to bin, we will result in thousands of ML stages. It is inefficient regarding query planning and execution.

We should have a type of bucketizer that can bin a lot of columns all at once. It would need to accept an list of arrays of split points to correspond to the columns to bin, but it might make things more efficient by replacing thousands of stages with just one.

This current approach in this patch is to add a new `MultipleBucketizerInterface` for this purpose. `Bucketizer` now extends this new interface.

### Performance

Benchmarking using the test dataset provided in JIRA SPARK-20392 (blockbuster.csv).

The ML pipeline includes 2 `StringIndexer`s and 1 `MultipleBucketizer` or 137 `Bucketizer`s to bin 137 input columns with the same splits. Then count the time to transform the dataset.

MultipleBucketizer: 3352 ms
Bucketizer: 51512 ms

## How was this patch tested?

Jenkins tests.

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

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

Closes #17819 from viirya/SPARK-20542.
2017-11-09 16:35:06 +02:00
jerryshao 6793a3dac0 [SPARK-22405][SQL] Add new alter table and alter database related ExternalCatalogEvent
## What changes were proposed in this pull request?

We're building a data lineage tool in which we need to monitor the metadata changes in ExternalCatalog, current ExternalCatalog already provides several useful events like "CreateDatabaseEvent" for custom SparkListener to use. But still there's some event missing, like alter database event and alter table event. So here propose to and new ExternalCatalogEvent.

## How was this patch tested?

Enrich the current UT and tested on local cluster.

CC hvanhovell please let me know your comments about current proposal, thanks.

Author: jerryshao <sshao@hortonworks.com>

Closes #19649 from jerryshao/SPARK-22405.
2017-11-09 11:57:56 +01:00
Liang-Chi Hsieh 40a8aefaf3 [SPARK-22442][SQL] ScalaReflection should produce correct field names for special characters
## What changes were proposed in this pull request?

For a class with field name of special characters, e.g.:
```scala
case class MyType(`field.1`: String, `field 2`: String)
```

Although we can manipulate DataFrame/Dataset, the field names are encoded:
```scala
scala> val df = Seq(MyType("a", "b"), MyType("c", "d")).toDF
df: org.apache.spark.sql.DataFrame = [field$u002E1: string, field$u00202: string]
scala> df.as[MyType].collect
res7: Array[MyType] = Array(MyType(a,b), MyType(c,d))
```

It causes resolving problem when we try to convert the data with non-encoded field names:
```scala
spark.read.json(path).as[MyType]
...
[info]   org.apache.spark.sql.AnalysisException: cannot resolve '`field$u002E1`' given input columns: [field 2, fie
ld.1];
[info]   at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
...
```

We should use decoded field name in Dataset schema.

## How was this patch tested?

Added tests.

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

Closes #19664 from viirya/SPARK-22442.
2017-11-09 11:54:50 +01:00
Dongjoon Hyun 98be55c0fa [SPARK-22222][CORE][TEST][FOLLOW-UP] Remove redundant and deprecated Timeouts
## What changes were proposed in this pull request?

Since SPARK-21939, Apache Spark uses `TimeLimits` instead of the deprecated `Timeouts`. This PR fixes the build warning `BufferHolderSparkSubmitSuite.scala` introduced at [SPARK-22222](https://github.com/apache/spark/pull/19460/files#diff-d8cf6e0c229969db94ec8ffc31a9239cR36) by removing the redundant `Timeouts`.
```scala
trait Timeouts in package concurrent is deprecated: Please use org.scalatest.concurrent.TimeLimits instead
[warn]     with Timeouts {
```
## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19697 from dongjoon-hyun/SPARK-22222.
2017-11-09 16:34:38 +09:00
hyukjinkwon 695647bf2e [SPARK-21640][SQL][PYTHON][R][FOLLOWUP] Add errorifexists in SparkR and other documentations
## What changes were proposed in this pull request?

This PR proposes to add `errorifexists` to SparkR API and fix the rest of them describing the mode, mainly, in API documentations as well.

This PR also replaces `convertToJSaveMode` to `setWriteMode` so that string as is is passed to JVM and executes:

b034f2565f/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala (L72-L82)

and remove the duplication here:

3f958a9992/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala (L187-L194)

## How was this patch tested?

Manually checked the built documentation. These were mainly found by `` grep -r `error` `` and `grep -r 'error'`.

Also, unit tests added in `test_sparkSQL.R`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19673 from HyukjinKwon/SPARK-21640-followup.
2017-11-09 15:00:31 +09:00
ptkool d01044233c [SPARK-22456][SQL] Add support for dayofweek function
## What changes were proposed in this pull request?
This PR adds support for a new function called `dayofweek` that returns the day of the week of the given argument as an integer value in the range 1-7, where 1 represents Sunday.

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

Author: ptkool <michael.styles@shopify.com>

Closes #19672 from ptkool/day_of_week_function.
2017-11-09 14:44:39 +09:00
Liang-Chi Hsieh 87343e1556 [SPARK-22446][SQL][ML] Declare StringIndexerModel indexer udf as nondeterministic
## What changes were proposed in this pull request?

UDFs that can cause runtime exception on invalid data are not safe to pushdown, because its behavior depends on its position in the query plan. Pushdown of it will risk to change its original behavior.

The example reported in the JIRA and taken as test case shows this issue. We should declare UDFs that can cause runtime exception on invalid data as non-determinstic.

This updates the document of `deterministic` property in `Expression` and states clearly an UDF that can cause runtime exception on some specific input, should be declared as non-determinstic.

## How was this patch tested?

Added test. Manually test.

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

Closes #19662 from viirya/SPARK-22446.
2017-11-08 12:17:52 +01:00
gatorsmile 0846a44736 [SPARK-22464][SQL] No pushdown for Hive metastore partition predicates containing null-safe equality
## What changes were proposed in this pull request?
`<=>` is not supported by Hive metastore partition predicate pushdown. We should not push down it to Hive metastore when they are be using in partition predicates.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19682 from gatorsmile/fixLimitPushDown.
2017-11-07 21:57:43 +01:00
Wenchen Fan d5202259d9 [SPARK-21127][SQL][FOLLOWUP] fix a config name typo
## What changes were proposed in this pull request?

`spark.sql.statistics.autoUpdate.size` should be `spark.sql.statistics.size.autoUpdate.enabled`. The previous name is confusing as users may treat it as a size config.

This config is in master branch only, no backward compatibility issue.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19667 from cloud-fan/minor.
2017-11-07 09:33:52 -08:00
Wenchen Fan 5014d6e256 [SPARK-22078][SQL] clarify exception behaviors for all data source v2 interfaces
## What changes were proposed in this pull request?

clarify exception behaviors for all data source v2 interfaces.

## How was this patch tested?

document change only

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19623 from cloud-fan/data-source-exception.
2017-11-06 22:25:11 +01:00
Wenchen Fan 472db58cb1 [SPARK-22445][SQL] move CodegenContext.copyResult to CodegenSupport
## What changes were proposed in this pull request?

`CodegenContext.copyResult` is kind of a global status for whole stage codegen. But the tricky part is, it is only used to transfer an information from child to parent when calling the `consume` chain. We have to be super careful in `produce`/`consume`, to set it to true when producing multiple result rows, and set it to false in operators that start new pipeline(like sort).

This PR moves the `copyResult` to `CodegenSupport`, and call it at `WholeStageCodegenExec`. This is much easier to reason about.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19656 from cloud-fan/whole-sage.
2017-11-06 15:10:44 +01:00
Huaxin Gao 572284c5b0 [SPARK-22443][SQL] add implementation of quoteIdentifier, getTableExistsQuery and getSchemaQuery in AggregatedDialect
…

## What changes were proposed in this pull request?

override JDBCDialects methods quoteIdentifier, getTableExistsQuery and getSchemaQuery in AggregatedDialect

## How was this patch tested?

Test the new implementation in JDBCSuite test("Aggregated dialects")

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

Closes #19658 from huaxingao/spark-22443.
2017-11-04 23:07:24 -07:00
Kazuaki Ishizaki 3bba8621cf [SPARK-22378][SQL] Eliminate redundant null check in generated code for extracting an element from complex types
## What changes were proposed in this pull request?

This PR eliminates redundant null check in generated code for extracting an element from complex types `GetArrayItem`, `GetMapValue`, and `GetArrayStructFields`. Since these code generation does not take care of `nullable` in `DataType` such as `ArrayType`, the generated code always has `isNullAt(index)`.
This PR avoids to generate `isNullAt(index)` if `nullable` is false in `DataType`.

Example
```
    val nonNullArray = Literal.create(Seq(1), ArrayType(IntegerType, false))
    checkEvaluation(GetArrayItem(nonNullArray, Literal(0)), 1)
```

Before this PR
```
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     boolean isNull = true;
/* 040 */     int value = -1;
/* 041 */
/* 042 */
/* 043 */
/* 044 */     isNull = false; // resultCode could change nullability.
/* 045 */
/* 046 */     final int index = (int) 0;
/* 047 */     if (index >= ((ArrayData) references[0]).numElements() || index < 0 || ((ArrayData) references[0]).isNullAt(index)) {
/* 048 */       isNull = true;
/* 049 */     } else {
/* 050 */       value = ((ArrayData) references[0]).getInt(index);
/* 051 */     }
/* 052 */     isNull_0 = isNull;
/* 053 */     value_0 = value;
/* 054 */
/* 055 */     // copy all the results into MutableRow
/* 056 */
/* 057 */     if (!isNull_0) {
/* 058 */       mutableRow.setInt(0, value_0);
/* 059 */     } else {
/* 060 */       mutableRow.setNullAt(0);
/* 061 */     }
/* 062 */
/* 063 */     return mutableRow;
/* 064 */   }
```

After this PR (Line 47 is changed)
```
/* 034 */   public java.lang.Object apply(java.lang.Object _i) {
/* 035 */     InternalRow i = (InternalRow) _i;
/* 036 */
/* 037 */
/* 038 */
/* 039 */     boolean isNull = true;
/* 040 */     int value = -1;
/* 041 */
/* 042 */
/* 043 */
/* 044 */     isNull = false; // resultCode could change nullability.
/* 045 */
/* 046 */     final int index = (int) 0;
/* 047 */     if (index >= ((ArrayData) references[0]).numElements() || index < 0) {
/* 048 */       isNull = true;
/* 049 */     } else {
/* 050 */       value = ((ArrayData) references[0]).getInt(index);
/* 051 */     }
/* 052 */     isNull_0 = isNull;
/* 053 */     value_0 = value;
/* 054 */
/* 055 */     // copy all the results into MutableRow
/* 056 */
/* 057 */     if (!isNull_0) {
/* 058 */       mutableRow.setInt(0, value_0);
/* 059 */     } else {
/* 060 */       mutableRow.setNullAt(0);
/* 061 */     }
/* 062 */
/* 063 */     return mutableRow;
/* 064 */   }
```

## How was this patch tested?

Added test cases into `ComplexTypeSuite`

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

Closes #19598 from kiszk/SPARK-22378.
2017-11-04 22:57:12 -07:00
Henry Robinson 6c6626614e [SPARK-22211][SQL] Remove incorrect FOJ limit pushdown
## What changes were proposed in this pull request?

It's not safe in all cases to push down a LIMIT below a FULL OUTER
JOIN. If the limit is pushed to one side of the FOJ, the physical
join operator can not tell if a row in the non-limited side would have a
match in the other side.

*If* the join operator guarantees that unmatched tuples from the limited
side are emitted before any unmatched tuples from the other side,
pushing down the limit is safe. But this is impractical for some join
implementations, e.g. SortMergeJoin.

For now, disable limit pushdown through a FULL OUTER JOIN, and we can
evaluate whether a more complicated solution is necessary in the future.

## How was this patch tested?

Ran org.apache.spark.sql.* tests. Altered full outer join tests in
LimitPushdownSuite.

Author: Henry Robinson <henry@cloudera.com>

Closes #19647 from henryr/spark-22211.
2017-11-04 22:47:25 -07:00
Vinitha Gankidi f7f4e9c2db [SPARK-22412][SQL] Fix incorrect comment in DataSourceScanExec
## What changes were proposed in this pull request?

Next fit decreasing bin packing algorithm is used to combine splits in DataSourceScanExec but the comment incorrectly states that first fit decreasing algorithm is used. The current implementation doesn't go back to a previously used bin other than the bin that the last element was put into.

Author: Vinitha Gankidi <vgankidi@netflix.com>

Closes #19634 from vgankidi/SPARK-22412.
2017-11-04 11:09:47 -07:00
Liang-Chi Hsieh 0c2aee69b0 [SPARK-22410][SQL] Remove unnecessary output from BatchEvalPython's children plans
## What changes were proposed in this pull request?

When we insert `BatchEvalPython` for Python UDFs into a query plan, if its child has some outputs that are not used by the original parent node, `BatchEvalPython` will still take those outputs and save into the queue. When the data for those outputs are big, it is easily to generate big spill on disk.

For example, the following reproducible code is from the JIRA ticket.

```python
from pyspark.sql.functions import *
from pyspark.sql.types import *

lines_of_file = [ "this is a line" for x in xrange(10000) ]
file_obj = [ "this_is_a_foldername/this_is_a_filename", lines_of_file ]
data = [ file_obj for x in xrange(5) ]

small_df = spark.sparkContext.parallelize(data).map(lambda x : (x[0], x[1])).toDF(["file", "lines"])
exploded = small_df.select("file", explode("lines"))

def split_key(s):
    return s.split("/")[1]

split_key_udf = udf(split_key, StringType())

with_filename = exploded.withColumn("filename", split_key_udf("file"))
with_filename.explain(True)
```

The physical plan before/after this change:

Before:

```
*Project [file#0, col#5, pythonUDF0#14 AS filename#9]
+- BatchEvalPython [split_key(file#0)], [file#0, lines#1, col#5, pythonUDF0#14]
   +- Generate explode(lines#1), true, false, [col#5]
      +- Scan ExistingRDD[file#0,lines#1]

```

After:

```
*Project [file#0, col#5, pythonUDF0#14 AS filename#9]
+- BatchEvalPython [split_key(file#0)], [col#5, file#0, pythonUDF0#14]
   +- *Project [col#5, file#0]
      +- Generate explode(lines#1), true, false, [col#5]
         +- Scan ExistingRDD[file#0,lines#1]
```

Before this change, `lines#1` is a redundant input to `BatchEvalPython`. This patch removes it by adding a Project.

## How was this patch tested?

Manually test.

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

Closes #19642 from viirya/SPARK-22410.
2017-11-04 13:11:09 +01:00
xubo245 7a8412352e [SPARK-22423][SQL] Scala test source files like TestHiveSingleton.scala should be in scala source root
## What changes were proposed in this pull request?

  Scala test source files like TestHiveSingleton.scala should be in scala source root

## How was this patch tested?

Just move scala file from java directory to scala directory
No new test case in this PR.

```
	renamed:    mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala -> mllib/src/test/scala/org/apache/spark/ml/util/IdentifiableSuite.scala
	renamed:    streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala -> streaming/src/test/scala/org/apache/spark/streaming/JavaTestUtils.scala
	renamed:    streaming/src/test/java/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala -> streaming/src/test/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala
	renamed:   sql/hive/src/test/java/org/apache/spark/sql/hive/test/TestHiveSingleton.scala  sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
```

Author: xubo245 <601450868@qq.com>

Closes #19639 from xubo245/scalaDirectory.
2017-11-04 11:51:10 +00:00
Marco Gaido 8915886608 [SPARK-22418][SQL][TEST] Add test cases for NULL Handling
## What changes were proposed in this pull request?

Added a test class to check NULL handling behavior.
The expected behavior is defined as the one of the most well-known databases as specified here: https://sqlite.org/nulls.html.

SparkSQL behaves like other DBs:
 - Adding anything to null gives null -> YES
 - Multiplying null by zero gives null -> YES
 - nulls are distinct in SELECT DISTINCT -> NO
 - nulls are distinct in a UNION -> NO
 - "CASE WHEN null THEN 1 ELSE 0 END" is 0? -> YES
 - "null OR true" is true -> YES
 - "not (null AND false)" is true -> YES
 - null in aggregation are skipped -> YES

## How was this patch tested?

Added test class

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19653 from mgaido91/SPARK-22418.
2017-11-03 22:03:58 -07:00
Wenchen Fan 2fd12af437 [SPARK-22306][SQL] alter table schema should not erase the bucketing metadata at hive side
forward-port https://github.com/apache/spark/pull/19622 to master branch.

This bug doesn't exist in master because we've added hive bucketing support and the hive bucketing metadata can be recognized by Spark, but we should still port it to master: 1) there may be other unsupported hive metadata removed by Spark. 2) reduce code difference between master and 2.2 to ease the backport in the feature.

***

When we alter table schema, we set the new schema to spark `CatalogTable`, convert it to hive table, and finally call `hive.alterTable`. This causes a problem in Spark 2.2, because hive bucketing metedata is not recognized by Spark, which means a Spark `CatalogTable` representing a hive table is always non-bucketed, and when we convert it to hive table and call `hive.alterTable`, the original hive bucketing metadata will be removed.

To fix this bug, we should read out the raw hive table metadata, update its schema, and call `hive.alterTable`. By doing this we can guarantee only the schema is changed, and nothing else.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19644 from cloud-fan/infer.
2017-11-02 23:41:16 +01:00
Dongjoon Hyun e3f67a97f1 [SPARK-22416][SQL] Move OrcOptions from sql/hive to sql/core
## What changes were proposed in this pull request?

According to the [discussion](https://github.com/apache/spark/pull/19571#issuecomment-339472976) on SPARK-15474, we will add new OrcFileFormat in `sql/core` module and allow users to use both old and new OrcFileFormat.

To do that, `OrcOptions` should be visible in `sql/core` module, too. Previously, it was `private[orc]` in `sql/hive`. This PR removes `private[orc]` because we don't use `private[sql]` in `sql/execution` package after [SPARK-16964](https://github.com/apache/spark/pull/14554).

## How was this patch tested?

Pass the Jenkins with the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19636 from dongjoon-hyun/SPARK-22416.
2017-11-02 18:28:56 +01:00
Patrick Woody 277b1924b4 [SPARK-22408][SQL] RelationalGroupedDataset's distinct pivot value calculation launches unnecessary stages
## What changes were proposed in this pull request?

Adding a global limit on top of the distinct values before sorting and collecting will reduce the overall work in the case where we have more distinct values. We will also eagerly perform a collect rather than a take because we know we only have at most (maxValues + 1) rows.

## How was this patch tested?

Existing tests cover sorted order

Author: Patrick Woody <pwoody@palantir.com>

Closes #19629 from pwoody/SPARK-22408.
2017-11-02 14:19:21 +01:00
Reynold Xin d43e1f06bd [MINOR] Data source v2 docs update.
## What changes were proposed in this pull request?
This patch includes some doc updates for data source API v2. I was reading the code and noticed some minor issues.

## How was this patch tested?
This is a doc only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #19626 from rxin/dsv2-update.
2017-11-01 18:39:15 +01:00
Jose Torres 73231860ba [SPARK-22305] Write HDFSBackedStateStoreProvider.loadMap non-recursively
## What changes were proposed in this pull request?
Write HDFSBackedStateStoreProvider.loadMap non-recursively. This prevents stack overflow if too many deltas stack up in a low memory environment.

## How was this patch tested?

existing unit tests for functional equivalence, new unit test to check for stack overflow

Author: Jose Torres <jose@databricks.com>

Closes #19611 from joseph-torres/SPARK-22305.
2017-10-31 11:53:50 -07:00
Wenchen Fan 4d9ebf3835 [SPARK-19611][SQL][FOLLOWUP] set dataSchema correctly in HiveMetastoreCatalog.convertToLogicalRelation
## What changes were proposed in this pull request?

We made a mistake in https://github.com/apache/spark/pull/16944 . In `HiveMetastoreCatalog#inferIfNeeded` we infer the data schema, merge with full schema, and return the new full schema. At caller side we treat the full schema as data schema and set it to `HadoopFsRelation`.

This doesn't cause any problem because both parquet and orc can work with a wrong data schema that has extra columns, but it's better to fix this mistake.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19615 from cloud-fan/infer.
2017-10-31 11:35:32 +01:00
Zhenhua Wang 59589bc654 [SPARK-22310][SQL] Refactor join estimation to incorporate estimation logic for different kinds of statistics
## What changes were proposed in this pull request?

The current join estimation logic is only based on basic column statistics (such as ndv, etc). If we want to add estimation for other kinds of statistics (such as histograms), it's not easy to incorporate into the current algorithm:
1. When we have multiple pairs of join keys, the current algorithm computes cardinality in a single formula. But if different join keys have different kinds of stats, the computation logic for each pair of join keys become different, so the previous formula does not apply.
2. Currently it computes cardinality and updates join keys' column stats separately. It's better to do these two steps together, since both computation and update logic are different for different kinds of stats.

## How was this patch tested?

Only refactor, covered by existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19531 from wzhfy/join_est_refactor.
2017-10-31 11:13:48 +01:00
Zhenhua Wang 44c4003155 [SPARK-22400][SQL] rename some APIs and classes to make their meaning clearer
## What changes were proposed in this pull request?

Both `ReadSupport` and `ReadTask` have a method called `createReader`, but they create different things. This could cause some confusion for data source developers. The same issue exists between `WriteSupport` and `DataWriterFactory`, both of which have a method called `createWriter`. This PR renames the method of `ReadTask`/`DataWriterFactory` to `createDataReader`/`createDataWriter`.

Besides, the name of `RowToInternalRowDataWriterFactory` is not correct, because it actually converts `InternalRow`s to `Row`s. It should be renamed `InternalRowDataWriterFactory`.

## How was this patch tested?

Only renaming, should be covered by existing tests.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19610 from wzhfy/rename.
2017-10-30 10:21:05 -07:00
gatorsmile 65338de5fb [SPARK-22396][SQL] Better Error Message for InsertIntoDir using Hive format without enabling Hive Support
## What changes were proposed in this pull request?
When Hive support is not on, users can hit unresolved plan node when trying to call `INSERT OVERWRITE DIRECTORY` using Hive format.
```
"unresolved operator 'InsertIntoDir true, Storage(Location: /private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/spark-b4227606-9311-46a8-8c02-56355bf0e2bc, Serde Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde, InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat), hive, true;;
```

This PR is to issue a better error message.
## How was this patch tested?
Added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19608 from gatorsmile/hivesupportInsertOverwrite.
2017-10-30 10:19:34 -07:00
Wenchen Fan 079a2609d7 [SPARK-17788][SPARK-21033][SQL] fix the potential OOM in UnsafeExternalSorter and ShuffleExternalSorter
## What changes were proposed in this pull request?

In `UnsafeInMemorySorter`, one record may take 32 bytes: 1 `long` for pointer, 1 `long` for key-prefix, and another 2 `long`s as the temporary buffer for radix sort.

In `UnsafeExternalSorter`, we set the `DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD` to be `1024 * 1024 * 1024 / 2`, and hoping the max size of point array to be 8 GB. However this is wrong, `1024 * 1024 * 1024 / 2 * 32` is actually 16 GB, and if we grow the point array before reach this limitation, we may hit the max-page-size error.

Users may see exception like this on large dataset:
```
Caused by: java.lang.IllegalArgumentException: Cannot allocate a page with more than 17179869176 bytes
at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:241)
at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:121)
at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:374)
at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:396)
at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:94)
...
```

Setting `DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD` to a smaller number is not enough, users can still set the config to a big number and trigger the too large page size issue. This PR fixes it by explicitly handling the too large page size exception in the sorter and spill.

This PR also change the type of `spark.shuffle.spill.numElementsForceSpillThreshold` to int, because it's only compared with `numRecords`, which is an int. This is an internal conf so we don't have a serious compatibility issue.

## How was this patch tested?

TODO

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18251 from cloud-fan/sort.
2017-10-30 17:53:06 +01:00
Wenchen Fan 9f02d7dc53 [SPARK-22385][SQL] MapObjects should not access list element by index
## What changes were proposed in this pull request?

This issue was discovered and investigated by Ohad Raviv and Sean Owen in https://issues.apache.org/jira/browse/SPARK-21657. The input data of `MapObjects` may be a `List` which has O(n) complexity for accessing by index. When converting input data to catalyst array, `MapObjects` gets element by index in each loop, and results to bad performance.

This PR fixes this issue by accessing elements via Iterator.

## How was this patch tested?

using the test script in https://issues.apache.org/jira/browse/SPARK-21657
```
val BASE = 100000000
val N = 100000
val df = sc.parallelize(List(("1234567890", (BASE to (BASE+N)).map(x => (x.toString, (x+1).toString, (x+2).toString, (x+3).toString)).toList ))).toDF("c1", "c_arr")
spark.time(df.queryExecution.toRdd.foreach(_ => ()))
```

We can see 50x speed up.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19603 from cloud-fan/map-objects.
2017-10-30 11:00:44 +01:00
Henry Robinson 9f5c77ae32 [SPARK-21983][SQL] Fix Antlr 4.7 deprecation warnings
## What changes were proposed in this pull request?

Fix three deprecation warnings introduced by move to ANTLR 4.7:

* Use ParserRuleContext.addChild(TerminalNode) in preference to
  deprecated ParserRuleContext.addChild(Token) interface.
* TokenStream.reset() is deprecated in favour of seek(0)
* Replace use of deprecated ANTLRInputStream with stream returned by
  CharStreams.fromString()

The last item changed the way we construct ANTLR's input stream (from
direct instantiation to factory construction), so necessitated a change
to how we override the LA() method to always return an upper-case
char. The ANTLR object is now wrapped, rather than inherited-from.

* Also fix incorrect usage of CharStream.getText() which expects the rhs
  of the supplied interval to be the last char to be returned, i.e. the
  interval is inclusive, and work around bug in ANTLR 4.7 where empty
  streams or intervals may cause getText() to throw an error.

## How was this patch tested?

Ran all the sql tests. Confirmed that LA() override has coverage by
breaking it, and noting that tests failed.

Author: Henry Robinson <henry@apache.org>

Closes #19578 from henryr/spark-21983.
2017-10-30 07:45:54 +00:00
gatorsmile 659acf18da Revert "[SPARK-22308] Support alternative unit testing styles in external applications"
This reverts commit 592cfeab9c.
2017-10-29 10:37:25 -07:00
Jen-Ming Chung bc7ca9786e [SPARK-22291][SQL] Conversion error when transforming array types of uuid, inet and cidr to StingType in PostgreSQL
## What changes were proposed in this pull request?

This PR fixes the conversion error when reads data from a PostgreSQL table that contains columns of `uuid[]`, `inet[]` and `cidr[]` data types.

For example, create a table with the uuid[] data type, and insert the test data.
```SQL
CREATE TABLE users
(
    id smallint NOT NULL,
    name character varying(50),
    user_ids uuid[],
    PRIMARY KEY (id)
)

INSERT INTO users ("id", "name","user_ids")
VALUES (1, 'foo', ARRAY
    ['7be8aaf8-650e-4dbb-8186-0a749840ecf2'
    ,'205f9bfc-018c-4452-a605-609c0cfad228']::UUID[]
)
```
Then it will throw the following exceptions when trying to load the data.
```
java.lang.ClassCastException: [Ljava.util.UUID; cannot be cast to [Ljava.lang.String;
    at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$14.apply(JdbcUtils.scala:459)
    at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$14.apply(JdbcUtils.scala:458)
...
```

## How was this patch tested?

Added test in `PostgresIntegrationSuite`.

Author: Jen-Ming Chung <jenmingisme@gmail.com>

Closes #19567 from jmchung/SPARK-22291.
2017-10-29 18:11:48 +01:00
Wenchen Fan 7fdacbc77b [SPARK-19727][SQL][FOLLOWUP] Fix for round function that modifies original column
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/17075 , to fix the bug in codegen path.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19576 from cloud-fan/bug.
2017-10-28 18:24:18 -07:00
Juliusz Sompolski e80da8129a [MINOR] Remove false comment from planStreamingAggregation
## What changes were proposed in this pull request?

AggUtils.planStreamingAggregation has some comments about DISTINCT aggregates,
while streaming aggregation does not support DISTINCT.
This seems to have been wrongly copy-pasted over.

## How was this patch tested?

Only a comment change.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #18937 from juliuszsompolski/streaming-agg-doc.
2017-10-28 17:20:35 -07:00
Takuya UESHIN 4c5269f1aa [SPARK-22370][SQL][PYSPARK] Config values should be captured in Driver.
## What changes were proposed in this pull request?

`ArrowEvalPythonExec` and `FlatMapGroupsInPandasExec` are refering config values of `SQLConf` in function for `mapPartitions`/`mapPartitionsInternal`, but we should capture them in Driver.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19587 from ueshin/issues/SPARK-22370.
2017-10-28 18:33:09 +01:00
Liang-Chi Hsieh 683ffe0620 [SPARK-22335][SQL] Clarify union behavior on Dataset of typed objects in the document
## What changes were proposed in this pull request?

Seems that end users can be confused by the union's behavior on Dataset of typed objects. We can clarity it more in the document of `union` function.

## How was this patch tested?

Only document change.

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

Closes #19570 from viirya/SPARK-22335.
2017-10-28 21:47:15 +09:00
Reynold Xin d28d5732ae [SPARK-21619][SQL] Fail the execution of canonicalized plans explicitly
## What changes were proposed in this pull request?
Canonicalized plans are not supposed to be executed. I ran into a case in which there's some code that accidentally calls execute on a canonicalized plan. This patch throws a more explicit exception when that happens.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18828 from rxin/SPARK-21619.
2017-10-27 23:44:24 -07:00
donnyzone c42d208e19 [SPARK-22333][SQL] timeFunctionCall(CURRENT_DATE, CURRENT_TIMESTAMP) has conflicts with columnReference
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-22333

In current version, users can use CURRENT_DATE() and CURRENT_TIMESTAMP() without specifying braces.
However, when a table has columns named as "current_date" or "current_timestamp", it will still be parsed as function call.

There are many such cases in our production cluster. We get the wrong answer due to this inappropriate behevior. In general, ColumnReference should get higher priority than timeFunctionCall.

## How was this patch tested?
unit test
manul test

Author: donnyzone <wellfengzhu@gmail.com>

Closes #19559 from DonnyZone/master.
2017-10-27 23:40:59 -07:00
Sathiya 01f6ba0e7a [SPARK-22181][SQL] Adds ReplaceExceptWithFilter rule
## What changes were proposed in this pull request?

Adds a new optimisation rule 'ReplaceExceptWithNotFilter' that replaces Except logical with Filter operator and schedule it before applying 'ReplaceExceptWithAntiJoin' rule. This way we can avoid expensive join operation if one or both of the datasets of the Except operation are fully derived out of Filters from a same parent.

## How was this patch tested?

The patch is tested locally using spark-shell + unit test.

Author: Sathiya <sathiya.kumar@polytechnique.edu>

Closes #19451 from sathiyapk/SPARK-22181-optimize-exceptWithFilter.
2017-10-27 18:57:08 -07:00
Marco Gaido b3d8fc3dc4 [SPARK-22226][SQL] splitExpression can create too many method calls in the outer class
## What changes were proposed in this pull request?

SPARK-18016 introduced `NestedClass` to avoid that the many methods generated by `splitExpressions` contribute to the outer class' constant pool, making it growing too much. Unfortunately, despite their definition is stored in the `NestedClass`, they all are invoked in the outer class and for each method invocation, there are two entries added to the constant pool: a `Methodref` and a `Utf8` entry (you can easily check this compiling a simple sample class with `janinoc` and looking at its Constant Pool). This limits the scalability of the solution with very large methods which are split in a lot of small ones. This means that currently we are generating classes like this one:

```
class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
  public UnsafeRow apply(InternalRow i) {
     rowWriter.zeroOutNullBytes();
     apply_0(i);
     apply_1(i);
...
    nestedClassInstance.apply_862(i);
    nestedClassInstance.apply_863(i);
...
    nestedClassInstance1.apply_1612(i);
    nestedClassInstance1.apply_1613(i);
...
  }
...
  private class NestedClass {
    private void apply_862(InternalRow i) { ... }
    private void apply_863(InternalRow i) { ... }
...
  }
  private class NestedClass1 {
    private void apply_1612(InternalRow i) { ... }
    private void apply_1613(InternalRow i) { ... }
...
  }
}
```

This PR reduce the Constant Pool size of the outer class by adding a new method to each nested class: in this method we invoke all the small methods generated by `splitExpression` in that nested class. In this way, in the outer class there is only one method invocation per nested class, reducing by orders of magnitude the entries in its constant pool because of method invocations. This means that after the patch the generated code becomes:

```
class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
  public UnsafeRow apply(InternalRow i) {
     rowWriter.zeroOutNullBytes();
     apply_0(i);
     apply_1(i);
     ...
     nestedClassInstance.apply(i);
     nestedClassInstance1.apply(i);
     ...
  }
...
  private class NestedClass {
    private void apply_862(InternalRow i) { ... }
    private void apply_863(InternalRow i) { ... }
...
    private void apply(InternalRow i) {
      apply_862(i);
      apply_863(i);
      ...
    }
  }
  private class NestedClass1 {
    private void apply_1612(InternalRow i) { ... }
    private void apply_1613(InternalRow i) { ... }
...
    private void apply(InternalRow i) {
      apply_1612(i);
      apply_1613(i);
      ...
    }
  }
}
```

## How was this patch tested?

Added UT and existing UTs

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

Closes #19480 from mgaido91/SPARK-22226.
2017-10-27 13:43:09 -07:00
gatorsmile 36b826f5d1 [TRIVIAL][SQL] Code cleaning in ResolveReferences
## What changes were proposed in this pull request?
This PR is to clean the related codes majorly based on the today's code review on  https://github.com/apache/spark/pull/19559

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19585 from gatorsmile/trivialFixes.
2017-10-27 07:52:10 -07:00
Bryan Cutler 17af727e38 [SPARK-21375][PYSPARK][SQL] Add Date and Timestamp support to ArrowConverters for toPandas() Conversion
## What changes were proposed in this pull request?

Adding date and timestamp support with Arrow for `toPandas()` and `pandas_udf`s.  Timestamps are stored in Arrow as UTC and manifested to the user as timezone-naive localized to the Python system timezone.

## How was this patch tested?

Added Scala tests for date and timestamp types under ArrowConverters, ArrowUtils, and ArrowWriter suites.  Added Python tests for `toPandas()` and `pandas_udf`s with date and timestamp types.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18664 from BryanCutler/arrow-date-timestamp-SPARK-21375.
2017-10-26 23:02:46 -07:00
Wenchen Fan 5c3a1f3fad [SPARK-22355][SQL] Dataset.collect is not threadsafe
## What changes were proposed in this pull request?

It's possible that users create a `Dataset`, and call `collect` of this `Dataset` in many threads at the same time. Currently `Dataset#collect` just call `encoder.fromRow` to convert spark rows to objects of type T, and this encoder is per-dataset. This means `Dataset#collect` is not thread-safe, because the encoder uses a projection to output the object to a re-usable row.

This PR fixes this problem, by creating a new projection when calling `Dataset#collect`, so that we have the re-usable row for each method call, instead of each Dataset.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19577 from cloud-fan/encoder.
2017-10-26 17:51:16 -07:00
Wenchen Fan 9b262f6a08 [SPARK-22356][SQL] data source table should support overlapped columns between data and partition schema
## What changes were proposed in this pull request?

This is a regression introduced by #14207. After Spark 2.1, we store the inferred schema when creating the table, to avoid inferring schema again at read path. However, there is one special case: overlapped columns between data and partition. For this case, it breaks the assumption of table schema that there is on ovelap between data and partition schema, and partition columns should be at the end. The result is, for Spark 2.1, the table scan has incorrect schema that puts partition columns at the end. For Spark 2.2, we add a check in CatalogTable to validate table schema, which fails at this case.

To fix this issue, a simple and safe approach is to fallback to old behavior when overlapeed columns detected, i.e. store empty schema in metastore.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19579 from cloud-fan/bug2.
2017-10-26 17:39:53 -07:00
Jose Torres 8e9863531b [SPARK-22366] Support ignoring missing files
## What changes were proposed in this pull request?

Add a flag "spark.sql.files.ignoreMissingFiles" to parallel the existing flag "spark.sql.files.ignoreCorruptFiles".

## How was this patch tested?

new unit test

Author: Jose Torres <jose@databricks.com>

Closes #19581 from joseph-torres/SPARK-22366.
2017-10-26 16:55:30 -07:00
Nathan Kronenfeld 592cfeab9c [SPARK-22308] Support alternative unit testing styles in external applications
## What changes were proposed in this pull request?
Support unit tests of external code (i.e., applications that use spark) using scalatest that don't want to use FunSuite.  SharedSparkContext already supports this, but SharedSQLContext does not.

I've introduced SharedSparkSession as a parent to SharedSQLContext, written in a way that it does support all scalatest styles.

## How was this patch tested?
There are three new unit test suites added that just test using FunSpec, FlatSpec, and WordSpec.

Author: Nathan Kronenfeld <nicole.oresme@gmail.com>

Closes #19529 from nkronenfeld/alternative-style-tests-2.
2017-10-26 00:29:49 -07:00
Liang-Chi Hsieh 1051ebec70 [SPARK-20783][SQL][FOLLOW-UP] Create ColumnVector to abstract existing compressed column
## What changes were proposed in this pull request?

Removed one unused method.

## How was this patch tested?

Existing tests.

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

Closes #19508 from viirya/SPARK-20783-followup.
2017-10-25 16:31:58 +01:00
Sean Owen 6c6950839d [SPARK-22322][CORE] Update FutureAction for compatibility with Scala 2.12 Future
## What changes were proposed in this pull request?

Scala 2.12's `Future` defines two new methods to implement, `transform` and `transformWith`. These can be implemented naturally in Spark's `FutureAction` extension and subclasses, but, only in terms of the new methods that don't exist in Scala 2.11. To support both at the same time, reflection is used to implement these.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #19561 from srowen/SPARK-22322.
2017-10-25 12:51:20 +01:00
Ruben Berenguel Montoro 427359f077 [SPARK-13947][SQL] The error message from using an invalid column reference is not clear
## What changes were proposed in this pull request?

 Rewritten error message for clarity. Added extra information in case of attribute name collision, hinting the user to double-check referencing two different tables

## How was this patch tested?

No functional changes, only final message has changed. It has been tested manually against the situation proposed in the JIRA ticket. Automated tests in repository pass.

This PR is original work from me and I license this work to the Spark project

Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>
Author: Ruben Berenguel Montoro <ruben@dreamattic.com>
Author: Ruben Berenguel <ruben@mostlymaths.net>

Closes #17100 from rberenguel/SPARK-13947-error-message.
2017-10-24 23:02:11 -07:00
Yuming Wang 524abb996a [SPARK-21101][SQL] Catch IllegalStateException when CREATE TEMPORARY FUNCTION
## What changes were proposed in this pull request?

It must `override` [`public StructObjectInspector initialize(ObjectInspector[] argOIs)`](https://github.com/apache/hive/blob/release-2.0.0/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTF.java#L70) when create a UDTF.

If you `override` [`public StructObjectInspector initialize(StructObjectInspector argOIs)`](https://github.com/apache/hive/blob/release-2.0.0/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTF.java#L49), `IllegalStateException` will throw. per: [HIVE-12377](https://issues.apache.org/jira/browse/HIVE-12377).

This PR catch `IllegalStateException` and point user to `override` `public StructObjectInspector initialize(ObjectInspector[] argOIs)`.

## How was this patch tested?

unit tests

Source code and binary jar: [SPARK-21101.zip](https://github.com/apache/spark/files/1123763/SPARK-21101.zip)
These two source code copy from : https://github.com/apache/hive/blob/release-2.0.0/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFStack.java

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18527 from wangyum/SPARK-21101.
2017-10-24 22:59:46 -07:00
Liang-Chi Hsieh bc1e76632d [SPARK-22348][SQL] The table cache providing ColumnarBatch should also do partition batch pruning
## What changes were proposed in this pull request?

We enable table cache `InMemoryTableScanExec` to provide `ColumnarBatch` now. But the cached batches are retrieved without pruning. In this case, we still need to do partition batch pruning.

## How was this patch tested?

Existing tests.

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

Closes #19569 from viirya/SPARK-22348.
2017-10-25 06:33:44 +01:00
Marco Gaido 3f5ba968c5 [SPARK-22301][SQL] Add rule to Optimizer for In with not-nullable value and empty list
## What changes were proposed in this pull request?

For performance reason, we should resolve in operation on an empty list as false in the optimizations phase, ad discussed in #19522.

## How was this patch tested?
Added UT

cc gatorsmile

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

Closes #19523 from mgaido91/SPARK-22301.
2017-10-24 09:11:52 -07:00
Sean Owen 8beeaed66b [SPARK-21936][SQL][FOLLOW-UP] backward compatibility test framework for HiveExternalCatalog
## What changes were proposed in this pull request?

Adjust Spark download in test to use Apache mirrors and respect its load balancer, and use Spark 2.1.2. This follows on a recent PMC list thread about removing the cloudfront download rather than update it further.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #19564 from srowen/SPARK-21936.2.
2017-10-24 13:56:10 +01:00
Kazuaki Ishizaki c30d5cfc71 [SPARK-20822][SQL] Generate code to directly get value from ColumnVector for table cache
## What changes were proposed in this pull request?

This PR generates the Java code to directly get a value for a column in `ColumnVector` without using an iterator (e.g. at lines 54-69 in the generated code example) for table cache (e.g. `dataframe.cache`). This PR improves runtime performance by eliminating data copy from column-oriented storage to `InternalRow` in a `SpecificColumnarIterator` iterator for primitive type. Another PR will support primitive type array.

Benchmark result: **1.2x**
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Int Sum with IntDelta cache:             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
InternalRow codegen                            731 /  812         43.0          23.2       1.0X
ColumnVector codegen                           616 /  772         51.0          19.6       1.2X
```
Benchmark program
```
  intSumBenchmark(sqlContext, 1024 * 1024 * 30)
  def intSumBenchmark(sqlContext: SQLContext, values: Int): Unit = {
    import sqlContext.implicits._
    val benchmarkPT = new Benchmark("Int Sum with IntDelta cache", values, 20)
    Seq(("InternalRow", "false"), ("ColumnVector", "true")).foreach {
      case (str, value) =>
        withSQLConf(sqlContext, SQLConf. COLUMN_VECTOR_CODEGEN.key -> value) { // tentatively added for benchmarking
          val dfPassThrough = sqlContext.sparkContext.parallelize(0 to values - 1, 1).toDF().cache()
          dfPassThrough.count()       // force to create df.cache()
          benchmarkPT.addCase(s"$str codegen") { iter =>
            dfPassThrough.agg(sum("value")).collect
          }
          dfPassThrough.unpersist(true)
        }
    }
    benchmarkPT.run()
  }
```

Motivating example
```
val dsInt = spark.range(3).cache
dsInt.count // force to build cache
dsInt.filter(_ > 0).collect
```
Generated code
```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inmemorytablescan_input;
/* 009 */   private org.apache.spark.sql.execution.metric.SQLMetric inmemorytablescan_numOutputRows;
/* 010 */   private org.apache.spark.sql.execution.metric.SQLMetric inmemorytablescan_scanTime;
/* 011 */   private long inmemorytablescan_scanTime1;
/* 012 */   private org.apache.spark.sql.execution.vectorized.ColumnarBatch inmemorytablescan_batch;
/* 013 */   private int inmemorytablescan_batchIdx;
/* 014 */   private org.apache.spark.sql.execution.vectorized.OnHeapColumnVector inmemorytablescan_colInstance0;
/* 015 */   private UnsafeRow inmemorytablescan_result;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder inmemorytablescan_holder;
/* 017 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter inmemorytablescan_rowWriter;
/* 018 */   private org.apache.spark.sql.execution.metric.SQLMetric filter_numOutputRows;
/* 019 */   private UnsafeRow filter_result;
/* 020 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder filter_holder;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter filter_rowWriter;
/* 022 */
/* 023 */   public GeneratedIterator(Object[] references) {
/* 024 */     this.references = references;
/* 025 */   }
/* 026 */
/* 027 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 028 */     partitionIndex = index;
/* 029 */     this.inputs = inputs;
/* 030 */     inmemorytablescan_input = inputs[0];
/* 031 */     inmemorytablescan_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0];
/* 032 */     inmemorytablescan_scanTime = (org.apache.spark.sql.execution.metric.SQLMetric) references[1];
/* 033 */     inmemorytablescan_scanTime1 = 0;
/* 034 */     inmemorytablescan_batch = null;
/* 035 */     inmemorytablescan_batchIdx = 0;
/* 036 */     inmemorytablescan_colInstance0 = null;
/* 037 */     inmemorytablescan_result = new UnsafeRow(1);
/* 038 */     inmemorytablescan_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(inmemorytablescan_result, 0);
/* 039 */     inmemorytablescan_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(inmemorytablescan_holder, 1);
/* 040 */     filter_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[2];
/* 041 */     filter_result = new UnsafeRow(1);
/* 042 */     filter_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(filter_result, 0);
/* 043 */     filter_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(filter_holder, 1);
/* 044 */
/* 045 */   }
/* 046 */
/* 047 */   protected void processNext() throws java.io.IOException {
/* 048 */     if (inmemorytablescan_batch == null) {
/* 049 */       inmemorytablescan_nextBatch();
/* 050 */     }
/* 051 */     while (inmemorytablescan_batch != null) {
/* 052 */       int inmemorytablescan_numRows = inmemorytablescan_batch.numRows();
/* 053 */       int inmemorytablescan_localEnd = inmemorytablescan_numRows - inmemorytablescan_batchIdx;
/* 054 */       for (int inmemorytablescan_localIdx = 0; inmemorytablescan_localIdx < inmemorytablescan_localEnd; inmemorytablescan_localIdx++) {
/* 055 */         int inmemorytablescan_rowIdx = inmemorytablescan_batchIdx + inmemorytablescan_localIdx;
/* 056 */         int inmemorytablescan_value = inmemorytablescan_colInstance0.getInt(inmemorytablescan_rowIdx);
/* 057 */
/* 058 */         boolean filter_isNull = false;
/* 059 */
/* 060 */         boolean filter_value = false;
/* 061 */         filter_value = inmemorytablescan_value > 1;
/* 062 */         if (!filter_value) continue;
/* 063 */
/* 064 */         filter_numOutputRows.add(1);
/* 065 */
/* 066 */         filter_rowWriter.write(0, inmemorytablescan_value);
/* 067 */         append(filter_result);
/* 068 */         if (shouldStop()) { inmemorytablescan_batchIdx = inmemorytablescan_rowIdx + 1; return; }
/* 069 */       }
/* 070 */       inmemorytablescan_batchIdx = inmemorytablescan_numRows;
/* 071 */       inmemorytablescan_batch = null;
/* 072 */       inmemorytablescan_nextBatch();
/* 073 */     }
/* 074 */     inmemorytablescan_scanTime.add(inmemorytablescan_scanTime1 / (1000 * 1000));
/* 075 */     inmemorytablescan_scanTime1 = 0;
/* 076 */   }
/* 077 */
/* 078 */   private void inmemorytablescan_nextBatch() throws java.io.IOException {
/* 079 */     long getBatchStart = System.nanoTime();
/* 080 */     if (inmemorytablescan_input.hasNext()) {
/* 081 */       org.apache.spark.sql.execution.columnar.CachedBatch inmemorytablescan_cachedBatch = (org.apache.spark.sql.execution.columnar.CachedBatch)inmemorytablescan_input.next();
/* 082 */       inmemorytablescan_batch = org.apache.spark.sql.execution.columnar.InMemoryRelation$.MODULE$.createColumn(inmemorytablescan_cachedBatch);
/* 083 */
/* 084 */       inmemorytablescan_numOutputRows.add(inmemorytablescan_batch.numRows());
/* 085 */       inmemorytablescan_batchIdx = 0;
/* 086 */       inmemorytablescan_colInstance0 = (org.apache.spark.sql.execution.vectorized.OnHeapColumnVector) inmemorytablescan_batch.column(0); org.apache.spark.sql.execution.columnar.ColumnAccessor$.MODULE$.decompress(inmemorytablescan_cachedBatch.buffers()[0], (org.apache.spark.sql.execution.vectorized.WritableColumnVector) inmemorytablescan_colInstance0, org.apache.spark.sql.types.DataTypes.IntegerType, inmemorytablescan_cachedBatch.numRows());
/* 087 */
/* 088 */     }
/* 089 */     inmemorytablescan_scanTime1 += System.nanoTime() - getBatchStart;
/* 090 */   }
/* 091 */ }
```

## How was this patch tested?

Add test cases into `DataFrameTungstenSuite` and `WholeStageCodegenSuite`

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

Closes #18747 from kiszk/SPARK-20822a.
2017-10-24 08:46:22 +01:00
Dongjoon Hyun 884d4f95f7 [SPARK-21912][SQL][FOLLOW-UP] ORC/Parquet table should not create invalid column names
## What changes were proposed in this pull request?

During [SPARK-21912](https://issues.apache.org/jira/browse/SPARK-21912), we skipped testing 'ADD COLUMNS' on ORC tables due to ORC limitation. Since [SPARK-21929](https://issues.apache.org/jira/browse/SPARK-21929) is resolved now, we can test both `ORC` and `PARQUET` completely.

## How was this patch tested?

Pass the updated test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19562 from dongjoon-hyun/SPARK-21912-2.
2017-10-23 17:21:49 -07:00
Zhenhua Wang f6290aea24 [SPARK-22285][SQL] Change implementation of ApproxCountDistinctForIntervals to TypedImperativeAggregate
## What changes were proposed in this pull request?

The current implementation of `ApproxCountDistinctForIntervals` is `ImperativeAggregate`. The number of `aggBufferAttributes` is the number of total words in the hllppHelper array. Each hllppHelper has 52 words by default relativeSD.

Since this aggregate function is used in equi-height histogram generation, and the number of buckets in histogram is usually hundreds, the number of `aggBufferAttributes` can easily reach tens of thousands or even more.

This leads to a huge method in codegen and causes error:
```
org.codehaus.janino.JaninoRuntimeException: Code of method "apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Lorg/apache/spark/sql/catalyst/expressions/UnsafeRow;" of class "org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection" grows beyond 64 KB.
```
Besides, huge generated methods also result in performance regression.

In this PR, we change its implementation to `TypedImperativeAggregate`. After the fix, `ApproxCountDistinctForIntervals` can deal with more than thousands endpoints without throwing codegen error, and improve performance from `20 sec` to `2 sec` in a test case of 500 endpoints.

## How was this patch tested?

Test by an added test case and existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19506 from wzhfy/change_forIntervals_typedAgg.
2017-10-23 23:02:36 +01:00
Kohki Nishio 5a5b6b7851 [SPARK-22303][SQL] Handle Oracle specific jdbc types in OracleDialect
TIMESTAMP (-101), BINARY_DOUBLE (101) and BINARY_FLOAT (100) are handled in OracleDialect

## What changes were proposed in this pull request?

When a oracle table contains columns whose type is BINARY_FLOAT or BINARY_DOUBLE, spark sql fails to load a table with SQLException

```
java.sql.SQLException: Unsupported type 101
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.org$apache$spark$sql$execution$datasources$jdbc$JdbcUtils$$getCatalystType(JdbcUtils.scala:235)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$8.apply(JdbcUtils.scala:292)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$8.apply(JdbcUtils.scala:292)
 at scala.Option.getOrElse(Option.scala:121)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.getSchema(JdbcUtils.scala:291)
 at org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD$.resolveTable(JDBCRDD.scala:64)
 at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation.<init>(JDBCRelation.scala:113)
 at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:47)
 at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:306)
 at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
 at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:146)
```

## How was this patch tested?

I updated a UT which covers type conversion test for types (-101, 100, 101), on top of that I tested this change against actual table with those columns and it was able to read and write to the table.

Author: Kohki Nishio <taroplus@me.com>

Closes #19548 from taroplus/oracle_sql_types_101.
2017-10-23 09:55:46 -07:00
Dongjoon Hyun ca2a780e7c [SPARK-21929][SQL] Support ALTER TABLE table_name ADD COLUMNS(..) for ORC data source
## What changes were proposed in this pull request?

When [SPARK-19261](https://issues.apache.org/jira/browse/SPARK-19261) implements `ALTER TABLE ADD COLUMNS`, ORC data source is omitted due to SPARK-14387, SPARK-16628, and SPARK-18355. Now, those issues are fixed and Spark 2.3 is [using Spark schema to read ORC table instead of ORC file schema](e6e36004af). This PR enables `ALTER TABLE ADD COLUMNS` for ORC data source.

## How was this patch tested?

Pass the updated and added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19545 from dongjoon-hyun/SPARK-21929.
2017-10-21 18:01:45 -07:00
gatorsmile a763607e4f [SPARK-21055][SQL][FOLLOW-UP] replace grouping__id with grouping_id()
## What changes were proposed in this pull request?
Simplifies the test cases that were added in the PR https://github.com/apache/spark/pull/18270.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19546 from gatorsmile/backportSPARK-21055.
2017-10-21 10:07:31 -07:00
gatorsmile d8cada8d1d [SPARK-20331][SQL][FOLLOW-UP] Add a SQLConf for enhanced Hive partition pruning predicate pushdown
## What changes were proposed in this pull request?
This is a follow-up PR of https://github.com/apache/spark/pull/17633.

This PR is to add a conf `spark.sql.hive.advancedPartitionPredicatePushdown.enabled`, which can be used to turn the enhancement off.

## How was this patch tested?
Add a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19547 from gatorsmile/Spark20331FollowUp.
2017-10-21 10:05:45 -07:00
Zhenhua Wang d9f286d261 [SPARK-22326][SQL] Remove unnecessary hashCode and equals methods
## What changes were proposed in this pull request?

Plan equality should be computed by `canonicalized`, so we can remove unnecessary `hashCode` and `equals` methods.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19539 from wzhfy/remove_equals.
2017-10-20 20:58:55 -07:00
Takuya UESHIN b8624b06e5 [SPARK-20396][SQL][PYSPARK][FOLLOW-UP] groupby().apply() with pandas udf
## What changes were proposed in this pull request?

This is a follow-up of #18732.
This pr modifies `GroupedData.apply()` method to convert pandas udf to grouped udf implicitly.

## How was this patch tested?

Exisiting tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19517 from ueshin/issues/SPARK-20396/fup2.
2017-10-20 12:44:30 -07:00
CenYuhai 16c9cc68c5 [SPARK-21055][SQL] replace grouping__id with grouping_id()
## What changes were proposed in this pull request?
spark does not support grouping__id, it has grouping_id() instead.
But it is not convenient for hive user to change to spark-sql
so this pr is to replace grouping__id with grouping_id()
hive user need not to alter their scripts

## How was this patch tested?

test with SQLQuerySuite.scala

Author: CenYuhai <yuhai.cen@ele.me>

Closes #18270 from cenyuhai/SPARK-21055.
2017-10-20 09:27:39 -07:00
Eric Perry b84f61cd79 [SQL] Mark strategies with override for clarity.
## What changes were proposed in this pull request?

This is a very trivial PR, simply marking `strategies` in `SparkPlanner` with the `override` keyword for clarity since it is overriding `strategies` in `QueryPlanner` two levels up in the class hierarchy. I was reading through the code to learn a bit and got stuck on this fact for a little while, so I figured this may be helpful so that another developer new to the project doesn't get stuck where I was.

I did not make a JIRA ticket for this because it is so trivial, but I'm happy to do so to adhere to the contribution guidelines if required.

## How was this patch tested?

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

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

Author: Eric Perry <eric@ericjperry.com>

Closes #19537 from ericjperry/override-strategies.
2017-10-19 23:57:41 -07:00
Wenchen Fan b034f2565f [SPARK-22026][SQL] data source v2 write path
## What changes were proposed in this pull request?

A working prototype for data source v2 write path.

The writing framework is similar to the reading framework. i.e. `WriteSupport` -> `DataSourceV2Writer` -> `DataWriterFactory` -> `DataWriter`.

Similar to the `FileCommitPotocol`, the writing API has job and task level commit/abort to support the transaction.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19269 from cloud-fan/data-source-v2-write.
2017-10-19 20:24:51 -07:00
Andrew Ash 7fae7995ba [SPARK-22268][BUILD] Fix lint-java
## What changes were proposed in this pull request?

Fix java style issues

## How was this patch tested?

Run `./dev/lint-java` locally since it's not run on Jenkins

Author: Andrew Ash <andrew@andrewash.com>

Closes #19486 from ash211/aash/fix-lint-java.
2017-10-20 09:40:00 +09:00
Marcelo Vanzin dc2714da50 [SPARK-22290][CORE] Avoid creating Hive delegation tokens when not necessary.
Hive delegation tokens are only needed when the Spark driver has no access
to the kerberos TGT. That happens only in two situations:

- when using a proxy user
- when using cluster mode without a keytab

This change modifies the Hive provider so that it only generates delegation
tokens in those situations, and tweaks the YARN AM so that it makes the proper
user visible to the Hive code when running with keytabs, so that the TGT
can be used instead of a delegation token.

The effect of this change is that now it's possible to initialize multiple,
non-concurrent SparkContext instances in the same JVM. Before, the second
invocation would fail to fetch a new Hive delegation token, which then could
make the second (or third or...) application fail once the token expired.
With this change, the TGT will be used to authenticate to the HMS instead.

This change also avoids polluting the current logged in user's credentials
when launching applications. The credentials are copied only when running
applications as a proxy user. This makes it possible to implement SPARK-11035
later, where multiple threads might be launching applications, and each app
should have its own set of credentials.

Tested by verifying HDFS and Hive access in following scenarios:
- client and cluster mode
- client and cluster mode with proxy user
- client and cluster mode with principal / keytab
- long-running cluster app with principal / keytab
- pyspark app that creates (and stops) multiple SparkContext instances
  through its lifetime

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19509 from vanzin/SPARK-22290.
2017-10-19 14:56:48 +08:00
Marco Gaido 1f25d8683a [SPARK-22249][FOLLOWUP][SQL] Check if list of value for IN is empty in the optimizer
## What changes were proposed in this pull request?

This PR addresses the comments by gatorsmile on [the previous PR](https://github.com/apache/spark/pull/19494).

## How was this patch tested?

Previous UT and added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19522 from mgaido91/SPARK-22249_FOLLOWUP.
2017-10-18 09:14:46 -07:00
maryannxue 72561ecf4b [SPARK-22266][SQL] The same aggregate function was evaluated multiple times
## What changes were proposed in this pull request?

To let the same aggregate function that appear multiple times in an Aggregate be evaluated only once, we need to deduplicate the aggregate expressions. The original code was trying to use a "distinct" call to get a set of aggregate expressions, but did not work, since the "distinct" did not compare semantic equality. And even if it did, further work should be done in result expression rewriting.
In this PR, I changed the "set" to a map mapping the semantic identity of a aggregate expression to itself. Thus, later on, when rewriting result expressions (i.e., output expressions), the aggregate expression reference can be fixed.

## How was this patch tested?

Added a new test in SQLQuerySuite

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

Closes #19488 from maryannxue/spark-22266.
2017-10-18 20:59:40 +08:00
Tathagata Das f3137feecd [SPARK-22278][SS] Expose current event time watermark and current processing time in GroupState
## What changes were proposed in this pull request?

Complex state-updating and/or timeout-handling logic in mapGroupsWithState functions may require taking decisions based on the current event-time watermark and/or processing time. Currently, you can use the SQL function `current_timestamp` to get the current processing time, but it needs to be passed inserted in every row with a select, and then passed through the encoder, which isn't efficient. Furthermore, there is no way to get the current watermark.

This PR exposes both of them through the GroupState API.
Additionally, it also cleans up some of the GroupState docs.

## How was this patch tested?

New unit tests

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

Closes #19495 from tdas/SPARK-22278.
2017-10-17 20:09:12 -07:00
Huaxin Gao 28f9f3f225 [SPARK-22271][SQL] mean overflows and returns null for some decimal variables
## What changes were proposed in this pull request?

In Average.scala, it has
```
  override lazy val evaluateExpression = child.dataType match {
    case DecimalType.Fixed(p, s) =>
      // increase the precision and scale to prevent precision loss
      val dt = DecimalType.bounded(p + 14, s + 4)
      Cast(Cast(sum, dt) / Cast(count, dt), resultType)
    case _ =>
      Cast(sum, resultType) / Cast(count, resultType)
  }

  def setChild (newchild: Expression) = {
    child = newchild
  }

```
It is possible that  Cast(count, dt), resultType) will make the precision of the decimal number bigger than 38, and this causes over flow.  Since count is an integer and doesn't need a scale, I will cast it using DecimalType.bounded(38,0)
## How was this patch tested?
In DataFrameSuite, I will add a test case.

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

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

Closes #19496 from huaxingao/spark-22271.
2017-10-17 12:50:41 -07:00
Jose Torres 75d666b95a [SPARK-22136][SS] Evaluate one-sided conditions early in stream-stream joins.
## What changes were proposed in this pull request?

Evaluate one-sided conditions early in stream-stream joins.

This is in addition to normal filter pushdown, because integrating it with the join logic allows it to take place in outer join scenarios. This means that rows which can never satisfy the join condition won't clog up the state.

## How was this patch tested?
new unit tests

Author: Jose Torres <jose@databricks.com>

Closes #19452 from joseph-torres/SPARK-22136.
2017-10-17 12:26:53 -07:00
Kent Yao 99e32f8ba5 [SPARK-22224][SQL] Override toString of KeyValue/Relational-GroupedDataset
## What changes were proposed in this pull request?
#### before

```scala
scala> val words = spark.read.textFile("README.md").flatMap(_.split(" "))
words: org.apache.spark.sql.Dataset[String] = [value: string]

scala> val grouped = words.groupByKey(identity)
grouped: org.apache.spark.sql.KeyValueGroupedDataset[String,String] = org.apache.spark.sql.KeyValueGroupedDataset65214862
```
#### after
```scala
scala> val words = spark.read.textFile("README.md").flatMap(_.split(" "))
words: org.apache.spark.sql.Dataset[String] = [value: string]

scala> val grouped = words.groupByKey(identity)
grouped: org.apache.spark.sql.KeyValueGroupedDataset[String,String] = [key: [value: string], value: [value: string]]
```

## How was this patch tested?
existing ut

cc gatorsmile cloud-fan

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19363 from yaooqinn/minor-dataset-tostring.
2017-10-17 17:58:45 +08:00
Marco Gaido 8148f19ca1 [SPARK-22249][SQL] isin with empty list throws exception on cached DataFrame
## What changes were proposed in this pull request?

As pointed out in the JIRA, there is a bug which causes an exception to be thrown if `isin` is called with an empty list on a cached DataFrame. The PR fixes it.

## How was this patch tested?

Added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19494 from mgaido91/SPARK-22249.
2017-10-17 09:41:23 +02:00
Dongjoon Hyun c09a2a76b5 [SPARK-22280][SQL][TEST] Improve StatisticsSuite to test convertMetastore properly
## What changes were proposed in this pull request?

This PR aims to improve **StatisticsSuite** to test `convertMetastore` configuration properly. Currently, some test logic in `test statistics of LogicalRelation converted from Hive serde tables` depends on the default configuration. New test case is shorter and covers both(true/false) cases explicitly.

This test case was previously modified by SPARK-17410 and SPARK-17284 in Spark 2.3.0.
- a2460be9c3 (diff-1c464c86b68c2d0b07e73b7354e74ce7R443)

## How was this patch tested?

Pass the Jenkins with the improved test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19500 from dongjoon-hyun/SPARK-22280.
2017-10-16 16:16:34 -07:00
Dongjoon Hyun 561505e2fc [SPARK-22282][SQL] Rename OrcRelation to OrcFileFormat and remove ORC_COMPRESSION
## What changes were proposed in this pull request?

This PR aims to
- Rename `OrcRelation` to `OrcFileFormat` object.
- Replace `OrcRelation.ORC_COMPRESSION` with `org.apache.orc.OrcConf.COMPRESS`. Since [SPARK-21422](https://issues.apache.org/jira/browse/SPARK-21422), we can use `OrcConf.COMPRESS` instead of Hive's.

```scala
// The references of Hive's classes will be minimized.
val ORC_COMPRESSION = "orc.compress"
```

## How was this patch tested?

Pass the Jenkins with the existing and updated test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19502 from dongjoon-hyun/SPARK-22282.
2017-10-16 11:27:08 -07:00
Liang-Chi Hsieh 0ae96495de [SPARK-22223][SQL] ObjectHashAggregate should not introduce unnecessary shuffle
## What changes were proposed in this pull request?

`ObjectHashAggregateExec` should override `outputPartitioning` in order to avoid unnecessary shuffle.

## How was this patch tested?

Added Jenkins test.

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

Closes #19501 from viirya/SPARK-22223.
2017-10-16 13:37:58 +08:00
Burak Yavuz e8547ffb49 [SPARK-22238] Fix plan resolution bug caused by EnsureStatefulOpPartitioning
## What changes were proposed in this pull request?

In EnsureStatefulOpPartitioning, we check that the inputRDD to a SparkPlan has the expected partitioning for Streaming Stateful Operators. The problem is that we are not allowed to access this information during planning.
The reason we added that check was because CoalesceExec could actually create RDDs with 0 partitions. We should fix it such that when CoalesceExec says that there is a SinglePartition, there is in fact an inputRDD of 1 partition instead of 0 partitions.

## How was this patch tested?

Regression test in StreamingQuerySuite

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #19467 from brkyvz/stateful-op.
2017-10-14 17:39:15 -07:00
Takuya UESHIN e0503a7223 [SPARK-22273][SQL] Fix key/value schema field names in HashMapGenerators.
## What changes were proposed in this pull request?

When fixing schema field names using escape characters with `addReferenceMinorObj()` at [SPARK-18952](https://issues.apache.org/jira/browse/SPARK-18952) (#16361), double-quotes around the names were remained and the names become something like `"((java.lang.String) references[1])"`.

```java
/* 055 */     private int maxSteps = 2;
/* 056 */     private int numRows = 0;
/* 057 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add("((java.lang.String) references[1])", org.apache.spark.sql.types.DataTypes.StringType);
/* 058 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add("((java.lang.String) references[2])", org.apache.spark.sql.types.DataTypes.LongType);
/* 059 */     private Object emptyVBase;
```

We should remove the double-quotes to refer the values in `references` properly:

```java
/* 055 */     private int maxSteps = 2;
/* 056 */     private int numRows = 0;
/* 057 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[1]), org.apache.spark.sql.types.DataTypes.StringType);
/* 058 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[2]), org.apache.spark.sql.types.DataTypes.LongType);
/* 059 */     private Object emptyVBase;
```

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19491 from ueshin/issues/SPARK-22273.
2017-10-13 23:24:36 -07:00
Steve Loughran e3536406ec [SPARK-21762][SQL] FileFormatWriter/BasicWriteTaskStatsTracker metrics collection fails if a new file isn't yet visible
## What changes were proposed in this pull request?

`BasicWriteTaskStatsTracker.getFileSize()` to catch `FileNotFoundException`, log  info and then return 0 as a file size.

This ensures that if a newly created file isn't visible due to the store not always having create consistency, the metric collection doesn't cause the failure.

## How was this patch tested?

New test suite included, `BasicWriteTaskStatsTrackerSuite`. This not only checks the resilience to missing files, but verifies the existing logic as to how file statistics are gathered.

Note that in the current implementation

1. if you call `Tracker..getFinalStats()` more than once, the file size count will increase by size of the last file. This could be fixed by clearing the filename field inside `getFinalStats()` itself.

2. If you pass in an empty or null string to `Tracker.newFile(path)` then IllegalArgumentException is raised, but only in `getFinalStats()`, rather than in `newFile`.  There's a test for this behaviour in the new suite, as it verifies that only FNFEs get swallowed.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #18979 from steveloughran/cloud/SPARK-21762-missing-files-in-metrics.
2017-10-13 23:08:17 -07:00
Liwei Lin 1bb8b76045 [MINOR][SS] keyWithIndexToNumValues" -> "keyWithIndexToValue"
## What changes were proposed in this pull request?

This PR changes `keyWithIndexToNumValues`  to `keyWithIndexToValue`.

There will be directories on HDFS named with this `keyWithIndexToNumValues`. So if we ever want to fix this, let's fix it now.

## How was this patch tested?

existing unit test cases.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #19435 from lw-lin/keyWithIndex.
2017-10-13 15:13:06 -07:00
Wenchen Fan 3823dc88d3 [SPARK-22252][SQL][FOLLOWUP] Command should not be a LeafNode
## What changes were proposed in this pull request?

This is a minor folllowup of #19474 .

#19474 partially reverted #18064 but accidentally introduced a behavior change. `Command` extended `LogicalPlan` before #18064 , but #19474 made it extend `LeafNode`. This is an internal behavior change as now all `Command` subclasses can't define children, and they have to implement `computeStatistic` method.

This PR fixes this by making `Command` extend `LogicalPlan`

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19493 from cloud-fan/minor.
2017-10-13 10:49:48 -07:00
Dongjoon Hyun 6412ea1759 [SPARK-21247][SQL] Type comparison should respect case-sensitive SQL conf
## What changes were proposed in this pull request?

This is an effort to reduce the difference between Hive and Spark. Spark supports case-sensitivity in columns. Especially, for Struct types, with `spark.sql.caseSensitive=true`, the following is supported.

```scala
scala> sql("select named_struct('a', 1, 'A', 2).a").show
+--------------------------+
|named_struct(a, 1, A, 2).a|
+--------------------------+
|                         1|
+--------------------------+

scala> sql("select named_struct('a', 1, 'A', 2).A").show
+--------------------------+
|named_struct(a, 1, A, 2).A|
+--------------------------+
|                         2|
+--------------------------+
```

And vice versa, with `spark.sql.caseSensitive=false`, the following is supported.
```scala
scala> sql("select named_struct('a', 1).A, named_struct('A', 1).a").show
+--------------------+--------------------+
|named_struct(a, 1).A|named_struct(A, 1).a|
+--------------------+--------------------+
|                   1|                   1|
+--------------------+--------------------+
```

However, types are considered different. For example, SET operations fail.
```scala
scala> sql("SELECT named_struct('a',1) union all (select named_struct('A',2))").show
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. struct<A:int> <> struct<a:int> at the first column of the second table;;
'Union
:- Project [named_struct(a, 1) AS named_struct(a, 1)#57]
:  +- OneRowRelation$
+- Project [named_struct(A, 2) AS named_struct(A, 2)#58]
   +- OneRowRelation$
```

This PR aims to support case-insensitive type equality. For example, in Set operation, the above operation succeed when `spark.sql.caseSensitive=false`.

```scala
scala> sql("SELECT named_struct('a',1) union all (select named_struct('A',2))").show
+------------------+
|named_struct(a, 1)|
+------------------+
|               [1]|
|               [2]|
+------------------+
```

## How was this patch tested?

Pass the Jenkins with a newly add test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18460 from dongjoon-hyun/SPARK-21247.
2017-10-14 00:35:12 +08:00
Dongjoon Hyun e6e36004af [SPARK-14387][SPARK-16628][SPARK-18355][SQL] Use Spark schema to read ORC table instead of ORC file schema
## What changes were proposed in this pull request?

Before Hive 2.0, ORC File schema has invalid column names like `_col1` and `_col2`. This is a well-known limitation and there are several Apache Spark issues with `spark.sql.hive.convertMetastoreOrc=true`. This PR ignores ORC File schema and use Spark schema.

## How was this patch tested?

Pass the newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19470 from dongjoon-hyun/SPARK-18355.
2017-10-13 23:09:12 +08:00
Wang Gengliang 2f00a71a87 [SPARK-22257][SQL] Reserve all non-deterministic expressions in ExpressionSet
## What changes were proposed in this pull request?

For non-deterministic expressions, they should be considered as not contained in the [[ExpressionSet]].
This is consistent with how we define `semanticEquals` between two expressions.
Otherwise, combining expressions will remove non-deterministic expressions which should be reserved.
E.g.
Combine filters of
```scala
testRelation.where(Rand(0) > 0.1).where(Rand(0) > 0.1)
```
should result in
```scala
testRelation.where(Rand(0) > 0.1 && Rand(0) > 0.1)
```

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19475 from gengliangwang/non-deterministic-expressionSet.
2017-10-12 22:45:19 -07:00
Wenchen Fan ec122209fb [SPARK-21165][SQL] FileFormatWriter should handle mismatched attribute ids between logical and physical plan
## What changes were proposed in this pull request?

Due to optimizer removing some unnecessary aliases, the logical and physical plan may have different output attribute ids. FileFormatWriter should handle this when creating the physical sort node.

## How was this patch tested?

new regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19483 from cloud-fan/bug2.
2017-10-13 13:09:35 +08:00
Wang Gengliang 3ff766f61a [SPARK-22263][SQL] Refactor deterministic as lazy value
## What changes were proposed in this pull request?

The method `deterministic` is frequently called in optimizer.
Refactor `deterministic` as lazy value, in order to avoid redundant computations.

## How was this patch tested?
Simple benchmark test over TPC-DS queries, run time from query string to optimized plan(continuous  20 runs, and get the average of last 5 results):
Before changes: 12601 ms
After changes: 11993ms
This is 4.8% performance improvement.

Also run test with Unit test.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19478 from gengliangwang/deterministicAsLazyVal.
2017-10-12 18:47:16 -07:00
Steve Loughran 9104add4c7 [SPARK-22217][SQL] ParquetFileFormat to support arbitrary OutputCommitters
## What changes were proposed in this pull request?

`ParquetFileFormat` to relax its requirement of output committer class from `org.apache.parquet.hadoop.ParquetOutputCommitter` or subclass thereof (and so implicitly Hadoop `FileOutputCommitter`) to any committer implementing `org.apache.hadoop.mapreduce.OutputCommitter`

This enables output committers which don't write to the filesystem the way `FileOutputCommitter` does to save parquet data from a dataframe: at present you cannot do this.

Before a committer which isn't a subclass of `ParquetOutputCommitter`, it checks to see if the context has requested summary metadata by setting `parquet.enable.summary-metadata`. If true, and the committer class isn't a parquet committer, it raises a RuntimeException with an error message.

(It could downgrade, of course, but raising an exception makes it clear there won't be an summary. It also makes the behaviour testable.)

Note that `SQLConf` already states that any `OutputCommitter` can be used, but that typically it's a subclass of ParquetOutputCommitter. That's not currently true. This patch will make the code consistent with the docs, adding tests to verify,

## How was this patch tested?

The patch includes a test suite, `ParquetCommitterSuite`, with a new committer, `MarkingFileOutputCommitter` which extends `FileOutputCommitter` and writes a marker file in the destination directory. The presence of the marker file can be used to verify the new committer was used. The tests then try the combinations of Parquet committer summary/no-summary and marking committer summary/no-summary.

| committer | summary | outcome |
|-----------|---------|---------|
| parquet   | true    | success |
| parquet   | false   | success |
| marking   | false   | success with marker |
| marking   | true    | exception |

All tests are happy.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #19448 from steveloughran/cloud/SPARK-22217-committer.
2017-10-13 08:40:26 +09:00
Ala Luszczak 02218c4c73 [SPARK-22251][SQL] Metric 'aggregate time' is incorrect when codegen is off
## What changes were proposed in this pull request?

Adding the code for setting 'aggregate time' metric to non-codegen path in HashAggregateExec and to ObjectHashAggregateExces.

## How was this patch tested?

Tested manually.

Author: Ala Luszczak <ala@databricks.com>

Closes #19473 from ala/fix-agg-time.
2017-10-12 17:00:22 +02:00
Wenchen Fan 73d80ec497 [SPARK-22197][SQL] push down operators to data source before planning
## What changes were proposed in this pull request?

As we discussed in https://github.com/apache/spark/pull/19136#discussion_r137023744 , we should push down operators to data source before planning, so that data source can report statistics more accurate.

This PR also includes some cleanup for the read path.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19424 from cloud-fan/follow.
2017-10-12 20:34:03 +08:00
Wenchen Fan 274f0efefa [SPARK-22252][SQL] FileFormatWriter should respect the input query schema
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/18064, we allowed `RunnableCommand` to have children in order to fix some UI issues. Then we made `InsertIntoXXX` commands take the input `query` as a child, when we do the actual writing, we just pass the physical plan to the writer(`FileFormatWriter.write`).

However this is problematic. In Spark SQL, optimizer and planner are allowed to change the schema names a little bit. e.g. `ColumnPruning` rule will remove no-op `Project`s, like `Project("A", Scan("a"))`, and thus change the output schema from "<A: int>" to `<a: int>`. When it comes to writing, especially for self-description data format like parquet, we may write the wrong schema to the file and cause null values at the read path.

Fortunately, in https://github.com/apache/spark/pull/18450 , we decided to allow nested execution and one query can map to multiple executions in the UI. This releases the major restriction in #18604 , and now we don't have to take the input `query` as child of `InsertIntoXXX` commands.

So the fix is simple, this PR partially revert #18064 and make `InsertIntoXXX` commands leaf nodes again.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19474 from cloud-fan/bug.
2017-10-12 20:20:44 +08:00
Shixiong Zhu 645e108eeb [SPARK-21988][SS] Implement StreamingRelation.computeStats to fix explain
## What changes were proposed in this pull request?

Implement StreamingRelation.computeStats to fix explain

## How was this patch tested?

- unit tests: `StreamingRelation.computeStats` and `StreamingExecutionRelation.computeStats`.
- regression tests: `explain join with a normal source` and `explain join with MemoryStream`.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19465 from zsxwing/SPARK-21988.
2017-10-11 13:51:33 -07:00
Zhenhua Wang 655f6f86f8 [SPARK-22208][SQL] Improve percentile_approx by not rounding up targetError and starting from index 0
## What changes were proposed in this pull request?

Currently percentile_approx never returns the first element when percentile is in (relativeError, 1/N], where relativeError default 1/10000, and N is the total number of elements. But ideally, percentiles in [0, 1/N] should all return the first element as the answer.

For example, given input data 1 to 10, if a user queries 10% (or even less) percentile, it should return 1, because the first value 1 already reaches 10%. Currently it returns 2.

Based on the paper, targetError is not rounded up, and searching index should start from 0 instead of 1. By following the paper, we should be able to fix the cases mentioned above.

## How was this patch tested?

Added a new test case and fix existing test cases.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19438 from wzhfy/improve_percentile_approx.
2017-10-11 00:16:12 -07:00
Kazuaki Ishizaki 76fb173dd6 [SPARK-21751][SQL] CodeGeneraor.splitExpressions counts code size more precisely
## What changes were proposed in this pull request?

Current `CodeGeneraor.splitExpressions` splits statements into methods if the total length of statements is more than 1024 characters. The length may include comments or empty line.

This PR excludes comment or empty line from the length to reduce the number of generated methods in a class, by using `CodeFormatter.stripExtraNewLinesAndComments()` method.

## How was this patch tested?

Existing tests

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

Closes #18966 from kiszk/SPARK-21751.
2017-10-10 20:29:02 -07:00
Marcelo Vanzin bd4eb9ce57 [SPARK-19558][SQL] Add config key to register QueryExecutionListeners automatically.
This change adds a new SQL config key that is equivalent to SparkContext's
"spark.extraListeners", allowing users to register QueryExecutionListener
instances through the Spark configuration system instead of having to
explicitly do it in code.

The code used by SparkContext to implement the feature was refactored into
a helper method in the Utils class, and SQL's ExecutionListenerManager was
modified to use it to initialize listener declared in the configuration.

Unit tests were added to verify all the new functionality.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19309 from vanzin/SPARK-19558.
2017-10-10 15:50:37 -07:00
Li Jin bfc7e1fe1a [SPARK-20396][SQL][PYSPARK] groupby().apply() with pandas udf
## What changes were proposed in this pull request?

This PR adds an apply() function on df.groupby(). apply() takes a pandas udf that is a transformation on `pandas.DataFrame` -> `pandas.DataFrame`.

Static schema
-------------------
```
schema = df.schema

pandas_udf(schema)
def normalize(df):
    df = df.assign(v1 = (df.v1 - df.v1.mean()) / df.v1.std()
    return df

df.groupBy('id').apply(normalize)
```
Dynamic schema
-----------------------
**This use case is removed from the PR and we will discuss this as a follow up. See discussion https://github.com/apache/spark/pull/18732#pullrequestreview-66583248**

Another example to use pd.DataFrame dtypes as output schema of the udf:

```
sample_df = df.filter(df.id == 1).toPandas()

def foo(df):
      ret = # Some transformation on the input pd.DataFrame
      return ret

foo_udf = pandas_udf(foo, foo(sample_df).dtypes)

df.groupBy('id').apply(foo_udf)
```
In interactive use case, user usually have a sample pd.DataFrame to test function `foo` in their notebook. Having been able to use `foo(sample_df).dtypes` frees user from specifying the output schema of `foo`.

Design doc: https://github.com/icexelloss/spark/blob/pandas-udf-doc/docs/pyspark-pandas-udf.md

## How was this patch tested?
* Added GroupbyApplyTest

Author: Li Jin <ice.xelloss@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>
Author: Bryan Cutler <cutlerb@gmail.com>

Closes #18732 from icexelloss/groupby-apply-SPARK-20396.
2017-10-11 07:32:01 +09:00
gatorsmile 633ffd816d rename the file. 2017-10-10 11:01:02 -07:00
Takuya UESHIN af8a34c787 [SPARK-22159][SQL][FOLLOW-UP] Make config names consistently end with "enabled".
## What changes were proposed in this pull request?

This is a follow-up of #19384.

In the previous pr, only definitions of the config names were modified, but we also need to modify the names in runtime or tests specified as string literal.

## How was this patch tested?

Existing tests but modified the config names.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19462 from ueshin/issues/SPARK-22159/fup1.
2017-10-09 22:35:34 -07:00
Feng Liu bebd2e1ce1 [SPARK-22222][CORE] Fix the ARRAY_MAX in BufferHolder and add a test
## What changes were proposed in this pull request?

We should not break the assumption that the length of the allocated byte array is word rounded:
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java#L170
So we want to use `Integer.MAX_VALUE - 15` instead of `Integer.MAX_VALUE - 8` as the upper bound of an allocated byte array.

cc: srowen gatorsmile
## How was this patch tested?

Since the Spark unit test JVM has less than 1GB heap, here we run the test code as a submit job, so it can run on a JVM has 4GB memory.

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

Author: Feng Liu <fengliu@databricks.com>

Closes #19460 from liufengdb/fix_array_max.
2017-10-09 21:34:37 -07:00
Jose Torres 71c2b81aa0 [SPARK-22230] Swap per-row order in state store restore.
## What changes were proposed in this pull request?
In state store restore, for each row, put the saved state before the row in the iterator instead of after.

This fixes an issue where agg(last('attr)) will forever return the last value of 'attr from the first microbatch.

## How was this patch tested?

new unit test

Author: Jose Torres <jose@databricks.com>

Closes #19461 from joseph-torres/SPARK-22230.
2017-10-09 16:34:39 -07:00
Ryan Blue 155ab6347e [SPARK-22170][SQL] Reduce memory consumption in broadcast joins.
## What changes were proposed in this pull request?

This updates the broadcast join code path to lazily decompress pages and
iterate through UnsafeRows to prevent all rows from being held in memory
while the broadcast table is being built.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #19394 from rdblue/broadcast-driver-memory.
2017-10-09 15:22:41 -07:00
Liang-Chi Hsieh debcbec749 [SPARK-21947][SS] Check and report error when monotonically_increasing_id is used in streaming query
## What changes were proposed in this pull request?

`monotonically_increasing_id` doesn't work in Structured Streaming. We should throw an exception if a streaming query uses it.

## How was this patch tested?

Added test.

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

Closes #19336 from viirya/SPARK-21947.
2017-10-06 13:10:04 -07:00
Xingbo Jiang 08b204fd2c [SPARK-22214][SQL] Refactor the list hive partitions code
## What changes were proposed in this pull request?

In this PR we make a few changes to the list hive partitions code, to make the code more extensible.
The following changes are made:
1. In `HiveClientImpl.getPartitions()`, call `client.getPartitions` instead of `shim.getAllPartitions` when `spec` is empty;
2. In `HiveTableScanExec`, previously we always call `listPartitionsByFilter` if the config `metastorePartitionPruning` is enabled, but actually, we'd better call `listPartitions` if `partitionPruningPred` is empty;
3.  We should use sessionCatalog instead of SharedState.externalCatalog in `HiveTableScanExec`.

## How was this patch tested?

Tested by existing test cases since this is code refactor, no regression or behavior change is expected.

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

Closes #19444 from jiangxb1987/hivePartitions.
2017-10-06 12:53:35 -07:00
gatorsmile 83488cc318 [SPARK-21871][SQL] Fix infinite loop when bytecode size is larger than spark.sql.codegen.hugeMethodLimit
## What changes were proposed in this pull request?
When exceeding `spark.sql.codegen.hugeMethodLimit`, the runtime fallbacks to the Volcano iterator solution. This could cause an infinite loop when `FileSourceScanExec` can use the columnar batch to read the data. This PR is to fix the issue.

## How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19440 from gatorsmile/testt.
2017-10-05 23:33:49 -07:00
Liang-Chi Hsieh ae61f187aa [SPARK-22206][SQL][SPARKR] gapply in R can't work on empty grouping columns
## What changes were proposed in this pull request?

Looks like `FlatMapGroupsInRExec.requiredChildDistribution` didn't consider empty grouping attributes. It should be a problem when running `EnsureRequirements` and `gapply` in R can't work on empty grouping columns.

## How was this patch tested?

Added test.

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

Closes #19436 from viirya/fix-flatmapinr-distribution.
2017-10-05 23:36:18 +09:00
Shixiong Zhu c8affec21c [SPARK-22203][SQL] Add job description for file listing Spark jobs
## What changes were proposed in this pull request?

The user may be confused about some 10000-tasks jobs. We can add a job description for these jobs so that the user can figure it out.

## How was this patch tested?

The new unit test.

Before:
<img width="343" alt="screen shot 2017-10-04 at 3 22 09 pm" src="https://user-images.githubusercontent.com/1000778/31202567-f78d15c0-a917-11e7-841e-11b8bf8f0032.png">

After:
<img width="473" alt="screen shot 2017-10-04 at 3 13 51 pm" src="https://user-images.githubusercontent.com/1000778/31202576-fc01e356-a917-11e7-9c2b-7bf80b153adb.png">

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19432 from zsxwing/SPARK-22203.
2017-10-04 20:58:48 -07:00
Tathagata Das 969ffd6317 [SPARK-22187][SS] Update unsaferow format for saved state such that we can set timeouts when state is null
## What changes were proposed in this pull request?

Currently, the group state of user-defined-type is encoded as top-level columns in the UnsafeRows stores in the state store. The timeout timestamp is also saved as (when needed) as the last top-level column. Since the group state is serialized to top-level columns, you cannot save "null" as a value of state (setting null in all the top-level columns is not equivalent). So we don't let the user set the timeout without initializing the state for a key. Based on user experience, this leads to confusion.

This PR is to change the row format such that the state is saved as nested columns. This would allow the state to be set to null, and avoid these confusing corner cases.

## How was this patch tested?
Refactored tests.

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

Closes #19416 from tdas/SPARK-22187.
2017-10-04 19:25:22 -07:00
Wenchen Fan bb035f1ee5 [SPARK-22169][SQL] support byte length literal as identifier
## What changes were proposed in this pull request?

By definition the table name in Spark can be something like `123x`, `25a`, etc., with exceptions for literals like `12L`, `23BD`, etc. However, Spark SQL has a special byte length literal, which stops users to use digits followed by `b`, `k`, `m`, `g` as identifiers.

byte length literal is not a standard sql literal and is only used in the `tableSample` parser rule. This PR move the parsing of byte length literal from lexer to parser, so that users can use it as identifiers.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19392 from cloud-fan/parser-bug.
2017-10-04 13:13:51 -07:00
Takeshi Yamamuro 4a779bdac3 [SPARK-21871][SQL] Check actual bytecode size when compiling generated code
## What changes were proposed in this pull request?
This pr added code to check actual bytecode size when compiling generated code. In #18810, we added code to give up code compilation and use interpreter execution in `SparkPlan` if the line number of generated functions goes over `maxLinesPerFunction`. But, we already have code to collect metrics for compiled bytecode size in `CodeGenerator` object. So,we could easily reuse the code for this purpose.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19083 from maropu/SPARK-21871.
2017-10-04 10:08:24 -07:00
Kazuaki Ishizaki 64df08b647 [SPARK-20783][SQL] Create ColumnVector to abstract existing compressed column (batch method)
## What changes were proposed in this pull request?

This PR abstracts data compressed by `CompressibleColumnAccessor` using `ColumnVector` in batch method. When `ColumnAccessor.decompress` is called, `ColumnVector` will have uncompressed data. This batch decompress does not use `InternalRow` to reduce the number of memory accesses.

As first step of this implementation, this JIRA supports primitive data types. Another PR will support array and other data types.

This implementation decompress data in batch into uncompressed column batch, as rxin suggested at [here](https://github.com/apache/spark/pull/18468#issuecomment-316914076). Another implementation uses adapter approach [as cloud-fan suggested](https://github.com/apache/spark/pull/18468).

## How was this patch tested?

Added test suites

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

Closes #18704 from kiszk/SPARK-20783a.
2017-10-04 15:06:44 +08:00
Rekha Joshi d54670192a [SPARK-22193][SQL] Minor typo fix
## What changes were proposed in this pull request?

[SPARK-22193][SQL] Minor typo fix in SortMergeJoinExec. Nothing major, but it bothered me going into.Hence fixing

## How was this patch tested?
existing tests

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: rjoshi2 <rekhajoshm@gmail.com>

Closes #19422 from rekhajoshm/SPARK-22193.
2017-10-04 07:11:00 +01:00
Jose Torres 3099c574c5 [SPARK-22136][SS] Implement stream-stream outer joins.
## What changes were proposed in this pull request?

Allow one-sided outer joins between two streams when a watermark is defined.

## How was this patch tested?

new unit tests

Author: Jose Torres <jose@databricks.com>

Closes #19327 from joseph-torres/outerjoin.
2017-10-03 21:42:51 -07:00
gatorsmile 5f69433453 [SPARK-22171][SQL] Describe Table Extended Failed when Table Owner is Empty
## What changes were proposed in this pull request?

Users could hit `java.lang.NullPointerException` when the tables were created by Hive and the table's owner is `null` that are got from Hive metastore. `DESC EXTENDED` failed with the error:

> SQLExecutionException: java.lang.NullPointerException at scala.collection.immutable.StringOps$.length$extension(StringOps.scala:47) at scala.collection.immutable.StringOps.length(StringOps.scala:47) at scala.collection.IndexedSeqOptimized$class.isEmpty(IndexedSeqOptimized.scala:27) at scala.collection.immutable.StringOps.isEmpty(StringOps.scala:29) at scala.collection.TraversableOnce$class.nonEmpty(TraversableOnce.scala:111) at scala.collection.immutable.StringOps.nonEmpty(StringOps.scala:29) at org.apache.spark.sql.catalyst.catalog.CatalogTable.toLinkedHashMap(interface.scala:300) at org.apache.spark.sql.execution.command.DescribeTableCommand.describeFormattedTableInfo(tables.scala:565) at org.apache.spark.sql.execution.command.DescribeTableCommand.run(tables.scala:543) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:66) at

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19395 from gatorsmile/desc.
2017-10-03 21:27:58 -07:00
gatorsmile e65b6b7ca1 [SPARK-22178][SQL] Refresh Persistent Views by REFRESH TABLE Command
## What changes were proposed in this pull request?
The underlying tables of persistent views are not refreshed when users issue the REFRESH TABLE command against the persistent views.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19405 from gatorsmile/refreshView.
2017-10-03 12:40:22 -07:00
Reynold Xin 4c5158eec9 [SPARK-21644][SQL] LocalLimit.maxRows is defined incorrectly
## What changes were proposed in this pull request?
The definition of `maxRows` in `LocalLimit` operator was simply wrong. This patch introduces a new `maxRowsPerPartition` method and uses that in pruning. The patch also adds more documentation on why we need local limit vs global limit.

Note that this previously has never been a bug because the way the code is structured, but future use of the maxRows could lead to bugs.

## How was this patch tested?
Should be covered by existing test cases.

Closes #18851

Author: gatorsmile <gatorsmile@gmail.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #19393 from gatorsmile/pr-18851.
2017-10-03 12:38:13 -07:00
Takeshi Yamamuro fa225da746 [SPARK-22176][SQL] Fix overflow issue in Dataset.show
## What changes were proposed in this pull request?
This pr fixed an overflow issue below in `Dataset.show`:
```
scala> Seq((1, 2), (3, 4)).toDF("a", "b").show(Int.MaxValue)
org.apache.spark.sql.AnalysisException: The limit expression must be equal to or greater than 0, but got -2147483648;;
GlobalLimit -2147483648
+- LocalLimit -2147483648
   +- Project [_1#27218 AS a#27221, _2#27219 AS b#27222]
      +- LocalRelation [_1#27218, _2#27219]

  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:41)
  at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:89)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.org$apache$spark$sql$catalyst$analysis$CheckAnalysis$$checkLimitClause(CheckAnalysis.scala:70)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:234)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:80)
  at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:127)
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19401 from maropu/MaxValueInShowString.
2017-10-02 15:25:33 -07:00
Dongjoon Hyun e5431f2cfd [SPARK-22158][SQL] convertMetastore should not ignore table property
## What changes were proposed in this pull request?

From the beginning, convertMetastoreOrc ignores table properties and use an empty map instead. This PR fixes that. For the diff, please see [this](https://github.com/apache/spark/pull/19382/files?w=1). convertMetastoreParquet also ignore.

```scala
val options = Map[String, String]()
```

- [SPARK-14070: HiveMetastoreCatalog.scala](https://github.com/apache/spark/pull/11891/files#diff-ee66e11b56c21364760a5ed2b783f863R650)
- [Master branch: HiveStrategies.scala](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala#L197
)

## How was this patch tested?

Pass the Jenkins with an updated test suite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19382 from dongjoon-hyun/SPARK-22158.
2017-10-02 15:00:26 -07:00
Liang-Chi Hsieh 3ca367083e [SPARK-22001][ML][SQL] ImputerModel can do withColumn for all input columns at one pass
## What changes were proposed in this pull request?

SPARK-21690 makes one-pass `Imputer` by parallelizing the computation of all input columns. When we transform dataset with `ImputerModel`, we do `withColumn` on all input columns sequentially. We can also do this on all input columns at once by adding a `withColumns` API to `Dataset`.

The new `withColumns` API is for internal use only now.

## How was this patch tested?

Existing tests for `ImputerModel`'s change. Added tests for `withColumns` API.

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

Closes #19229 from viirya/SPARK-22001.
2017-10-01 10:49:22 -07:00
Takeshi Yamamuro c6610a997f [SPARK-22122][SQL] Use analyzed logical plans to count input rows in TPCDSQueryBenchmark
## What changes were proposed in this pull request?
Since the current code ignores WITH clauses to check input relations in TPCDS queries, this leads to inaccurate per-row processing time for benchmark results. For example, in `q2`, this fix could catch all the input relations: `web_sales`, `date_dim`, and `catalog_sales` (the current code catches `date_dim` only). The one-third of the TPCDS queries uses WITH clauses, so I think it is worth fixing this.

## How was this patch tested?
Manually checked.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19344 from maropu/RespectWithInTPCDSBench.
2017-09-29 21:36:52 -07:00
gatorsmile 530fe68329 [SPARK-21904][SQL] Rename tempTables to tempViews in SessionCatalog
### What changes were proposed in this pull request?
`tempTables` is not right. To be consistent, we need to rename the internal variable names/comments to tempViews in SessionCatalog too.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19117 from gatorsmile/renameTempTablesToTempViews.
2017-09-29 19:35:32 -07:00
gatorsmile 9ed7394a68 [SPARK-22161][SQL] Add Impala-modified TPC-DS queries
## What changes were proposed in this pull request?

Added IMPALA-modified TPCDS queries to TPC-DS query suites.

- Ref: https://github.com/cloudera/impala-tpcds-kit/tree/master/queries

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19386 from gatorsmile/addImpalaQueries.
2017-09-29 08:59:42 -07:00
Wang Gengliang 0fa4dbe4f4 [SPARK-22141][FOLLOWUP][SQL] Add comments for the order of batches
## What changes were proposed in this pull request?
Add comments for specifying the position of  batch "Check Cartesian Products", as rxin suggested in https://github.com/apache/spark/pull/19362 .

## How was this patch tested?
Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19379 from gengliangwang/SPARK-22141-followup.
2017-09-28 23:23:30 -07:00
Marco Gaido 161ba7eaa4 [SPARK-22146] FileNotFoundException while reading ORC files containing special characters
## What changes were proposed in this pull request?

Reading ORC files containing special characters like '%' fails with a FileNotFoundException.
This PR aims to fix the problem.

## How was this patch tested?

Added UT.

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

Closes #19368 from mgaido91/SPARK-22146.
2017-09-28 23:14:53 -07:00
Reynold Xin 323806e68f [SPARK-22160][SQL] Make sample points per partition (in range partitioner) configurable and bump the default value up to 100
## What changes were proposed in this pull request?
Spark's RangePartitioner hard codes the number of sampling points per partition to be 20. This is sometimes too low. This ticket makes it configurable, via spark.sql.execution.rangeExchange.sampleSizePerPartition, and raises the default in Spark SQL to be 100.

## How was this patch tested?
Added a pretty sophisticated test based on chi square test ...

Author: Reynold Xin <rxin@databricks.com>

Closes #19387 from rxin/SPARK-22160.
2017-09-28 21:07:12 -07:00
Reynold Xin d29d1e8799 [SPARK-22159][SQL] Make config names consistently end with "enabled".
## What changes were proposed in this pull request?
spark.sql.execution.arrow.enable and spark.sql.codegen.aggregate.map.twolevel.enable -> enabled

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #19384 from rxin/SPARK-22159.
2017-09-28 15:59:05 -07:00
Reynold Xin d74dee1336 [SPARK-22153][SQL] Rename ShuffleExchange -> ShuffleExchangeExec
## What changes were proposed in this pull request?
For some reason when we added the Exec suffix to all physical operators, we missed this one. I was looking for this physical operator today and couldn't find it, because I was looking for ExchangeExec.

## How was this patch tested?
This is a simple rename and should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #19376 from rxin/SPARK-22153.
2017-09-28 09:20:37 -07:00
gatorsmile 9244957b50 [SPARK-22140] Add TPCDSQuerySuite
## What changes were proposed in this pull request?
Now, we are not running TPC-DS queries as regular test cases. Thus, we need to add a test suite using empty tables for ensuring the new code changes will not break them. For example, optimizer/analyzer batches should not exceed the max iteration.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19361 from gatorsmile/tpcdsQuerySuite.
2017-09-27 17:03:42 -07:00
Herman van Hovell 02bb0682e6 [SPARK-22143][SQL] Fix memory leak in OffHeapColumnVector
## What changes were proposed in this pull request?
`WriteableColumnVector` does not close its child column vectors. This can create memory leaks for `OffHeapColumnVector` where we do not clean up the memory allocated by a vectors children. This can be especially bad for string columns (which uses a child byte column vector).

## How was this patch tested?
I have updated the existing tests to always use both on-heap and off-heap vectors. Testing and diagnoses was done locally.

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

Closes #19367 from hvanhovell/SPARK-22143.
2017-09-27 23:08:30 +02:00
Takuya UESHIN 09cbf3df20 [SPARK-22125][PYSPARK][SQL] Enable Arrow Stream format for vectorized UDF.
## What changes were proposed in this pull request?

Currently we use Arrow File format to communicate with Python worker when invoking vectorized UDF but we can use Arrow Stream format.

This pr replaces the Arrow File format with the Arrow Stream format.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19349 from ueshin/issues/SPARK-22125.
2017-09-27 23:21:44 +09:00
guoxiaolong d2b8b63b93 [SAPRK-20785][WEB-UI][SQL] Spark should provide jump links and add (count) in the SQL web ui.
## What changes were proposed in this pull request?

propose:

it provide links that jump to Running Queries,Completed Queries and Failed Queries.
it add (count) about Running Queries,Completed Queries and Failed Queries.
This is a small optimization in in the SQL web ui.

fix before:

![1](https://user-images.githubusercontent.com/26266482/30840686-36025cc0-a2ab-11e7-8d8d-1de0122a84fb.png)

fix after:
![2](https://user-images.githubusercontent.com/26266482/30840723-6cc67a52-a2ab-11e7-8002-9191a55895a6.png)

## How was this patch tested?

manual tests

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

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #19346 from guoxiaolongzte/SPARK-20785.
2017-09-27 20:48:55 +08:00
Wang Gengliang 9c5935d00b [SPARK-22141][SQL] Propagate empty relation before checking Cartesian products
## What changes were proposed in this pull request?

When inferring constraints from children, Join's condition can be simplified as None.
For example,
```
val testRelation = LocalRelation('a.int)
val x = testRelation.as("x")
val y = testRelation.where($"a" === 2 && !($"a" === 2)).as("y")
x.join.where($"x.a" === $"y.a")
```
The plan will become
```
Join Inner
:- LocalRelation <empty>, [a#23]
+- LocalRelation <empty>, [a#224]
```
And the Cartesian products check will throw exception for above plan.

Propagate empty relation before checking Cartesian products, and the issue is resolved.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19362 from gengliangwang/MoveCheckCartesianProducts.
2017-09-27 12:44:10 +02:00
Juliusz Sompolski f21f6ce998 [SPARK-22103][FOLLOWUP] Rename addExtraCode to addInnerClass
## What changes were proposed in this pull request?

Address PR comments that appeared post-merge, to rename `addExtraCode` to `addInnerClass`,
and not count the size of the inner class to the size of the outer class.

## How was this patch tested?

YOLO.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19353 from juliuszsompolski/SPARK-22103followup.
2017-09-26 10:04:34 -07:00
Liang-Chi Hsieh 64fbd1cef3 [SPARK-22124][SQL] Sample and Limit should also defer input evaluation under codegen
## What changes were proposed in this pull request?

We can override `usedInputs` to claim that an operator defers input evaluation. `Sample` and `Limit` are two operators which should claim it but don't. We should do it.

## How was this patch tested?

Existing tests.

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

Closes #19345 from viirya/SPARK-22124.
2017-09-26 15:23:13 +08:00
Bryan Cutler d8e825e3bc [SPARK-22106][PYSPARK][SQL] Disable 0-parameter pandas_udf and add doctests
## What changes were proposed in this pull request?

This change disables the use of 0-parameter pandas_udfs due to the API being overly complex and awkward, and can easily be worked around by using an index column as an input argument.  Also added doctests for pandas_udfs which revealed bugs for handling empty partitions and using the pandas_udf decorator.

## How was this patch tested?

Reworked existing 0-parameter test to verify error is raised, added doctest for pandas_udf, added new tests for empty partition and decorator usage.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19325 from BryanCutler/arrow-pandas_udf-0-param-remove-SPARK-22106.
2017-09-26 10:54:00 +09:00
Greg Owen ce204780ee [SPARK-22120][SQL] TestHiveSparkSession.reset() should clean out Hive warehouse directory
## What changes were proposed in this pull request?
During TestHiveSparkSession.reset(), which is called after each TestHiveSingleton suite, we now delete and recreate the Hive warehouse directory.

## How was this patch tested?
Ran full suite of tests locally, verified that they pass.

Author: Greg Owen <greg@databricks.com>

Closes #19341 from GregOwen/SPARK-22120.
2017-09-25 14:16:11 -07:00
Juliusz Sompolski 038b185736 [SPARK-22103] Move HashAggregateExec parent consume to a separate function in codegen
## What changes were proposed in this pull request?

HashAggregateExec codegen uses two paths for fast hash table and a generic one.
It generates code paths for iterating over both, and both code paths generate the consume code of the parent operator, resulting in that code being expanded twice.
This leads to a long generated function that might be an issue for the compiler (see e.g. SPARK-21603).
I propose to remove the double expansion by generating the consume code in a helper function that can just be called from both iterating loops.

An issue with separating the `consume` code to a helper function was that a number of places relied and assumed on being in the scope of an outside `produce` loop and e.g. use `continue` to jump out.
I replaced such code flows with nested scopes. It is code that should be handled the same by compiler, while getting rid of depending on assumptions that are outside of the `consume`'s own scope.

## How was this patch tested?

Existing test coverage.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19324 from juliuszsompolski/aggrconsumecodegen.
2017-09-25 12:50:25 -07:00
Zhenhua Wang 365a29bdbf [SPARK-22100][SQL] Make percentile_approx support date/timestamp type and change the output type to be the same as input type
## What changes were proposed in this pull request?

The `percentile_approx` function previously accepted numeric type input and output double type results.

But since all numeric types, date and timestamp types are represented as numerics internally, `percentile_approx` can support them easily.

After this PR, it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles.

This change is also required when we generate equi-height histograms for these types.

## How was this patch tested?

Added a new test and modified some existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19321 from wzhfy/approx_percentile_support_types.
2017-09-25 09:28:42 -07:00
Sean Owen 576c43fb42 [SPARK-22087][SPARK-14650][WIP][BUILD][REPL][CORE] Compile Spark REPL for Scala 2.12 + other 2.12 fixes
## What changes were proposed in this pull request?

Enable Scala 2.12 REPL. Fix most remaining issues with 2.12 compilation and warnings, including:

- Selecting Kafka 0.10.1+ for Scala 2.12 and patching over a minor API difference
- Fixing lots of "eta expansion of zero arg method deprecated" warnings
- Resolving the SparkContext.sequenceFile implicits compile problem
- Fixing an odd but valid jetty-server missing dependency in hive-thriftserver

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19307 from srowen/Scala212.
2017-09-24 09:40:13 +01:00
hyukjinkwon 9d48bd0b34 [SPARK-22093][TESTS] Fixes assume in UtilsSuite and HiveDDLSuite
## What changes were proposed in this pull request?

This PR proposes to remove `assume` in `Utils.resolveURIs` and replace `assume` to `assert` in `Utils.resolveURI` in the test cases in `UtilsSuite`.

It looks `Utils.resolveURIs` supports multiple but also single paths as input. So, it looks not meaningful to check if the input has `,`.

For the test for `Utils.resolveURI`, I replaced it to `assert` because it looks taking single path and in order to prevent future mistakes when adding more tests here.

For `assume` in `HiveDDLSuite`, it looks it should be `assert` to test at the last
## How was this patch tested?

Fixed unit tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19332 from HyukjinKwon/SPARK-22093.
2017-09-24 17:11:29 +09:00
Liang-Chi Hsieh 2274d84efc [SPARK-21338][SQL][FOLLOW-UP] Implement isCascadingTruncateTable() method in AggregatedDialect
## What changes were proposed in this pull request?

The implemented `isCascadingTruncateTable` in `AggregatedDialect` is wrong. When no dialect claims cascading, once there is an unknown cascading truncate in the dialects, we should return unknown cascading, instead of false.

## How was this patch tested?

Added test.

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

Closes #19286 from viirya/SPARK-21338-followup.
2017-09-23 21:51:04 -07:00
Kevin Yu 4a8c9e29bc [SPARK-22110][SQL][DOCUMENTATION] Add usage and improve documentation with arguments and examples for trim function
## What changes were proposed in this pull request?

This PR proposes to enhance the documentation for `trim` functions in the function description session.

- Add more `usage`, `arguments` and `examples` for the trim function
- Adjust space in the `usage` session

After the changes, the trim function documentation will look like this:

- `trim`

```trim(str) - Removes the leading and trailing space characters from str.

trim(BOTH trimStr FROM str) - Remove the leading and trailing trimStr characters from str

trim(LEADING trimStr FROM str) - Remove the leading trimStr characters from str

trim(TRAILING trimStr FROM str) - Remove the trailing trimStr characters from str

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
BOTH, FROM - these are keywords to specify trimming string characters from both ends of the string
LEADING, FROM - these are keywords to specify trimming string characters from the left end of the string
TRAILING, FROM - these are keywords to specify trimming string characters from the right end of the string
Examples:

> SELECT trim('    SparkSQL   ');
 SparkSQL
> SELECT trim('SL', 'SSparkSQLS');
 parkSQ
> SELECT trim(BOTH 'SL' FROM 'SSparkSQLS');
 parkSQ
> SELECT trim(LEADING 'SL' FROM 'SSparkSQLS');
 parkSQLS
> SELECT trim(TRAILING 'SL' FROM 'SSparkSQLS');
 SSparkSQ
```

- `ltrim`

```ltrim

ltrim(str) - Removes the leading space characters from str.

ltrim(trimStr, str) - Removes the leading string contains the characters from the trim string

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
Examples:

> SELECT ltrim('    SparkSQL   ');
 SparkSQL
> SELECT ltrim('Sp', 'SSparkSQLS');
 arkSQLS
```

- `rtrim`
```rtrim

rtrim(str) - Removes the trailing space characters from str.

rtrim(trimStr, str) - Removes the trailing string which contains the characters from the trim string from the str

Arguments:

str - a string expression
trimStr - the trim string characters to trim, the default value is a single space
Examples:

> SELECT rtrim('    SparkSQL   ');
 SparkSQL
> SELECT rtrim('LQSa', 'SSparkSQLS');
 SSpark
```

This is the trim characters function jira: [trim function](https://issues.apache.org/jira/browse/SPARK-14878)

## How was this patch tested?

Manually tested
```
spark-sql> describe function extended trim;
17/09/22 17:03:04 INFO CodeGenerator: Code generated in 153.026533 ms
Function: trim
Class: org.apache.spark.sql.catalyst.expressions.StringTrim
Usage:
    trim(str) - Removes the leading and trailing space characters from `str`.

    trim(BOTH trimStr FROM str) - Remove the leading and trailing `trimStr` characters from `str`

    trim(LEADING trimStr FROM str) - Remove the leading `trimStr` characters from `str`

    trim(TRAILING trimStr FROM str) - Remove the trailing `trimStr` characters from `str`

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space
      * BOTH, FROM - these are keywords to specify trimming string characters from both ends of
          the string
      * LEADING, FROM - these are keywords to specify trimming string characters from the left
          end of the string
      * TRAILING, FROM - these are keywords to specify trimming string characters from the right
          end of the string

    Examples:
      > SELECT trim('    SparkSQL   ');
       SparkSQL
      > SELECT trim('SL', 'SSparkSQLS');
       parkSQ
      > SELECT trim(BOTH 'SL' FROM 'SSparkSQLS');
       parkSQ
      > SELECT trim(LEADING 'SL' FROM 'SSparkSQLS');
       parkSQLS
      > SELECT trim(TRAILING 'SL' FROM 'SSparkSQLS');
       SSparkSQ
  ```
```
spark-sql> describe function extended ltrim;
Function: ltrim
Class: org.apache.spark.sql.catalyst.expressions.StringTrimLeft
Usage:
    ltrim(str) - Removes the leading space characters from `str`.

    ltrim(trimStr, str) - Removes the leading string contains the characters from the trim string

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space

    Examples:
      > SELECT ltrim('    SparkSQL   ');
       SparkSQL
      > SELECT ltrim('Sp', 'SSparkSQLS');
       arkSQLS

```

```
spark-sql> describe function extended rtrim;
Function: rtrim
Class: org.apache.spark.sql.catalyst.expressions.StringTrimRight
Usage:
    rtrim(str) - Removes the trailing space characters from `str`.

    rtrim(trimStr, str) - Removes the trailing string which contains the characters from the trim string from the `str`

Extended Usage:
    Arguments:
      * str - a string expression
      * trimStr - the trim string characters to trim, the default value is a single space

    Examples:
      > SELECT rtrim('    SparkSQL   ');
       SparkSQL
      > SELECT rtrim('LQSa', 'SSparkSQLS');
       SSpark

```

Author: Kevin Yu <qyu@us.ibm.com>

Closes #19329 from kevinyu98/spark-14878-5.
2017-09-23 10:27:40 -07:00
hyukjinkwon 04975a68b5 [SPARK-22109][SQL] Resolves type conflicts between strings and timestamps in partition column
## What changes were proposed in this pull request?

This PR proposes to resolve the type conflicts in strings and timestamps in partition column values.
It looks we need to set the timezone as it needs a cast between strings and timestamps.

```scala
val df = Seq((1, "2015-01-01 00:00:00"), (2, "2014-01-01 00:00:00"), (3, "blah")).toDF("i", "str")
val path = "/tmp/test.parquet"
df.write.format("parquet").partitionBy("str").save(path)
spark.read.parquet(path).show()
```

**Before**

```
java.util.NoSuchElementException: None.get
  at scala.None$.get(Option.scala:347)
  at scala.None$.get(Option.scala:345)
  at org.apache.spark.sql.catalyst.expressions.TimeZoneAwareExpression$class.timeZone(datetimeExpressions.scala:46)
  at org.apache.spark.sql.catalyst.expressions.Cast.timeZone$lzycompute(Cast.scala:172)
  at org.apache.spark.sql.catalyst.expressions.Cast.timeZone(Cast.scala:172)
  at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3$$anonfun$apply$16.apply(Cast.scala:208)
  at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3$$anonfun$apply$16.apply(Cast.scala:208)
  at org.apache.spark.sql.catalyst.expressions.Cast.org$apache$spark$sql$catalyst$expressions$Cast$$buildCast(Cast.scala:201)
  at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3.apply(Cast.scala:207)
  at org.apache.spark.sql.catalyst.expressions.Cast.nullSafeEval(Cast.scala:533)
  at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:331)
  at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningUtils$$resolveTypeConflicts$1.apply(PartitioningUtils.scala:481)
  at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningUtils$$resolveTypeConflicts$1.apply(PartitioningUtils.scala:480)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
```

**After**

```
+---+-------------------+
|  i|                str|
+---+-------------------+
|  2|2014-01-01 00:00:00|
|  1|2015-01-01 00:00:00|
|  3|               blah|
+---+-------------------+
```

## How was this patch tested?

Unit tests added in `ParquetPartitionDiscoverySuite` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19331 from HyukjinKwon/SPARK-22109.
2017-09-24 00:05:17 +09:00
Sean Owen 50ada2a4d3 [SPARK-22033][CORE] BufferHolder, other size checks should account for the specific VM array size limitations
## What changes were proposed in this pull request?

Try to avoid allocating an array bigger than Integer.MAX_VALUE - 8, which is the actual max size on some JVMs, in several places

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19266 from srowen/SPARK-22033.
2017-09-23 15:40:59 +01:00
guoxiaolong 3920af7d1d [SPARK-22099] The 'job ids' list style needs to be changed in the SQL page.
## What changes were proposed in this pull request?

The 'job ids' list style needs to be changed in the SQL page. There are two reasons:
1. If a job id is a line, there are a lot of job ids, then the table row height will be high. As shown below:
![3](https://user-images.githubusercontent.com/26266482/30732242-2fb11442-9fa4-11e7-98ea-80a98f280243.png)

2. should be consistent with the 'JDBC / ODBC Server' page style, I am in this way to modify the style. As shown below:
![2](https://user-images.githubusercontent.com/26266482/30732257-3c550820-9fa4-11e7-9d8e-467d3011e0ac.png)

My changes are as follows:
![6](https://user-images.githubusercontent.com/26266482/30732318-8f61d8b8-9fa4-11e7-8af5-037ed12b13c9.png)

![5](https://user-images.githubusercontent.com/26266482/30732284-5b6a6c00-9fa4-11e7-8db9-3a2291f37ae6.png)

## How was this patch tested?
manual tests

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

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #19320 from guoxiaolongzte/SPARK-22099.
2017-09-23 15:39:53 +01:00
Ala Luszczak d2b2932d8b [SPARK-22092] Reallocation in OffHeapColumnVector.reserveInternal corrupts struct and array data
## What changes were proposed in this pull request?

`OffHeapColumnVector.reserveInternal()` will only copy already inserted values during reallocation if `data != null`. In vectors containing arrays or structs this is incorrect, since there field `data` is not used at all. We need to check `nulls` instead.

## How was this patch tested?

Adds new tests to `ColumnVectorSuite` that reproduce the errors.

Author: Ala Luszczak <ala@databricks.com>

Closes #19308 from ala/vector-realloc.
2017-09-22 15:31:43 +02:00
Bryan Cutler 27fc536d9a [SPARK-21190][PYSPARK] Python Vectorized UDFs
This PR adds vectorized UDFs to the Python API

**Proposed API**
Introduce a flag to turn on vectorization for a defined UDF, for example:

```
pandas_udf(DoubleType())
def plus(a, b)
    return a + b
```
or

```
plus = pandas_udf(lambda a, b: a + b, DoubleType())
```
Usage is the same as normal UDFs

0-parameter UDFs
pandas_udf functions can declare an optional `**kwargs` and when evaluated, will contain a key "size" that will give the required length of the output.  For example:

```
pandas_udf(LongType())
def f0(**kwargs):
    return pd.Series(1).repeat(kwargs["size"])

df.select(f0())
```

Added new unit tests in pyspark.sql that are enabled if pyarrow and Pandas are available.

- [x] Fix support for promoted types with null values
- [ ] Discuss 0-param UDF API (use of kwargs)
- [x] Add tests for chained UDFs
- [ ] Discuss behavior when pyarrow not installed / enabled
- [ ] Cleanup pydoc and add user docs

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18659 from BryanCutler/arrow-vectorized-udfs-SPARK-21404.
2017-09-22 16:17:50 +08:00
maryannxue 5960686e79 [SPARK-21998][SQL] SortMergeJoinExec did not calculate its outputOrdering correctly during physical planning
## What changes were proposed in this pull request?

Right now the calculation of SortMergeJoinExec's outputOrdering relies on the fact that its children have already been sorted on the join keys, while this is often not true until EnsureRequirements has been applied. So we ended up not getting the correct outputOrdering during physical planning stage before Sort nodes are added to the children.

For example, J = {A join B on key1 = key2}
1. if A is NOT ordered on key1 ASC, J's outputOrdering should include "key1 ASC"
2. if A is ordered on key1 ASC, J's outputOrdering should include "key1 ASC"
3. if A is ordered on key1 ASC, with sameOrderExp=c1, J's outputOrdering should include "key1 ASC, sameOrderExp=c1"

So to fix this I changed the  behavior of <code>getKeyOrdering(keys, childOutputOrdering)</code> to:
1. If the childOutputOrdering satisfies (is a superset of) the required child ordering => childOutputOrdering
2. Otherwise => required child ordering

In addition, I organized the logic for deciding the relationship between two orderings into SparkPlan, so that it can be reused by EnsureRequirements and SortMergeJoinExec, and potentially other classes.

## How was this patch tested?

Added new test cases.
Passed all integration tests.

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

Closes #19281 from maryannxue/spark-21998.
2017-09-21 23:54:16 -07:00
Shixiong Zhu fedf6961be [SPARK-22094][SS] processAllAvailable should check the query state
## What changes were proposed in this pull request?

`processAllAvailable` should also check the query state and if the query is stopped, it should return.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19314 from zsxwing/SPARK-22094.
2017-09-21 21:55:07 -07:00
Tathagata Das f32a842505 [SPARK-22053][SS] Stream-stream inner join in Append Mode
## What changes were proposed in this pull request?

#### Architecture
This PR implements stream-stream inner join using a two-way symmetric hash join. At a high level, we want to do the following.

1. For each stream, we maintain the past rows as state in State Store.
  - For each joining key, there can be multiple rows that have been received.
  - So, we have to effectively maintain a key-to-list-of-values multimap as state for each stream.
2. In each batch, for each input row in each stream
  - Look up the other streams state to see if there are matching rows, and output them if they satisfy the joining condition
  - Add the input row to corresponding stream’s state.
  - If the data has a timestamp/window column with watermark, then we will use that to calculate the threshold for keys that are required to buffered for future matches and drop the rest from the state.

Cleaning up old unnecessary state rows depends completely on whether watermark has been defined and what are join conditions. We definitely want to support state clean up two types of queries that are likely to be common.

- Queries to time range conditions - E.g. `SELECT * FROM leftTable, rightTable ON leftKey = rightKey AND leftTime > rightTime - INTERVAL 8 MINUTES AND leftTime < rightTime + INTERVAL 1 HOUR`
- Queries with windows as the matching key - E.g. `SELECT * FROM leftTable, rightTable ON leftKey = rightKey AND window(leftTime, "1 hour") = window(rightTime, "1 hour")` (pseudo-SQL)

#### Implementation
The stream-stream join is primarily implemented in three classes
- `StreamingSymmetricHashJoinExec` implements the above symmetric join algorithm.
- `SymmetricsHashJoinStateManagers` manages the streaming state for the join. This essentially is a fault-tolerant key-to-list-of-values multimap built on the StateStore APIs. `StreamingSymmetricHashJoinExec` instantiates two such managers, one for each join side.
- `StreamingSymmetricHashJoinExecHelper` is a helper class to extract threshold for the state based on the join conditions and the event watermark.

Refer to the scaladocs class for more implementation details.

Besides the implementation of stream-stream inner join SparkPlan. Some additional changes are
- Allowed inner join in append mode in UnsupportedOperationChecker
- Prevented stream-stream join on an empty batch dataframe to be collapsed by the optimizer

## How was this patch tested?
- New tests in StreamingJoinSuite
- Updated tests UnsupportedOperationSuite

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

Closes #19271 from tdas/SPARK-22053.
2017-09-21 15:39:07 -07:00
Zheng RuiFeng a8a5cd24e2 [SPARK-22009][ML] Using treeAggregate improve some algs
## What changes were proposed in this pull request?

I test on a dataset of about 13M instances, and found that using `treeAggregate` give a speedup in following algs:

|Algs| SpeedUp |
|------|-----------|
|OneHotEncoder| 5% |
|StatFunctions.calculateCov| 7% |
|StatFunctions.multipleApproxQuantiles|  9% |
|RegressionEvaluator| 8% |

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #19232 from zhengruifeng/use_treeAggregate.
2017-09-21 20:06:42 +01:00
Liang-Chi Hsieh 9cac249fd5 [SPARK-22088][SQL] Incorrect scalastyle comment causes wrong styles in stringExpressions
## What changes were proposed in this pull request?

There is an incorrect `scalastyle:on` comment in `stringExpressions.scala` and causes the line size limit check ineffective in the file. There are many lines of code and comment which are more than 100 chars.

## How was this patch tested?

Code style change only.

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

Closes #19305 from viirya/fix-wrong-style.
2017-09-21 11:51:00 -07:00
Sean Owen f10cbf17dc [SPARK-21977][HOTFIX] Adjust EnsureStatefulOpPartitioningSuite to use scalatest lifecycle normally instead of constructor
## What changes were proposed in this pull request?

Adjust EnsureStatefulOpPartitioningSuite to use scalatest lifecycle normally instead of constructor; fixes:

```
*** RUN ABORTED ***
  org.apache.spark.SparkException: Only one SparkContext may be running in this JVM (see SPARK-2243). To ignore this error, set spark.driver.allowMultipleContexts = true. The currently running SparkContext was created at:
org.apache.spark.sql.streaming.EnsureStatefulOpPartitioningSuite.<init>(EnsureStatefulOpPartitioningSuite.scala:35)
```

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19306 from srowen/SPARK-21977.2.
2017-09-21 18:00:19 +01:00
Liang-Chi Hsieh 1270e71753 [SPARK-22086][DOCS] Add expression description for CASE WHEN
## What changes were proposed in this pull request?

In SQL conditional expressions, only CASE WHEN lacks for expression description. This patch fills the gap.

## How was this patch tested?

Only documentation change.

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

Closes #19304 from viirya/casewhen-doc.
2017-09-21 22:45:06 +09:00
Zhenhua Wang 1d1a09be9f [SPARK-17997][SQL] Add an aggregation function for counting distinct values for multiple intervals
## What changes were proposed in this pull request?

This work is a part of [SPARK-17074](https://issues.apache.org/jira/browse/SPARK-17074) to compute equi-height histograms. Equi-height histogram is an array of bins. A bin consists of two endpoints which form an interval of values and the ndv in that interval.

This PR creates a new aggregate function, given an array of endpoints, counting distinct values (ndv) in intervals among those endpoints.

This PR also refactors `HyperLogLogPlusPlus` by extracting a helper class `HyperLogLogPlusPlusHelper`, where the underlying HLLPP algorithm locates.

## How was this patch tested?

Add new test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #15544 from wzhfy/countIntervals.
2017-09-21 21:43:02 +08:00
Wenchen Fan 352bea5457 [SPARK-22076][SQL][FOLLOWUP] Expand.projections should not be a Stream
## What changes were proposed in this pull request?

This a follow-up of https://github.com/apache/spark/pull/19289 , we missed another place: `rollup`. `Seq.init.toSeq` also returns a `Stream`, we should fix it too.

## How was this patch tested?

manually

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19298 from cloud-fan/bug.
2017-09-20 21:13:46 -07:00
Wenchen Fan ce6a71e013 [SPARK-22076][SQL] Expand.projections should not be a Stream
## What changes were proposed in this pull request?

Spark with Scala 2.10 fails with a group by cube:
```
spark.range(1).select($"id" as "a", $"id" as "b").write.partitionBy("a").mode("overwrite").saveAsTable("rollup_bug")
spark.sql("select 1 from rollup_bug group by rollup ()").show
```

It can be traced back to https://github.com/apache/spark/pull/15484 , which made `Expand.projections` a lazy `Stream` for group by cube.

In scala 2.10 `Stream` captures a lot of stuff, and in this case it captures the entire query plan which has some un-serializable parts.

This change is also good for master branch, to reduce the serialized size of `Expand.projections`.

## How was this patch tested?

manually verified with Spark with Scala 2.10.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19289 from cloud-fan/bug.
2017-09-20 09:00:43 -07:00
Sean Owen e17901d6df [SPARK-22049][DOCS] Confusing behavior of from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

Clarify behavior of to_utc_timestamp/from_utc_timestamp with an example

## How was this patch tested?

Doc only change / existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19276 from srowen/SPARK-22049.
2017-09-20 20:47:17 +09:00
Sean Owen 3d4dd14cd5 [SPARK-22066][BUILD] Update checkstyle to 8.2, enable it, fix violations
## What changes were proposed in this pull request?

Update plugins, including scala-maven-plugin, to latest versions. Update checkstyle to 8.2. Remove bogus checkstyle config and enable it. Fix existing and new Java checkstyle errors.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19282 from srowen/SPARK-22066.
2017-09-20 10:01:46 +01:00
Burak Yavuz 280ff523f4 [SPARK-21977] SinglePartition optimizations break certain Streaming Stateful Aggregation requirements
## What changes were proposed in this pull request?

This is a bit hard to explain as there are several issues here, I'll try my best. Here are the requirements:
  1. A StructuredStreaming Source that can generate empty RDDs with 0 partitions
  2. A StructuredStreaming query that uses the above source, performs a stateful aggregation
     (mapGroupsWithState, groupBy.count, ...), and coalesce's by 1

The crux of the problem is that when a dataset has a `coalesce(1)` call, it receives a `SinglePartition` partitioning scheme. This scheme satisfies most required distributions used for aggregations such as HashAggregateExec. This causes a world of problems:
  Symptom 1. If the input RDD has 0 partitions, the whole lineage will receive 0 partitions, nothing will be executed, the state store will not create any delta files. When this happens, the next trigger fails, because the StateStore fails to load the delta file for the previous trigger
  Symptom 2. Let's say that there was data. Then in this case, if you stop your stream, and change `coalesce(1)` with `coalesce(2)`, then restart your stream, your stream will fail, because `spark.sql.shuffle.partitions - 1` number of StateStores will fail to find its delta files.

To fix the issues above, we must check that the partitioning of the child of a `StatefulOperator` satisfies:
If the grouping expressions are empty:
  a) AllTuple distribution
  b) Single physical partition
If the grouping expressions are non empty:
  a) Clustered distribution
  b) spark.sql.shuffle.partition # of partitions
whether or not `coalesce(1)` exists in the plan, and whether or not the input RDD for the trigger has any data.

Once you fix the above problem by adding an Exchange to the plan, you come across the following bug:
If you call `coalesce(1).groupBy().count()` on a Streaming DataFrame, and if you have a trigger with no data, `StateStoreRestoreExec` doesn't return the prior state. However, for this specific aggregation, `HashAggregateExec` after the restore returns a (0, 0) row, since we're performing a count, and there is no data. Then this data gets stored in `StateStoreSaveExec` causing the previous counts to be overwritten and lost.

## How was this patch tested?

Regression tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #19196 from brkyvz/sa-0.
2017-09-20 00:01:21 -07:00
Marcelo Vanzin c6ff59a230 [SPARK-18838][CORE] Add separate listener queues to LiveListenerBus.
This change modifies the live listener bus so that all listeners are
added to queues; each queue has its own thread to dispatch events,
making it possible to separate slow listeners from other more
performance-sensitive ones.

The public API has not changed - all listeners added with the existing
"addListener" method, which after this change mostly means all
user-defined listeners, end up in a default queue. Internally, there's
an API allowing listeners to be added to specific queues, and that API
is used to separate the internal Spark listeners into 3 categories:
application status listeners (e.g. UI), executor management (e.g. dynamic
allocation), and the event log.

The queueing logic, while abstracted away in a separate class, is kept
as much as possible hidden away from consumers. Aside from choosing their
queue, there's no code change needed to take advantage of queues.

Test coverage relies on existing tests; a few tests had to be tweaked
because they relied on `LiveListenerBus.postToAll` being synchronous,
and the change makes that method asynchronous. Other tests were simplified
not to use the asynchronous LiveListenerBus.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19211 from vanzin/SPARK-18838.
2017-09-20 13:41:29 +08:00
Bryan Cutler 718bbc9390 [SPARK-22067][SQL] ArrowWriter should use position when setting UTF8String ByteBuffer
## What changes were proposed in this pull request?

The ArrowWriter StringWriter was setting Arrow data using a position of 0 instead of the actual position in the ByteBuffer.  This was currently working because of a bug ARROW-1443, and has been fixed as of
Arrow 0.7.0.  Testing with this version revealed the error in ArrowConvertersSuite test string conversion.

## How was this patch tested?

Existing tests, manually verified working with Arrow 0.7.0

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19284 from BryanCutler/arrow-ArrowWriter-StringWriter-position-SPARK-22067.
2017-09-20 10:51:00 +09:00
aokolnychyi ee13f3e3dc [SPARK-21969][SQL] CommandUtils.updateTableStats should call refreshTable
## What changes were proposed in this pull request?

Tables in the catalog cache are not invalidated once their statistics are updated. As a consequence, existing sessions will use the cached information even though it is not valid anymore. Consider and an example below.

```
// step 1
spark.range(100).write.saveAsTable("tab1")
// step 2
spark.sql("analyze table tab1 compute statistics")
// step 3
spark.sql("explain cost select distinct * from tab1").show(false)
// step 4
spark.range(100).write.mode("append").saveAsTable("tab1")
// step 5
spark.sql("explain cost select distinct * from tab1").show(false)
```

After step 3, the table will be present in the catalog relation cache. Step 4 will correctly update the metadata inside the catalog but will NOT invalidate the cache.

By the way, ``spark.sql("analyze table tab1 compute statistics")`` between step 3 and step 4 would also solve the problem.

## How was this patch tested?

Current and additional unit tests.

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

Closes #19252 from aokolnychyi/spark-21969.
2017-09-19 14:19:13 -07:00
Huaxin Gao d5aefa83ad [SPARK-21338][SQL] implement isCascadingTruncateTable() method in AggregatedDialect
## What changes were proposed in this pull request?

org.apache.spark.sql.jdbc.JdbcDialect's method:
def isCascadingTruncateTable(): Option[Boolean] = None
is not overriden in org.apache.spark.sql.jdbc.AggregatedDialect class.
Because of this issue, when you add more than one dialect Spark doesn't truncate table because isCascadingTruncateTable always returns default None for Aggregated Dialect.
Will implement isCascadingTruncateTable in AggregatedDialect class in this PR.

## How was this patch tested?

In JDBCSuite, inside test("Aggregated dialects"), will add one line to test AggregatedDialect.isCascadingTruncateTable

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

Closes #19256 from huaxingao/spark-21338.
2017-09-19 09:27:05 -07:00
Kent Yao 581200af71 [SPARK-21428][SQL][FOLLOWUP] CliSessionState should point to the actual metastore not a dummy one
## What changes were proposed in this pull request?

While running bin/spark-sql, we will reuse cliSessionState, but the Hive configurations generated here just points to a dummy meta store which actually should be the real one. And the warehouse is determined later in SharedState, HiveClient should respect this config changing in this case too.

## How was this patch tested?
existing ut

cc cloud-fan jiangxb1987

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19068 from yaooqinn/SPARK-21428-FOLLOWUP.
2017-09-19 19:35:36 +08:00
Taaffy 1bc17a6b8a [SPARK-22052] Incorrect Metric assigned in MetricsReporter.scala
Current implementation for processingRate-total uses wrong metric:
mistakenly uses inputRowsPerSecond instead of processedRowsPerSecond

## What changes were proposed in this pull request?
Adjust processingRate-total from using inputRowsPerSecond to processedRowsPerSecond

## How was this patch tested?

Built spark from source with proposed change and tested output with correct parameter. Before change the csv metrics file for inputRate-total and processingRate-total displayed the same values due to the error. After changing MetricsReporter.scala the processingRate-total csv file displayed the correct metric.
<img width="963" alt="processed rows per second" src="https://user-images.githubusercontent.com/32072374/30554340-82eea12c-9ca4-11e7-8370-8168526ff9a2.png">

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

Author: Taaffy <32072374+Taaffy@users.noreply.github.com>

Closes #19268 from Taaffy/patch-1.
2017-09-19 10:20:04 +01:00
Armin 7c92351f43 [MINOR][CORE] Cleanup dead code and duplication in Mem. Management
## What changes were proposed in this pull request?

* Removed the method `org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter#alignToWords`.
It became unused as a result of 85b0a15754
(SPARK-15962) introducing word alignment for unsafe arrays.
* Cleaned up duplicate code in memory management and unsafe sorters
  * The change extracting the exception paths is more than just cosmetics since it def. reduces the size the affected methods compile to

## How was this patch tested?

* Build still passes after removing the method, grepping the codebase for `alignToWords` shows no reference to it anywhere either.
* Dried up code is covered by existing tests.

Author: Armin <me@obrown.io>

Closes #19254 from original-brownbear/cleanup-mem-consumer.
2017-09-19 10:06:32 +01:00
Wenchen Fan 10f45b3c84 [SPARK-22047][FLAKY TEST] HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

This PR tries to download Spark for each test run, to make sure each test run is absolutely isolated.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19265 from cloud-fan/test.
2017-09-19 11:53:50 +08:00
Kevin Yu c66d64b3df [SPARK-14878][SQL] Trim characters string function support
#### What changes were proposed in this pull request?

This PR enhances the TRIM function support in Spark SQL by allowing the specification
of trim characters set. Below is the SQL syntax :

``` SQL
<trim function> ::= TRIM <left paren> <trim operands> <right paren>
<trim operands> ::= [ [ <trim specification> ] [ <trim character set> ] FROM ] <trim source>
<trim source> ::= <character value expression>
<trim specification> ::=
  LEADING
| TRAILING
| BOTH
<trim character set> ::= <characters value expression>
```
or
``` SQL
LTRIM (source-exp [, trim-exp])
RTRIM (source-exp [, trim-exp])
```

Here are the documentation link of support of this feature by other mainstream databases.
- **Oracle:** [TRIM function](http://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2126.htm#OLADM704)
- **DB2:** [TRIM scalar function](https://www.ibm.com/support/knowledgecenter/en/SSMKHH_10.0.0/com.ibm.etools.mft.doc/ak05270_.htm)
- **MySQL:** [Trim function](http://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim)
- **Oracle:** [ltrim](https://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2018.htm#OLADM594)
- **DB2:** [ltrim](https://www.ibm.com/support/knowledgecenter/en/SSEPEK_11.0.0/sqlref/src/tpc/db2z_bif_ltrim.html)

This PR is to implement the above enhancement. In the implementation, the design principle is to keep the changes to the minimum. Also, the exiting trim functions (which handles a special case, i.e., trimming space characters) are kept unchanged for performane reasons.
#### How was this patch tested?

The unit test cases are added in the following files:
- UTF8StringSuite.java
- StringExpressionsSuite.scala
- sql/SQLQuerySuite.scala
- StringFunctionsSuite.scala

Author: Kevin Yu <qyu@us.ibm.com>

Closes #12646 from kevinyu98/spark-14878.
2017-09-18 12:12:35 -07:00
Feng Liu 3b049abf10 [SPARK-22003][SQL] support array column in vectorized reader with UDF
## What changes were proposed in this pull request?

The UDF needs to deserialize the `UnsafeRow`. When the column type is Array, the `get` method from the `ColumnVector`, which is used by the vectorized reader, is called, but this method is not implemented.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #19230 from liufengdb/fix_array_open.
2017-09-18 08:49:32 -07:00
Wenchen Fan 894a7561de [SPARK-22047][TEST] ignore HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

As reported in https://issues.apache.org/jira/browse/SPARK-22047 , HiveExternalCatalogVersionsSuite is failing frequently, let's disable this test suite to unblock other PRs, I'm looking into the root cause.

## How was this patch tested?
N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19264 from cloud-fan/test.
2017-09-18 16:42:08 +08:00
Jose Torres 0bad10d3e3 [SPARK-22017] Take minimum of all watermark execs in StreamExecution.
## What changes were proposed in this pull request?

Take the minimum of all watermark exec nodes as the "real" watermark in StreamExecution, rather than picking one arbitrarily.

## How was this patch tested?

new unit test

Author: Jose Torres <jose@databricks.com>

Closes #19239 from joseph-torres/SPARK-22017.
2017-09-15 21:10:07 -07:00
Wenchen Fan c7307acdad [SPARK-15689][SQL] data source v2 read path
## What changes were proposed in this pull request?

This PR adds the infrastructure for data source v2, and implement features which Spark already have in data source v1, i.e. column pruning, filter push down, catalyst expression filter push down, InternalRow scan, schema inference, data size report. The write path is excluded to avoid making this PR growing too big, and will be added in follow-up PR.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19136 from cloud-fan/data-source-v2.
2017-09-15 22:18:36 +08:00
Wenchen Fan 3c6198c86e [SPARK-21987][SQL] fix a compatibility issue of sql event logs
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/18600 we removed the `metadata` field from `SparkPlanInfo`. This causes a problem when we replay event logs that are generated by older Spark versions.

## How was this patch tested?

a regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19237 from cloud-fan/event.
2017-09-15 00:47:44 -07:00
Yuming Wang 4decedfdbd [SPARK-22002][SQL] Read JDBC table use custom schema support specify partial fields.
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/18266 add a new feature to support read JDBC table use custom schema, but we must specify all the fields. For simplicity, this PR support  specify partial fields.

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #19231 from wangyum/SPARK-22002.
2017-09-14 23:35:55 -07:00
Tathagata Das 88661747f5 [SPARK-22018][SQL] Preserve top-level alias metadata when collapsing projects
## What changes were proposed in this pull request?
If there are two projects like as follows.
```
Project [a_with_metadata#27 AS b#26]
+- Project [a#0 AS a_with_metadata#27]
   +- LocalRelation <empty>, [a#0, b#1]
```
Child Project has an output column with a metadata in it, and the parent Project has an alias that implicitly forwards the metadata. So this metadata is visible for higher operators. Upon applying CollapseProject optimizer rule, the metadata is not preserved.
```
Project [a#0 AS b#26]
+- LocalRelation <empty>, [a#0, b#1]
```
This is incorrect, as downstream operators that expect certain metadata (e.g. watermark in structured streaming) to identify certain fields will fail to do so. This PR fixes it by preserving the metadata of top-level aliases.

## How was this patch tested?
New unit test

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

Closes #19240 from tdas/SPARK-22018.
2017-09-14 22:32:16 -07:00
goldmedal a28728a9af [SPARK-21513][SQL][FOLLOWUP] Allow UDF to_json support converting MapType to json for PySpark and SparkR
## What changes were proposed in this pull request?
In previous work SPARK-21513, we has allowed `MapType` and `ArrayType` of `MapType`s convert to a json string but only for Scala API. In this follow-up PR, we will make SparkSQL support it for PySpark and SparkR, too. We also fix some little bugs and comments of the previous work in this follow-up PR.

### For PySpark
```
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice")']
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
```
### For SparkR
```
# Converts a map into a JSON object
df2 <- sql("SELECT map('name', 'Bob')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
# Converts an array of maps into a JSON array
df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
```
## How was this patch tested?
Add unit test cases.

cc viirya HyukjinKwon

Author: goldmedal <liugs963@gmail.com>

Closes #19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
2017-09-15 11:53:10 +09:00
Jose Torres 054ddb2f54 [SPARK-21988] Add default stats to StreamingExecutionRelation.
## What changes were proposed in this pull request?

Add default stats to StreamingExecutionRelation.

## How was this patch tested?

existing unit tests and an explain() test to be sure

Author: Jose Torres <jose@databricks.com>

Closes #19212 from joseph-torres/SPARK-21988.
2017-09-14 11:06:25 -07:00
Zhenhua Wang ddd7f5e11d [SPARK-17642][SQL][FOLLOWUP] drop test tables and improve comments
## What changes were proposed in this pull request?

Drop test tables and improve comments.

## How was this patch tested?

Modified existing test.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19213 from wzhfy/useless_comment.
2017-09-14 23:14:21 +08:00
gatorsmile 4e6fc69014 [SPARK-4131][FOLLOW-UP] Support "Writing data into the filesystem from queries"
## What changes were proposed in this pull request?
This PR is clean the codes in https://github.com/apache/spark/pull/18975

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19225 from gatorsmile/refactorSPARK-4131.
2017-09-14 14:48:04 +08:00
Dilip Biswal dcbb229433 [MINOR][SQL] Only populate type metadata for required types such as CHAR/VARCHAR.
## What changes were proposed in this pull request?
When reading column descriptions from hive catalog, we currently populate the metadata for all types to record the raw hive type string. In terms of processing , we need this additional metadata information for CHAR/VARCHAR types or complex type containing the CHAR/VARCHAR types.

Its a minor cleanup. I haven't created a JIRA for it.

## How was this patch tested?
Test added in HiveMetastoreCatalogSuite

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #19215 from dilipbiswal/column_metadata.
2017-09-13 22:45:44 -07:00
Takeshi Yamamuro 8be7e6bb3c [SPARK-21973][SQL] Add an new option to filter queries in TPC-DS
## What changes were proposed in this pull request?
This pr added a new option to filter TPC-DS queries to run in `TPCDSQueryBenchmark`.
By default, `TPCDSQueryBenchmark` runs all the TPC-DS queries.
This change could enable developers to run some of the TPC-DS queries by this option,
e.g., to run q2, q4, and q6 only:
```
spark-submit --class <this class> --conf spark.sql.tpcds.queryFilter="q2,q4,q6" --jars <spark sql test jar>
```

## How was this patch tested?
Manually checked.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19188 from maropu/RunPartialQueriesInTPCDS.
2017-09-13 21:54:10 -07:00
Yuming Wang 17edfec59d [SPARK-20427][SQL] Read JDBC table use custom schema
## What changes were proposed in this pull request?

Auto generated Oracle schema some times not we expect:

- `number(1)` auto mapped to BooleanType, some times it's not we expect, per [SPARK-20921](https://issues.apache.org/jira/browse/SPARK-20921).
-  `number` auto mapped to Decimal(38,10), It can't read big data, per [SPARK-20427](https://issues.apache.org/jira/browse/SPARK-20427).

This PR fix this issue by custom schema as follows:
```scala
val props = new Properties()
props.put("customSchema", "ID decimal(38, 0), N1 int, N2 boolean")
val dfRead = spark.read.schema(schema).jdbc(jdbcUrl, "tableWithCustomSchema", props)
dfRead.show()
```
or
```sql
CREATE TEMPORARY VIEW tableWithCustomSchema
USING org.apache.spark.sql.jdbc
OPTIONS (url '$jdbcUrl', dbTable 'tableWithCustomSchema', customSchema'ID decimal(38, 0), N1 int, N2 boolean')
```

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18266 from wangyum/SPARK-20427.
2017-09-13 16:34:17 -07:00
Jane Wang 8c7e19a37d [SPARK-4131] Merge HiveTmpFile.scala to SaveAsHiveFile.scala
## What changes were proposed in this pull request?

The code is already merged to master:
https://github.com/apache/spark/pull/18975

This is a following up PR to merge HiveTmpFile.scala to SaveAsHiveFile.

## How was this patch tested?

Build successfully

Author: Jane Wang <janewang@fb.com>

Closes #19221 from janewangfb/merge_savehivefile_hivetmpfile.
2017-09-13 15:12:36 -07:00
donnyzone 21c4450fb2 [SPARK-21980][SQL] References in grouping functions should be indexed with semanticEquals
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-21980

This PR fixes the issue in ResolveGroupingAnalytics rule, which indexes the column references in grouping functions without considering case sensitive configurations.

The problem can be reproduced by:

`val df = spark.createDataFrame(Seq((1, 1), (2, 1), (2, 2))).toDF("a", "b")
 df.cube("a").agg(grouping("A")).show()`

## How was this patch tested?
unit tests

Author: donnyzone <wellfengzhu@gmail.com>

Closes #19202 from DonnyZone/ResolveGroupingAnalytics.
2017-09-13 10:06:53 -07:00
Armin b6ef1f57bc [SPARK-21970][CORE] Fix Redundant Throws Declarations in Java Codebase
## What changes were proposed in this pull request?

1. Removing all redundant throws declarations from Java codebase.
2. Removing dead code made visible by this from `ShuffleExternalSorter#closeAndGetSpills`

## How was this patch tested?

Build still passes.

Author: Armin <me@obrown.io>

Closes #19182 from original-brownbear/SPARK-21970.
2017-09-13 14:04:26 +01:00
goldmedal 371e4e2053 [SPARK-21513][SQL] Allow UDF to_json support converting MapType to json
# What changes were proposed in this pull request?
UDF to_json only supports converting `StructType` or `ArrayType` of `StructType`s to a json output string now.
According to the discussion of JIRA SPARK-21513, I allow to `to_json` support converting `MapType` and `ArrayType` of `MapType`s to a json output string.
This PR is for SQL and Scala API only.

# How was this patch tested?
Adding unit test case.

cc viirya HyukjinKwon

Author: goldmedal <liugs963@gmail.com>
Author: Jia-Xuan Liu <liugs963@gmail.com>

Closes #18875 from goldmedal/SPARK-21513.
2017-09-13 09:43:00 +09:00
Wang Gengliang 1a98574766 [SPARK-21979][SQL] Improve QueryPlanConstraints framework
## What changes were proposed in this pull request?

Improve QueryPlanConstraints framework, make it robust and simple.
In https://github.com/apache/spark/pull/15319, constraints for expressions like `a = f(b, c)` is resolved.
However, for expressions like
```scala
a = f(b, c) && c = g(a, b)
```
The current QueryPlanConstraints framework will produce non-converging constraints.
Essentially, the problem is caused by having both the name and child of aliases in the same constraint set.   We infer constraints, and push down constraints as predicates in filters, later on these predicates are propagated as constraints, etc..
Simply using the alias names only can resolve these problems.  The size of constraints is reduced without losing any information. We can always get these inferred constraints on child of aliases when pushing down filters.

Also, the EqualNullSafe between name and child in propagating alias is meaningless
```scala
allConstraints += EqualNullSafe(e, a.toAttribute)
```
It just produces redundant constraints.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19201 from gengliangwang/QueryPlanConstraints.
2017-09-12 13:02:29 -07:00
sarutak b9b54b1c88 [SPARK-21368][SQL] TPCDSQueryBenchmark can't refer query files.
## What changes were proposed in this pull request?

TPCDSQueryBenchmark packaged into a jar doesn't work with spark-submit.
It's because of the failure of reference query files in the jar file.

## How was this patch tested?

Ran the benchmark.

Author: sarutak <sarutak@oss.nttdata.co.jp>
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #18592 from sarutak/fix-tpcds-benchmark.
2017-09-12 10:49:46 -07:00
Zhenhua Wang 515910e9bd [SPARK-17642][SQL] support DESC EXTENDED/FORMATTED table column commands
## What changes were proposed in this pull request?

Support DESC (EXTENDED | FORMATTED) ? TABLE COLUMN command.
Support DESC EXTENDED | FORMATTED TABLE COLUMN command to show column-level statistics.
Do NOT support describe nested columns.

## How was this patch tested?

Added test cases.

Author: Zhenhua Wang <wzh_zju@163.com>
Author: Zhenhua Wang <wangzhenhua@huawei.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16422 from wzhfy/descColumn.
2017-09-12 08:59:52 -07:00
Jen-Ming Chung 7d0a3ef4ce [SPARK-21610][SQL][FOLLOWUP] Corrupt records are not handled properly when creating a dataframe from a file
## What changes were proposed in this pull request?

When the `requiredSchema` only contains `_corrupt_record`, the derived `actualSchema` is empty and the `_corrupt_record` are all null for all rows. This PR captures above situation and raise an exception with a reasonable workaround messag so that users can know what happened and how to fix the query.

## How was this patch tested?

Added unit test in `CSVSuite`.

Author: Jen-Ming Chung <jenmingisme@gmail.com>

Closes #19199 from jmchung/SPARK-21610-FOLLOWUP.
2017-09-12 22:47:12 +09:00
caoxuewen dc74c0e67d [MINOR][SQL] remove unuse import class
## What changes were proposed in this pull request?

this PR describe remove the import class that are unused.

## How was this patch tested?

N/A

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #19131 from heary-cao/unuse_import.
2017-09-11 10:09:20 +01:00
Jen-Ming Chung 6273a711b6 [SPARK-21610][SQL] Corrupt records are not handled properly when creating a dataframe from a file
## What changes were proposed in this pull request?
```
echo '{"field": 1}
{"field": 2}
{"field": "3"}' >/tmp/sample.json
```

```scala
import org.apache.spark.sql.types._

val schema = new StructType()
  .add("field", ByteType)
  .add("_corrupt_record", StringType)

val file = "/tmp/sample.json"

val dfFromFile = spark.read.schema(schema).json(file)

scala> dfFromFile.show(false)
+-----+---------------+
|field|_corrupt_record|
+-----+---------------+
|1    |null           |
|2    |null           |
|null |{"field": "3"} |
+-----+---------------+

scala> dfFromFile.filter($"_corrupt_record".isNotNull).count()
res1: Long = 0

scala> dfFromFile.filter($"_corrupt_record".isNull).count()
res2: Long = 3
```
When the `requiredSchema` only contains `_corrupt_record`, the derived `actualSchema` is empty and the `_corrupt_record` are all null for all rows. This PR captures above situation and raise an exception with a reasonable workaround messag so that users can know what happened and how to fix the query.

## How was this patch tested?

Added test case.

Author: Jen-Ming Chung <jenmingisme@gmail.com>

Closes #18865 from jmchung/SPARK-21610.
2017-09-10 17:26:43 -07:00
Jane Wang f76790557b [SPARK-4131] Support "Writing data into the filesystem from queries"
## What changes were proposed in this pull request?

This PR implements the sql feature:
INSERT OVERWRITE [LOCAL] DIRECTORY directory1
  [ROW FORMAT row_format] [STORED AS file_format]
  SELECT ... FROM ...

## How was this patch tested?
Added new unittests and also pulled the code to fb-spark so that we could test writing to hdfs directory.

Author: Jane Wang <janewang@fb.com>

Closes #18975 from janewangfb/port_local_directory.
2017-09-09 11:48:34 -07:00
Yanbo Liang e4d8f9a36a [MINOR][SQL] Correct DataFrame doc.
## What changes were proposed in this pull request?
Correct DataFrame doc.

## How was this patch tested?
Only doc change, no tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #19173 from yanboliang/df-doc.
2017-09-09 09:25:12 -07:00
Liang-Chi Hsieh 6b45d7e941 [SPARK-21954][SQL] JacksonUtils should verify MapType's value type instead of key type
## What changes were proposed in this pull request?

`JacksonUtils.verifySchema` verifies if a data type can be converted to JSON. For `MapType`, it now verifies the key type. However, in `JacksonGenerator`, when converting a map to JSON, we only care about its values and create a writer for the values. The keys in a map are treated as strings by calling `toString` on the keys.

Thus, we should change `JacksonUtils.verifySchema` to verify the value type of `MapType`.

## How was this patch tested?

Added tests.

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

Closes #19167 from viirya/test-jacksonutils.
2017-09-09 19:10:52 +09:00
Andrew Ash 8a5eb50681 [SPARK-21941] Stop storing unused attemptId in SQLTaskMetrics
## What changes were proposed in this pull request?

In a driver heap dump containing 390,105 instances of SQLTaskMetrics this
would have saved me approximately 3.2MB of memory.

Since we're not getting any benefit from storing this unused value, let's
eliminate it until a future PR makes use of it.

## How was this patch tested?

Existing unit tests

Author: Andrew Ash <andrew@andrewash.com>

Closes #19153 from ash211/aash/trim-sql-listener.
2017-09-08 23:33:15 -07:00
Kazuaki Ishizaki 8a4f228dc0 [SPARK-21946][TEST] fix flaky test: "alter table: rename cached table" in InMemoryCatalogedDDLSuite
## What changes were proposed in this pull request?

This PR fixes flaky test `InMemoryCatalogedDDLSuite "alter table: rename cached table"`.
Since this test validates distributed DataFrame, the result should be checked by using `checkAnswer`. The original version used `df.collect().Seq` method that does not guaranty an order of each element of the result.

## How was this patch tested?

Use existing test case

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

Closes #19159 from kiszk/SPARK-21946.
2017-09-08 09:39:20 -07:00
Liang-Chi Hsieh 0dfc1ec59e [SPARK-21726][SQL][FOLLOW-UP] Check for structural integrity of the plan in Optimzer in test mode
## What changes were proposed in this pull request?

The condition in `Optimizer.isPlanIntegral` is wrong. We should always return `true` if not in test mode.

## How was this patch tested?

Manually test.

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

Closes #19161 from viirya/SPARK-21726-followup.
2017-09-08 20:21:37 +09:00
Wenchen Fan dbb824125d [SPARK-21936][SQL] backward compatibility test framework for HiveExternalCatalog
## What changes were proposed in this pull request?

`HiveExternalCatalog` is a semi-public interface. When creating tables, `HiveExternalCatalog` converts the table metadata to hive table format and save into hive metastore. It's very import to guarantee backward compatibility here, i.e., tables created by previous Spark versions should still be readable in newer Spark versions.

Previously we find backward compatibility issues manually, which is really easy to miss bugs. This PR introduces a test framework to automatically test `HiveExternalCatalog` backward compatibility, by downloading Spark binaries with different versions, and create tables with these Spark versions, and read these tables with current Spark version.

## How was this patch tested?

test-only change

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19148 from cloud-fan/test.
2017-09-07 23:21:49 -07:00
Liang-Chi Hsieh 6e37524a1f [SPARK-21726][SQL] Check for structural integrity of the plan in Optimzer in test mode.
## What changes were proposed in this pull request?

We have many optimization rules now in `Optimzer`. Right now we don't have any checks in the optimizer to check for the structural integrity of the plan (e.g. resolved). When debugging, it is difficult to identify which rules return invalid plans.

It would be great if in test mode, we can check whether a plan is still resolved after the execution of each rule, so we can catch rules that return invalid plans.

## How was this patch tested?

Added tests.

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

Closes #18956 from viirya/SPARK-21726.
2017-09-07 23:12:18 -07:00
liuxian f62b20f39c [SPARK-21949][TEST] Tables created in unit tests should be dropped after use
## What changes were proposed in this pull request?
 Tables should be dropped after use in unit tests.
## How was this patch tested?
N/A

Author: liuxian <liu.xian3@zte.com.cn>

Closes #19155 from 10110346/droptable.
2017-09-07 23:09:26 -07:00
Dongjoon Hyun c26976fe14 [SPARK-21939][TEST] Use TimeLimits instead of Timeouts
Since ScalaTest 3.0.0, `org.scalatest.concurrent.Timeouts` is deprecated.
This PR replaces the deprecated one with `org.scalatest.concurrent.TimeLimits`.

```scala
-import org.scalatest.concurrent.Timeouts._
+import org.scalatest.concurrent.TimeLimits._
```

Pass the existing test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19150 from dongjoon-hyun/SPARK-21939.

Change-Id: I1a1b07f1b97e51e2263dfb34b7eaaa099b2ded5e
2017-09-08 09:31:13 +08:00
Dongjoon Hyun e00f1a1da1 [SPARK-13656][SQL] Delete spark.sql.parquet.cacheMetadata from SQLConf and docs
## What changes were proposed in this pull request?

Since [SPARK-15639](https://github.com/apache/spark/pull/13701), `spark.sql.parquet.cacheMetadata` and `PARQUET_CACHE_METADATA` is not used. This PR removes from SQLConf and docs.

## How was this patch tested?

Pass the existing Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19129 from dongjoon-hyun/SPARK-13656.
2017-09-07 16:26:56 -07:00
Dongjoon Hyun eea2b877cf [SPARK-21912][SQL] ORC/Parquet table should not create invalid column names
## What changes were proposed in this pull request?

Currently, users meet job abortions while creating or altering ORC/Parquet tables with invalid column names. We had better prevent this by raising **AnalysisException** with a guide to use aliases instead like Paquet data source tables.

**BEFORE**
```scala
scala> sql("CREATE TABLE orc1 USING ORC AS SELECT 1 `a b`")
17/09/04 13:28:21 ERROR Utils: Aborting task
java.lang.IllegalArgumentException: Error: : expected at the position 8 of 'struct<a b:int>' but ' ' is found.
17/09/04 13:28:21 ERROR FileFormatWriter: Job job_20170904132821_0001 aborted.
17/09/04 13:28:21 ERROR Executor: Exception in task 0.0 in stage 1.0 (TID 1)
org.apache.spark.SparkException: Task failed while writing rows.
```

**AFTER**
```scala
scala> sql("CREATE TABLE orc1 USING ORC AS SELECT 1 `a b`")
17/09/04 13:27:40 ERROR CreateDataSourceTableAsSelectCommand: Failed to write to table orc1
org.apache.spark.sql.AnalysisException: Attribute name "a b" contains invalid character(s) among " ,;{}()\n\t=". Please use alias to rename it.;
```

## How was this patch tested?

Pass the Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19124 from dongjoon-hyun/SPARK-21912.
2017-09-06 22:20:48 -07:00
Liang-Chi Hsieh ce7293c150 [SPARK-21835][SQL][FOLLOW-UP] RewritePredicateSubquery should not produce unresolved query plans
## What changes were proposed in this pull request?

This is a follow-up of #19050 to deal with `ExistenceJoin` case.

## How was this patch tested?

Added test.

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

Closes #19151 from viirya/SPARK-21835-followup.
2017-09-06 22:15:25 -07:00
Jacek Laskowski fa0092bddf [SPARK-21901][SS] Define toString for StateOperatorProgress
## What changes were proposed in this pull request?

Just `StateOperatorProgress.toString` + few formatting fixes

## How was this patch tested?

Local build. Waiting for OK from Jenkins.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #19112 from jaceklaskowski/SPARK-21901-StateOperatorProgress-toString.
2017-09-06 15:48:48 -07:00