Commit graph

3583 commits

Author SHA1 Message Date
gatorsmile 9c2a376e41 [SPARK-15297][SQL] Fix Set -V Command
#### What changes were proposed in this pull request?
The command `SET -v` always outputs the default values even if we set the parameter. This behavior is incorrect. Instead, if users override it, we should output the user-specified value.

In addition, the output schema of `SET -v` is wrong. We should use the column `value` instead of `default` for the parameter value.

This PR is to fix the above two issues.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13081 from gatorsmile/setVcommand.
2016-05-19 10:05:53 +08:00
Wenchen Fan ebfe3a1f2c [SPARK-15192][SQL] null check for SparkSession.createDataFrame
## What changes were proposed in this pull request?

This PR adds null check in `SparkSession.createDataFrame`, so that we can make sure the passed in rows matches the given schema.

## How was this patch tested?

new tests in `DatasetSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13008 from cloud-fan/row-encoder.
2016-05-18 18:06:38 -07:00
Jurriaan Pruis 32be51fba4 [SPARK-15323][SPARK-14463][SQL] Fix reading of partitioned format=text datasets
https://issues.apache.org/jira/browse/SPARK-15323

I was using partitioned text datasets in Spark 1.6.1 but it broke in Spark 2.0.0.

It would be logical if you could also write those,
but not entirely sure how to solve this with the new DataSet implementation.

Also it doesn't work using `sqlContext.read.text`, since that method returns a `DataSet[String]`.
See https://issues.apache.org/jira/browse/SPARK-14463 for that issue.

Author: Jurriaan Pruis <email@jurriaanpruis.nl>

Closes #13104 from jurriaan/fix-partitioned-text-reads.
2016-05-18 16:15:09 -07:00
Davies Liu 84b23453dd Revert "[SPARK-15392][SQL] fix default value of size estimation of logical plan"
This reverts commit fc29b896da.
2016-05-18 16:02:52 -07:00
Davies Liu fc29b896da [SPARK-15392][SQL] fix default value of size estimation of logical plan
## What changes were proposed in this pull request?

We use  autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD.

This PR change the default value to Long.MaxValue.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #13179 from davies/fix_default_size.
2016-05-18 15:45:59 -07:00
Davies Liu 8fb1d1c7f3 [SPARK-15357] Cooperative spilling should check consumer memory mode
## What changes were proposed in this pull request?

Since we support forced spilling for Spillable, which only works in OnHeap mode, different from other SQL operators (could be OnHeap or OffHeap), we should considering the mode of consumer before calling trigger forced spilling.

## How was this patch tested?

Add new test.

Author: Davies Liu <davies@databricks.com>

Closes #13151 from davies/fix_mode.
2016-05-18 09:44:21 -07:00
Cheng Lian c4a45fd855 [SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10
## What changes were proposed in this pull request?

This PR fixes a Scala 2.10 compilation failure introduced in PR #13127.

## How was this patch tested?

Jenkins build.

Author: Cheng Lian <lian@databricks.com>

Closes #13166 from liancheng/hotfix-for-scala-2.10.
2016-05-18 18:58:24 +08:00
Dongjoon Hyun d2f81df1ba [MINOR][SQL] Remove unused pattern matching variables in Optimizers.
## What changes were proposed in this pull request?

This PR removes unused pattern matching variable in Optimizers in order to improve readability.

## How was this patch tested?

Pass the existing Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13145 from dongjoon-hyun/remove_unused_pattern_matching_variables.
2016-05-18 11:51:50 +01:00
WeichenXu 2f9047b5eb [SPARK-15322][MLLIB][CORE][SQL] update deprecate accumulator usage into accumulatorV2 in spark project
## What changes were proposed in this pull request?

I use Intellj-IDEA to search usage of deprecate SparkContext.accumulator in the whole spark project, and update the code.(except those test code for accumulator method itself)

## How was this patch tested?

Exisiting unit tests

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #13112 from WeichenXu123/update_accuV2_in_mllib.
2016-05-18 11:48:46 +01:00
Davies Liu 33814f887a [SPARK-15307][SQL] speed up listing files for data source
## What changes were proposed in this pull request?

Currently, listing files is very slow if there is thousands files, especially on local file system, because:
1) FileStatus.getPermission() is very slow on local file system, which is launch a subprocess and parse the stdout.
2) Create an JobConf is very expensive (ClassUtil.findContainingJar() is slow).

This PR improve these by:
1) Use another constructor of LocatedFileStatus to avoid calling FileStatus.getPermission, the permissions are not used for data sources.
2) Only create an JobConf once within one task.

## How was this patch tested?

Manually tests on a partitioned table with 1828 partitions, decrease the time to load the table from 22 seconds to 1.6 seconds (Most of time are spent in merging schema now).

Author: Davies Liu <davies@databricks.com>

Closes #13094 from davies/listing.
2016-05-18 18:46:57 +08:00
Sean Zhong 6e02aec44b [SPARK-15334][SQL] HiveClient facade not compatible with Hive 0.12
## What changes were proposed in this pull request?

HiveClient facade is not compatible with Hive 0.12.

This PR Fixes the following compatibility issues:
1. `org.apache.spark.sql.hive.client.HiveClientImpl` use `AddPartitionDesc(db, table, ignoreIfExists)` to create partitions, however, Hive 0.12 doesn't have this constructor for `AddPartitionDesc`.
2. `HiveClientImpl` uses `PartitionDropOptions` when dropping partition, however, class `PartitionDropOptions` doesn't exist in Hive 0.12.
3. Hive 0.12 doesn't support adding permanent functions. It is not valid to call `org.apache.hadoop.hive.ql.metadata.Hive.createFunction`, `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction`, and `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction`
4. `org.apache.spark.sql.hive.client.VersionsSuite` doesn't have enough test coverage for different hive versions 0.12, 0.13, 0.14, 1.0.0, 1.1.0, 1.2.0.

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13127 from clockfly/versionSuite.
2016-05-18 16:00:02 +08:00
Yin Huai 2a5db9c140 [SPARK-14346] Fix scala-2.10 build
## What changes were proposed in this pull request?
Scala 2.10 build was broken by #13079. I am reverting the change of that line.

Author: Yin Huai <yhuai@databricks.com>

Closes #13157 from yhuai/SPARK-14346-fix-scala2.10.
2016-05-17 18:02:31 -07:00
Sean Zhong 25b315e6ca [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable
## What changes were proposed in this pull request?

Update the unit test code, examples, and documents to remove calls to deprecated method `dataset.registerTempTable`.

## How was this patch tested?

This PR only changes the unit test code, examples, and comments. It should be safe.
This is a follow up of PR https://github.com/apache/spark/pull/12945 which was merged.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13098 from clockfly/spark-15171-remove-deprecation.
2016-05-18 09:01:59 +08:00
Cheng Lian b674e67c22 [SPARK-14346][SQL] Native SHOW CREATE TABLE for Hive tables/views
## What changes were proposed in this pull request?

This is a follow-up of #12781. It adds native `SHOW CREATE TABLE` support for Hive tables and views. A new field `hasUnsupportedFeatures` is added to `CatalogTable` to indicate whether all table metadata retrieved from the concrete underlying external catalog (i.e. Hive metastore in this case) can be mapped to fields in `CatalogTable`. This flag is useful when the target Hive table contains structures that can't be handled by Spark SQL, e.g., skewed columns and storage handler, etc..

## How was this patch tested?

New test cases are added in `ShowCreateTableSuite` to do round-trip tests.

Author: Cheng Lian <lian@databricks.com>

Closes #13079 from liancheng/spark-14346-show-create-table-for-hive-tables.
2016-05-17 15:56:44 -07:00
Shixiong Zhu 8e8bc9f957 [SPARK-11735][CORE][SQL] Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped
## What changes were proposed in this pull request?

Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped.

## How was this patch tested?

Jenkins unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13154 from zsxwing/check-spark-context-stop.
2016-05-17 14:57:21 -07:00
Dongjoon Hyun 9f176dd391 [MINOR][DOCS] Replace remaining 'sqlContext' in ScalaDoc/JavaDoc.
## What changes were proposed in this pull request?

According to the recent change, this PR replaces all the remaining `sqlContext` usage with `spark` in ScalaDoc/JavaDoc (.scala/.java files) except `SQLContext.scala`, `SparkPlan.scala', and `DatasetHolder.scala`.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13125 from dongjoon-hyun/minor_doc_sparksession.
2016-05-17 20:50:22 +02:00
hyukjinkwon 8d05a7a98b [SPARK-10216][SQL] Avoid creating empty files during overwriting with group by query
## What changes were proposed in this pull request?

