Commit graph

2730 commits

Author SHA1 Message Date
Daoyuan Wang 96e9afaae9 [SPARK-16515][SQL] set default record reader and writer for script transformation
## What changes were proposed in this pull request?
In ScriptInputOutputSchema, we read default RecordReader and RecordWriter from conf. Since Spark 2.0 has deleted those config keys from hive conf, we have to set default reader/writer class name by ourselves. Otherwise we will get None for LazySimpleSerde, the data written would not be able to read by script. The test case added worked fine with previous version of Spark, but would fail now.

## How was this patch tested?
added a test case in SQLQuerySuite.

Closes #14169

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14249 from yhuai/scriptTransformation.
2016-07-18 13:58:12 -07:00
hyukjinkwon 2877f1a522 [SPARK-16351][SQL] Avoid per-record type dispatch in JSON when writing
## What changes were proposed in this pull request?

Currently, `JacksonGenerator.apply` is doing type-based dispatch for each row to write appropriate values.
It might not have to be done like this because the schema is already kept.

So, appropriate writers can be created first according to the schema once, and then apply them to each row. This approach is similar with `CatalystWriteSupport`.

This PR corrects `JacksonGenerator` so that it creates all writers for the schema once and then applies them to each row rather than type dispatching for every row.

Benchmark was proceeded with the codes below:

```scala
test("Benchmark for JSON writer") {
  val N = 500 << 8
  val row =
    """{"struct":{"field1": true, "field2": 92233720368547758070},
      "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]},
      "arrayOfString":["str1", "str2"],
      "arrayOfInteger":[1, 2147483647, -2147483648],
      "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808],
      "arrayOfBigInteger":[922337203685477580700, -922337203685477580800],
      "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308],
      "arrayOfBoolean":[true, false, true],
      "arrayOfNull":[null, null, null, null],
      "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
      "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]],
      "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]]
     }"""
  val df = spark.sqlContext.read.json(spark.sparkContext.parallelize(List.fill(N)(row)))
  val benchmark = new Benchmark("JSON writer", N)
  benchmark.addCase("writing JSON file", 10) { _ =>
    withTempPath { path =>
      df.write.format("json").save(path.getCanonicalPath)
    }
  }
  benchmark.run()
}
```

This produced the results below

- **Before**

```
JSON writer:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
writing JSON file                             1675 / 1767          0.1       13087.5       1.0X
```

- **After**

```
JSON writer:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
writing JSON file                             1597 / 1686          0.1       12477.1       1.0X
```

In addition, I ran this benchmark 10 times for each and calculated the average elapsed time as below:

| **Before** | **After**|
|---------------|------------|
|17478ms  |16669ms |

It seems roughly ~5% is improved.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14028 from HyukjinKwon/SPARK-16351.
2016-07-18 09:49:14 -07:00
Reynold Xin 480c870644 [SPARK-16588][SQL] Deprecate monotonicallyIncreasingId in Scala/Java
This patch deprecates monotonicallyIncreasingId in Scala/Java, as done in Python.

This patch was originally written by HyukjinKwon. Closes #14236.
2016-07-17 22:48:00 -07:00
Dongjoon Hyun c576f9fb90 [SPARK-16529][SQL][TEST] withTempDatabase should set default database before dropping
## What changes were proposed in this pull request?

`SQLTestUtils.withTempDatabase` is a frequently used test harness to setup a temporary table and clean up finally. This issue improves like the following for usability.

```scala
-    try f(dbName) finally spark.sql(s"DROP DATABASE $dbName CASCADE")
+    try f(dbName) finally {
+      if (spark.catalog.currentDatabase == dbName) {
+        spark.sql(s"USE ${DEFAULT_DATABASE}")
+      }
+      spark.sql(s"DROP DATABASE $dbName CASCADE")
+    }
```

In case of forgetting to reset the databaes, `withTempDatabase` will not raise Exception.

## How was this patch tested?

This improves test harness.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14184 from dongjoon-hyun/SPARK-16529.
2016-07-15 00:51:11 +08:00
Dongjoon Hyun 56183b84fb [SPARK-16543][SQL] Rename the columns of SHOW PARTITION/COLUMNS commands
## What changes were proposed in this pull request?

This PR changes the name of columns returned by `SHOW PARTITION` and `SHOW COLUMNS` commands. Currently, both commands uses `result` as a column name.

**Comparison: Column Name**

Command|Spark(Before)|Spark(After)|Hive
----------|--------------|------------|-----
SHOW PARTITIONS|result|partition|partition
SHOW COLUMNS|result|col_name|field

