GitHub user ithjz opened a pull request:

    https://github.com/apache/spark/pull/19347

    Branch 2.2       sparkmlib's    output of many algorithms is not clear

    
    
    What's the use of these **results?**
    
    
    JavaGradientBoostingRegressionExample 
    
    
    
    Test Mean Squared Error: 0.12500000000000003
    Learned regression GBT model:
    TreeEnsembleModel regressor with 3 trees
    
      Tree 0:
        If (feature 351 <= 15.0)
         Predict: 0.0
        Else (feature 351 > 15.0)
         Predict: 1.0
      Tree 1:
        Predict: 0.0
      Tree 2:
        Predict: 0.0

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/apache/spark branch-2.2

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19347.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19347
    
----
commit e936a96badfeeb2051ee35dc4b0fbecefa9bf4cb
Author: Peng <peng.m...@intel.com>
Date:   2017-05-24T11:54:17Z

    [SPARK-20764][ML][PYSPARK][FOLLOWUP] Fix visibility discrepancy with 
numInstances and degreesOfFreedom in LR and GLR - Python version
    
    ## What changes were proposed in this pull request?
    Add test cases for PR-18062
    
    ## How was this patch tested?
    The existing UT
    
    Author: Peng <peng.m...@intel.com>
    
    Closes #18068 from mpjlu/moreTest.
    
    (cherry picked from commit 9afcf127d31b5477a539dde6e5f01861532a1c4c)
    Signed-off-by: Yanbo Liang <yblia...@gmail.com>

commit 1d107242f8ec842c009e0b427f6e4a8313d99aa2
Author: zero323 <zero...@users.noreply.github.com>
Date:   2017-05-24T11:57:44Z

    [SPARK-20631][FOLLOW-UP] Fix incorrect tests.
    
    ## What changes were proposed in this pull request?
    
    - Fix incorrect tests for `_check_thresholds`.
    - Move test to `ParamTests`.
    
    ## How was this patch tested?
    
    Unit tests.
    
    Author: zero323 <zero...@users.noreply.github.com>
    
    Closes #18085 from zero323/SPARK-20631-FOLLOW-UP.
    
    (cherry picked from commit 1816eb3bef930407dc9e083de08f5105725c55d1)
    Signed-off-by: Yanbo Liang <yblia...@gmail.com>

commit 83aeac9e0590e99010d0af8e067822d0ed0971fe
Author: Bago Amirbekian <b...@databricks.com>
Date:   2017-05-24T14:55:38Z

    [SPARK-20862][MLLIB][PYTHON] Avoid passing float to ndarray.reshape in 
LogisticRegressionModel
    
    ## What changes were proposed in this pull request?
    
    Fixed TypeError with python3 and numpy 1.12.1. Numpy's `reshape` no longer 
takes floats as arguments as of 1.12. Also, python3 uses float division for 
`/`, we should be using `//` to ensure that `_dataWithBiasSize` doesn't get set 
to a float.
    
    ## How was this patch tested?
    
    Existing tests run using python3 and numpy 1.12.
    
    Author: Bago Amirbekian <b...@databricks.com>
    
    Closes #18081 from MrBago/BF-py3floatbug.
    
    (cherry picked from commit bc66a77bbe2120cc21bd8da25194efca4cde13c3)
    Signed-off-by: Yanbo Liang <yblia...@gmail.com>

commit c59ad420b5fda29567f4a06b5f71df76e70e269a
Author: Liang-Chi Hsieh <vii...@gmail.com>
Date:   2017-05-24T16:35:40Z

    [SPARK-20848][SQL] Shutdown the pool after reading parquet files
    
    ## What changes were proposed in this pull request?
    
    From JIRA: On each call to spark.read.parquet, a new ForkJoinPool is 
created. One of the threads in the pool is kept in the WAITING state, and never 
stopped, which leads to unbounded growth in number of threads.
    
    We should shutdown the pool after reading parquet files.
    
    ## How was this patch tested?
    
    Added a test to ParquetFileFormatSuite.
    
    Please review http://spark.apache.org/contributing.html before opening a 
pull request.
    
    Author: Liang-Chi Hsieh <vii...@gmail.com>
    
    Closes #18073 from viirya/SPARK-20848.
    
    (cherry picked from commit f72ad303f05a6d99513ea3b121375726b177199c)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>

commit b7a2a16b1e01375292938fc48b0a333ec4e7cd30
Author: Reynold Xin <r...@databricks.com>
Date:   2017-05-24T20:57:19Z

    [SPARK-20867][SQL] Move hints from Statistics into HintInfo class
    
    ## What changes were proposed in this pull request?
    This is a follow-up to SPARK-20857 to move the broadcast hint from 
Statistics into a new HintInfo class, so we can be more flexible in adding new 
hints in the future.
    
    ## How was this patch tested?
    Updated test cases to reflect the change.
    
    Author: Reynold Xin <r...@databricks.com>
    
    Closes #18087 from rxin/SPARK-20867.
    
    (cherry picked from commit a64746677bf09ef67e3fd538355a6ee9b5ce8cf4)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit 2405afce4e87c0486f2aef1d068f17aea2480b17