Currently, `INSERT INTO` with `GROUP BY` query tries to make at least 200 files (default value of `spark.sql.shuffle.partition`), which results in lots of empty files.

This PR makes it avoid creating empty files during overwriting into Hive table and in internal data sources  with group by query.

This checks whether the given partition has data in it or not and creates/writes file only when it actually has data.

## How was this patch tested?

Unittests in `InsertIntoHiveTableSuite` and `HadoopFsRelationTest`.

Closes #8411

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Keuntae Park <sirpkt@apache.org>

Closes #12855 from HyukjinKwon/pr/8411.
2016-05-17 11:18:51 -07:00
Wenchen Fan 20a89478e1 [SPARK-14346][SQL][FOLLOW-UP] add tests for CREAT TABLE USING with partition and bucket
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/12781 introduced PARTITIONED BY, CLUSTERED BY, and SORTED BY keywords to CREATE TABLE USING. This PR adds tests to make sure those keywords are handled correctly.

This PR also fixes a mistake that we should create non-hive-compatible table if partition or bucket info exists.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13144 from cloud-fan/add-test.
2016-05-17 10:12:51 -07:00
Kousuke Saruta c0c3ec3547 [SPARK-15165] [SQL] Codegen can break because toCommentSafeString is not actually safe
## What changes were proposed in this pull request?