Note that Spark/Hive uses `col_name` in `DESC TABLES`. So, this PR chooses `col_name` for consistency among Spark commands.

**Before**
```scala
scala> sql("show partitions p").show()
+------+
|result|
+------+
|   b=2|
+------+

scala> sql("show columns in p").show()
+------+
|result|
+------+
|     a|
|     b|
+------+
```

**After**
```scala
scala> sql("show partitions p").show
+---------+
|partition|
+---------+
|      b=2|
+---------+

scala> sql("show columns in p").show
+--------+
|col_name|
+--------+
|       a|
|       b|
+--------+
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14199 from dongjoon-hyun/SPARK-16543.
2016-07-14 17:18:34 +02:00
Liwei Lin 39c836e976 [SPARK-16503] SparkSession should provide Spark version
## What changes were proposed in this pull request?

This patch enables SparkSession to provide spark version.

## How was this patch tested?

Manual test:

```
scala> sc.version
res0: String = 2.1.0-SNAPSHOT

scala> spark.version
res1: String = 2.1.0-SNAPSHOT
```

```
>>> sc.version
u'2.1.0-SNAPSHOT'
>>> spark.version
u'2.1.0-SNAPSHOT'
```

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14165 from lw-lin/add-version.
2016-07-13 22:30:46 -07:00
gatorsmile c5ec879828 [SPARK-16482][SQL] Describe Table Command for Tables Requiring Runtime Inferred Schema
#### What changes were proposed in this pull request?
If we create a table pointing to a parquet/json datasets without specifying the schema, describe table command does not show the schema at all. It only shows `# Schema of this table is inferred at runtime`. In 1.6, describe table does show the schema of such a table.

~~For data source tables, to infer the schema, we need to load the data source tables at runtime. Thus, this PR calls the function `lookupRelation`.~~

For data source tables, we infer the schema before table creation. Thus, this PR set the inferred schema as the table schema when table creation.

#### How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14148 from gatorsmile/describeSchema.
2016-07-13 15:23:37 -07:00
Burak Yavuz 0744d84c91 [SPARK-16531][SQL][TEST] Remove timezone setting from DataFrameTimeWindowingSuite
## What changes were proposed in this pull request?

It's unnecessary. `QueryTest` already sets it.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #14170 from brkyvz/test-tz.
2016-07-13 12:54:57 -07:00
Maciej Brynski 83879ebc58 [SPARK-16439] Fix number formatting in SQL UI
## What changes were proposed in this pull request?

Spark SQL UI display numbers greater than 1000 with u00A0 as grouping separator.
Problem exists when server locale has no-breaking space as separator. (for example pl_PL)
This patch turns off grouping and remove this separator.

The problem starts with this PR.
https://github.com/apache/spark/pull/12425/files#diff-803f475b01acfae1c5c96807c2ea9ddcR125

## How was this patch tested?

Manual UI tests. Screenshot attached.

