Commit graph

7298 commits

Author SHA1 Message Date
Shixiong Zhu 4035c98a0c
[SPARK-26042][SS][TESTS] Fix a potential hang in KafkaContinuousSourceTopicDeletionSuite
## What changes were proposed in this pull request?

As initializing lazy vals shares the same lock, a thread is trying to initialize `executedPlan` when `isRDD` is running, this thread will hang forever.

This PR just materializes `executedPlan` so that accessing it when `toRdd` is running doesn't need to wait for a lock

## How was this patch tested?

Jenkins

Closes #23023 from zsxwing/SPARK-26042.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-11-14 10:19:20 -08:00
hyukjinkwon a7a331df6e [SPARK-26032][PYTHON] Break large sql/tests.py files into smaller files
## What changes were proposed in this pull request?

This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file!

This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context.

We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy.

Basically this PR proposes to break down `pyspark/sql/tests.py` into ...:

```bash
pyspark
...
├── sql
...
│   ├── tests  # Includes all tests broken down from 'pyspark/sql/tests.py'
│   │   │      # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can
│   │   │      # be added. For instance, 'test_arrow.py', 'test_datasources.py' ...
│   │   ├── __init__.py
│   │   ├── test_appsubmit.py
│   │   ├── test_arrow.py
│   │   ├── test_catalog.py
│   │   ├── test_column.py
│   │   ├── test_conf.py
│   │   ├── test_context.py
│   │   ├── test_dataframe.py
│   │   ├── test_datasources.py
│   │   ├── test_functions.py
│   │   ├── test_group.py
│   │   ├── test_pandas_udf.py
│   │   ├── test_pandas_udf_grouped_agg.py
│   │   ├── test_pandas_udf_grouped_map.py
│   │   ├── test_pandas_udf_scalar.py
│   │   ├── test_pandas_udf_window.py
│   │   ├── test_readwriter.py
│   │   ├── test_serde.py
│   │   ├── test_session.py
│   │   ├── test_streaming.py
│   │   ├── test_types.py
│   │   ├── test_udf.py
│   │   └── test_utils.py
...
├── testing  # Includes testing utils that can be used in unittests.
│   ├── __init__.py
│   └── sqlutils.py
...
```

## How was this patch tested?

Existing tests should cover.

`cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran.

Each test (not officially) can be ran via:

```
SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar
```

Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`.

Closes #23021 from HyukjinKwon/SPARK-25344.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-14 14:51:11 +08:00
Liang-Chi Hsieh f26cd18816 [SPARK-25942][SQL] Aggregate expressions shouldn't be resolved on AppendColumns
## What changes were proposed in this pull request?

`Dataset.groupByKey` will bring in new attributes from serializer. If key type is the same as original Dataset's object type, they have same serializer output and so the attribute names will conflict.

This won't be a problem at most of cases, if we don't refer conflict attributes:

```scala
val ds: Dataset[(ClassData, Long)] = Seq(ClassData("one", 1), ClassData("two", 2)).toDS()
  .map(c => ClassData(c.a, c.b + 1))
  .groupByKey(p => p).count()
```

But if we use conflict attributes, `Analyzer` will complain about ambiguous references:

```scala
val ds = Seq(1, 2, 3).toDS()
val agg = ds.groupByKey(_ >= 2).agg(sum("value").as[Long], sum($"value" + 1).as[Long])
```

We have discussed two fixes https://github.com/apache/spark/pull/22944#discussion_r230977212:

1. Implicitly add alias to key attribute:

Works for primitive type. But for product type, we can't implicitly add aliases to key attributes because we might need to access key attributes by  names in methods like `mapGroups`.

2. Detect conflict from key attributes and warn users to add alias manually

This might work, but needs to add some hacks to Analyzer or AttributeSeq.resolve.

This patch applies another simpler fix. We resolve aggregate expressions with `AppendColumns`'s children, instead of `AppendColumns`. `AppendColumns`'s output contains its children's output and serializer output, aggregate expressions shouldn't use serializer output.

## How was this patch tested?

Added test.

Closes #22944 from viirya/dataset_agg.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-14 01:57:10 +08:00
Marco Gaido 4b955625ee [SPARK-25482][SQL] Avoid pushdown of subqueries to data source filters
## What changes were proposed in this pull request?

An expressions with a subquery can be pushed down as a data source filter. Despite the filter is not actively used, this causes anyway a re-execution of the subquery becuase the `ReuseSubquery` optimization rule is ineffective in this case.

The PR avoids this problem by forbidding the push down of filters containing a subquery.
## How was this patch tested?

added UT

Closes #22518 from mgaido91/SPARK-25482.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-14 01:52:33 +08:00
mu5358271 a7a051afa8 [SPARK-25947][SQL] Reduce memory usage in ShuffleExchangeExec by selecting only the sort columns
## What changes were proposed in this pull request?

When sorting rows, ShuffleExchangeExec uses the entire row instead of just the columns references in SortOrder to create the RangePartitioner. This causes the RangePartitioner to sample entire rows to create rangeBounds and can cause OOM issues on the driver when rows contain large fields.

This change creates a projection and only use columns involved in the SortOrder for the RangePartitioner

## How was this patch tested?

Existing tests in spark-sql.

Plus

Started a local spark-shell with a small spark.driver.maxResultSize:

```
spark-shell --master 'local[16]' --conf spark.driver.maxResultSize=128M --driver-memory 4g
```

and ran the following script:

```
import com.google.common.io.Files
import org.apache.spark.SparkContext
import org.apache.spark.sql.SparkSession

import scala.util.Random

transient val sc = SparkContext.getOrCreate()
transient val spark = SparkSession.builder().getOrCreate()

import spark.implicits._

val path = Files.createTempDir().toString

// this creates a dataset with 1024 entries, each 1MB in size, across 16 partitions
sc.parallelize(0 until (1 << 10), sc.defaultParallelism).
  map(_ => Array.fill(1 << 18)(Random.nextInt)).
  toDS.
  write.mode("overwrite").parquet(path)

spark.read.parquet(path).
  orderBy('value (0)).
  write.mode("overwrite").parquet(s"$path-sorted")

spark.read.parquet(s"$path-sorted").show
```
execution would fail when initializing RangePartitioner without this change.
execution succeeds and generates a correctly sorted dataset with this change.

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

Closes #22961 from mu5358271/sort-improvement.

Authored-by: mu5358271 <shuheng.dai@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-14 00:25:57 +08:00
Maxim Gekk 44683e0f7b [SPARK-26023][SQL] Dumping truncated plans and generated code to a file
## What changes were proposed in this pull request?

In the PR, I propose new method for debugging queries by dumping info about their execution to a file. It saves logical, optimized and physical plan similar to the `explain()` method + generated code. One of the advantages of the method over `explain` is it does not materializes full output as one string in memory which can cause OOMs.

## How was this patch tested?

Added a few tests to `QueryExecutionSuite` to check positive and negative scenarios.

Closes #23018 from MaxGekk/truncated-plan-to-file.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2018-11-13 15:23:35 +01:00
Marco Gaido 8d7dbde914 [SPARK-26003] Improve SQLAppStatusListener.aggregateMetrics performance
## What changes were proposed in this pull request?

In `SQLAppStatusListener.aggregateMetrics`, we use the `metricIds` only to filter the relevant metrics. And this is a Seq which is also sorted. When there are many metrics involved, this can be pretty inefficient. The PR proposes to use a Set for it.

## How was this patch tested?

NA

Closes #23002 from mgaido91/SPARK-26003.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-13 13:47:25 +08:00
Maxim Gekk c491934377 [SPARK-26007][SQL] DataFrameReader.csv() respects to spark.sql.columnNameOfCorruptRecord
## What changes were proposed in this pull request?

Passing current value of SQL config `spark.sql.columnNameOfCorruptRecord` to `CSVOptions` inside of `DataFrameReader`.`csv()`.

## How was this patch tested?

Added a test where default value of `spark.sql.columnNameOfCorruptRecord` is changed.

Closes #23006 from MaxGekk/csv-corrupt-sql-config.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-13 12:26:19 +08:00
Yuanjian Li d0ae48497c [SPARK-25949][SQL] Add test for PullOutPythonUDFInJoinCondition
## What changes were proposed in this pull request?

As comment in https://github.com/apache/spark/pull/22326#issuecomment-424923967, we test the new added optimizer rule by end-to-end test in python side, need to add suites under `org.apache.spark.sql.catalyst.optimizer` like other optimizer rules.

## How was this patch tested?
new added UT

Closes #22955 from xuanyuanking/SPARK-25949.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-12 15:16:15 +08:00
Sean Owen 2d085c13b7 [SPARK-25984][CORE][SQL][STREAMING] Remove deprecated .newInstance(), primitive box class constructor calls
## What changes were proposed in this pull request?

Deprecated in Java 11, replace Class.newInstance with Class.getConstructor.getInstance, and primtive wrapper class constructors with valueOf or equivalent

## How was this patch tested?

Existing tests.

Closes #22988 from srowen/SPARK-25984.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-10 09:52:14 -06:00
Dongjoon Hyun d66a4e82ec [SPARK-25102][SQL] Write Spark version to ORC/Parquet file metadata
## What changes were proposed in this pull request?

Currently, Spark writes Spark version number into Hive Table properties with `spark.sql.create.version`.
```
parameters:{
  spark.sql.sources.schema.part.0={
    "type":"struct",
    "fields":[{"name":"a","type":"integer","nullable":true,"metadata":{}}]
  },
  transient_lastDdlTime=1541142761,
  spark.sql.sources.schema.numParts=1,
  spark.sql.create.version=2.4.0
}
```

This PR aims to write Spark versions to ORC/Parquet file metadata with `org.apache.spark.sql.create.version` because we used `org.apache.` prefix in Parquet metadata already. It's different from Hive Table property key `spark.sql.create.version`, but it seems that we cannot change Hive Table property for backward compatibility.

After this PR, ORC and Parquet file generated by Spark will have the following metadata.

**ORC (`native` and `hive` implmentation)**
```
$ orc-tools meta /tmp/o
File Version: 0.12 with ...
...
User Metadata:
  org.apache.spark.sql.create.version=3.0.0
```

**PARQUET**
```
$ parquet-tools meta /tmp/p
...
creator:     parquet-mr version 1.10.0 (build 031a6654009e3b82020012a18434c582bd74c73a)
extra:       org.apache.spark.sql.create.version = 3.0.0
extra:       org.apache.spark.sql.parquet.row.metadata = {"type":"struct","fields":[{"name":"id","type":"long","nullable":false,"metadata":{}}]}
```

## How was this patch tested?

Pass the Jenkins with newly added test cases.

This closes #22255.

Closes #22932 from dongjoon-hyun/SPARK-25102.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-09 22:42:48 -08:00
Gengliang Wang 1db799795c [SPARK-25979][SQL] Window function: allow parentheses around window reference
## What changes were proposed in this pull request?

