Commit graph

21826 commits

Author SHA1 Message Date
Wenchen Fan b42ad165bb [SPARK-24072][SQL] clearly define pushed filters
## What changes were proposed in this pull request?

filters like parquet row group filter, which is actually pushed to the data source but still to be evaluated by Spark, should also count as `pushedFilters`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21143 from cloud-fan/step1.
2018-04-30 09:13:32 -07:00
Maxim Gekk 3121b411f7 [SPARK-23846][SQL] The samplingRatio option for CSV datasource
## What changes were proposed in this pull request?

I propose to support the `samplingRatio` option for schema inferring of CSV datasource similar to the same option of JSON datasource:
b14993e1fc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala (L49-L50)

## How was this patch tested?

Added 2 tests for json and 2 tests for csv datasources. The tests checks that only subset of input dataset is used for schema inferring.

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

Closes #20959 from MaxGekk/csv-sampling.
2018-04-30 09:45:22 +08:00
hyukjinkwon 56f501e1c0 [MINOR][DOCS] Fix a broken link for Arrow's supported types in the programming guide
## What changes were proposed in this pull request?

This PR fixes a broken link for Arrow's supported types in the programming guide.

## How was this patch tested?

Manually tested via `SKIP_API=1 jekyll watch`.

"Supported SQL Types" here in https://spark.apache.org/docs/latest/sql-programming-guide.html#enabling-for-conversion-tofrom-pandas is broken. It should be https://spark.apache.org/docs/latest/sql-programming-guide.html#supported-sql-types

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21191 from HyukjinKwon/minor-arrow-link.
2018-04-30 09:40:46 +08:00
Maxim Gekk bd14da6fd5 [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support custom encoding for json files
## What changes were proposed in this pull request?

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

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

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

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

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

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

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

## How was this patch tested?

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

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

Closes #20937 from MaxGekk/json-encoding-line-sep.
2018-04-29 11:25:31 +08:00
Yuming Wang 4df51361a5 [SPARK-22732][SS][FOLLOW-UP] Fix MemorySinkV2 toString error
## What changes were proposed in this pull request?

Fix `MemorySinkV2` toString() error

## How was this patch tested?

N/A

Author: Yuming Wang <yumwang@ebay.com>

Closes #21170 from wangyum/SPARK-22732.
2018-04-28 16:57:41 +08:00
Marco Gaido ad94e8592b [SPARK-23736][SQL][FOLLOWUP] Error message should contains SQL types
## What changes were proposed in this pull request?

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

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21181 from mgaido91/SPARK-23736_followup.
2018-04-28 10:47:43 +08:00
Jungtaek Lim 1fb46f30f8 [SPARK-23688][SS] Refactor tests away from rate source
## What changes were proposed in this pull request?

Replace rate source with memory source in continuous mode test suite. Keep using "rate" source if the tests intend to put data periodically in background, or need to put short source name to load, since "memory" doesn't have provider for source.

## How was this patch tested?

Ran relevant test suite from IDE.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21152 from HeartSaVioR/SPARK-23688.
2018-04-28 09:55:56 +08:00
Juliusz Sompolski 8614edd445 [SPARK-24104] SQLAppStatusListener overwrites metrics onDriverAccumUpdates instead of updating them
## What changes were proposed in this pull request?

Event `SparkListenerDriverAccumUpdates` may happen multiple times in a query - e.g. every `FileSourceScanExec` and `BroadcastExchangeExec` call `postDriverMetricUpdates`.
In Spark 2.2 `SQLListener` updated the map with new values. `SQLAppStatusListener` overwrites it.
Unless `update` preserved it in the KV store (dependant on `exec.lastWriteTime`), only the metrics from the last operator that does `postDriverMetricUpdates` are preserved.

## How was this patch tested?

Unit test added.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #21171 from juliuszsompolski/SPARK-24104.
2018-04-27 14:14:28 -07:00
Dilip Biswal 3fd297af6d [SPARK-24085][SQL] Query returns UnsupportedOperationException when scalar subquery is present in partitioning expression
## What changes were proposed in this pull request?
In this case, the partition pruning happens before the planning phase of scalar subquery expressions.
For scalar subquery expressions, the planning occurs late in the cycle (after the physical planning)  in "PlanSubqueries" just before execution. Currently we try to execute the scalar subquery expression as part of partition pruning and fail as it implements Unevaluable.

The fix attempts to ignore the Subquery expressions from partition pruning computation. Another option can be to somehow plan the subqueries before the partition pruning. Since this may not be a commonly occuring expression, i am opting for a simpler fix.

Repro
``` SQL
CREATE TABLE test_prc_bug (
id_value string
)
partitioned by (id_type string)
location '/tmp/test_prc_bug'
stored as parquet;

insert into test_prc_bug values ('1','a');
insert into test_prc_bug values ('2','a');
insert into test_prc_bug values ('3','b');
insert into test_prc_bug values ('4','b');

select * from test_prc_bug
where id_type = (select 'b');
```
## How was this patch tested?
Added test in SubquerySuite and hive/SQLQuerySuite

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

Closes #21174 from dilipbiswal/spark-24085.
2018-04-27 11:43:29 -07:00
Patrick McGloin 2824f12b8b [SPARK-23565][SS] New error message for structured streaming sources assertion
## What changes were proposed in this pull request?

A more informative message to tell you why a structured streaming query cannot continue if you have added more sources, than there are in the existing checkpoint offsets.

## How was this patch tested?

I added a Unit Test.

Author: Patrick McGloin <mcgloin.patrick@gmail.com>

Closes #20946 from patrickmcgloin/master.
2018-04-27 23:04:14 +08:00
eric-maynard 109935fc5d [SPARK-23830][YARN] added check to ensure main method is found
## What changes were proposed in this pull request?

When a user specifies the wrong class -- or, in fact, a class instead of an object -- Spark throws an NPE which is not useful for debugging. This was reported in [SPARK-23830](https://issues.apache.org/jira/browse/SPARK-23830). This PR adds a check to ensure the main method was found and logs a useful error in the event that it's null.

## How was this patch tested?

* Unit tests + Manual testing
* The scope of the changes is very limited

Author: eric-maynard <emaynard@cloudera.com>
Author: Eric Maynard <emaynard@cloudera.com>

Closes #21168 from eric-maynard/feature/SPARK-23830.
2018-04-27 15:25:07 +08:00
Dongjoon Hyun 8aa1d7b0ed [SPARK-23355][SQL] convertMetastore should not ignore table properties
## What changes were proposed in this pull request?

Previously, SPARK-22158 fixed for `USING hive` syntax. This PR aims to fix for `STORED AS` syntax. Although the test case covers ORC part, the patch considers both `convertMetastoreOrc` and `convertMetastoreParquet`.

## How was this patch tested?

Pass newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20522 from dongjoon-hyun/SPARK-22158-2.
2018-04-27 11:00:41 +08:00
zhoukang 9ee9fcf522 [SPARK-24083][YARN] Log stacktrace for uncaught exception
## What changes were proposed in this pull request?

Log stacktrace for uncaught exception

## How was this patch tested?
UT and manually test

Author: zhoukang <zhoukang199191@gmail.com>

Closes #21151 from caneGuy/zhoukang/log-stacktrace.
2018-04-26 15:38:11 -07:00
hyukjinkwon f7435bec6a [SPARK-24044][PYTHON] Explicitly print out skipped tests from unittest module
## What changes were proposed in this pull request?

This PR proposes to remove duplicated dependency checking logics and also print out skipped tests from unittests.

For example, as below:

```
Skipped tests in pyspark.sql.tests with pypy:
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
...

Skipped tests in pyspark.sql.tests with python3:
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
...
```

Currently, it's not printed out in the console. I think we should better print out skipped tests in the console.

## How was this patch tested?

Manually tested. Also, fortunately, Jenkins has good environment to test the skipped output.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21107 from HyukjinKwon/skipped-tests-print.
2018-04-26 15:11:42 -07:00
Huaxin Gao 4f1e38649e [SPARK-24057][PYTHON] put the real data type in the AssertionError message
## What changes were proposed in this pull request?

Print out the data type in the AssertionError message to make it more meaningful.

## How was this patch tested?

I manually tested the changed code on my local, but didn't add any test.

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

Closes #21159 from huaxingao/spark-24057.
2018-04-26 14:21:22 -07:00
gatorsmile ce2f919f8d [SPARK-23799][SQL][FOLLOW-UP] FilterEstimation.evaluateInSet produces wrong stats for STRING
## What changes were proposed in this pull request?
`colStat.min` AND `colStat.max` are empty for string type. Thus, `evaluateInSet` should not return zero when either `colStat.min` or `colStat.max`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21147 from gatorsmile/cached.
2018-04-26 19:07:13 +08:00
Tathagata Das d1eb8d3ddc [SPARK-24094][SS][MINOR] Change description strings of v2 streaming sources to reflect the change
## What changes were proposed in this pull request?

This makes it easy to understand at runtime which version is running. Great for debugging production issues.

## How was this patch tested?
Not necessary.

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

Closes #21160 from tdas/SPARK-24094.
2018-04-25 23:24:05 -07:00
jerryshao ffaf0f9fd4 [SPARK-24062][THRIFT SERVER] Fix SASL encryption cannot enabled issue in thrift server
## What changes were proposed in this pull request?

For the details of the exception please see [SPARK-24062](https://issues.apache.org/jira/browse/SPARK-24062).

The issue is:

Spark on Yarn stores SASL secret in current UGI's credentials, this credentials will be distributed to AM and executors, so that executors and drive share the same secret to communicate. But STS/Hive library code will refresh the current UGI by UGI's loginFromKeytab() after Spark application is started, this will create a new UGI in the current driver's context with empty tokens and secret keys, so secret key is lost in the current context's UGI, that's why Spark driver throws secret key not found exception.

In Spark 2.2 code, Spark also stores this secret key in SecurityManager's class variable, so even UGI is refreshed, the secret is still existed in the object, so STS with SASL can still be worked in Spark 2.2. But in Spark 2.3, we always search key from current UGI, which makes it fail to work in Spark 2.3.

To fix this issue, there're two possible solutions:

1. Fix in STS/Hive library, when a new UGI is refreshed, copy the secret key from original UGI to the new one. The difficulty is that some codes to refresh the UGI is existed in Hive library, which makes us hard to change the code.
2. Roll back the logics in SecurityManager to match Spark 2.2, so that this issue can be fixed.

2nd solution seems a simple one. So I will propose a PR with 2nd solution.

## How was this patch tested?

Verified in local cluster.

CC vanzin  tgravescs  please help to review. Thanks!

Author: jerryshao <sshao@hortonworks.com>

Closes #21138 from jerryshao/SPARK-24062.
2018-04-26 13:27:33 +08:00
Marco Gaido cd10f9df82 [SPARK-23916][SQL] Add array_join function
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

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

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

## How was this patch tested?

modified existing UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21008 from mgaido91/SPARK-23902.
2018-04-26 12:19:20 +09:00
Maxim Gekk 3f1e999d3d [SPARK-23849][SQL] Tests for samplingRatio of json datasource
## What changes were proposed in this pull request?

Added the `samplingRatio` option to the `json()` method of PySpark DataFrame Reader. Improving existing tests for Scala API according to review of the PR: https://github.com/apache/spark/pull/20959

## How was this patch tested?

Added new test for PySpark, updated 2 existing tests according to reviews of https://github.com/apache/spark/pull/20959 and added new negative test

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

Closes #21056 from MaxGekk/json-sampling.
2018-04-26 09:14:24 +08:00
hyukjinkwon 95a651339e [SPARK-24069][R] Add array_min / array_max functions
## What changes were proposed in this pull request?

This PR proposes to add array_max and array_min in R side too.

array_max:

```r
df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))
mutated <- mutate(df, v1 = create_array(df$gear, df$am, df$carb))
head(select(mutated, array_max(mutated$v1)))
```

```
  array_max(v1)
1             4
2             4
3             4
4             3
5             3
6             3
```

array_min:

```r
df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))
mutated <- mutate(df, v1 = create_array(df$mpg, df$cyl, df$hp))
head(select(mutated, array_min(mutated$v1)))
```

```
  array_min(v1)
1             6
2             6
3             4
4             6
5             8
6             6
```

## How was this patch tested?

Unit tests were added in `R/pkg/tests/fulltests/test_sparkSQL.R` and manually tested. Documentation was manually built and verified.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21142 from HyukjinKwon/sparkr_array_min_array_max.
2018-04-26 09:12:38 +08:00
Wenchen Fan ac4ca7c4dd [SPARK-24012][SQL][TEST][FOLLOWUP] add unit test
## What changes were proposed in this pull request?

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

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21154 from cloud-fan/test.
2018-04-25 13:42:44 -07:00
Tathagata Das 396938ef02 [SPARK-24050][SS] Calculate input / processing rates correctly for DataSourceV2 streaming sources
## What changes were proposed in this pull request?

In some streaming queries, the input and processing rates are not calculated at all (shows up as zero) because MicroBatchExecution fails to associated metrics from the executed plan of a trigger with the sources in the logical plan of the trigger. The way this executed-plan-leaf-to-logical-source attribution works is as follows. With V1 sources, there was no way to identify which execution plan leaves were generated by a streaming source. So did a best-effort attempt to match logical and execution plan leaves when the number of leaves were same. In cases where the number of leaves is different, we just give up and report zero rates. An example where this may happen is as follows.

```
val cachedStaticDF = someStaticDF.union(anotherStaticDF).cache()
val streamingInputDF = ...

val query = streamingInputDF.join(cachedStaticDF).writeStream....
```
In this case, the `cachedStaticDF` has multiple logical leaves, but in the trigger's execution plan it only has leaf because a cached subplan is represented as a single InMemoryTableScanExec leaf. This leads to a mismatch in the number of leaves causing the input rates to be computed as zero.

With DataSourceV2, all inputs are represented in the executed plan using `DataSourceV2ScanExec`, each of which has a reference to the associated logical `DataSource` and `DataSourceReader`. So its easy to associate the metrics to the original streaming sources.

In this PR, the solution is as follows. If all the streaming sources in a streaming query as v2 sources, then use a new code path where the execution-metrics-to-source mapping is done directly. Otherwise we fall back to existing mapping logic.

## How was this patch tested?
- New unit tests using V2 memory source
- Existing unit tests using V1 source

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

Closes #21126 from tdas/SPARK-24050.
2018-04-25 12:21:55 -07:00
Takeshi Yamamuro 20ca208bcd [SPARK-23880][SQL] Do not trigger any jobs for caching data
## What changes were proposed in this pull request?
This pr fixed code so that `cache` could prevent any jobs from being triggered.
For example, in the current master, an operation below triggers a actual job;
```
val df = spark.range(10000000000L)
  .filter('id > 1000)
  .orderBy('id.desc)
  .cache()
```
This triggers a job while the cache should be lazy. The problem is that, when creating `InMemoryRelation`, we build the RDD, which calls `SparkPlan.execute` and may trigger jobs, like sampling job for range partitioner, or broadcast job.

This pr removed the code to build a cached `RDD` in the constructor of `InMemoryRelation` and added `CachedRDDBuilder` to lazily build the `RDD` in `InMemoryRelation`. Then, the first call of `CachedRDDBuilder.cachedColumnBuffers` triggers a job to materialize the cache in  `InMemoryTableScanExec` .

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21018 from maropu/SPARK-23880.
2018-04-25 19:06:18 +08:00
liutang123 64e8408e6f [SPARK-24012][SQL] Union of map and other compatible column
## What changes were proposed in this pull request?
Union of map and other compatible column result in unresolved operator 'Union; exception

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

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

Author: liutang123 <liutang123@yeah.net>

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

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

## How was this patch tested?

New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

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

Author: mn-mikke <mrkAha12346github>

Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-25 11:19:08 +09:00
Jose Torres d6c26d1c9a [SPARK-24038][SS] Refactor continuous writing to its own class
## What changes were proposed in this pull request?

Refactor continuous writing to its own class.

See WIP https://github.com/jose-torres/spark/pull/13 for the overall direction this is going, but I think this PR is very isolated and necessary anyway.

## How was this patch tested?

existing unit tests - refactoring only

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

Closes #21116 from jose-torres/SPARK-24038.
2018-04-24 17:06:03 -07:00
Tathagata Das 7b1e6523af [SPARK-24056][SS] Make consumer creation lazy in Kafka source for Structured streaming
## What changes were proposed in this pull request?

Currently, the driver side of the Kafka source (i.e. KafkaMicroBatchReader) eagerly creates a consumer as soon as the Kafk aMicroBatchReader is created. However, we create dummy KafkaMicroBatchReader to get the schema and immediately stop it. Its better to make the consumer creation lazy, it will be created on the first attempt to fetch offsets using the KafkaOffsetReader.

## How was this patch tested?
Existing unit tests

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

Closes #21134 from tdas/SPARK-24056.
2018-04-24 14:33:33 -07:00
WeichenXu 379bffa052 [SPARK-23990][ML] Instruments logging improvements - ML regression package
## What changes were proposed in this pull request?

Instruments logging improvements - ML regression package

I add an `OptionalInstrument` class which used in `WeightLeastSquares` and `IterativelyReweightedLeastSquares`.

## How was this patch tested?

N/A

Author: WeichenXu <weichen.xu@databricks.com>

Closes #21078 from WeichenXu123/inst_reg.
2018-04-24 11:02:22 -07:00
Liang-Chi Hsieh 83013752e3 [SPARK-23455][ML] Default Params in ML should be saved separately in metadata
## What changes were proposed in this pull request?

We save ML's user-supplied params and default params as one entity in metadata. During loading the saved models, we set all the loaded params into created ML model instances as user-supplied params.

It causes some problems, e.g., if we strictly disallow some params to be set at the same time, a default param can fail the param check because it is treated as user-supplied param after loading.

The loaded default params should not be set as user-supplied params. We should save ML default params separately in metadata.

For backward compatibility, when loading metadata, if it is a metadata file from previous Spark, we shouldn't raise error if we can't find the default param field.

## How was this patch tested?

Pass existing tests and added tests.

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

Closes #20633 from viirya/save-ml-default-params.
2018-04-24 10:40:25 -07:00
Steve Loughran ce7ba2e98e [SPARK-23807][BUILD] Add Hadoop 3.1 profile with relevant POM fix ups
## What changes were proposed in this pull request?

1. Adds a `hadoop-3.1` profile build depending on the hadoop-3.1 artifacts.
1. In the hadoop-cloud module, adds an explicit hadoop-3.1 profile which switches from explicitly pulling in cloud connectors (hadoop-openstack, hadoop-aws, hadoop-azure) to depending on the hadoop-cloudstorage POM artifact, which pulls these in, has pre-excluded things like hadoop-common, and stays up to date with new connectors (hadoop-azuredatalake, hadoop-allyun). Goal: it becomes the Hadoop projects homework of keeping this clean, and the spark project doesn't need to handle new hadoop releases adding more dependencies.
1. the hadoop-cloud/hadoop-3.1 profile also declares support for jetty-ajax and jetty-util to ensure that these jars get into the distribution jar directory when needed by unshaded libraries.
1. Increases the curator and zookeeper versions to match those in hadoop-3, fixing spark core to build in sbt with the hadoop-3 dependencies.

## How was this patch tested?

* Everything this has been built and tested against both ASF Hadoop branch-3.1 and hadoop trunk.
* spark-shell was used to create connectors to all the stores and verify that file IO could take place.

The spark hive-1.2.1 JAR has problems here, as it's version check logic fails for Hadoop versions > 2.

This can be avoided with either of

* The hadoop JARs built to declare their version as Hadoop 2.11  `mvn install -DskipTests -DskipShade -Ddeclared.hadoop.version=2.11` . This is safe for local test runs, not for deployment (HDFS is very strict about cross-version deployment).
* A modified version of spark hive whose version check switch statement is happy with hadoop 3.

I've done both, with maven and SBT.

Three issues surfaced

1. A spark-core test failure —fixed in SPARK-23787.
1. SBT only: Zookeeper not being found in spark-core. Somehow curator 2.12.0 triggers some slightly different dependency resolution logic from previous versions, and Ivy was missing zookeeper.jar entirely. This patch adds the explicit declaration for all spark profiles, setting the ZK version = 3.4.9 for hadoop-3.1
1. Marking jetty-utils as provided in spark was stopping hadoop-azure from being able to instantiate the azure wasb:// client; it was using jetty-util-ajax, which could then not find a class in jetty-util.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #20923 from steveloughran/cloud/SPARK-23807-hadoop-31.
2018-04-24 09:57:09 -07:00
Lu WANG 2a24c481da [SPARK-23975][ML] Allow Clustering to take Arrays of Double as input features
## What changes were proposed in this pull request?

- Multiple possible input types is added in validateAndTransformSchema() and computeCost() while checking column type

- Add if statement in transform() to support array type as featuresCol

- Add the case statement in fit() while selecting columns from dataset

These changes will be applied to KMeans first, then to other clustering method

## How was this patch tested?

unit test is added

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

Author: Lu WANG <lu.wang@databricks.com>

Closes #21081 from ludatabricks/SPARK-23975.
2018-04-24 09:25:41 -07:00
Julien Cuquemelle 55c4ca88a3 [SPARK-22683][CORE] Add a executorAllocationRatio parameter to throttle the parallelism of the dynamic allocation
## What changes were proposed in this pull request?

By default, the dynamic allocation will request enough executors to maximize the
parallelism according to the number of tasks to process. While this minimizes the
latency of the job, with small tasks this setting can waste a lot of resources due to
executor allocation overhead, as some executor might not even do any work.
This setting allows to set a ratio that will be used to reduce the number of
target executors w.r.t. full parallelism.

The number of executors computed with this setting is still fenced by
`spark.dynamicAllocation.maxExecutors` and `spark.dynamicAllocation.minExecutors`

## How was this patch tested?
Units tests and runs on various actual workloads on a Yarn Cluster

Author: Julien Cuquemelle <j.cuquemelle@criteo.com>

Closes #19881 from jcuquemelle/AddTaskPerExecutorSlot.
2018-04-24 10:56:55 -05:00
Takeshi Yamamuro 4926a7c2f0 [SPARK-23589][SQL][FOLLOW-UP] Reuse InternalRow in ExternalMapToCatalyst eval
## What changes were proposed in this pull request?
This pr is a follow-up of #20980 and fixes code to reuse `InternalRow` for converting input keys/values in `ExternalMapToCatalyst` eval.

## How was this patch tested?
Existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21137 from maropu/SPARK-23589-FOLLOWUP.
2018-04-24 17:52:05 +02:00
hyukjinkwon 87e8a572be [SPARK-24054][R] Add array_position function / element_at functions
## What changes were proposed in this pull request?

This PR proposes to add array_position and element_at in R side too.

array_position:

```r
df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))
mutated <- mutate(df, v1 = create_array(df$gear, df$am, df$carb))
head(select(mutated, array_position(mutated$v1, 1)))
```

```
  array_position(v1, 1.0)
1                       2
2                       2
3                       2
4                       3
5                       0
6                       3
```

element_at:

```r
df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))
mutated <- mutate(df, v1 = create_array(df$mpg, df$cyl, df$hp))
head(select(mutated, element_at(mutated$v1, 1)))
```

```
  element_at(v1, 1.0)
1                21.0
2                21.0
3                22.8
4                21.4
5                18.7
6                18.1
```

```r
df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))
mutated <- mutate(df, v1 = create_map(df$model, df$cyl))
head(select(mutated, element_at(mutated$v1, "Valiant")))
```

```
  element_at(v3, Valiant)
1                      NA
2                      NA
3                      NA
4                      NA
5                      NA
6                       6
```

## How was this patch tested?

Unit tests were added in `R/pkg/tests/fulltests/test_sparkSQL.R` and manually tested. Documentation was manually built and verified.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21130 from HyukjinKwon/sparkr_array_position_element_at.
2018-04-24 16:18:20 +08:00
seancxmao c303b1b676 [MINOR][DOCS] Fix comments of SQLExecution#withExecutionId
## What changes were proposed in this pull request?
Fix comment. Change `BroadcastHashJoin.broadcastFuture` to `BroadcastExchangeExec.relationFuture`: d28d5732ae/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala (L66)

## How was this patch tested?
N/A

Author: seancxmao <seancxmao@gmail.com>

Closes #21113 from seancxmao/SPARK-13136.
2018-04-24 16:16:07 +08:00
Marco Gaido 281c1ca0dc [SPARK-23973][SQL] Remove consecutive Sorts
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21072 from mgaido91/SPARK-23973.
2018-04-24 10:11:09 +08:00
Marcelo Vanzin 428b903859 [SPARK-24029][CORE] Follow up: set SO_REUSEADDR on the server socket.
"childOption" is for the remote connections, not for the server socket
that actually listens for incoming connections.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21132 from vanzin/SPARK-24029.2.
2018-04-24 09:10:29 +08:00
wuyi c8f3ac69d1 [SPARK-23888][CORE] correct the comment of hasAttemptOnHost()
TaskSetManager.hasAttemptOnHost had a misleading comment.  The comment
said that it only checked for running tasks, but really it checked for
any tasks that might have run in the past as well.  This updates to line
up with the implementation.

Author: wuyi <ngone_5451@163.com>

Closes #20998 from Ngone51/SPARK-23888.
2018-04-23 15:35:50 -05:00
Holden Karau e82cb68349
[SPARK-11237][ML] Add pmml export for k-means in Spark ML
## What changes were proposed in this pull request?

Adding PMML export to Spark ML's KMeans Model.

## How was this patch tested?

New unit test for Spark ML PMML export based on the old Spark MLlib unit test.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #20907 from holdenk/SPARK-11237-Add-PMML-Export-for-KMeans.
2018-04-23 13:23:02 -07:00
Tathagata Das 770add81c3 [SPARK-23004][SS] Ensure StateStore.commit is called only once in a streaming aggregation task
## What changes were proposed in this pull request?

A structured streaming query with a streaming aggregation can throw the following error in rare cases. 

```
java.lang.IllegalStateException: Cannot commit after already committed or aborted
	at org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$verify(HDFSBackedStateStoreProvider.scala:643)
	at org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$HDFSBackedStateStore.commit(HDFSBackedStateStoreProvider.scala:135)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2$$anonfun$hasNext$2.apply$mcV$sp(statefulOperators.scala:359)
	at org.apache.spark.sql.execution.streaming.StateStoreWriter$class.timeTakenMs(statefulOperators.scala:102)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec.timeTakenMs(statefulOperators.scala:251)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2.hasNext(statefulOperators.scala:359)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:188)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:114)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:105)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:42)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:336)
```

This can happen when the following conditions are accidentally hit. 
 - Streaming aggregation with aggregation function that is a subset of [`TypedImperativeAggregation`](76b8b840dd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala (L473)) (for example, `collect_set`, `collect_list`, `percentile`, etc.). 
 - Query running in `update}` mode
 - After the shuffle, a partition has exactly 128 records. 

This causes StateStore.commit to be called twice. See the [JIRA](https://issues.apache.org/jira/browse/SPARK-23004) for a more detailed explanation. The solution is to use `NextIterator` or `CompletionIterator`, each of which has a flag to prevent the "onCompletion" task from being called more than once. In this PR, I chose to implement using `NextIterator`.

## How was this patch tested?

Added unit test that I have confirm will fail without the fix.

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

Closes #21124 from tdas/SPARK-23004.
2018-04-23 13:20:32 -07:00
liuzhaokun 448d248f89 [SPARK-21168] KafkaRDD should always set kafka clientId.
[https://issues.apache.org/jira/browse/SPARK-21168](https://issues.apache.org/jira/browse/SPARK-21168)
There are no a number of other places that a client ID should be set,and I think we should use consumer.clientId in the clientId method,because the fetch request  will be used by the same consumer behind.

Author: liuzhaokun <liu.zhaokun@zte.com.cn>

Closes #19887 from liu-zhaokun/master1205.
2018-04-23 13:56:11 -05:00
Teng Peng 293a0f29e3
[Spark-24024][ML] Fix poisson deviance calculations in GLM to handle y = 0
## What changes were proposed in this pull request?

It is reported by Spark users that the deviance calculation for poisson regression does not handle y = 0. Thus, the correct model summary cannot be obtained. The user has confirmed the the issue is in
```
override def deviance(y: Double, mu: Double, weight: Double): Double =
{ 2.0 * weight * (y * math.log(y / mu) - (y - mu)) }
when y = 0.
```

The user also mentioned there are many other places he believe we should check the same thing. However, no other changes are needed, including Gamma distribution.

## How was this patch tested?
Add a comparison with R deviance calculation to the existing unit test.

Author: Teng Peng <josephtengpeng@gmail.com>

Closes #21125 from tengpeng/Spark24024GLM.
2018-04-23 10:29:47 -07:00
Takeshi Yamamuro afbdf42730 [SPARK-23589][SQL] ExternalMapToCatalyst should support interpreted execution
## What changes were proposed in this pull request?
This pr supported interpreted mode for `ExternalMapToCatalyst`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

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

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

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

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

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

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

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

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21111 from cloud-fan/refactor.
2018-04-23 20:18:50 +08:00
Kazuaki Ishizaki c3a86faa53 [SPARK-10399][SPARK-23879][FOLLOWUP][CORE] Free unused off-heap memory in MemoryBlockSuite
## What changes were proposed in this pull request?

As viirya pointed out [here](https://github.com/apache/spark/pull/19222#discussion_r179910484), this PR explicitly frees unused off-heap memory in `MemoryBlockSuite`

## How was this patch tested?

Existing UTs

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

Closes #21117 from kiszk/SPARK-10399-free-offheap.
2018-04-23 10:45:25 +08:00
Mykhailo Shtelma c48085aa91 [SPARK-23799][SQL] FilterEstimation.evaluateInSet produces devision by zero in a case of empty table with analyzed statistics
>What changes were proposed in this pull request?

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

>How was this patch tested?

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

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21115 from gatorsmile/fixLag.
2018-04-21 10:45:12 -07:00