![image](https://cloud.githubusercontent.com/assets/4006010/16749556/5cb5a372-47cb-11e6-9a95-67fd3f9d1c71.png)

Author: Maciej Brynski <maciej.brynski@adpilot.pl>

Closes #14142 from maver1ck/master.
2016-07-13 10:50:26 +01:00
Xin Ren f73891e0b9 [MINOR] Fix Java style errors and remove unused imports
## What changes were proposed in this pull request?

Fix Java style errors and remove unused imports, which are randomly found

## How was this patch tested?

Tested on my local machine.

Author: Xin Ren <iamshrek@126.com>

Closes #14161 from keypointt/SPARK-16437.
2016-07-13 10:47:07 +01:00
Sean Owen c190d89bd3 [SPARK-15889][STREAMING] Follow-up fix to erroneous condition in StreamTest
## What changes were proposed in this pull request?

A second form of AssertQuery now actually invokes the condition; avoids a build warning too

## How was this patch tested?

Jenkins; running StreamTest

Author: Sean Owen <sowen@cloudera.com>

Closes #14133 from srowen/SPARK-15889.2.
2016-07-13 10:44:07 +01:00
petermaxlee 56bd399a86 [SPARK-16284][SQL] Implement reflect SQL function
## What changes were proposed in this pull request?
This patch implements reflect SQL function, which can be used to invoke a Java method in SQL. Slightly different from Hive, this implementation requires the class name and the method name to be literals. This implementation also supports only a smaller number of data types, and requires the function to be static, as suggested by rxin in #13969.

java_method is an alias for reflect, so this should also resolve SPARK-16277.

## How was this patch tested?
Added expression unit tests and an end-to-end test.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14138 from petermaxlee/reflect-static.
2016-07-13 08:05:20 +08:00
Marcelo Vanzin 7f968867ff [SPARK-16119][SQL] Support PURGE option to drop table / partition.
This option is used by Hive to directly delete the files instead of
moving them to the trash. This is needed in certain configurations
where moving the files does not work. For non-Hive tables and partitions,
Spark already behaves as if the PURGE option was set, so there's no
need to do anything.

Hive support for PURGE was added in 0.14 (for tables) and 1.2 (for
partitions), so the code reflects that: trying to use the option with
older versions of Hive will cause an exception to be thrown.

The change is a little noisier than I would like, because of the code
to propagate the new flag through all the interfaces and implementations;
the main changes are in the parser and in HiveShim, aside from the tests
(DDLCommandSuite, VersionsSuite).

Tested by running sql and catalyst unit tests, plus VersionsSuite which
has been updated to test the version-specific behavior. I also ran an
internal test suite that uses PURGE and would not pass previously.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #13831 from vanzin/SPARK-16119.
2016-07-12 12:47:46 -07:00
Reynold Xin c377e49e38 [SPARK-16489][SQL] Guard against variable reuse mistakes in expression code generation
## What changes were proposed in this pull request?
In code generation, it is incorrect for expressions to reuse variable names across different instances of itself. As an example, SPARK-16488 reports a bug in which pmod expression reuses variable name "r".

This patch updates ExpressionEvalHelper test harness to always project two instances of the same expression, which will help us catch variable reuse problems in expression unit tests. This patch also fixes the bug in crc32 expression.

## How was this patch tested?
This is a test harness change, but I also created a new test suite for testing the test harness.

Author: Reynold Xin <rxin@databricks.com>

Closes #14146 from rxin/SPARK-16489.
2016-07-12 10:07:23 -07:00
Lianhui Wang 5ad68ba5ce [SPARK-15752][SQL] Optimize metadata only query that has an aggregate whose children are deterministic project or filter operators.
## What changes were proposed in this pull request?
when query only use metadata (example: partition key), it can return results based on metadata without scanning files. Hive did it in HIVE-1003.

## How was this patch tested?
add unit tests

Author: Lianhui Wang <lianhuiwang09@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Lianhui Wang <lianhuiwang@users.noreply.github.com>

Closes #13494 from lianhuiwang/metadata-only.
2016-07-12 18:52:15 +02:00
Takuya UESHIN 5b28e02584 [SPARK-16189][SQL] Add ExternalRDD logical plan for input with RDD to have a chance to eliminate serialize/deserialize.
## What changes were proposed in this pull request?

Currently the input `RDD` of `Dataset` is always serialized to `RDD[InternalRow]` prior to being as `Dataset`, but there is a case that we use `map` or `mapPartitions` just after converted to `Dataset`.
In this case, serialize and then deserialize happens but it would not be needed.

This pr adds `ExistingRDD` logical plan for input with `RDD` to have a chance to eliminate serialize/deserialize.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #13890 from ueshin/issues/SPARK-16189.
2016-07-12 17:16:59 +08:00
petermaxlee c9a6762150 [SPARK-16199][SQL] Add a method to list the referenced columns in data source Filter
## What changes were proposed in this pull request?
It would be useful to support listing the columns that are referenced by a filter. This can help simplify data source planning, because with this we would be able to implement unhandledFilters method in HadoopFsRelation.

This is based on rxin's patch (#13901) and adds unit tests.

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

Author: petermaxlee <petermaxlee@gmail.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #14120 from petermaxlee/SPARK-16199.
2016-07-11 22:23:32 -07:00
Russell Spitzer b1e5281c5c [SPARK-12639][SQL] Mark Filters Fully Handled By Sources with *
## What changes were proposed in this pull request?

In order to make it clear which filters are fully handled by the
underlying datasource we will mark them with an *. This will give a
clear visual queue to users that the filter is being treated differently
by catalyst than filters which are just presented to the underlying
DataSource.

Examples from the FilteredScanSuite, in this example `c IN (...)` is handled by the source, `b < ...` is not
### Before
```
//SELECT a FROM oneToTenFiltered WHERE a + b > 9 AND b < 16 AND c IN ('bbbbbBBBBB', 'cccccCCCCC', 'dddddDDDDD', 'foo')
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

### After
```
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), *In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

## How was the this patch tested?

