## What changes were proposed in this pull request?
This pr added code to print the same warning messages with `===` cases when using NULL-safe equals (`<=>`).
## How was this patch tested?
Existing tests.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18436 from maropu/SPARK-20073.
### Idea
This PR adds validation to REFRESH sql statements. Currently, users can specify whatever they want as resource path. For example, spark.sql("REFRESH ! $ !") will be executed without any exceptions.
### Implementation
I am not sure that my current implementation is the most optimal, so any feedback is appreciated. My first idea was to make the grammar as strict as possible. Unfortunately, there were some problems. I tried the approach below:
SqlBase.g4
```
...
| REFRESH TABLE tableIdentifier #refreshTable
| REFRESH resourcePath #refreshResource
...
resourcePath
: STRING
| (IDENTIFIER | number | nonReserved | '/' | '-')+ // other symbols can be added if needed
;
```
It is not flexible enough and requires to explicitly mention all possible symbols. Therefore, I came up with the current approach that is implemented in the code.
Let me know your opinion on which one is better.
Author: aokolnychyi <anton.okolnychyi@sap.com>
Closes#18368 from aokolnychyi/spark-21102.
## What changes were proposed in this pull request?
It is strange that we will get "table not found" error if **the first sql** uses upper case table names, when developers write tests with `TestHiveSingleton`, **although case insensitivity**. This is because in `TestHiveQueryExecution`, test tables are loaded based on exact matching instead of case sensitivity.
## How was this patch tested?
Added a new test case.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18504 from wzhfy/testHive.
## What changes were proposed in this pull request?
Parallelize FileInputFormat.listStatus in Hadoop API via LIST_STATUS_NUM_THREADS to speed up examination of file sizes for wholeTextFiles et al
## How was this patch tested?
Existing tests, which will exercise the key path here: using a local file system.
Author: Sean Owen <sowen@cloudera.com>
Closes#18441 from srowen/SPARK-21137.
## What changes were proposed in this pull request?
Move `compileValue` method in JDBCRDD to JdbcDialect, and override the `compileValue` method in OracleDialect to rewrite the Oracle-specific timestamp and date literals in where clause.
## How was this patch tested?
An integration test has been added.
Author: Rui Zha <zrdt713@gmail.com>
Author: Zharui <zrdt713@gmail.com>
Closes#18451 from SharpRay/extend-compileValue-to-dialects.
## What changes were proposed in this pull request?
This PR is to maintain API parity with changes made in SPARK-17498 to support a new option
'keep' in StringIndexer to handle unseen labels or NULL values with PySpark.
Note: This is updated version of #17237 , the primary author of this PR is VinceShieh .
## How was this patch tested?
Unit tests.
Author: VinceShieh <vincent.xie@intel.com>
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#18453 from yanboliang/spark-19852.
## What changes were proposed in this pull request?
OutputFakerExec was added long ago and is not used anywhere now so we should remove it.
## How was this patch tested?
N/A
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Closes#18473 from jiangxb1987/OutputFakerExec.
## What changes were proposed in this pull request?
Not adding the exception to the suppressed if it is the same instance as originalThrowable.
## How was this patch tested?
Added new tests to verify this, these tests fail without source code changes and passes with the change.
Author: Devaraj K <devaraj@apache.org>
Closes#18384 from devaraj-kavali/SPARK-21170.
## What changes were proposed in this pull request?
1, make param support non-final with `finalFields` option
2, generate `HasSolver` with `finalFields = false`
3, override `solver` in LiR, GLR, and make MLPC inherit `HasSolver`
## How was this patch tested?
existing tests
Author: Ruifeng Zheng <ruifengz@foxmail.com>
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#16028 from zhengruifeng/param_non_final.
## What changes were proposed in this pull request?
We currently implement statistics propagation directly in logical plan. Given we already have two different implementations, it'd make sense to actually decouple the two and add stats propagation using mixin. This would reduce the coupling between logical plan and statistics handling.
This can also be a powerful pattern in the future to add additional properties (e.g. constraints).
## How was this patch tested?
Should be covered by existing test cases.
Author: Reynold Xin <rxin@databricks.com>
Closes#18479 from rxin/stats-trait.
## What changes were proposed in this pull request?
Update stats after the following data changing commands:
- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand
## How was this patch tested?
Added new test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18334 from wzhfy/changeStatsForOperation.
## What changes were proposed in this pull request?
For performance reasons, `UnsafeRow.getString`, `getStruct`, etc. return a "pointer" that points to a memory region of this unsafe row. This makes the unsafe projection a little dangerous, because all of its output rows share one instance.
When we implement SQL operators, we should be careful to not cache the input rows because they may be produced by unsafe projection from child operator and thus its content may change overtime.
However, when we updating values of InternalRow(e.g. in mutable projection and safe projection), we only copy UTF8String, we should also copy InternalRow, ArrayData and MapData. This PR fixes this, and also fixes the copy of vairous InternalRow, ArrayData and MapData implementations.
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18483 from cloud-fan/fix-copy.
## What changes were proposed in this pull request?
Remove `numHashCollisions` in `BytesToBytesMap`. And change `getAverageProbesPerLookup()` to `getAverageProbesPerLookup` as suggested.
## How was this patch tested?
Existing tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18480 from viirya/SPARK-21052-followup.
### What changes were proposed in this pull request?
Function argument should not be named expressions. It could cause two issues:
- Misleading error message
- Unexpected query results when the column name is `distinct`, which is not a reserved word in our parser.
```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query: cannot resolve '`distinct`' given input columns: [c1, c2]; line 1 pos 26;
'Project [unresolvedalias('count(c1#30, 'distinct), None)]
+- SubqueryAlias t1
+- CatalogRelation `default`.`t1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#30, c2#31]
```
After the fix, the error message becomes
```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query:
extraneous input 'c2' expecting {')', ',', '.', '[', 'OR', 'AND', 'IN', NOT, 'BETWEEN', 'LIKE', RLIKE, 'IS', EQ, '<=>', '<>', '!=', '<', LTE, '>', GTE, '+', '-', '*', '/', '%', 'DIV', '&', '|', '||', '^'}(line 1, pos 35)
== SQL ==
select count(distinct c1, distinct c2) from t1
-----------------------------------^^^
```
### How was this patch tested?
Added a test case to parser suite.
Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18338 from gatorsmile/parserDistinctAggFunc.
# What issue does this PR address ?
Jira:https://issues.apache.org/jira/browse/SPARK-21223
fix the Thread-safety issue in FsHistoryProvider
Currently, Spark HistoryServer use a HashMap named fileToAppInfo in class FsHistoryProvider to store the map of eventlog path and attemptInfo.
When use ThreadPool to Replay the log files in the list and merge the list of old applications with new ones, multi thread may update fileToAppInfo at the same time, which may cause Thread-safety issues, such as falling into an infinite loop because of calling resize func of the hashtable.
Author: 曾林西 <zenglinxi@meituan.com>
Closes#18430 from zenglinxi0615/master.
## What changes were proposed in this pull request?
Fix scala-2.10 build failure of ```GeneralizedLinearRegressionSuite```.
## How was this patch tested?
Build with scala-2.10.
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#18489 from yanboliang/glr.
## What changes were proposed in this pull request?
This PR makes the following changes:
- Implement a new commit protocol `HadoopMapRedCommitProtocol` which support the old `mapred` package's committer;
- Refactor SparkHadoopWriter and SparkHadoopMapReduceWriter, now they are combined together, thus we can support write through both mapred and mapreduce API by the new SparkHadoopWriter, a lot of duplicated codes are removed.
After this change, it should be pretty easy for us to support the committer from both the new and the old hadoop API at high level.
## How was this patch tested?
No major behavior change, passed the existing test cases.
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Closes#18438 from jiangxb1987/SparkHadoopWriter.
## What changes were proposed in this pull request?
Add support for offset in GLM. This is useful for at least two reasons:
1. Account for exposure: e.g., when modeling the number of accidents, we may need to use miles driven as an offset to access factors on frequency.
2. Test incremental effects of new variables: we can use predictions from the existing model as offset and run a much smaller model on only new variables. This avoids re-estimating the large model with all variables (old + new) and can be very important for efficient large-scaled analysis.
## How was this patch tested?
New test.
yanboliang srowen felixcheung sethah
Author: actuaryzhang <actuaryzhang10@gmail.com>
Closes#16699 from actuaryzhang/offset.
## What changes were proposed in this pull request?
Grouped documentation for column collection methods.
Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>
Closes#18458 from actuaryzhang/sparkRDocCollection.
## What changes were proposed in this pull request?
Grouped documentation for column misc methods.
Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>
Closes#18448 from actuaryzhang/sparkRDocMisc.
## What changes were proposed in this pull request?
`WindowExec` currently improperly stores complex objects (UnsafeRow, UnsafeArrayData, UnsafeMapData, UTF8String) during aggregation by keeping a reference in the buffer used by `GeneratedMutableProjections` to the actual input data. Things go wrong when the input object (or the backing bytes) are reused for other things. This could happen in window functions when it starts spilling to disk. When reading the back the spill files the `UnsafeSorterSpillReader` reuses the buffer to which the `UnsafeRow` points, leading to weird corruption scenario's. Note that this only happens for aggregate functions that preserve (parts of) their input, for example `FIRST`, `LAST`, `MIN` & `MAX`.
This was not seen before, because the spilling logic was not doing actual spills as much and actually used an in-memory page. This page was not cleaned up during window processing and made sure unsafe objects point to their own dedicated memory location. This was changed by https://github.com/apache/spark/pull/16909, after this PR Spark spills more eagerly.
This PR provides a surgical fix because we are close to releasing Spark 2.2. This change just makes sure that there cannot be any object reuse at the expensive of a little bit of performance. We will follow-up with a more subtle solution at a later point.
## How was this patch tested?
Added a regression test to `DataFrameWindowFunctionsSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#18470 from hvanhovell/SPARK-21258.
## What changes were proposed in this pull request?
A follow up PR to fix Scala 2.10 build for #18472
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18478 from zsxwing/SPARK-21253-2.
## What changes were proposed in this pull request?
Please see also https://issues.apache.org/jira/browse/SPARK-21176
This change limits the number of selector threads that jetty creates to maximum 8 per proxy servlet (Jetty default is number of processors / 2).
The newHttpClient for Jettys ProxyServlet class is overwritten to avoid the Jetty defaults (which are designed for high-performance http servers).
Once https://github.com/eclipse/jetty.project/issues/1643 is available, the code could be cleaned up to avoid the method override.
I really need this on v2.1.1 - what is the best way for a backport automatic merge works fine)? Shall I create another PR?
## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
The patch was tested manually on a Spark cluster with a head node that has 88 processors using JMX to verify that the number of selector threads is now limited to 8 per proxy.
gurvindersingh zsxwing can you please review the change?
Author: IngoSchuster <ingo.schuster@de.ibm.com>
Author: Ingo Schuster <ingo.schuster@de.ibm.com>
Closes#18437 from IngoSchuster/master.
## What changes were proposed in this pull request?
Disable spark.reducer.maxReqSizeShuffleToMem because it breaks the old shuffle service.
Credits to wangyum
Closes#18466
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18467 from zsxwing/SPARK-21253.
## What changes were proposed in this pull request?
If a network error happens before processing StreamResponse/StreamFailure events, StreamCallback.onFailure won't be called.
This PR fixes `failOutstandingRequests` to also notify outstanding StreamCallbacks.
## How was this patch tested?
The new unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18472 from zsxwing/fix-stream-2.
## What changes were proposed in this pull request?
Since the objects `readLocksByTask`, `writeLocksByTask` and `info`s are coupled and supposed to be modified by other threads concurrently, all the read and writes of them in the method `releaseAllLocksForTask` should be protected by a single synchronized block like other similar methods.
## How was this patch tested?
existing tests
Author: Feng Liu <fengliu@databricks.com>
Closes#18400 from liufengdb/synchronize.
## What changes were proposed in this pull request?
This adds the average hash map probe metrics to join operator such as `BroadcastHashJoin` and `ShuffledHashJoin`.
This PR adds the API to `HashedRelation` to get average hash map probe.
## How was this patch tested?
Related test cases are added.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18301 from viirya/SPARK-21052.
JIRA Issue:https://issues.apache.org/jira/browse/SPARK-21225
In the function "resourceOffers", It declare a variable "tasks" for storage the tasks which have allocated a executor. It declared like this:
`val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores))`
But, I think this code only conside a situation for that one task per core. If the user set "spark.task.cpus" as 2 or 3, It really don't need so much Mem. I think It can motify as follow:
val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK))
to instead.
Motify like this the other earning is that it's more easy to understand the way how the tasks allocate offers.
Author: 杨治国10192065 <yang.zhiguo@zte.com.cn>
Closes#18435 from JackYangzg/motifyTaskCoreDisp.
## What changes were proposed in this pull request?
Hide duration of incompleted applications.
## How was this patch tested?
manual tests
Author: fjh100456 <fu.jinhua6@zte.com.cn>
Closes#18351 from fjh100456/master.
## What changes were proposed in this pull request?
Same with SPARK-20985.
Fix code style for constructing and stopping a `SparkContext`. Assure the context is stopped to avoid other tests complain that there's only one `SparkContext` can exist.
Author: jinxing <jinxing6042@126.com>
Closes#18454 from jinxing64/SPARK-21240.
PR #15999 included fixes for doc strings in the ML shared param traits (occurrences of `>` and `>=`).
This PR simply uses the HTML-escaped version of the param doc to embed into the Scaladoc, to ensure that when `SharedParamsCodeGen` is run, the generated javadoc will be compliant for Java 8.
## How was this patch tested?
Existing tests
Author: Nick Pentreath <nickp@za.ibm.com>
Closes#18420 from MLnick/shared-params-javadoc8.
## What changes were proposed in this pull request?
Grouped documentation for nonaggregate column methods.
Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>
Closes#18422 from actuaryzhang/sparkRDocNonAgg.
## What changes were proposed in this pull request?
This is kind of another follow-up for https://github.com/apache/spark/pull/18064 .
In #18064 , we wrap every SQL command with SQL execution, which makes nested SQL execution very likely to happen. #18419 trid to improve it a little bit, by introduing `SQLExecition.ignoreNestedExecutionId`. However, this is not friendly to data source developers, they may need to update their code to use this `ignoreNestedExecutionId` API.
This PR proposes a new solution, to just allow nested execution. The downside is that, we may have multiple executions for one query. We can improve this by updating the data organization in SQLListener, to have 1-n mapping from query to execution, instead of 1-1 mapping. This can be done in a follow-up.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18450 from cloud-fan/execution-id.
## What changes were proposed in this pull request?
Report Spill size on disk for UnsafeExternalSorter
## How was this patch tested?
Tested by running a job on cluster and verify the spill size on disk.
Author: Sital Kedia <skedia@fb.com>
Closes#17471 from sitalkedia/fix_disk_spill_size.
## What changes were proposed in this pull request?
Invalidate spark's stats after data changing commands:
- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand
## How was this patch tested?
Added test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#18449 from wzhfy/removeStats.
## What changes were proposed in this pull request?
`QueryPlan.preCanonicalized` is only overridden in a few places, and it does introduce an extra concept to `QueryPlan` which may confuse people.
This PR removes it and override `canonicalized` in these places
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18440 from cloud-fan/minor.
## What changes were proposed in this pull request?
This PR proposes to support a DDL-formetted string as schema as below:
```r
mockLines <- c("{\"name\":\"Michael\"}",
"{\"name\":\"Andy\", \"age\":30}",
"{\"name\":\"Justin\", \"age\":19}")
jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp")
writeLines(mockLines, jsonPath)
df <- read.df(jsonPath, "json", "name STRING, age DOUBLE")
collect(df)
```
## How was this patch tested?
Tests added in `test_streaming.R` and `test_sparkSQL.R` and manual tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18431 from HyukjinKwon/r-ddl-schema.
## What changes were proposed in this pull request?
Please see [SPARK-14657](https://issues.apache.org/jira/browse/SPARK-14657) for detail of this bug.
I searched online and test some other cases, found when we fit R glm model(or other models powered by R formula) w/o intercept on a dataset including string/category features, one of the categories in the first category feature is being used as reference category, we will not drop any category for that feature.
I think we should keep consistent semantics between Spark RFormula and R formula.
## How was this patch tested?
Add standard unit tests.
cc mengxr
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#12414 from yanboliang/spark-14657.
## What changes were proposed in this pull request?
Grouped documentation for string column methods.
Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>
Closes#18366 from actuaryzhang/sparkRDocString.
## What changes were proposed in this pull request?
Move elimination of Distinct clause from analyzer to optimizer
Distinct clause is useless after MAX/MIN clause. For example,
"Select MAX(distinct a) FROM src from"
is equivalent of
"Select MAX(a) FROM src from"
However, this optimization is implemented in analyzer. It should be in optimizer.
## How was this patch tested?
Unit test
gatorsmile cloud-fan
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#18429 from gengliangwang/distinct_opt.
## What changes were proposed in this pull request?
If someone creates a HiveSession, the planner in `IncrementalExecution` doesn't take into account the Hive scan strategies. This causes joins of Streaming DataFrame's with Hive tables to fail.
## How was this patch tested?
Regression test
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#18426 from brkyvz/hive-join.
## What changes were proposed in this pull request?
Grouped documentation for math column methods.
Author: actuaryzhang <actuaryzhang10@gmail.com>
Author: Wayne Zhang <actuaryzhang10@gmail.com>
Closes#18371 from actuaryzhang/sparkRDocMath.
## What changes were proposed in this pull request?
Add metric on number of running tasks to status bar on Jobs / Active Jobs.
## How was this patch tested?
Run a long running (1 minute) query in spark-shell and use localhost:4040 web UI to observe progress. See jira for screen snapshot.
Author: Eric Vandenberg <ericvandenberg@fb.com>
Closes#18369 from ericvandenbergfb/runningTasks.
## What changes were proposed in this pull request?
The issue happens in `ExternalMapToCatalyst`. For example, the following codes create `ExternalMapToCatalyst` to convert Scala Map to catalyst map format.
val data = Seq.tabulate(10)(i => NestedData(1, Map("key" -> InnerData("name", i + 100))))
val ds = spark.createDataset(data)
The `valueConverter` in `ExternalMapToCatalyst` looks like:
if (isnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true))) null else named_struct(name, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).name, true), value, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).value)
There is a `CreateNamedStruct` expression (`named_struct`) to create a row of `InnerData.name` and `InnerData.value` that are referred by `ExternalMapToCatalyst_value52`.
Because `ExternalMapToCatalyst_value52` are local variable, when `CreateNamedStruct` splits expressions to individual functions, the local variable can't be accessed anymore.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18418 from viirya/SPARK-19104.