toCommentSafeString method replaces "\u" with "\\\\u" to avoid codegen breaking.
But if the even number of "\" is put before "u", like "\\\\u", in the string literal in the query, codegen can break.

Following code causes compilation error.

```
val df = Seq(...).toDF
df.select("'\\\\\\\\u002A/'").show
```

The reason of the compilation error is because "\\\\\\\\\\\\\\\\u002A/" is translated into "*/" (the end of comment).

Due to this unsafety, arbitrary code can be injected like as follows.

```
val df = Seq(...).toDF
// Inject "System.exit(1)"
df.select("'\\\\\\\\u002A/{System.exit(1);}/*'").show
```

## How was this patch tested?

Added new test cases.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Author: sarutak <sarutak@oss.nttdata.co.jp>

Closes #12939 from sarutak/SPARK-15165.
2016-05-17 10:07:01 -07:00
Wenchen Fan c36ca651f9 [SPARK-15351][SQL] RowEncoder should support array as the external type for ArrayType
## What changes were proposed in this pull request?

This PR improves `RowEncoder` and `MapObjects`, to support array as the external type for `ArrayType`. The idea is straightforward, we use `Object` as the external input type for `ArrayType`, and determine its type at runtime in `MapObjects`.

## How was this patch tested?

new test in `RowEncoderSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13138 from cloud-fan/map-object.
2016-05-17 17:02:52 +08:00
Sean Owen 122302cbf5 [SPARK-15290][BUILD] Move annotations, like @Since / @DeveloperApi, into spark-tags
## What changes were proposed in this pull request?

(See https://github.com/apache/spark/pull/12416 where most of this was already reviewed and committed; this is just the module structure and move part. This change does not move the annotations into test scope, which was the apparently problem last time.)

Rename `spark-test-tags` -> `spark-tags`; move common annotations like `Since` to `spark-tags`

## How was this patch tested?

Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #13074 from srowen/SPARK-15290.
2016-05-17 09:55:53 +01:00
Liwei Lin 95f4fbae52 [SPARK-14942][SQL][STREAMING] Reduce delay between batch construction and execution
## Problem

Currently in `StreamExecution`, [we first run the batch, then construct the next](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L165):
```scala
if (dataAvailable) runBatch()
constructNextBatch()
```

This is good when we run batches ASAP, where data would get processed in the **very next batch**:

![1](https://cloud.githubusercontent.com/assets/15843379/14779964/2786e698-0b0d-11e6-9d2c-bb41513488b2.png)

However, when we run batches at trigger like `ProcessTime("1 minute")`, data - such as _y_ below - may not get processed in the very next batch i.e. _batch 1_, but in _batch 2_:

![2](https://cloud.githubusercontent.com/assets/15843379/14779818/6f3bb064-0b0c-11e6-9f16-c1ce4897186b.png)

## What changes were proposed in this pull request?

This patch reverses the order of `constructNextBatch()` and `runBatch()`. After this patch, data would get processed in the **very next batch**, i.e. _batch 1_:

![3](https://cloud.githubusercontent.com/assets/15843379/14779816/6f36ee62-0b0c-11e6-9e53-bc8397fade18.png)

In addition, this patch alters when we do `currentBatchId += 1`: let's do that when the processing of the current batch's data is completed, so we won't bother passing `currentBatchId + 1` or  `currentBatchId - 1` to states or sinks.

## How was this patch tested?

New added test case. Also this should be covered by existing test suits, e.g. stress tests and others.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #12725 from lw-lin/construct-before-run-3.
2016-05-16 12:59:55 -07:00
Sean Zhong 4a5ee1954a [SPARK-15253][SQL] Support old table schema config key "spark.sql.sources.schema" for DESCRIBE TABLE
## What changes were proposed in this pull request?

"DESCRIBE table" is broken when table schema is stored at key "spark.sql.sources.schema".

Originally, we used spark.sql.sources.schema to store the schema of a data source table.
After SPARK-6024, we removed this flag. Although we are not using spark.sql.sources.schema any more, we need to still support it.

## How was this patch tested?

Unit test.

When using spark2.0 to load a table generated by spark 1.2.
Before change:
`DESCRIBE table` => Schema of this table is inferred at runtime,,

After change:
`DESCRIBE table` => correct output.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13073 from clockfly/spark-15253.
2016-05-16 10:41:20 +08:00
Zheng RuiFeng c7efc56c7b [MINOR] Fix Typos
## What changes were proposed in this pull request?
1,Rename matrix args in BreezeUtil to upper to match the doc
2,Fix several typos in ML and SQL

## How was this patch tested?
manual tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #13078 from zhengruifeng/fix_ann.
2016-05-15 15:59:49 +01:00
Sean Owen f5576a052d [SPARK-12972][CORE] Update org.apache.httpcomponents.httpclient
## What changes were proposed in this pull request?

(Retry of https://github.com/apache/spark/pull/13049)

- update to httpclient 4.5 / httpcore 4.4
- remove some defunct exclusions
- manage httpmime version to match
- update selenium / httpunit to support 4.5 (possible now that Jetty 9 is used)

## How was this patch tested?

Jenkins tests. Also, locally running the same test command of one Jenkins profile that failed: `mvn -Phadoop-2.6 -Pyarn -Phive -Phive-thriftserver -Pkinesis-asl ...`

Author: Sean Owen <sowen@cloudera.com>

Closes #13117 from srowen/SPARK-12972.2.
2016-05-15 15:56:46 +01:00
Tejas Patil 4210e2a6b7 [TRIVIAL] Add () to SparkSession's builder function
## What changes were proposed in this pull request?

Was trying out `SparkSession` for the first time and the given class doc (when copied as is) did not work over Spark shell:

```
scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate()
<console>:27: error: org.apache.spark.sql.SparkSession.Builder does not take parameters
       SparkSession.builder().master("local").appName("Word Count").getOrCreate()