Manually tested with the Spark Cassandra Connector, a source which fully handles underlying filters. Now fully handled filters appear with an * next to their names. I can add an automated test as well if requested

Post 1.6.1
Tested by modifying the FilteredScanSuite to run explains.

Author: Russell Spitzer <Russell.Spitzer@gmail.com>

Closes #11317 from RussellSpitzer/SPARK-12639-Star.
2016-07-11 21:40:09 -07:00
Sameer Agarwal 9cc74f95ed [SPARK-16488] Fix codegen variable namespace collision in pmod and partitionBy
## What changes were proposed in this pull request?

This patch fixes a variable namespace collision bug in pmod and partitionBy

## How was this patch tested?

Regression test for one possible occurrence. A more general fix in `ExpressionEvalHelper.checkEvaluation` will be in a subsequent PR.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #14144 from sameeragarwal/codegen-bug.
2016-07-11 20:26:01 -07:00
Tathagata Das e50efd53f0 [SPARK-16430][SQL][STREAMING] Fixed bug in the maxFilesPerTrigger in FileStreamSource
## What changes were proposed in this pull request?

Incorrect list of files were being allocated to a batch. This caused a file to read multiple times in the multiple batches.

## How was this patch tested?

Added unit tests

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

Closes #14143 from tdas/SPARK-16430-1.
2016-07-11 18:41:36 -07:00
Shixiong Zhu 91a443b849 [SPARK-16433][SQL] Improve StreamingQuery.explain when no data arrives
## What changes were proposed in this pull request?

Display `No physical plan. Waiting for data.` instead of `N/A`  for StreamingQuery.explain when no data arrives because `N/A` doesn't provide meaningful information.

## How was this patch tested?

Existing unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #14100 from zsxwing/SPARK-16433.
2016-07-11 18:11:06 -07:00
James Thomas 9e2c763dbb [SPARK-16114][SQL] structured streaming event time window example
## What changes were proposed in this pull request?

A structured streaming example with event time windowing.

## How was this patch tested?

Run locally

Author: James Thomas <jamesjoethomas@gmail.com>

Closes #13957 from jjthomas/current.
2016-07-11 17:57:51 -07:00
Dongjoon Hyun 840853ed06 [SPARK-16458][SQL] SessionCatalog should support listColumns for temporary tables
## What changes were proposed in this pull request?

Temporary tables are used frequently, but `spark.catalog.listColumns` does not support those tables. This PR make `SessionCatalog` supports temporary table column listing.

**Before**
```scala
scala> spark.range(10).createOrReplaceTempView("t1")

scala> spark.catalog.listTables().collect()
res1: Array[org.apache.spark.sql.catalog.Table] = Array(Table[name=`t1`, tableType=`TEMPORARY`, isTemporary=`true`])

scala> spark.catalog.listColumns("t1").collect()
org.apache.spark.sql.AnalysisException: Table `t1` does not exist in database `default`.;
```

**After**
```
scala> spark.catalog.listColumns("t1").collect()
res2: Array[org.apache.spark.sql.catalog.Column] = Array(Column[name='id', description='id', dataType='bigint', nullable='false', isPartition='false', isBucket='false'])
```
## How was this patch tested?

Pass the Jenkins tests including a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14114 from dongjoon-hyun/SPARK-16458.
2016-07-11 22:45:22 +02:00
Reynold Xin ffcb6e055a [SPARK-16477] Bump master version to 2.1.0-SNAPSHOT
## What changes were proposed in this pull request?
After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14130 from rxin/SPARK-16477.
2016-07-11 09:42:56 -07:00
Dongjoon Hyun 7ac79da0e4 [SPARK-16459][SQL] Prevent dropping current database
## What changes were proposed in this pull request?

This PR prevents dropping the current database to avoid errors like the followings.

```scala
scala> sql("create database delete_db")
scala> sql("use delete_db")
scala> sql("drop database delete_db")
scala> sql("create table t as select 1")
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database `delete_db` not found;
```

## How was this patch tested?

Pass the Jenkins tests including an updated testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14115 from dongjoon-hyun/SPARK-16459.
2016-07-11 15:15:47 +02:00
gatorsmile e226278941 [SPARK-16355][SPARK-16354][SQL] Fix Bugs When LIMIT/TABLESAMPLE is Non-foldable, Zero or Negative
#### What changes were proposed in this pull request?
**Issue 1:** When a query containing LIMIT/TABLESAMPLE 0, the statistics could be zero. Results are correct but it could cause a huge performance regression. For example,
```Scala
Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v")
  .createOrReplaceTempView("test")
val df1 = spark.table("test")
val df2 = spark.table("test").limit(0)
val df = df1.join(df2, Seq("k"), "left")
```
The statistics of both `df` and `df2` are zero. The statistics values should never be zero; otherwise `sizeInBytes` of `BinaryNode` will also be zero (product of children). This PR is to increase it to `1` when the num of rows is equal to 0.