Very minor parser bug, but possibly problematic for code-generated queries:

Consider the following two queries:
```
SELECT avg(k) OVER (w) FROM kv WINDOW w AS (PARTITION BY v ORDER BY w) ORDER BY 1
```
and
```
SELECT avg(k) OVER w FROM kv WINDOW w AS (PARTITION BY v ORDER BY w) ORDER BY 1
```
The former, with parens around the OVER condition, fails to parse while the latter, without parens, succeeds:
```
Error in SQL statement: ParseException:
mismatched input '(' expecting {<EOF>, ',', 'FROM', 'WHERE', 'GROUP', 'ORDER', 'HAVING', 'LIMIT', 'LATERAL', 'WINDOW', 'UNION', 'EXCEPT', 'MINUS', 'INTERSECT', 'SORT', 'CLUSTER', 'DISTRIBUTE'}(line 1, pos 19)

== SQL ==
SELECT avg(k) OVER (w) FROM kv WINDOW w AS (PARTITION BY v ORDER BY w) ORDER BY 1
-------------------^^^
```
This was found when running the cockroach DB tests.

I tried PostgreSQL, The SQL with parentheses  is also workable.

## How was this patch tested?

Unit test

Closes #22987 from gengliangwang/windowParentheses.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-09 09:44:04 -08:00
gatorsmile 657fd00b52 [SPARK-25988][SQL] Keep names unchanged when deduplicating the column names in Analyzer
## What changes were proposed in this pull request?
When the queries do not use the column names with the same case, users might hit various errors. Below is a typical test failure they can hit.
```
Expected only partition pruning predicates: ArrayBuffer(isnotnull(tdate#237), (cast(tdate#237 as string) >= 2017-08-15));
org.apache.spark.sql.AnalysisException: Expected only partition pruning predicates: ArrayBuffer(isnotnull(tdate#237), (cast(tdate#237 as string) >= 2017-08-15));
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.prunePartitionsByFilter(ExternalCatalogUtils.scala:146)
	at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.listPartitionsByFilter(InMemoryCatalog.scala:560)
	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.listPartitionsByFilter(SessionCatalog.scala:925)
```

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

Closes #22990 from gatorsmile/fix1283.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-09 08:22:26 -08:00
Yuming Wang 0558d021cc [SPARK-25510][SQL][TEST][FOLLOW-UP] Remove BenchmarkWithCodegen
## What changes were proposed in this pull request?

Remove `BenchmarkWithCodegen` as we don't use it anymore.
More details: https://github.com/apache/spark/pull/22484#discussion_r221397904

## How was this patch tested?

N/A

Closes #22985 from wangyum/SPARK-25510.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-09 11:45:03 +08:00
Maxim Gekk 79551f558d [SPARK-25945][SQL] Support locale while parsing date/timestamp from CSV/JSON
## What changes were proposed in this pull request?

In the PR, I propose to add new option `locale` into CSVOptions/JSONOptions to make parsing date/timestamps in local languages possible. Currently the locale is hard coded to `Locale.US`.

## How was this patch tested?

Added two tests for parsing a date from CSV/JSON - `ноя 2018`.

Closes #22951 from MaxGekk/locale.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-09 09:45:06 +08:00
Wenchen Fan 973f7c01df
[MINOR] update HiveExternalCatalogVersionsSuite to test 2.4.0
## What changes were proposed in this pull request?

Since Spark 2.4.0 is released, we should test it in HiveExternalCatalogVersionsSuite

## How was this patch tested?

N/A

Closes #22984 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-08 15:49:36 -08:00
Gengliang Wang 7bb901aa28
[SPARK-25964][SQL][MINOR] Revise OrcReadBenchmark/DataSourceReadBenchmark case names and execution instructions
## What changes were proposed in this pull request?

1. OrcReadBenchmark is under hive module, so the way to run it should be
```
build/sbt "hive/test:runMain <this class>"
```

2. The benchmark "String with Nulls Scan" should be with case "String with Nulls Scan(5%/50%/95%)", not "(0.05%/0.5%/0.95%)"

3. Add the null value percentages in the test case names of DataSourceReadBenchmark, for the benchmark "String with Nulls Scan" .

## How was this patch tested?

Re-run benchmarks

Closes #22965 from gengliangwang/fixHiveOrcReadBenchmark.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-08 10:08:14 -08:00
Dongjoon Hyun 6abe90625e [SPARK-25676][SQL][FOLLOWUP] Use 'foreach(_ => ())'
## What changes were proposed in this pull request?

#22970 fixed Scala 2.12 build error, and this PR updates the function according to the review comments.

## How was this patch tested?

This is also manually tested with Scala 2.12 build.

Closes #22978 from dongjoon-hyun/SPARK-25676-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-08 23:37:14 +08:00
Takuya UESHIN 0d7396f3af
[SPARK-22827][SQL][FOLLOW-UP] Throw SparkOutOfMemoryError in HashAggregateExec, too.
## What changes were proposed in this pull request?

This is a follow-up pr of #20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError` is thrown.
We should throw `SparkOutOfMemoryError` in `HashAggregateExec`, too.

## How was this patch tested?

Existing tests.

Closes #22969 from ueshin/issues/SPARK-22827/oome.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-08 03:51:55 -08:00
Dongjoon Hyun a3004d084c
[SPARK-25971][SQL] Ignore partition byte-size statistics in SQLQueryTestSuite
## What changes were proposed in this pull request?