```

Adding () to the builder method in SparkSession.

## How was this patch tested?

```
scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate()
res0: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38

scala> SparkSession.builder.master("local").appName("Word Count").getOrCreate()
res1: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38
```

Author: Tejas Patil <tejasp@fb.com>

Closes #13086 from tejasapatil/doc_correction.
2016-05-13 18:10:22 -07:00
hyukjinkwon 3ded5bc4db [SPARK-15267][SQL] Refactor options for JDBC and ORC data sources and change default compression for ORC
## What changes were proposed in this pull request?

Currently, Parquet, JSON and CSV data sources have a class for thier options, (`ParquetOptions`, `JSONOptions` and `CSVOptions`).

It is convenient to manage options for sources to gather options into a class. Currently, `JDBC`, `Text`, `libsvm` and `ORC` datasources do not have this class. This might be nicer if these options are in a unified format so that options can be added and

This PR refactors the options in Spark internal data sources adding new classes, `OrcOptions`, `TextOptions`, `JDBCOptions` and `LibSVMOptions`.

Also, this PR change the default compression codec for ORC from `NONE` to `SNAPPY`.

## How was this patch tested?

Existing tests should cover this for refactoring and unittests in `OrcHadoopFsRelationSuite` for changing the default compression codec for ORC.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13048 from HyukjinKwon/SPARK-15267.
2016-05-13 09:04:37 -07:00
Reynold Xin e1dc853737 [SPARK-15310][SQL] Rename HiveTypeCoercion -> TypeCoercion
## What changes were proposed in this pull request?
We originally designed the type coercion rules to match Hive, but over time we have diverged. It does not make sense to call it HiveTypeCoercion anymore. This patch renames it TypeCoercion.

## How was this patch tested?
Updated unit tests to reflect the rename.

Author: Reynold Xin <rxin@databricks.com>

Closes #13091 from rxin/SPARK-15310.
2016-05-13 00:15:39 -07:00
hyukjinkwon 51841d77d9 [SPARK-13866] [SQL] Handle decimal type in CSV inference at CSV data source.
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-13866

This PR adds the support to infer `DecimalType`.
Here are the rules between `IntegerType`, `LongType` and `DecimalType`.

#### Infering Types

1. `IntegerType` and then `LongType`are tried first.

  ```scala
  Int.MaxValue => IntegerType
  Long.MaxValue => LongType
  ```

2. If it fails, try `DecimalType`.

  ```scala
  (Long.MaxValue + 1) => DecimalType(20, 0)
  ```
  This does not try to infer this as `DecimalType` when scale is less than 0.

3. if it fails, try `DoubleType`
  ```scala
  0.1 => DoubleType // This is failed to be inferred as `DecimalType` because it has the scale, 1.
  ```

#### Compatible Types (Merging Types)

For merging types, this is the same with JSON data source. If `DecimalType` is not capable, then it becomes `DoubleType`

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for code style test.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #11724 from HyukjinKwon/SPARK-13866.
2016-05-12 22:31:14 -07:00
Reynold Xin eda2800d44 [SPARK-14541][SQL] Support IFNULL, NULLIF, NVL and NVL2
## What changes were proposed in this pull request?
This patch adds support for a few SQL functions to improve compatibility with other databases: IFNULL, NULLIF, NVL and NVL2. In order to do this, this patch introduced a RuntimeReplaceable expression trait that allows replacing an unevaluable expression in the optimizer before evaluation.

Note that the semantics are not completely identical to other databases in esoteric cases.

## How was this patch tested?
Added a new test suite SQLCompatibilityFunctionSuite.

Closes #12373.

Author: Reynold Xin <rxin@databricks.com>

Closes #13084 from rxin/SPARK-14541.
2016-05-12 22:18:39 -07:00
Reynold Xin ba169c3230 [SPARK-15306][SQL] Move object expressions into expressions.objects package
## What changes were proposed in this pull request?
This patch moves all the object related expressions into expressions.objects package, for better code organization.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #13085 from rxin/SPARK-15306.
2016-05-12 21:35:14 -07:00
Herman van Hovell bb1362eb3b [SPARK-10605][SQL] Create native collect_list/collect_set aggregates
## What changes were proposed in this pull request?
We currently use the Hive implementations for the collect_list/collect_set aggregate functions. This has a few major drawbacks: the use of HiveUDAF (which has quite a bit of overhead) and the lack of support for struct datatypes. This PR adds native implementation of these functions to Spark.

The size of the collected list/set may vary, this means we cannot use the fast, Tungsten, aggregation path to perform the aggregation, and that we fallback to the slower sort based path. Another big issue with these operators is that when the size of the collected list/set grows too large, we can start experiencing large GC pauzes and OOMEs.

This `collect*` aggregates implemented in this PR rely on the sort based aggregate path for correctness. They maintain their own internal buffer which holds the rows for one group at a time. The sortbased aggregation path is triggered by disabling `partialAggregation` for these aggregates (which is kinda funny); this technique is also employed in `org.apache.spark.sql.hiveHiveUDAFFunction`.

I have done some performance testing:
```scala
import org.apache.spark.sql.{Dataset, Row}