**Issue 2:** When a query containing negative LIMIT/TABLESAMPLE, we should issue exceptions. Negative values could break the implementation assumption of multiple parts. For example, statistics calculation.  Below is the example query.
```SQL
SELECT * FROM testData TABLESAMPLE (-1 rows)
SELECT * FROM testData LIMIT -1
```
This PR is to issue an appropriate exception in this case.

**Issue 3:** Spark SQL follows the restriction of LIMIT clause in Hive. The argument to the LIMIT clause must evaluate to a constant value. It can be a numeric literal, or another kind of numeric expression involving operators, casts, and function return values. You cannot refer to a column or use a subquery. Currently, we do not detect whether the expression in LIMIT clause is foldable or not. If non-foldable, we might issue a strange error message. For example,
```SQL
SELECT * FROM testData LIMIT rand() > 0.2
```
Then, a misleading error message is issued, like
```
assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
   +- LocalLimit (_nondeterministic#202 > 0.2)
      +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
         +- LogicalRDD [key#11, value#12]

java.lang.AssertionError: assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2)
+- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203]
   +- LocalLimit (_nondeterministic#202 > 0.2)
      +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202]
         +- LogicalRDD [key#11, value#12]
```
This PR detects it and then issues a meaningful error message.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14034 from gatorsmile/limit.
2016-07-11 16:21:13 +08:00
petermaxlee 82f0874453 [SPARK-16318][SQL] Implement all remaining xpath functions
## What changes were proposed in this pull request?
This patch implements all remaining xpath functions that Hive supports and not natively supported in Spark: xpath_int, xpath_short, xpath_long, xpath_float, xpath_double, xpath_string, and xpath.

## How was this patch tested?
Added unit tests and end-to-end tests.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13991 from petermaxlee/SPARK-16318.
2016-07-11 13:28:34 +08:00
gatorsmile 7374e518e2 [SPARK-16401][SQL] Data Source API: Enable Extending RelationProvider and CreatableRelationProvider without Extending SchemaRelationProvider
#### What changes were proposed in this pull request?
When users try to implement a data source API with extending only `RelationProvider` and `CreatableRelationProvider`, they will hit an error when resolving the relation.
```Scala
spark.read
.format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema")
  .load()
  .write.
format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema")
  .save()
```

The error they hit is like
```
org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.;
org.apache.spark.sql.AnalysisException: org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.;
	at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:319)
	at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:494)
	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:211)
```

Actually, the bug fix is simple. [`DataSource.createRelation(sparkSession.sqlContext, mode, options, data)`](dd644f8117/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L429)) already returns a BaseRelation. We should not assign schema to `userSpecifiedSchema`. That schema assignment only makes sense for the data sources that extend `FileFormat`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14075 from gatorsmile/dataSource.
2016-07-09 20:35:45 +08:00
Dongjoon Hyun 3b22291b5f [SPARK-16387][SQL] JDBC Writer should use dialect to quote field names.
## What changes were proposed in this pull request?

Currently, JDBC Writer uses dialects to get datatypes, but doesn't to quote field names. This PR uses dialects to quote the field names, too.

**Reported Error Scenario (MySQL case)**
```scala
scala> val url="jdbc:mysql://localhost:3306/temp"
scala> val prop = new java.util.Properties
scala> prop.setProperty("user","root")
scala> spark.createDataset(Seq("a","b","c")).toDF("order")
scala> df.write.mode("overwrite").jdbc(url, "temptable", prop)
...MySQLSyntaxErrorException: ... near 'order TEXT )
```

## How was this patch tested?

Pass the Jenkins tests and manually do the above case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14107 from dongjoon-hyun/SPARK-16387.
2016-07-08 16:07:12 -07:00
wujian f5fef69143 [SPARK-16281][SQL] Implement parse_url SQL function
## What changes were proposed in this pull request?

This PR adds parse_url SQL functions in order to remove Hive fallback.

A new implementation of #13999

## How was this patch tested?

Pass the exist tests including new testcases.

Author: wujian <jan.chou.wu@gmail.com>