Currently, `SQLQueryTestSuite` is sensitive in terms of the bytes of parquet files in table partitions. If we change the default file format (from Parquet to ORC) or update the metadata of them, the test case should be changed accordingly. This PR aims to make `SQLQueryTestSuite` more robust by ignoring the partition byte statistics.
```
-Partition Statistics   1144 bytes, 2 rows
+Partition Statistics   [not included in comparison] bytes, 2 rows
```

## How was this patch tested?

Pass the Jenkins with the newly updated test cases.

Closes #22972 from dongjoon-hyun/SPARK-25971.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-08 03:40:28 -08:00
hyukjinkwon 0a2e45fdb8 Revert "[SPARK-23831][SQL] Add org.apache.derby to IsolatedClientLoader"
This reverts commit a75571b46f.
2018-11-08 16:32:25 +08:00
Maxim Gekk ee03f760b3 [SPARK-25955][TEST] Porting JSON tests for CSV functions
## What changes were proposed in this pull request?

In the PR, I propose to port existing JSON tests from `JsonFunctionsSuite` that are applicable for CSV, and put them to `CsvFunctionsSuite`. In particular:
- roundtrip `from_csv` to `to_csv`, and `to_csv` to `from_csv`
- using `schema_of_csv` in `from_csv`
- Java API `from_csv`
- using `from_csv` and `to_csv` in exprs.

Closes #22960 from MaxGekk/csv-additional-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-08 14:51:29 +08:00
Maxim Gekk 17449a2e6b [SPARK-25952][SQL] Passing actual schema to JacksonParser
## What changes were proposed in this pull request?

The PR fixes an issue when the corrupt record column specified via `spark.sql.columnNameOfCorruptRecord` or JSON options `columnNameOfCorruptRecord` is propagated to JacksonParser, and returned row breaks an assumption in `FailureSafeParser` that the row must contain only actual data. The issue is fixed by passing actual schema without the corrupt record field into `JacksonParser`.

## How was this patch tested?

Added a test with the corrupt record column in the middle of user's schema.

Closes #22958 from MaxGekk/from_json-corrupt-record-schema.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-08 14:48:23 +08:00
Dongjoon Hyun d68f3a726f
[SPARK-25676][FOLLOWUP][BUILD] Fix Scala 2.12 build error
## What changes were proposed in this pull request?

This PR fixes the Scala-2.12 build.

## How was this patch tested?

Manual build with Scala-2.12 profile.

Closes #22970 from dongjoon-hyun/SPARK-25676-2.12.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-11-08 05:54:48 +00:00
Sean Owen 0025a8397f [SPARK-25908][CORE][SQL] Remove old deprecated items in Spark 3
## What changes were proposed in this pull request?

- Remove some AccumulableInfo .apply() methods
- Remove non-label-specific multiclass precision/recall/fScore in favor of accuracy
- Remove toDegrees/toRadians in favor of degrees/radians (SparkR: only deprecated)
- Remove approxCountDistinct in favor of approx_count_distinct (SparkR: only deprecated)
- Remove unused Python StorageLevel constants
- Remove Dataset unionAll in favor of union
- Remove unused multiclass option in libsvm parsing
- Remove references to deprecated spark configs like spark.yarn.am.port
- Remove TaskContext.isRunningLocally
- Remove ShuffleMetrics.shuffle* methods
- Remove BaseReadWrite.context in favor of session
- Remove Column.!== in favor of =!=
- Remove Dataset.explode
- Remove Dataset.registerTempTable
- Remove SQLContext.getOrCreate, setActive, clearActive, constructors

Not touched yet

- everything else in MLLib
- HiveContext
- Anything deprecated more recently than 2.0.0, generally

## How was this patch tested?

Existing tests