sql("create function collect_list2 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectList'")

val df = range(0, 10000000).select($"id", (rand(213123L) * 100000).cast("int").as("grp"))
df.select(countDistinct($"grp")).show

def benchmark(name: String, plan: Dataset[Row], maxItr: Int = 5): Unit = {
   // Do not measure planning.
   plan1.queryExecution.executedPlan

   // Execute the plan a number of times and average the result.
   val start = System.nanoTime
   var i = 0
   while (i < maxItr) {
     plan.rdd.foreach(row => Unit)
     i += 1
   }
   val time = (System.nanoTime - start) / (maxItr * 1000000L)
   println(s"[$name] $maxItr iterations completed in an average time of $time ms.")
}

val plan1 = df.groupBy($"grp").agg(collect_list($"id"))
val plan2 = df.groupBy($"grp").agg(callUDF("collect_list2", $"id"))

benchmark("Spark collect_list", plan1)
...
> [Spark collect_list] 5 iterations completed in an average time of 3371 ms.

benchmark("Hive collect_list", plan2)
...
> [Hive collect_list] 5 iterations completed in an average time of 9109 ms.
```
Performance is improved by a factor 2-3.

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

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #12874 from hvanhovell/implode.
2016-05-12 13:56:00 -07:00
bomeng 81bf870848 [SPARK-14897][SQL] upgrade to jetty 9.2.16
## What changes were proposed in this pull request?

Since Jetty 8 is EOL (end of life) and has critical security issue [http://www.securityweek.com/critical-vulnerability-found-jetty-web-server], I think upgrading to 9 is necessary. I am using latest 9.2 since 9.3 requires Java 8+.

`javax.servlet` and `derby` were also upgraded since Jetty 9.2 needs corresponding version.

## How was this patch tested?

Manual test and current test cases should cover it.

Author: bomeng <bmeng@us.ibm.com>

Closes #12916 from bomeng/SPARK-14897.
2016-05-12 20:07:44 +01:00
gatorsmile be617f3d06 [SPARK-14684][SPARK-15277][SQL] Partition Spec Validation in SessionCatalog and Checking Partition Spec Existence Before Dropping
#### What changes were proposed in this pull request?
~~Currently, multiple partitions are allowed to drop by using a single DDL command: Alter Table Drop Partition. However, the internal implementation could break atomicity. That means, we could just drop a subset of qualified partitions, if hitting an exception when dropping one of qualified partitions~~

~~This PR contains the following behavior changes:~~
~~- disallow dropping multiple partitions by a single command ~~
~~- allow users to input predicates in partition specification and issue a nicer error message if the predicate's comparison operator is not `=`.~~
~~- verify the partition spec in SessionCatalog. This can ensure each partition spec in `Drop Partition` does not correspond to multiple partitions.~~

This PR has two major parts:
- Verify the partition spec in SessionCatalog for fixing the following issue:
  ```scala
  sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')")
  ```
  Above example uses an invalid partition spec. Without this PR, we will drop all the partitions. The reason is Hive megastores getPartitions API returns all the partitions if we provide an invalid spec.

- Re-implemented the `dropPartitions` in `HiveClientImpl`. Now, we always check if all the user-specified partition specs exist before attempting to drop the partitions. Previously, we start drop the partition before completing checking the existence of all the partition specs. If any failure happened after we start to drop the partitions, we will log an error message to indicate which partitions have been dropped and which partitions have not been dropped.

#### How was this patch tested?
Modified the existing test cases and added new test cases.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12801 from gatorsmile/banDropMultiPart.
2016-05-12 11:14:40 -07:00
Liang-Chi Hsieh 470de743ec [SPARK-15094][SPARK-14803][SQL] Remove extra Project added in EliminateSerialization
## What changes were proposed in this pull request?

We will eliminate the pair of `DeserializeToObject` and `SerializeFromObject` in `Optimizer` and add extra `Project`. However, when DeserializeToObject's outputObjectType is ObjectType and its cls can't be processed by unsafe project, it will be failed.

To fix it, we can simply remove the extra `Project` and replace the output attribute of `DeserializeToObject` in another rule.

## How was this patch tested?
`DatasetSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #12926 from viirya/fix-eliminate-serialization-projection.
2016-05-12 10:11:12 -07:00
Sean Zhong 33c6eb5218 [SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView
## What changes were proposed in this pull request?

Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #12945 from clockfly/spark-15171.
2016-05-12 15:51:53 +08:00
Wenchen Fan 46991448aa [SPARK-15160][SQL] support data source table in InMemoryCatalog
## What changes were proposed in this pull request?

This PR adds a new rule to convert `SimpleCatalogRelation` to data source table if its table property contains data source information.

## How was this patch tested?

new test in SQLQuerySuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12935 from cloud-fan/ds-table.
2016-05-11 23:55:42 -07:00
Cheng Lian f036dd7ce7 [SPARK-14346] SHOW CREATE TABLE for data source tables
## What changes were proposed in this pull request?

This PR adds native `SHOW CREATE TABLE` DDL command for data source tables. Support for Hive tables will be added in follow-up PR(s).

To show table creation DDL for data source tables created by CTAS statements, this PR also added partitioning and bucketing support for normal `CREATE TABLE ... USING ...` syntax.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

A new test suite `ShowCreateTableSuite` is added in sql/hive package to test the new feature.

Author: Cheng Lian <lian@databricks.com>

Closes #12781 from liancheng/spark-14346-show-create-table.
2016-05-11 20:44:04 -07:00
Sandeep Singh ff92eb2e80 [SPARK-15080][CORE] Break copyAndReset into copy and reset
## What changes were proposed in this pull request?
Break copyAndReset into two methods copy and reset instead of just one.

## How was this patch tested?
Existing Tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #12936 from techaddict/SPARK-15080.
2016-05-12 11:12:09 +08:00
Sandeep Singh db573fc743 [SPARK-15072][SQL][PYSPARK] FollowUp: Remove SparkSession.withHiveSupport in PySpark
## What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/12851
Remove `SparkSession.withHiveSupport` in PySpark and instead use `SparkSession.builder. enableHiveSupport`

## How was this patch tested?
Existing tests.

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13063 from techaddict/SPARK-15072-followup.
2016-05-11 17:44:00 -07:00
Bill Chambers 603f4453a1 [SPARK-15264][SPARK-15274][SQL] CSV Reader Error on Blank Column Names
## What changes were proposed in this pull request?

When a CSV begins with:
- `,,`
OR
- `"","",`

meaning that the first column names are either empty or blank strings and `header` is specified to be `true`, then the column name is replaced with `C` + the index number of that given column. For example, if you were to read in the CSV:
```
"","second column"
"hello", "there"
```
Then column names would become `"C0", "second column"`.

This behavior aligns with what currently happens when `header` is specified to be `false` in recent versions of Spark.

### Current Behavior in Spark <=1.6
In Spark <=1.6, a CSV with a blank column name becomes a blank string, `""`, meaning that this column cannot be accessed. However the CSV reads in without issue.

### Current Behavior in Spark 2.0
Spark throws a NullPointerError and will not read in the file.

#### Reproduction in 2.0
https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/346304/2828750690305044/484361/latest.html

## How was this patch tested?
A new test was added to `CSVSuite` to account for this issue. We then have asserts that test for being able to select both the empty column names as well as the regular column names.

Author: Bill Chambers <bill@databricks.com>
Author: Bill Chambers <wchambers@ischool.berkeley.edu>

Closes #13041 from anabranch/master.
2016-05-11 17:42:13 -07:00
Andrew Or f14c4ba001 [SPARK-15276][SQL] CREATE TABLE with LOCATION should imply EXTERNAL
## What changes were proposed in this pull request?

Before:
```sql
-- uses that location but issues a warning
CREATE TABLE my_tab LOCATION /some/path
-- deletes any existing data in the specified location
DROP TABLE my_tab
```

After:
```sql
-- uses that location but creates an EXTERNAL table instead
CREATE TABLE my_tab LOCATION /some/path
-- does not delete the data at /some/path
DROP TABLE my_tab
```

This patch essentially makes the `EXTERNAL` field optional. This is related to #13032.

## How was this patch tested?

New test in `DDLCommandSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #13060 from andrewor14/location-implies-external.
2016-05-11 17:29:58 -07:00
Andrew Or 8881765ac7 [SPARK-15257][SQL] Require CREATE EXTERNAL TABLE to specify LOCATION
## What changes were proposed in this pull request?

Before:
```sql
-- uses warehouse dir anyway
CREATE EXTERNAL TABLE my_tab
-- doesn't actually delete the data
DROP TABLE my_tab
```
After:
```sql
-- no location is provided, throws exception
CREATE EXTERNAL TABLE my_tab
-- creates an external table using that location
CREATE EXTERNAL TABLE my_tab LOCATION '/path/to/something'
-- doesn't delete the data, which is expected
DROP TABLE my_tab
```

## How was this patch tested?

New test in `DDLCommandSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #13032 from andrewor14/create-external-table-location.
2016-05-11 15:30:53 -07:00
Tathagata Das 81c68eceba [SPARK-15248][SQL] Make MetastoreFileCatalog consider directories from partition specs of a partitioned metastore table
Table partitions can be added with locations different from default warehouse location of a hive table.
`CREATE TABLE parquetTable (a int) PARTITIONED BY (b int) STORED AS parquet `
`ALTER TABLE parquetTable ADD PARTITION (b=1) LOCATION '/partition'`
Querying such a table throws error as the MetastoreFileCatalog does not list the added partition directory, it only lists the default base location.

```
[info] - SPARK-15248: explicitly added partitions should be readable *** FAILED *** (1 second, 8 milliseconds)
[info]   java.util.NoSuchElementException: key not found: file:/Users/tdas/Projects/Spark/spark2/target/tmp/spark-b39ad224-c5d1-4966-8981-fb45a2066d61/partition
[info]   at scala.collection.MapLike$class.default(MapLike.scala:228)
[info]   at scala.collection.AbstractMap.default(Map.scala:59)
[info]   at scala.collection.MapLike$class.apply(MapLike.scala:141)
[info]   at scala.collection.AbstractMap.apply(Map.scala:59)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:59)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:55)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
[info]   at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
[info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
[info]   at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
[info]   at scala.collection.AbstractTraversable.map(Traversable.scala:104)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog.listFiles(PartitioningAwareFileCatalog.scala:55)
[info]   at org.apache.spark.sql.execution.datasources.FileSourceStrategy$.apply(FileSourceStrategy.scala:93)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
[info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:55)
[info]   at org.apache.spark.sql.execution.SparkStrategies$SpecialLimits$.apply(SparkStrategies.scala:55)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
[info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
[info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:77)
[info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:75)
[info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:82)
[info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:82)
[info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:330)
[info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:146)
[info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:159)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:554)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:535)
[info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:125)
[info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTempDir(parquetSuites.scala:726)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7.apply$mcV$sp(parquetSuites.scala:535)
[info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:166)
[info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTable(parquetSuites.scala:726)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply$mcV$sp(parquetSuites.scala:534)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
```

The solution in this PR to get the paths to list from the partition spec and not rely on the default table path alone.

unit tests.

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

Closes #13022 from tdas/SPARK-15248.
2016-05-11 12:36:25 -07:00
Eric Liang 6d0368ab8d [SPARK-15259] Sort time metric should not include spill and record insertion time
## What changes were proposed in this pull request?

After SPARK-14669 it seems the sort time metric includes both spill and record insertion time. This makes it not very useful since the metric becomes close to the total execution time of the node.

We should track just the time spent for in-memory sort, as before.

## How was this patch tested?

Verified metric in the UI, also unit test on UnsafeExternalRowSorter.

cc davies

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #13035 from ericl/fix-metrics.
2016-05-11 11:25:46 -07:00
Wenchen Fan d8935db5ec [SPARK-15241] [SPARK-15242] [SQL] fix 2 decimal-related issues in RowEncoder
## What changes were proposed in this pull request?

SPARK-15241: We now support java decimal and catalyst decimal in external row, it makes sense to also support scala decimal.

SPARK-15242: This is a long-standing bug, and is exposed after https://github.com/apache/spark/pull/12364, which eliminate the `If` expression if the field is not nullable:
```
val fieldValue = serializerFor(
  GetExternalRowField(inputObject, i, externalDataTypeForInput(f.dataType)),
  f.dataType)
if (f.nullable) {
  If(
    Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil),
    Literal.create(null, f.dataType),
    fieldValue)
} else {
  fieldValue
}
```

Previously, we always use `DecimalType.SYSTEM_DEFAULT` as the output type of converted decimal field, which is wrong as it doesn't match the real decimal type. However, it works well because we always put converted field into `If` expression to do the null check, and `If` use its `trueValue`'s data type as its output type.
Now if we have a not nullable decimal field, then the converted field's output type will be `DecimalType.SYSTEM_DEFAULT`, and we will write wrong data into unsafe row.

The fix is simple, just use the given decimal type as the output type of converted decimal field.

These 2 issues was found at https://github.com/apache/spark/pull/13008

## How was this patch tested?

new tests in RowEncoderSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13019 from cloud-fan/encoder-decimal.
2016-05-11 11:16:05 -07:00
Dongjoon Hyun e1576478bd [SPARK-14933][HOTFIX] Replace sqlContext with spark.
## What changes were proposed in this pull request?

This fixes compile errors.

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13053 from dongjoon-hyun/hotfix_sqlquerysuite.
2016-05-11 10:03:51 -07:00
Liang-Chi Hsieh a5f9fdbba3 [SPARK-15268][SQL] Make JavaTypeInference work with UDTRegistration
## What changes were proposed in this pull request?

We have a private `UDTRegistration` API to register user defined type. Currently `JavaTypeInference` can't work with it. So `SparkSession.createDataFrame` from a bean class will not correctly infer the schema of the bean class.

## How was this patch tested?
`VectorUDTSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #13046 from viirya/fix-udt-registry-javatypeinference.
2016-05-11 09:31:22 -07:00
xin Wu 427c20dd6d [SPARK-14933][SQL] Failed to create view out of a parquet or orc table
## What changes were proposed in this pull request?
#### Symptom
 If a table is created as parquet or ORC table with hive syntaxt DDL, such as
```SQL
create table t1 (c1 int, c2 string) stored as parquet
```
The following command will fail
```SQL
create view v1 as select * from t1
```
#### Root Cause
Currently, `HiveMetaStoreCatalog` converts Paruqet/Orc tables to `LogicalRelation` without giving any `tableIdentifier`. `SQLBuilder` expects the `LogicalRelation` to have an associated `tableIdentifier`. However, the `LogicalRelation` created earlier does not have such a `tableIdentifier`. Thus, `SQLBuilder.toSQL` can not recognize this logical plan and issue an exception.

This PR is to assign a `TableIdentifier` to the `LogicalRelation` when resolving parquet or orc tables in `HiveMetaStoreCatalog`.

## How was this patch tested?
testcases created and dev/run-tests is run.

Author: xin Wu <xinwu@us.ibm.com>

Closes #12716 from xwu0226/SPARK_14933.
2016-05-11 22:17:59 +08:00
Shixiong Zhu 875ef76428 [SPARK-15231][SQL] Document the semantic of saveAsTable and insertInto and don't drop columns silently
## What changes were proposed in this pull request?

This PR adds documents about the different behaviors between `insertInto` and `saveAsTable`, and throws an exception when the user try to add too man columns using `saveAsTable with append`.

## How was this patch tested?

Unit tests added in this PR.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13013 from zsxwing/SPARK-15231.
2016-05-10 23:53:55 -07:00