Closes #14008 from janplus/SPARK-16281.
2016-07-08 14:38:05 -07:00
Dongjoon Hyun 142df4834b [SPARK-16429][SQL] Include StringType columns in describe()
## What changes were proposed in this pull request?

Currently, Spark `describe` supports `StringType`. However, `describe()` returns a dataset for only all numeric columns. This PR aims to include `StringType` columns in `describe()`, `describe` without argument.

**Background**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe("age", "name").show()
+-------+------------------+-------+
|summary|               age|   name|
+-------+------------------+-------+
|  count|                 2|      3|
|   mean|              24.5|   null|
| stddev|7.7781745930520225|   null|
|    min|                19|   Andy|
|    max|                30|Michael|
+-------+------------------+-------+
```

**Before**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe().show()
+-------+------------------+
|summary|               age|
+-------+------------------+
|  count|                 2|
|   mean|              24.5|
| stddev|7.7781745930520225|
|    min|                19|
|    max|                30|
+-------+------------------+
```

**After**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe().show()
+-------+------------------+-------+
|summary|               age|   name|
+-------+------------------+-------+
|  count|                 2|      3|
|   mean|              24.5|   null|
| stddev|7.7781745930520225|   null|
|    min|                19|   Andy|
|    max|                30|Michael|
+-------+------------------+-------+
```

## How was this patch tested?

Pass the Jenkins with a update testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14095 from dongjoon-hyun/SPARK-16429.
2016-07-08 14:36:50 -07:00
Jurriaan Pruis 38cf8f2a50 [SPARK-13638][SQL] Add quoteAll option to CSV DataFrameWriter
## What changes were proposed in this pull request?

Adds an quoteAll option for writing CSV which will quote all fields.
See https://issues.apache.org/jira/browse/SPARK-13638

## How was this patch tested?

Added a test to verify the output columns are quoted for all fields in the Dataframe

Author: Jurriaan Pruis <email@jurriaanpruis.nl>

Closes #13374 from jurriaan/csv-quote-all.
2016-07-08 11:45:41 -07:00
Dongjoon Hyun a54438cb23 [SPARK-16285][SQL] Implement sentences SQL functions
## What changes were proposed in this pull request?

This PR implements `sentences` SQL function.

## How was this patch tested?

Pass the Jenkins tests with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14004 from dongjoon-hyun/SPARK_16285.
2016-07-08 17:05:24 +08:00
Tathagata Das 5bce458093 [SPARK-16430][SQL][STREAMING] Add option maxFilesPerTrigger
## What changes were proposed in this pull request?

An option that limits the file stream source to read 1 file at a time enables rate limiting. It has the additional convenience that a static set of files can be used like a stream for testing as this will allows those files to be considered one at a time.

This PR adds option `maxFilesPerTrigger`.

## How was this patch tested?

New unit test

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

Closes #14094 from tdas/SPARK-16430.
2016-07-07 23:19:41 -07:00
Liwei Lin 0f7175def9 [SPARK-16350][SQL] Fix support for incremental planning in wirteStream.foreach()
## What changes were proposed in this pull request?

There are cases where `complete` output mode does not output updated aggregated value; for details please refer to [SPARK-16350](https://issues.apache.org/jira/browse/SPARK-16350).

The cause is that, as we do `data.as[T].foreachPartition { iter => ... }` in `ForeachSink.addBatch()`, `foreachPartition()` does not support incremental planning for now.

This patches makes `foreachPartition()` support incremental planning in `ForeachSink`, by making a special version of `Dataset` with its `rdd()` method supporting incremental planning.

## How was this patch tested?

Added a unit test which failed before the change

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14030 from lw-lin/fix-foreach-complete.
2016-07-07 10:40:42 -07:00
Reynold Xin 986b251401 [SPARK-16400][SQL] Remove InSet filter pushdown from Parquet
## What changes were proposed in this pull request?
This patch removes InSet filter pushdown from Parquet data source, since row-based pushdown is not beneficial to Spark and brings extra complexity to the code base.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14076 from rxin/SPARK-16400.
2016-07-07 18:09:18 +08:00
gatorsmile ab05db0b48 [SPARK-16368][SQL] Fix Strange Errors When Creating View With Unmatched Column Num
#### What changes were proposed in this pull request?
When creating a view, a common user error is the number of columns produced by the `SELECT` clause does not match the number of column names specified by `CREATE VIEW`.

For example, given Table `t1` only has 3 columns
```SQL
create view v1(col2, col4, col3, col5) as select * from t1
```
Currently, Spark SQL reports the following error:
```
requirement failed
java.lang.IllegalArgumentException: requirement failed
	at scala.Predef$.require(Predef.scala:212)
	at org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:90)