Author: Kris Mok <kris....@databricks.com>
Date:   2017-05-25T00:19:35Z

    [SPARK-20872][SQL] ShuffleExchange.nodeName should handle null coordinator
    
    ## What changes were proposed in this pull request?
    
    A one-liner change in `ShuffleExchange.nodeName` to cover the case when 
`coordinator` is `null`, so that the match expression is exhaustive.
    
    Please refer to 
[SPARK-20872](https://issues.apache.org/jira/browse/SPARK-20872) for a 
description of the symptoms.
    TL;DR is that inspecting a `ShuffleExchange` (directly or transitively) on 
the Executor side can hit a case where the `coordinator` field of a 
`ShuffleExchange` is null, and thus will trigger a `MatchError` in 
`ShuffleExchange.nodeName()`'s inexhaustive match expression.
    
    Also changed two other match conditions in `ShuffleExchange` on the 
`coordinator` field to be consistent.
    
    ## How was this patch tested?
    
    Manually tested this change with a case where the `coordinator` is null to 
make sure `ShuffleExchange.nodeName` doesn't throw a `MatchError` any more.
    
    Author: Kris Mok <kris....@databricks.com>
    
    Closes #18095 from rednaxelafx/shuffleexchange-nodename.
    
    (cherry picked from commit c0b3e45e3b46a5235b748cb85ad200c9ec1bb426)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit ae65d3014941344a924da583959e6b4b1d1d64f2
Author: Jacek Laskowski <ja...@japila.pl>
Date:   2017-05-25T00:24:23Z

    [SPARK-16202][SQL][DOC] Follow-up to Correct The Description of 
CreatableRelationProvider's createRelation
    
    ## What changes were proposed in this pull request?
    
    Follow-up to SPARK-16202:
    
    1. Remove the duplication of the meaning of `SaveMode` (as one was in fact 
missing that had proven that the duplication may be incomplete in the future 
again)
    
    2. Use standard scaladoc tags
    
    /cc gatorsmile rxin yhuai (as they were involved previously)
    
    ## How was this patch tested?
    
    local build
    
    Author: Jacek Laskowski <ja...@japila.pl>
    
    Closes #18026 from jaceklaskowski/CreatableRelationProvider-SPARK-16202.
    
    (cherry picked from commit 5f8ff2fc9a859ceeaa8f1d03060fdbb30951e706)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit 3f82d65bf6a628b0d46bb2eded9ed12f1d5aa9d2
Author: liuxian <liu.xi...@zte.com.cn>
Date:   2017-05-25T00:32:02Z

    [SPARK-20403][SQL] Modify the instructions of some functions
    
    ## What changes were proposed in this pull request?
    1.    add  instructions of  'cast'  function When using 'show functions'  
and 'desc function cast'
           command in spark-sql
    2.    Modify the  instructions of functions,such as
         
boolean,tinyint,smallint,int,bigint,float,double,decimal,date,timestamp,binary,string
    
    ## How was this patch tested?
    Before modification:
    spark-sql>desc function boolean;
    Function: boolean
    Class: org.apache.spark.sql.catalyst.expressions.Cast
    Usage: boolean(expr AS type) - Casts the value `expr` to the target data 
type `type`.
    
    After modification:
    spark-sql> desc function boolean;
    Function: boolean
    Class: org.apache.spark.sql.catalyst.expressions.Cast
    Usage: boolean(expr) - Casts the value `expr` to the target data type 
`boolean`.
    
    spark-sql> desc function cast
    Function: cast
    Class: org.apache.spark.sql.catalyst.expressions.Cast
    Usage: cast(expr AS type) - Casts the value `expr` to the target data type 
`type`.
    
    Author: liuxian <liu.xi...@zte.com.cn>
    
    Closes #17698 from 10110346/wip_lx_0418.
    
    (cherry picked from commit 197f9018a4641c8fc0725905ebfb535b61bed791)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit e0aa23939a4cbf95f2cc83a7f5adee841b491358
Author: Liang-Chi Hsieh <vii...@gmail.com>
Date:   2017-05-25T01:55:45Z

    [SPARK-20848][SQL][FOLLOW-UP] Shutdown the pool after reading parquet files
    
    ## What changes were proposed in this pull request?
    
    This is a follow-up to #18073. Taking a safer approach to shutdown the pool 
to prevent possible issue. Also using `ThreadUtils.newForkJoinPool` instead to 
set a better thread name.
    
    ## How was this patch tested?
    
    Manually test.
    
    Please review http://spark.apache.org/contributing.html before opening a 
pull request.
    
    Author: Liang-Chi Hsieh <vii...@gmail.com>
    
    Closes #18100 from viirya/SPARK-20848-followup.
    
    (cherry picked from commit 6b68d61cf31748a088778dfdd66491b2f89a3c7b)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>

commit b52a06d7034b3d392f7f0ee69a2fba098783e70d
Author: Xianyang Liu <xianyang....@intel.com>
Date:   2017-05-25T07:47:59Z

    [SPARK-20250][CORE] Improper OOM error when a task been killed while 
spilling data
    
    ## What changes were proposed in this pull request?
    
    Currently, when a task is calling spill() but it receives a killing request 
from driver (e.g., speculative task), the `TaskMemoryManager` will throw an 
`OOM` exception.  And we don't catch `Fatal` exception when a error caused by 
`Thread.interrupt`. So for `ClosedByInterruptException`, we should throw 
`RuntimeException` instead of `OutOfMemoryError`.
    
    https://issues.apache.org/jira/browse/SPARK-20250?jql=project%20%3D%20SPARK
    
    ## How was this patch tested?
    
    Existing unit tests.
    
    Author: Xianyang Liu <xianyang....@intel.com>
    
    Closes #18090 from ConeyLiu/SPARK-20250.
    
    (cherry picked from commit 731462a04f8e33ac507ad19b4270c783a012a33e)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>

commit 8896c4ee9ea315a7dcd1a05b7201e7ad0539a5ed
Author: jinxing <jinxing6...@126.com>
Date:   2017-05-25T08:11:30Z

    [SPARK-19659] Fetch big blocks to disk when shuffle-read.
    
    ## What changes were proposed in this pull request?
    
    Currently the whole block is fetched into memory(off heap by default) when 
shuffle-read. A block is defined by (shuffleId, mapId, reduceId). Thus it can 
be large when skew situations. If OOM happens during shuffle read, job will be 
killed and users will be notified to "Consider boosting 
spark.yarn.executor.memoryOverhead". Adjusting parameter and allocating more 
memory can resolve the OOM. However the approach is not perfectly suitable for 
production environment, especially for data warehouse.
    Using Spark SQL as data engine in warehouse, users hope to have a unified 
parameter(e.g. memory) but less resource wasted(resource is allocated but not 
used). The hope is strong especially when migrating data engine to Spark from 
another one(e.g. Hive). Tuning the parameter for thousands of SQLs one by one 
is very time consuming.
    It's not always easy to predict skew situations, when happen, it make sense 
to fetch remote blocks to disk for shuffle-read, rather than kill the job 
because of OOM.
    
    In this pr, I propose to fetch big blocks to disk(which is also mentioned 
in SPARK-3019):
    
    1. Track average size and also the outliers(which are larger than 
2*avgSize) in MapStatus;
    2. Request memory from `MemoryManager` before fetch blocks and release the 
memory to `MemoryManager` when `ManagedBuffer` is released.
    3. Fetch remote blocks to disk when failing acquiring memory from 
`MemoryManager`, otherwise fetch to memory.
    
    This is an improvement for memory control when shuffle blocks and help to 
avoid OOM in scenarios like below:
    1. Single huge block;
    2. Sizes of many blocks are underestimated in `MapStatus` and the actual 
footprint of blocks is much larger than the estimated.
    
    ## How was this patch tested?
    Added unit test in `MapStatusSuite` and `ShuffleBlockFetcherIteratorSuite`.
    
    Author: jinxing <jinxing6...@126.com>
    
    Closes #16989 from jinxing64/SPARK-19659.
    
    (cherry picked from commit 3f94e64aa8fd806ae1fa0156d846ce96afacddd3)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>

commit 9cbf39f1c74f16483865cd93d6ffc3c521e878a7
Author: Yanbo Liang <yblia...@gmail.com>
Date:   2017-05-25T12:15:15Z

    [SPARK-19281][FOLLOWUP][ML] Minor fix for PySpark FPGrowth.
    
    ## What changes were proposed in this pull request?
    Follow-up for #17218, some minor fix for PySpark ```FPGrowth```.
    
    ## How was this patch tested?
    Existing UT.
    
    Author: Yanbo Liang <yblia...@gmail.com>
    
    Closes #18089 from yanboliang/spark-19281.
    
    (cherry picked from commit 913a6bfe4b0eb6b80a03b858ab4b2767194103de)
    Signed-off-by: Yanbo Liang <yblia...@gmail.com>

commit e01f1f222bcb7c469b1e1595e9338ed478d99894
Author: Yan Facai (颜发才) <facai....@gmail.com>
Date:   2017-05-25T13:40:39Z

    [SPARK-20768][PYSPARK][ML] Expose numPartitions (expert) param of PySpark 
FPGrowth.
    
    ## What changes were proposed in this pull request?
    
    Expose numPartitions (expert) param of PySpark FPGrowth.
    
    ## How was this patch tested?
    
    + [x] Pass all unit tests.
    
    Author: Yan Facai (颜发才) <facai....@gmail.com>
    
    Closes #18058 from facaiy/ENH/pyspark_fpg_add_num_partition.
    
    (cherry picked from commit 139da116f130ed21481d3e9bdee5df4b8d7760ac)
    Signed-off-by: Yanbo Liang <yblia...@gmail.com>

commit 022a4957d8dc8d6049e0a8c9191fcfd1bd95a4a4
Author: Lior Regev <liore...@gmail.com>
Date:   2017-05-25T16:08:19Z

    [SPARK-20741][SPARK SUBMIT] Added cleanup of JARs archive generated by 
SparkSubmit
    
    ## What changes were proposed in this pull request?
    
    Deleted generated JARs archive after distribution to HDFS
    
    ## How was this patch tested?
    
    Please review http://spark.apache.org/contributing.html before opening a 
pull request.
    
    Author: Lior Regev <liore...@gmail.com>
    
    Closes #17986 from liorregev/master.
    
    (cherry picked from commit 7306d556903c832984c7f34f1e8fe738a4b2343c)
    Signed-off-by: Sean Owen <so...@cloudera.com>

commit 5ae1c652147aba9c5087335b0c6916a1035090b2
Author: hyukjinkwon <gurwls...@gmail.com>
Date:   2017-05-25T16:10:30Z

    [SPARK-19707][SPARK-18922][TESTS][SQL][CORE] Fix test failures/the invalid 
path check for sc.addJar on Windows
    
    ## What changes were proposed in this pull request?
    
    This PR proposes two things:
    
    - A follow up for SPARK-19707 (Improving the invalid path check for 
sc.addJar on Windows as well).
    
    ```
    org.apache.spark.SparkContextSuite:
     - add jar with invalid path *** FAILED *** (32 milliseconds)
       2 was not equal to 1 (SparkContextSuite.scala:309)
       ...
    ```
    
    - Fix path vs URI related test failures on Windows.
    
    ```
    org.apache.spark.storage.LocalDirsSuite:
     - SPARK_LOCAL_DIRS override also affects driver *** FAILED *** (0 
milliseconds)
       new java.io.File("/NONEXISTENT_PATH").exists() was true 
(LocalDirsSuite.scala:50)
       ...
    
     - Utils.getLocalDir() throws an exception if any temporary directory 
cannot be retrieved *** FAILED *** (15 milliseconds)
       Expected exception java.io.IOException to be thrown, but no exception 
was thrown. (LocalDirsSuite.scala:64)
       ...
    ```
    
    ```
    org.apache.spark.sql.hive.HiveSchemaInferenceSuite:
     - orc: schema should be inferred and saved when INFER_AND_SAVE is 
specified *** FAILED *** (203 milliseconds)
       java.net.URISyntaxException: Illegal character in opaque part at index 
2: C:\projects\spark\target\tmp\spark-dae61ab3-a851-4dd3-bf4e-be97c501f254
       ...
    
     - parquet: schema should be inferred and saved when INFER_AND_SAVE is 
specified *** FAILED *** (203 milliseconds)
       java.net.URISyntaxException: Illegal character in opaque part at index 
2: C:\projects\spark\target\tmp\spark-fa3aff89-a66e-4376-9a37-2a9b87596939
       ...
    
     - orc: schema should be inferred but not stored when INFER_ONLY is 
specified *** FAILED *** (141 milliseconds)
       java.net.URISyntaxException: Illegal character in opaque part at index 
2: C:\projects\spark\target\tmp\spark-fb464e59-b049-481b-9c75-f53295c9fc2c
       ...
    
     - parquet: schema should be inferred but not stored when INFER_ONLY is 
specified *** FAILED *** (125 milliseconds)
       java.net.URISyntaxException: Illegal character in opaque part at index 
2: C:\projects\spark\target\tmp\spark-9487568e-80a4-42b3-b0a5-d95314c4ccbc
       ...
    
     - orc: schema should not be inferred when NEVER_INFER is specified *** 
FAILED *** (156 milliseconds)
       java.net.URISyntaxException: Illegal character in opaque part at index 
2: C:\projects\spark\target\tmp\spark-0d2dfa45-1b0f-4958-a8be-1074ed0135a
       ...
    
     - parquet: schema should not be inferred when NEVER_INFER is specified *** 
FAILED *** (547 milliseconds)
       java.net.URISyntaxException: Illegal character in opaque part at index 
2: C:\projects\spark\target\tmp\spark-6d95d64e-613e-4a59-a0f6-d198c5aa51ee
       ...
    ```
    
    ```
    org.apache.spark.sql.execution.command.DDLSuite:
     - create temporary view using *** FAILED *** (15 milliseconds)
       org.apache.spark.sql.AnalysisException: Path does not exist: 
file:/C:projectsspark       arget   
mpspark-3881d9ca-561b-488d-90b9-97587472b853    mp;
       ...
    
     - insert data to a data source table which has a non-existing location 
should succeed *** FAILED *** (109 milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-4cad3d19-6085-4b75-b407-fe5e9d21df54 
did not equal 
file:///C:/projects/spark/target/tmp/spark-4cad3d19-6085-4b75-b407-fe5e9d21df54 
(DDLSuite.scala:1869)
       ...
    
     - insert into a data source table with a non-existing partition location 
should succeed *** FAILED *** (94 milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d 
did not equal 
file:///C:/projects/spark/target/tmp/spark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d 
(DDLSuite.scala:1910)
       ...
    
     - read data from a data source table which has a non-existing location 
should succeed *** FAILED *** (93 milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-f8c281e2-08c2-4f73-abbf-f3865b702c34 
did not equal 
file:///C:/projects/spark/target/tmp/spark-f8c281e2-08c2-4f73-abbf-f3865b702c34 
(DDLSuite.scala:1937)
       ...
    
     - read data from a data source table with non-existing partition location 
should succeed *** FAILED *** (110 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - create datasource table with a non-existing location *** FAILED *** (94 
milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-387316ae-070c-4e78-9b78-19ebf7b29ec8 
did not equal 
file:///C:/projects/spark/target/tmp/spark-387316ae-070c-4e78-9b78-19ebf7b29ec8 
(DDLSuite.scala:1982)
       ...
    
     - CTAS for external data source table with a non-existing location *** 
FAILED *** (16 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - CTAS for external data source table with a existed location *** FAILED 
*** (15 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - data source table:partition column name containing a b *** FAILED *** 
(125 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - data source table:partition column name containing a:b *** FAILED *** 
(143 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - data source table:partition column name containing a%b *** FAILED *** 
(109 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - data source table:partition column name containing a,b *** FAILED *** 
(109 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - location uri contains a b for datasource table *** FAILED *** (94 
milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-5739cda9-b702-4e14-932c-42e8c4174480a%20b
 did not equal 
file:///C:/projects/spark/target/tmp/spark-5739cda9-b702-4e14-932c-42e8c4174480/a%20b
 (DDLSuite.scala:2084)
       ...
    
     - location uri contains a:b for datasource table *** FAILED *** (78 
milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-9bdd227c-840f-4f08-b7c5-4036638f098da:b 
did not equal 
file:///C:/projects/spark/target/tmp/spark-9bdd227c-840f-4f08-b7c5-4036638f098d/a:b
 (DDLSuite.scala:2084)
       ...
    
     - location uri contains a%b for datasource table *** FAILED *** (78 
milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-62bb5f1d-fa20-460a-b534-cb2e172a3640a%25b
 did not equal 
file:///C:/projects/spark/target/tmp/spark-62bb5f1d-fa20-460a-b534-cb2e172a3640/a%25b
 (DDLSuite.scala:2084)
       ...
    
     - location uri contains a b for database *** FAILED *** (16 milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - location uri contains a:b for database *** FAILED *** (15 milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - location uri contains a%b for database *** FAILED *** (0 milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    ```
    
    ```
    org.apache.spark.sql.hive.execution.HiveDDLSuite:
     - create hive table with a non-existing location *** FAILED *** (16 
milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - CTAS for external hive table with a non-existing location *** FAILED *** 
(16 milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - CTAS for external hive table with a existed location *** FAILED *** (16 
milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - partition column name of parquet table containing a b *** FAILED *** 
(156 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - partition column name of parquet table containing a:b *** FAILED *** (94 
milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - partition column name of parquet table containing a%b *** FAILED *** 
(125 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - partition column name of parquet table containing a,b *** FAILED *** 
(110 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    
     - partition column name of hive table containing a b *** FAILED *** (15 
milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - partition column name of hive table containing a:b *** FAILED *** (16 
milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - partition column name of hive table containing a%b *** FAILED *** (16 
milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - partition column name of hive table containing a,b *** FAILED *** (0 
milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - hive table: location uri contains a b *** FAILED *** (0 milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - hive table: location uri contains a:b *** FAILED *** (0 milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    
     - hive table: location uri contains a%b *** FAILED *** (0 milliseconds)
       org.apache.spark.sql.AnalysisException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
MetaException(message:java.lang.IllegalArgumentException: Can not create a Path 
from an empty string);
       ...
    ```
    
    ```
    org.apache.spark.sql.sources.PathOptionSuite:
     - path option also exist for write path *** FAILED *** (94 milliseconds)
       
file:/C:projectsspark%09arget%09mpspark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc 
did not equal 
file:///C:/projects/spark/target/tmp/spark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc 
(PathOptionSuite.scala:98)
       ...
    ```
    
    ```
    org.apache.spark.sql.CachedTableSuite:
     - SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer 
to this table *** FAILED *** (110 milliseconds)
       java.lang.IllegalArgumentException: Can not create a Path from an empty 
string
       ...
    ```
    
    ```
    org.apache.spark.sql.execution.DataSourceScanExecRedactionSuite:
     - treeString is redacted *** FAILED *** (250 milliseconds)
       
"file:/C:/projects/spark/target/tmp/spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" 
did not contain 
"C:\projects\spark\target\tmp\spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" 
(DataSourceScanExecRedactionSuite.scala:46)
       ...
    ```
    
    ## How was this patch tested?
    
    Tested via AppVeyor for each and checked it passed once each. These should 
be retested via AppVeyor in this PR.
    
    Author: hyukjinkwon <gurwls...@gmail.com>
    
    Closes #17987 from HyukjinKwon/windows-20170515.
    
    (cherry picked from commit e9f983df275c138626af35fd263a7abedf69297f)
    Signed-off-by: Sean Owen <so...@cloudera.com>

commit 7a21de9e2bb0d9344a371a8570b2fffa68c3236e
Author: Shixiong Zhu <shixi...@databricks.com>
Date:   2017-05-25T17:49:14Z

    [SPARK-20874][EXAMPLES] Add Structured Streaming Kafka Source to examples 
project
    
    ## What changes were proposed in this pull request?
    
    Add Structured Streaming Kafka Source to the `examples` project so that 
people can run `bin/run-example StructuredKafkaWordCount ...`.
    
    ## How was this patch tested?
    
    manually tested it.
    
    Author: Shixiong Zhu <shixi...@databricks.com>
    
    Closes #18101 from zsxwing/add-missing-example-dep.
    
    (cherry picked from commit 98c3852986a2cb5f2d249d6c8ef602be283bd90e)
    Signed-off-by: Shixiong Zhu <shixi...@databricks.com>

commit 289dd170cb3e0b9eca9af5841a0155ceaffee447
Author: Michael Allman <mich...@videoamp.com>
Date:   2017-05-26T01:25:43Z

    [SPARK-20888][SQL][DOCS] Document change of default setting of 
spark.sql.hive.caseSensitiveInferenceMode
    
    (Link to Jira: https://issues.apache.org/jira/browse/SPARK-20888)
    
    ## What changes were proposed in this pull request?
    
    Document change of default setting of 
spark.sql.hive.caseSensitiveInferenceMode configuration key from NEVER_INFO to 
INFER_AND_SAVE in the Spark SQL 2.1 to 2.2 migration notes.
    
    Author: Michael Allman <mich...@videoamp.com>
    
    Closes #18112 from mallman/spark-20888-document_infer_and_save.
    
    (cherry picked from commit c1e7989c4ffd83c51f5c97998b4ff6fe8dd83cf4)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>

commit fafe283277b50974c26684b06449086acd0cf05a
Author: Wenchen Fan <wenc...@databricks.com>
Date:   2017-05-26T07:01:28Z

    [SPARK-20868][CORE] UnsafeShuffleWriter should verify the position after 
FileChannel.transferTo
    
    ## What changes were proposed in this pull request?
    
    Long time ago we fixed a 
[bug](https://issues.apache.org/jira/browse/SPARK-3948) in shuffle writer about 
`FileChannel.transferTo`. We were not very confident about that fix, so we 
added a position check after the writing, try to discover the bug earlier.
    
     However this checking is missing in the new `UnsafeShuffleWriter`, this PR 
adds it.
    
    https://issues.apache.org/jira/browse/SPARK-18105 maybe related to that 
`FileChannel.transferTo` bug, hopefully we can find out the root cause after 
adding this position check.
    
    ## How was this patch tested?
    
    N/A
    
    Author: Wenchen Fan <wenc...@databricks.com>
    
    Closes #18091 from cloud-fan/shuffle.
    
    (cherry picked from commit d9ad78908f6189719cec69d34557f1a750d2e6af)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>

commit f99456b5f6225a534ce52cf2b817285eb8853926
Author: NICHOLAS T. MARION <nmar...@us.ibm.com>
Date:   2017-05-10T09:59:57Z

    [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities
    
    ## What changes were proposed in this pull request?
    
    Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these 
functions at any point that getParameter is called against a HttpServletRequest.
    
    ## How was this patch tested?
    
    Unit tests, IBM Security AppScan Standard no longer showing 
vulnerabilities, manual verification of WebUI pages.
    
    Author: NICHOLAS T. MARION <nmar...@us.ibm.com>
    
    Closes #17686 from n-marion/xss-fix.
    
    (cherry picked from commit b512233a457092b0e2a39d0b42cb021abc69d375)
    Signed-off-by: Sean Owen <so...@cloudera.com>

commit 92837aeb47fc3427166e4b6e62f6130f7480d7fa
Author: Kazuaki Ishizaki <ishiz...@jp.ibm.com>
Date:   2017-05-16T21:47:21Z

    [SPARK-19372][SQL] Fix throwing a Java exception at df.fliter() due to 64KB 
bytecode size limit
    
    ## What changes were proposed in this pull request?
    
    When an expression for `df.filter()` has many nodes (e.g. 400), the size of 
Java bytecode for the generated Java code is more than 64KB. It produces an 
Java exception. As a result, the execution fails.
    This PR continues to execute by calling `Expression.eval()` disabling code 
generation if an exception has been caught.
    
    ## How was this patch tested?
    
    Add a test suite into `DataFrameSuite`
    
    Author: Kazuaki Ishizaki <ishiz...@jp.ibm.com>
    
    Closes #17087 from kiszk/SPARK-19372.

commit 2b59ed4f1d4e859d5987b6eaaee074260b2a12f8
Author: Michael Armbrust <mich...@databricks.com>
Date:   2017-05-26T20:33:23Z

    [SPARK-20844] Remove experimental from Structured Streaming APIs
    
    Now that Structured Streaming has been out for several Spark release and 
has large production use cases, the `Experimental` label is no longer 
appropriate.  I've left `InterfaceStability.Evolving` however, as I think we 
may make a few changes to the pluggable Source & Sink API in Spark 2.3.
    
    Author: Michael Armbrust <mich...@databricks.com>
    
    Closes #18065 from marmbrus/streamingGA.

commit 30922dec8a8cc598b6715f85281591208a91df00
Author: zero323 <zero...@users.noreply.github.com>
Date:   2017-05-26T22:01:01Z

    [SPARK-20694][DOCS][SQL] Document DataFrameWriter partitionBy, bucketBy and 
sortBy in SQL guide
    
    ## What changes were proposed in this pull request?
    
    - Add Scala, Python and Java examples for `partitionBy`, `sortBy` and 
`bucketBy`.
    - Add _Bucketing, Sorting and Partitioning_ section to SQL Programming Guide
    - Remove bucketing from Unsupported Hive Functionalities.
    
    ## How was this patch tested?
    
    Manual tests, docs build.
    
    Author: zero323 <zero...@users.noreply.github.com>
    
    Closes #17938 from zero323/DOCS-BUCKETING-AND-PARTITIONING.
    
    (cherry picked from commit ae33abf71b353c638487948b775e966c7127cd46)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit fc799d730304c6a176636b414fc15184e89367d7
Author: Yu Peng <loneknigh...@gmail.com>
Date:   2017-05-26T23:28:36Z

    [SPARK-10643][CORE] Make spark-submit download remote files to local in 
client mode
    
    ## What changes were proposed in this pull request?
    
    This PR makes spark-submit script download remote files to local file 
system for local/standalone client mode.
    
    ## How was this patch tested?
    
    - Unit tests
    - Manual tests by adding s3a jar and testing against file on s3.
    
    Please review http://spark.apache.org/contributing.html before opening a 
pull request.
    
    Author: Yu Peng <loneknigh...@gmail.com>
    
    Closes #18078 from loneknightpy/download-jar-in-spark-submit.
    
    (cherry picked from commit 4af37812915763ac3bfd91a600a7f00a4b84d29a)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit 39f76657ef2967f4c87230e06cbbb1611c276375
Author: Wenchen Fan <wenc...@databricks.com>
Date:   2017-05-27T02:57:43Z

    [SPARK-19659][CORE][FOLLOW-UP] Fetch big blocks to disk when shuffle-read
    
    ## What changes were proposed in this pull request?
    
    This PR includes some minor improvement for the comments and tests in 
https://github.com/apache/spark/pull/16989
    
    ## How was this patch tested?
    
    N/A
    
    Author: Wenchen Fan <wenc...@databricks.com>
    
    Closes #18117 from cloud-fan/follow.
    
    (cherry picked from commit 1d62f8aca82601506c44b6fd852f4faf3602d7e2)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>

commit f2408bdd7a0950385ee1364e006d55bfa6e5a200
Author: Shixiong Zhu <shixi...@databricks.com>
Date:   2017-05-27T05:25:38Z

    [SPARK-20843][CORE] Add a config to set driver terminate timeout
    
    ## What changes were proposed in this pull request?
    
    Add a `worker` configuration to set how long to wait before forcibly 
killing driver.
    
    ## How was this patch tested?
    
    Jenkins
    
    Author: Shixiong Zhu <shixi...@databricks.com>
    
    Closes #18126 from zsxwing/SPARK-20843.
    
    (cherry picked from commit 6c1dbd6fc8d49acf7c1c902d2ebf89ed5e788a4e)
    Signed-off-by: Shixiong Zhu <shixi...@databricks.com>

commit 25e87d80c483785dc4a79fb283bc80f68197bf4a
Author: Wenchen Fan <wenc...@databricks.com>
Date:   2017-05-27T23:16:51Z

    [SPARK-20897][SQL] cached self-join should not fail
    
    ## What changes were proposed in this pull request?
    
    The failed test case is, we have a `SortMergeJoinExec` for a self-join, 
which means we have a `ReusedExchange` node in the query plan. It works fine 
without caching, but throws an exception in 
`SortMergeJoinExec.outputPartitioning` if we cache it.
    
    The root cause is, `ReusedExchange` doesn't propagate the output 
partitioning from its child, so in `SortMergeJoinExec.outputPartitioning` we 
create `PartitioningCollection` with a hash partitioning and an unknown 
partitioning, and fail.
    
    This bug is mostly fine, because inserting the `ReusedExchange` is the last 
step to prepare the physical plan, we won't call 
`SortMergeJoinExec.outputPartitioning` anymore after this.
    
    However, if the dataframe is cached, the physical plan of it becomes 
`InMemoryTableScanExec`, which contains another physical plan representing the 
cached query, and it has gone through the entire planning phase and may have 
`ReusedExchange`. Then the planner call 
`InMemoryTableScanExec.outputPartitioning`, which then calls 
`SortMergeJoinExec.outputPartitioning` and trigger this bug.
    
    ## How was this patch tested?
    
    a new regression test
    
    Author: Wenchen Fan <wenc...@databricks.com>
    
    Closes #18121 from cloud-fan/bug.
    
    (cherry picked from commit 08ede46b897b7e52cfe8231ffc21d9515122cf49)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit dc51be1e79b89d143da5df16b893df86a306f059
Author: Xiao Li <gatorsm...@gmail.com>
Date:   2017-05-28T04:32:18Z

    [SPARK-20908][SQL] Cache Manager: Hint should be ignored in plan matching
    
    ### What changes were proposed in this pull request?
    
    In Cache manager, the plan matching should ignore Hint.
    ```Scala
          val df1 = spark.range(10).join(broadcast(spark.range(10)))
          df1.cache()
          spark.range(10).join(spark.range(10)).explain()
    ```
    The output plan of the above query shows that the second query is  not 
using the cached data of the first query.
    ```
    BroadcastNestedLoopJoin BuildRight, Inner
    :- *Range (0, 10, step=1, splits=2)
    +- BroadcastExchange IdentityBroadcastMode
       +- *Range (0, 10, step=1, splits=2)
    ```
    
    After the fix, the plan becomes
    ```
    InMemoryTableScan [id#20L, id#23L]
       +- InMemoryRelation [id#20L, id#23L], true, 10000, StorageLevel(disk, 
memory, deserialized, 1 replicas)
             +- BroadcastNestedLoopJoin BuildRight, Inner
                :- *Range (0, 10, step=1, splits=2)
                +- BroadcastExchange IdentityBroadcastMode
                   +- *Range (0, 10, step=1, splits=2)
    ```
    
    ### How was this patch tested?
    Added a test.
    
    Author: Xiao Li <gatorsm...@gmail.com>
    
    Closes #18131 from gatorsmile/HintCache.
    
    (cherry picked from commit 06c155c90dc784b07002f33d98dcfe9be1e38002)
    Signed-off-by: Xiao Li <gatorsm...@gmail.com>

commit 26640a26984bac4fc1037714e60bd3607929b377
Author: Kazuaki Ishizaki <ishiz...@jp.ibm.com>
Date:   2017-05-29T19:17:14Z

    [SPARK-20907][TEST] Use testQuietly for test suites that generate long log 
output
    
    ## What changes were proposed in this pull request?
    
    Supress console output by using `testQuietly` in test suites
    
    ## How was this patch tested?
    
    Tested by `"SPARK-19372: Filter can be executed w/o generated code due to 
JVM code size limit"` in `DataFrameSuite`
    
    Author: Kazuaki Ishizaki <ishiz...@jp.ibm.com>
    
    Closes #18135 from kiszk/SPARK-20907.
    
    (cherry picked from commit c9749068ecf8e0acabdfeeceeedff0f1f73293b7)
    Signed-off-by: Shixiong Zhu <shixi...@databricks.com>

commit 3b79e4cda74e0bf82ec55e673beb8f84e7cfaca4
Author: Yuming Wang <wgy...@gmail.com>
Date:   2017-05-29T23:10:22Z

    [SPARK-8184][SQL] Add additional function description for weekofyear
    
    ## What changes were proposed in this pull request?
    
    Add additional function description for weekofyear.
    
    ## How was this patch tested?
    
     manual tests
    
    
![weekofyear](https://cloud.githubusercontent.com/assets/5399861/26525752/08a1c278-4394-11e7-8988-7cbf82c3a999.gif)
    
    Author: Yuming Wang <wgy...@gmail.com>
    
    Closes #18132 from wangyum/SPARK-8184.
    
    (cherry picked from commit 1c7db00c74ec6a91c7eefbdba85cbf41fbe8634a)
    Signed-off-by: Reynold Xin <r...@databricks.com>

commit f6730a70cb47ebb3df7f42209df7b076aece1093
Author: Prashant Sharma <prash...@in.ibm.com>
Date:   2017-05-30T01:12:01Z

    [SPARK-19968][SS] Use a cached instance of `KafkaProducer` instead of 
creating one every batch.
    
    ## What changes were proposed in this pull request?
    
    In summary, cost of recreating a KafkaProducer for writing every batch is 
high as it starts a lot threads and make connections and then closes them. A 
KafkaProducer instance is promised to be thread safe in Kafka docs. Reuse of 
KafkaProducer instance while writing via multiple threads is encouraged.
    
    Furthermore, I have performance improvement of 10x in latency, with this 
patch.
    
    ### These are times that addBatch took in ms. Without applying this patch
    
![with-out_patch](https://cloud.githubusercontent.com/assets/992952/23994612/a9de4a42-0a6b-11e7-9d5b-7ae18775bee4.png)
    ### These are times that addBatch took in ms. After applying this patch
    
![with_patch](https://cloud.githubusercontent.com/assets/992952/23994616/ad8c11ec-0a6b-11e7-8634-2266ebb5033f.png)
    
    ## How was this patch tested?
    Running distributed benchmarks comparing runs with this patch and without 
it.
    Added relevant unit tests.
    
    Author: Prashant Sharma <prash...@in.ibm.com>
    
    Closes #17308 from ScrapCodes/cached-kafka-producer.
    
    (cherry picked from commit 96a4d1d0827fc3fba83f174510b061684f0d00f7)
    Signed-off-by: Shixiong Zhu <shixi...@databricks.com>

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to