Closes #22921 from srowen/SPARK-25908.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-07 22:48:50 -06:00
Imran Rashid 8fbc1830f9 [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue
JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never try to allocate an array that big.  This commit changes some defaults & configs to gracefully fallover to something that doesn't require one large array in some cases; in other cases it simply improves an error message for cases which will still fail.

Closes #22818 from squito/SPARK-25827.

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

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

## How was this patch tested?

unit tests

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

Closes #22943 from wangyum/SPARK-25098.

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

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

## How was this patch tested?

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

Closes #22956 from MaxGekk/csv-tests.

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

Refactor BenchmarkWideTable to use main method.
Generate benchmark result:

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

## How was this patch tested?

manual tests

Closes #22823 from yucai/BenchmarkWideTable.

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

Upgrade ASM to 7.x to support JDK11

## How was this patch tested?

Existing tests.

Closes #22953 from dbtsai/asm7.

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

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

## How was this patch tested?

Existing tests.

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

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

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

## How was this patch tested?

Existing tests.

Closes #22950 from srowen/Typos.

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

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

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

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

Closes #22914 from shahidki31/pageFallBack.

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

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

## How was this patch tested?
manual tests

Closes #22847 from yucai/splitThreshold.

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

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

## How was this patch tested?

Existing tests.

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

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

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

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

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

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

## How was this patch tested?

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

Closes #22626 from MaxGekk/to_csv.

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

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

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

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

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

## How was this patch tested?
Existing tests

Closes #22864 from shahidki31/unusedCode.

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

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

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

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

## How was this patch tested?

Pass the Jenkins

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

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

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

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

## How was this patch tested?

existing tests

Closes #22898 from cloud-fan/minor.

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

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

## How was this patch tested?

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

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

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

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

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

## How was this patch tested?

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

Closes #22666 from MaxGekk/schema_of_csv-function.

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

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

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

Let’s consider a few examples.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

## How was this patch tested?

This PR comes with a set of dedicated tests.

Closes #22857 from aokolnychyi/spark-25860.

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

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

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

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

## How was this patch tested?

Manual.

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

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

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

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

## How was this patch tested?
Unit test.

Closes #22895 from gengliangwang/improve_error_msg.

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

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

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

```

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

```
## How was this patch tested?

the Existed test cases.

Closes #21860 from heary-cao/fastHashMap.

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

Refactor BuiltInDataSourceWriteBenchmark, DataSourceWriteBenchmark and AvroWriteBenchmark to use main method.

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

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

manual tests

Closes #22861 from yucai/BuiltInDataSourceWriteBenchmark.

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

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

Closes #22870 from rxin/SPARK-25862.

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

Refactor JSONBenchmark to use main method

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

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

## How was this patch tested?

manual tests

Closes #22844 from heary-cao/JSONBenchmarks.

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

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

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

## How was this patch tested?

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

Closes #22713 from mgaido91/SPARK-25691.

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

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

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

## How was this patch tested?

manual tests

Closes #22845 from heary-cao/CSVBenchmarks.

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

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

## How was this patch tested?

add new unit tested.

Closes #22755 from heary-cao/AddTestToBroadcastHashJoinExec.

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

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

## How was this patch tested?

Manually verified.

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

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

Closes #22856 from dilipbiswal/average_cleanup.

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

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

## How was this patch tested?

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

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

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

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

## How was this patch tested?

manual tests

Closes #22872 from yucai/main_args.

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

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

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

Tests also bear this out.

## How was this patch tested?

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

Closes #22865 from bersprockets/confdocfix.

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

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

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

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

## How was this patch tested?

added UT

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

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

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

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

Closes #22802 from 10110346/FileSplitnotneed.

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

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

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

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

## How was this patch tested?

Add new test case in `DataFrameWindowFramesSuite`

Closes #22853 from jiangxb1987/windowBoundary.

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

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

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

## How was this patch tested?

enable a old test case

Closes #22812 from cloud-fan/map.

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

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

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

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

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

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

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

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

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

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

## How was this patch tested?

N/A

Closes #22850 from laskfla/HiveUtils-Comment.

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

Remove SQLContext methods deprecated in 1.4

## How was this patch tested?

Existing tests.

Closes #22815 from srowen/SPARK-25821.

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

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

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

## How was this patch tested?

Existing unit tests.

Closes #22848 from gengliangwang/fixSQLListenerWarning.

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

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

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

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

## How was this patch tested?

Unit tests were added.

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

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

This patch also reverts 8acb51f08b which applies to 2.4.

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

Closes #22841 from rxin/SPARK-25842.

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

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

0e07b483d2 is a patch to reproduce this issue.

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

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

## How was this patch tested?

Jenkins.

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

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

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

This PR removes `GetArrayFromMap`

## How was this patch tested?

existing tests

Closes #22825 from cloud-fan/minor.

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

Refactor ObjectHashAggregateExecBenchmark to use main method

## How was this patch tested?

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

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

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

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

To summarize the proposed changes:

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

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

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

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

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

## How was this patch tested?

Existing tests.

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

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

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

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

Author: adrian555 <v2ave10p>

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

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

## How was this patch tested?

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

Closes #22237 from MaxGekk/from_json-failuresafe.

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

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

## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

Added a test case.
Built complete project on travis.

viirya kiszk cloud-fan michalsenkyr marmbrus liancheng

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

Lead-authored-by: Vladimir Kuriatkov <vofque@gmail.com>
Co-authored-by: Vladimir Kuriatkov <Vladimir_Kuriatkov@epam.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-24 09:29:40 +08:00
Liang-Chi Hsieh 736fc03930 [SPARK-25791][SQL] Datatype of serializers in RowEncoder should be accessible
## What changes were proposed in this pull request?

The serializers of `RowEncoder` use few `If` Catalyst expression which inherits `ComplexTypeMergingExpression` that will check input data types.

It is possible to generate serializers which fail the check and can't to access the data type of serializers. When producing If expression, we should use the same data type at its input expressions.

## How was this patch tested?

Added test.

Closes #22785 from viirya/SPARK-25791.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-23 22:02:14 +08:00
Imran Rashid 78c8bd2e68 [SPARK-25805][SQL][TEST] Fix test for SPARK-25159
The original test would sometimes fail if the listener bus did not keep
up, so just wait till the listener bus is empty.  Tested by adding a
sleep in the listener, which made the test consistently fail without the
fix, but pass consistently after the fix.

Closes #22799 from squito/SPARK-25805.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-23 14:20:29 +08:00
Liang-Chi Hsieh 03e82e3689 [SPARK-25040][SQL] Empty string for non string types should be disallowed
## What changes were proposed in this pull request?

This takes over original PR at #22019. The original proposal is to have null for float and double types. Later a more reasonable proposal is to disallow empty strings. This patch adds logic to throw exception when finding empty strings for non string types.

## How was this patch tested?

Added test.

Closes #22787 from viirya/SPARK-25040.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-23 13:43:53 +08:00
Liang-Chi Hsieh ff9ede0929 [SPARK-25627][TEST] Reduce test time for ContinuousStressSuite
## What changes were proposed in this pull request?

This goes to reduce test time for ContinuousStressSuite - from 8 mins 13 sec to 43 seconds.

The approach taken by this is to reduce the triggers and epochs to wait and to reduce the expected rows accordingly.

## How was this patch tested?

Existing tests.

Closes #22662 from viirya/SPARK-25627.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-22 13:18:29 -05:00
hyukjinkwon b8c6ba9e64
[SPARK-25779][SQL][TESTS] Remove SQL query tests for function documentation by DESCRIBE FUNCTION at SQLQueryTestSuite
Currently, there are some tests testing function descriptions:

```bash
$ grep -ir "describe function" sql/core/src/test/resources/sql-tests/inputs
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function to_json;
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function extended to_json;
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function from_json;
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function extended from_json;
```

Looks there are not quite good points about testing them since we're not going to test documentation itself.
For `DESCRIBE FCUNTION` functionality itself, they are already being tested here and there.
See the test failures in https://github.com/apache/spark/pull/18749 (where I added examples to function descriptions)

We better remove those tests so that people don't add such tests in the SQL tests.

## How was this patch tested?

Manual.

Closes #22776 from HyukjinKwon/SPARK-25779.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-20 18:02:38 -07:00
Wenchen Fan ab5752cb95
[SPARK-25747][SQL] remove ColumnarBatchScan.needsUnsafeRowConversion
## What changes were proposed in this pull request?

`needsUnsafeRowConversion` is used in 2 places:
1. `ColumnarBatchScan.produceRows`
2. `FileSourceScanExec.doExecute`

When we hit `ColumnarBatchScan.produceRows`, it means whole stage codegen is on but the vectorized reader is off. The vectorized reader can be off for several reasons:
1. the file format doesn't have a vectorized reader(json, csv, etc.)
2. the vectorized reader config is off
3. the schema is not supported

Anyway when the vectorized reader is off, file format reader will always return unsafe rows, and other `ColumnarBatchScan` implementations also always return unsafe rows, so `ColumnarBatchScan.needsUnsafeRowConversion` is not needed.

When we hit `FileSourceScanExec.doExecute`, it means whole stage codegen is off. For this case, we need the `needsUnsafeRowConversion` to convert `ColumnarRow` to `UnsafeRow`, if the file format reader returns batch.

This PR removes `ColumnarBatchScan.needsUnsafeRowConversion`, and keep this flag only in `FileSourceScanExec`

## How was this patch tested?

existing tests

Closes #22750 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-20 17:45:04 -07:00
Yuming Wang 62551cceeb
[SPARK-25492][TEST] Refactor WideSchemaBenchmark to use main method
## What changes were proposed in this pull request?

Refactor `WideSchemaBenchmark` to use main method.
1. use `spark-submit`:
```console
bin/spark-submit --class  org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark --jars ./core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar ./sql/core/target/spark-sql_2.11-3.0.0-SNAPSHOT-tests.jar
```

2. Generate benchmark result:
```console
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark"
```

## How was this patch tested?

manual tests

Closes #22501 from wangyum/SPARK-25492.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-20 17:31:13 -07:00
hyukjinkwon 3370865b0e [SPARK-25785][SQL] Add prettyNames for from_json, to_json, from_csv, and schema_of_json
## What changes were proposed in this pull request?

This PR adds `prettyNames` for `from_json`, `to_json`, `from_csv`, and `schema_of_json` so that appropriate names are used.

## How was this patch tested?

Unit tests

Closes #22773 from HyukjinKwon/minor-prettyNames.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-20 10:15:53 +08:00
Yuming Wang 9ad0f6ea89
[SPARK-25269][SQL] SQL interface support specify StorageLevel when cache table
## What changes were proposed in this pull request?

SQL interface support specify `StorageLevel` when cache table. The semantic is:
```sql
CACHE TABLE tableName OPTIONS('storageLevel' 'DISK_ONLY');
```
All supported `StorageLevel` are:
eefdf9f9dd/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala (L172-L183)

## How was this patch tested?

unit tests and manual tests.

manual tests configuration:
```
--executor-memory 15G --executor-cores 5 --num-executors 50
```
Data:
Input Size / Records: 1037.7 GB / 11732805788

Result:
![image](https://user-images.githubusercontent.com/5399861/47213362-56a1c980-d3cd-11e8-82e7-28d7abc5923e.png)

Closes #22263 from wangyum/SPARK-25269.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-19 09:15:55 -07:00
Peter Toth f38594fc56 [SPARK-25768][SQL] fix constant argument expecting UDAFs
## What changes were proposed in this pull request?

Without this PR some UDAFs like `GenericUDAFPercentileApprox` can throw an exception because expecting a constant parameter (object inspector) as a particular argument.

The exception is thrown because `toPrettySQL` call in `ResolveAliases` analyzer rule transforms a `Literal` parameter to a `PrettyAttribute` which is then transformed to an `ObjectInspector` instead of a `ConstantObjectInspector`.
The exception comes from `getEvaluator` method of `GenericUDAFPercentileApprox` that actually shouldn't be called during `toPrettySQL` transformation. The reason why it is called are the non lazy fields in `HiveUDAFFunction`.

This PR makes all fields of `HiveUDAFFunction` lazy.

## How was this patch tested?

added new UT

Closes #22766 from peter-toth/SPARK-25768.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-19 21:17:14 +08:00
maryannxue e8167768cf [SPARK-25044][FOLLOW-UP] Change ScalaUDF constructor signature
## What changes were proposed in this pull request?

This is a follow-up PR for #22259. The extra field added in `ScalaUDF` with the original PR was declared optional, but should be indeed required, otherwise callers of `ScalaUDF`'s constructor could ignore this new field and cause the result to be incorrect. This PR makes the new field required and changes its name to `handleNullForInputs`.

#22259 breaks the previous behavior for null-handling of primitive-type input parameters. For example, for `val f = udf({(x: Int, y: Any) => x})`, `f(null, "str")` should return `null` but would return `0` after #22259. In this PR, all UDF methods except `def udf(f: AnyRef, dataType: DataType): UserDefinedFunction` have been restored with the original behavior. The only exception is documented in the Spark SQL migration guide.

In addition, now that we have this extra field indicating if a null-test should be applied on the corresponding input value, we can also make use of this flag to avoid the rule `HandleNullInputsForUDF` being applied infinitely.

## How was this patch tested?
Added UT in UDFSuite

Passed affected existing UTs:
AnalysisSuite
UDFSuite

Closes #22732 from maryannxue/spark-25044-followup.

Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-19 21:03:59 +08:00
Russell Spitzer 6e0fc8b0fc [SPARK-25560][SQL] Allow FunctionInjection in SparkExtensions
This allows an implementer of Spark Session Extensions to utilize a
method "injectFunction" which will add a new function to the default
Spark Session Catalogue.

## What changes were proposed in this pull request?

Adds a new function to SparkSessionExtensions

    def injectFunction(functionDescription: FunctionDescription)

Where function description is a new type

  type FunctionDescription = (FunctionIdentifier, FunctionBuilder)

The functions are loaded in BaseSessionBuilder when the function registry does not have a parent
function registry to get loaded from.

## How was this patch tested?

New unit tests are added for the extension in SparkSessionExtensionSuite

Closes #22576 from RussellSpitzer/SPARK-25560.

Authored-by: Russell Spitzer <Russell.Spitzer@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2018-10-19 10:40:56 +02:00
Justin Uang 1e6c1d8bfb [SPARK-25493][SQL] Use auto-detection for CRLF in CSV datasource multiline mode
## What changes were proposed in this pull request?

CSVs with windows style crlf ('\r\n') don't work in multiline mode. They work fine in single line mode because the line separation is done by Hadoop, which can handle all the different types of line separators. This PR fixes it by enabling Univocity's line separator detection in multiline mode, which will detect '\r\n', '\r', or '\n' automatically as it is done by hadoop in single line mode.

## How was this patch tested?

Unit test with a file with crlf line endings.

Closes #22503 from justinuang/fix-clrf-multiline.

Authored-by: Justin Uang <juang@palantir.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-19 11:13:02 +08:00
Yuming Wang 1117fc35ff
[SPARK-25760][SQL] Set AddJarCommand return empty
## What changes were proposed in this pull request?

Only `AddJarCommand` return `0`, the user will be confused about what it means. This PR sets it to empty.

```sql
spark-sql> add jar /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar;
ADD JAR /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar
0
spark-sql>
```

## How was this patch tested?

manual tests
```sql
spark-sql> add jar /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar;
ADD JAR /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar
spark-sql>
```

Closes #22747 from wangyum/AddJarCommand.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-18 09:19:42 -07:00
Russell Spitzer c3eaee7765 [SPARK-25003][PYSPARK] Use SessionExtensions in Pyspark
Master

## What changes were proposed in this pull request?

Previously Pyspark used the private constructor for SparkSession when
building that object. This resulted in a SparkSession without checking
the sql.extensions parameter for additional session extensions. To fix
this we instead use the Session.builder() path as SparkR uses, this
loads the extensions and allows their use in PySpark.

## How was this patch tested?

An integration test was added which mimics the Scala test for the same feature.

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

Closes #21990 from RussellSpitzer/SPARK-25003-master.

Authored-by: Russell Spitzer <Russell.Spitzer@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-18 12:29:09 +08:00
Gengliang Wang 24f5bbd770 [SPARK-25735][CORE][MINOR] Improve start-thriftserver.sh: print clean usage and exit with code 1
## What changes were proposed in this pull request?
Currently if we run
```
sh start-thriftserver.sh -h
```

we get
```
...
Thrift server options:
2018-10-15 21:45:39 INFO HiveThriftServer2:54 - Starting SparkContext
2018-10-15 21:45:40 INFO SparkContext:54 - Running Spark version 2.3.2
2018-10-15 21:45:40 WARN NativeCodeLoader:62 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2018-10-15 21:45:40 ERROR SparkContext:91 - Error initializing SparkContext.
org.apache.spark.SparkException: A master URL must be set in your configuration
at org.apache.spark.SparkContext.<init>(SparkContext.scala:367)
at org.apache.spark.SparkContext$.getOrCreate(SparkContext.scala:2493)
at org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:934)
at org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:925)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:925)
at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.init(SparkSQLEnv.scala:48)
at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2$.main(HiveThriftServer2.scala:79)
at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.main(HiveThriftServer2.scala)
2018-10-15 21:45:40 ERROR Utils:91 - Uncaught exception in thread main
```

After fix, the usage output is clean:
```
...
Thrift server options:
--hiveconf <property=value> Use value for given property
```

Also exit with code 1, to follow other scripts(this is the behavior of parsing option `-h` for other linux commands as well).

## How was this patch tested?

Manual test.

Closes #22727 from gengliangwang/stsUsage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-17 09:56:17 -05:00
Vladimir Kuriatkov e5b8136f47 [SPARK-21402][SQL] Fix java array of structs deserialization
When deserializing values of ArrayType with struct elements in java beans, fields of structs get mixed up.
I suggest using struct data types retrieved from resolved input data instead of inferring them from java beans.

## What changes were proposed in this pull request?

MapObjects expression is used to map array elements to java beans. Struct type of elements is inferred from java bean structure and ends up with mixed up field order.
I used UnresolvedMapObjects instead of MapObjects, which allows to provide element type for MapObjects during analysis based on the resolved input data, not on the java bean.

## How was this patch tested?

Added a test case.
Built complete project on travis.

michalsenkyr cloud-fan marmbrus liancheng

Closes #22708 from vofque/SPARK-21402.

Lead-authored-by: Vladimir Kuriatkov <vofque@gmail.com>
Co-authored-by: Vladimir Kuriatkov <Vladimir_Kuriatkov@epam.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 22:13:05 +08:00
Wenchen Fan 9690eba16e [SPARK-25680][SQL] SQL execution listener shouldn't happen on execution thread
## What changes were proposed in this pull request?

The SQL execution listener framework was created from scratch(see https://github.com/apache/spark/pull/9078). It didn't leverage what we already have in the spark listener framework, and one major problem is, the listener runs on the spark execution thread, which means a bad listener can block spark's query processing.

This PR re-implements the SQL execution listener framework. Now `ExecutionListenerManager` is just a normal spark listener, which watches the `SparkListenerSQLExecutionEnd` events and post events to the
user-provided SQL execution listeners.

## How was this patch tested?

existing tests.

Closes #22674 from cloud-fan/listener.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 16:06:07 +08:00
彭灿00244106 e9332f600e [SQL][CATALYST][MINOR] update some error comments
## What changes were proposed in this pull request?

this PR correct some comment error:
1. change from "as low a possible" to "as low as possible" in RewriteDistinctAggregates.scala
2. delete redundant word “with” in HiveTableScanExec’s  doExecute()  method

## How was this patch tested?

Existing unit tests.

Closes #22694 from CarolinePeng/update_comment.

Authored-by: 彭灿00244106 <00244106@zte.intra>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-17 12:45:13 +08:00
Takeshi Yamamuro a9f685bb70 [SPARK-25734][SQL] Literal should have a value corresponding to dataType
## What changes were proposed in this pull request?
`Literal.value` should have a value a value corresponding to `dataType`. This pr added code to verify it and fixed the existing tests to do so.

## How was this patch tested?
Modified the existing tests.

Closes #22724 from maropu/SPARK-25734.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 11:02:39 +08:00
Maxim Gekk e9af9460bc [SPARK-25393][SQL] Adding new function from_csv()
## What changes were proposed in this pull request?

The PR adds new function `from_csv()` similar to `from_json()` to parse columns with CSV strings. I added the following methods:
```Scala
def from_csv(e: Column, schema: StructType, options: Map[String, String]): Column
```
and this signature to call it from Python, R and Java:
```Scala
def from_csv(e: Column, schema: String, options: java.util.Map[String, String]): Column
```

## How was this patch tested?

Added new test suites `CsvExpressionsSuite`, `CsvFunctionsSuite` and sql tests.

Closes #22379 from MaxGekk/from_csv.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-17 09:32:05 +08:00
Dongjoon Hyun 2c664edc06 [SPARK-25579][SQL] Use quoted attribute names if needed in pushed ORC predicates
## What changes were proposed in this pull request?

This PR aims to fix an ORC performance regression at Spark 2.4.0 RCs from Spark 2.3.2. Currently, for column names with `.`, the pushed predicates are ignored.

**Test Data**
```scala
scala> val df = spark.range(Int.MaxValue).sample(0.2).toDF("col.with.dot")
scala> df.write.mode("overwrite").orc("/tmp/orc")
```

**Spark 2.3.2**
```scala
scala> spark.sql("set spark.sql.orc.impl=native")
scala> spark.sql("set spark.sql.orc.filterPushdown=true")
scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 1542 ms

scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 152 ms
```

**Spark 2.4.0 RC3**
```scala
scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 4074 ms

scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 1771 ms
```

## How was this patch tested?

Pass the Jenkins with a newly added test case.

Closes #22597 from dongjoon-hyun/SPARK-25579.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-16 20:30:23 +08:00
Wenchen Fan e028fd3aed [SPARK-25736][SQL][TEST] add tests to verify the behavior of multi-column count
## What changes were proposed in this pull request?

AFAIK multi-column count is not widely supported by the mainstream databases(postgres doesn't support), and the SQL standard doesn't define it clearly, as near as I can tell.

Since Spark supports it, we should clearly document the current behavior and add tests to verify it.

## How was this patch tested?

N/A

Closes #22728 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-16 15:13:01 +08:00
Yuming Wang 5c7f6b6636 [SPARK-25629][TEST] Reduce ParquetFilterSuite: filter pushdown test time costs in Jenkins
## What changes were proposed in this pull request?

Only test these 4 cases is enough:
be2238fb50/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala (L269-L279)

## How was this patch tested?

Manual tests on my local machine.
before:
```
- filter pushdown - decimal (13 seconds, 683 milliseconds)
```
after:
```
- filter pushdown - decimal (9 seconds, 713 milliseconds)
```

Closes #22636 from wangyum/SPARK-25629.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-16 12:30:02 +08:00
Imran Rashid fdaa99897a [SPARK-25738][SQL] Fix LOAD DATA INPATH for hdfs port
## What changes were proposed in this pull request?

LOAD DATA INPATH didn't work if the defaultFS included a port for hdfs.
Handling this just requires a small change to use the correct URI
constructor.

## How was this patch tested?

Added a unit test, ran all tests via jenkins

Closes #22733 from squito/SPARK-25738.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-15 18:34:30 -07:00