```

This error message is very confusing. This PR is to detect the error and issue a meaningful error message.

#### How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14047 from gatorsmile/viewMismatchedColumns.
2016-07-07 00:07:25 -07:00
hyukjinkwon 34283de160 [SPARK-14839][SQL] Support for other types for tableProperty rule in SQL syntax
## What changes were proposed in this pull request?

Currently, Scala API supports to take options with the types, `String`, `Long`, `Double` and `Boolean` and Python API also supports other types.

This PR corrects `tableProperty` rule to support other types (string, boolean, double and integer) so that support the options for data sources in a consistent way. This will affect other rules such as DBPROPERTIES and TBLPROPERTIES (allowing other types as values).

Also, `TODO add bucketing and partitioning.` was removed because it was resolved in 24bea00047

## How was this patch tested?

Unit test in `MetastoreDataSourcesSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13517 from HyukjinKwon/SPARK-14839.
2016-07-06 23:57:18 -04:00
Reynold Xin 8e3e4ed6c0 [SPARK-16371][SQL] Two follow-up tasks
## What changes were proposed in this pull request?
This is a small follow-up for SPARK-16371:

1. Hide removeMetadata from public API.
2. Add JIRA ticket number to test case name.

## How was this patch tested?
Updated a test comment.

Author: Reynold Xin <rxin@databricks.com>

Closes #14074 from rxin/parquet-filter.
2016-07-06 15:04:37 -07:00
hyukjinkwon 4f8ceed593 [SPARK-16371][SQL] Do not push down filters incorrectly when inner name and outer name are the same in Parquet
## What changes were proposed in this pull request?

Currently, if there is a schema as below:

```
root
  |-- _1: struct (nullable = true)
  |    |-- _1: integer (nullable = true)
```

and if we execute the codes below:

```scala
df.filter("_1 IS NOT NULL").count()
```

This pushes down a filter although this filter is being applied to `StructType`.(If my understanding is correct, Spark does not pushes down filters for those).

The reason is, `ParquetFilters.getFieldMap` produces results below:

```
(_1,StructType(StructField(_1,IntegerType,true)))
(_1,IntegerType)
```

and then it becomes a `Map`

```
(_1,IntegerType)
```

Now, because of ` ....lift(dataTypeOf(name)).map(_(name, value))`, this pushes down filters for `_1` which Parquet thinks is `IntegerType`. However, it is actually `StructType`.

So, Parquet filter2 produces incorrect results, for example, the codes below:

```
df.filter("_1 IS NOT NULL").count()
```

produces always 0.

This PR prevents this by not finding nested fields.

## How was this patch tested?

Unit test in `ParquetFilterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14067 from HyukjinKwon/SPARK-16371.
2016-07-06 12:42:16 -07:00
Cheng Lian 23eff5e512 [SPARK-15979][SQL] Renames CatalystWriteSupport to ParquetWriteSupport
## What changes were proposed in this pull request?

PR #13696 renamed various Parquet support classes but left `CatalystWriteSupport` behind. This PR is renames it as a follow-up.

## How was this patch tested?

N/A.

Author: Cheng Lian <lian@databricks.com>

Closes #14070 from liancheng/spark-15979-follow-up.
2016-07-06 10:36:45 -07:00
Reynold Xin 7e28fabdff [SPARK-16388][SQL] Remove spark.sql.nativeView and spark.sql.nativeView.canonical config
## What changes were proposed in this pull request?
These two configs should always be true after Spark 2.0. This patch removes them from the config list. Note that ideally this should've gone into branch-2.0, but due to the timing of the release we should only merge this in master for Spark 2.1.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #14061 from rxin/SPARK-16388.
2016-07-06 17:40:55 +08:00
Dongjoon Hyun ec79183ac5 [SPARK-16340][SQL] Support column arguments for regexp_replace Dataset operation
## What changes were proposed in this pull request?

Currently, `regexp_replace` function supports `Column` arguments in a query. This PR supports that in a `Dataset` operation, too.

## How was this patch tested?

Pass the Jenkins tests with a updated testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14060 from dongjoon-hyun/SPARK-16340.
2016-07-05 22:11:40 -07:00
Dongjoon Hyun d0d28507ca [SPARK-16286][SQL] Implement stack table generating function
## What changes were proposed in this pull request?

This PR implements `stack` table generating function.

## How was this patch tested?

Pass the Jenkins tests including new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14033 from dongjoon-hyun/SPARK-16286.
2016-07-06 10:54:43 +08:00
Dongjoon Hyun 4db63fd2b4 [SPARK-16383][SQL] Remove SessionState.executeSql
## What changes were proposed in this pull request?

This PR removes `SessionState.executeSql` in favor of `SparkSession.sql`. We can remove this safely since the visibility `SessionState` is `private[sql]` and `executeSql` is only used in one **ignored** test, `test("Multiple Hive Instances")`.

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14055 from dongjoon-hyun/SPARK-16383.
2016-07-05 16:47:32 -07:00
Reynold Xin 16a2a7d714 [SPARK-16311][SQL] Metadata refresh should work on temporary views
## What changes were proposed in this pull request?
This patch fixes the bug that the refresh command does not work on temporary views. This patch is based on https://github.com/apache/spark/pull/13989, but removes the public Dataset.refresh() API as well as improved test coverage.

Note that I actually think the public refresh() API is very useful. We can in the future implement it by also invalidating the lazy vals in QueryExecution (or alternatively just create a new QueryExecution).

## How was this patch tested?
Re-enabled a previously ignored test, and added a new test suite for Hive testing behavior of temporary views against MetastoreRelation.

Author: Reynold Xin <rxin@databricks.com>
Author: petermaxlee <petermaxlee@gmail.com>

Closes #14009 from rxin/SPARK-16311.
2016-07-05 11:36:05 -07:00
hyukjinkwon 07d9c5327f [SPARK-9876][SQL][FOLLOWUP] Enable string and binary tests for Parquet predicate pushdown and replace deprecated fromByteArray.
## What changes were proposed in this pull request?

It seems Parquet has been upgraded to 1.8.1 by https://github.com/apache/spark/pull/13280. So,  this PR enables string and binary predicate push down which was disabled due to [SPARK-11153](https://issues.apache.org/jira/browse/SPARK-11153) and [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251) and cleans up some comments unremoved (I think by mistake).

This PR also replace the API, `fromByteArray()` deprecated in [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251).

## How was this patch tested?

Unit tests in `ParquetFilters`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13389 from HyukjinKwon/parquet-1.8-followup.
2016-07-05 16:59:40 +08:00
Dongjoon Hyun 7f7eb3934e [SPARK-16360][SQL] Speed up SQL query performance by removing redundant executePlan call
## What changes were proposed in this pull request?

Currently, there are a few reports about Spark 2.0 query performance regression for large queries.

This PR speeds up SQL query processing performance by removing redundant **consecutive `executePlan`** call in `Dataset.ofRows` function and `Dataset` instantiation. Specifically, this PR aims to reduce the overhead of SQL query execution plan generation, not real query execution. So, we can not see the result in the Spark Web UI. Please use the following query script. The result is **25.78 sec** -> **12.36 sec** as expected.

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

def time[R](block: => R): R = {
  val t0 = System.nanoTime()
  val result = block
  println("Elapsed time: " + ((System.nanoTime - t0) / 1e9) + "s")
  result
}
```

**Before**
```scala
scala> time(sql(query))
Elapsed time: 30.138142577s  // First query has a little overhead of initialization.
res0: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
scala> time(sql(query))
Elapsed time: 25.787751452s  // Let's compare this one.
res1: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
```

**After**
```scala
scala> time(sql(query))
Elapsed time: 17.500279659s  // First query has a little overhead of initialization.
res0: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
scala> time(sql(query))
Elapsed time: 12.364812255s  // This shows the real difference. The speed up is about 2 times.
res1: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields]
```

## How was this patch tested?

Manual by the above script.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14044 from dongjoon-hyun/SPARK-16360.
2016-07-05 16:19:22 +08:00
Koert Kuipers 8cdb81fa82 [SPARK-15204][SQL] improve nullability inference for Aggregator
## What changes were proposed in this pull request?

TypedAggregateExpression sets nullable based on the schema of the outputEncoder

## How was this patch tested?

Add test in DatasetAggregatorSuite

Author: Koert Kuipers <koert@tresata.com>

Closes #13532 from koertkuipers/feat-aggregator-nullable.
2016-07-04 12:14:14 +08:00
Dongjoon Hyun 88134e7368 [SPARK-16288][SQL] Implement inline table generating function
## What changes were proposed in this pull request?

This PR implements `inline` table generating function.

## How was this patch tested?

Pass the Jenkins tests with new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13976 from dongjoon-hyun/SPARK-16288.
2016-07-04 01:57:45 +08:00