You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by Parth-Brahmbhatt <gi...@git.apache.org> on 2016/07/21 19:16:18 UTC

[GitHub] spark pull request #14305: Spark-16669:Adding partition prunning to Metastor...

GitHub user Parth-Brahmbhatt opened a pull request:

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

    Spark-16669:Adding partition prunning to Metastore statistics for better join selection.

    ## What changes were proposed in this pull request?
    Currently the metastore statistics returns the size of entire table which results in Join selection strategy to not use broadcast joins even when only a single partition from a large table is selected. This PR addresses that issue by only estimating the size of the partition by applying partition pruning during size estimation. Currently it only works with partition columns used with equality checks under AND,OR,IN Operators. If a partition column is used in any other operator, it defaults back to total table size. A config controls the behavior which is off by default.
    
    ## How was this patch tested?
    Unit tests added, and manually tested in local environment.
    


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

    $ git pull https://github.com/Parth-Brahmbhatt/spark SPARK-16669

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

    https://github.com/apache/spark/pull/14305.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 #14305
    
----
commit 76d0ef34e4a5b91b883141f839adc493205fa429
Author: Xiangrui Meng <me...@databricks.com>
Date:   2016-06-22T17:06:43Z

    [MINOR][MLLIB] DefaultParamsReadable/Writable should be DeveloperApi
    
    ## What changes were proposed in this pull request?
    
    `DefaultParamsReadable/Writable` are not user-facing. Only developers who implement `Transformer/Estimator` would use it. So this PR changes the annotation to `DeveloperApi`.
    
    Author: Xiangrui Meng <me...@databricks.com>
    
    Closes #13828 from mengxr/default-readable-should-be-developer-api.
    
    (cherry picked from commit 6a6010f0015542dc2753b2cb12fdd1204db63ea6)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 520828c90d25acf733ffa70fe269dcfe93b56a31
Author: Ahmed Mahran <ah...@mashin.io>
Date:   2016-06-22T17:39:24Z

    [SPARK-16120][STREAMING] getCurrentLogFiles in ReceiverSuite WAL generating and cleaning case uses external variable instead of the passed parameter
    
    ## What changes were proposed in this pull request?
    
    In `ReceiverSuite.scala`, in the test case "write ahead log - generating and cleaning", the inner method `getCurrentLogFiles` uses external variable `logDirectory1` instead of the passed parameter `logDirectory`. This PR fixes this by using the passed method argument instead of variable from the outer scope.
    
    ## How was this patch tested?
    
    The unit test was re-run and the output logs were checked for the correct paths used.
    
    tdas
    
    Author: Ahmed Mahran <ah...@mashin.io>
    
    Closes #13825 from ahmed-mahran/b-receiver-suite-wal-gen-cln.
    
    (cherry picked from commit c2cebdb7ddff3d041d548fe1cd8de4efb31b294f)
    Signed-off-by: Shixiong Zhu <sh...@databricks.com>

commit e043c02d039809be149622a4d7562f332cfa25aa
Author: Eric Liang <ek...@databricks.com>
Date:   2016-06-22T19:12:34Z

    [SPARK-16003] SerializationDebugger runs into infinite loop
    
    ## What changes were proposed in this pull request?
    
    This fixes SerializationDebugger to not recurse forever when `writeReplace` returns an object of the same class, which is the case for at least the `SQLMetrics` class.
    
    See also the OpenJDK unit tests on the behavior of recursive `writeReplace()`:
    https://github.com/openjdk-mirror/jdk7u-jdk/blob/f4d80957e89a19a29bb9f9807d2a28351ed7f7df/test/java/io/Serializable/nestedReplace/NestedReplace.java
    
    cc davies cloud-fan
    
    ## How was this patch tested?
    
    Unit tests for SerializationDebugger.
    
    Author: Eric Liang <ek...@databricks.com>
    
    Closes #13814 from ericl/spark-16003.
    
    (cherry picked from commit 6f915c9ec24003877d1ef675a59145699780a2ff)
    Signed-off-by: Davies Liu <da...@gmail.com>

commit 299f427b70f8dedbc0b554f83c4fde408caf4d15
Author: Kai Jiang <ji...@gmail.com>
Date:   2016-06-22T19:50:36Z

    [SPARK-15672][R][DOC] R programming guide update
    
    ## What changes were proposed in this pull request?
    Guide for
    - UDFs with dapply, dapplyCollect
    - spark.lapply for running parallel R functions
    
    ## How was this patch tested?
    build locally
    <img width="654" alt="screen shot 2016-06-14 at 03 12 56" src="https://cloud.githubusercontent.com/assets/3419881/16039344/12a3b6a0-31de-11e6-8d77-fe23308075c0.png">
    
    Author: Kai Jiang <ji...@gmail.com>
    
    Closes #13660 from vectorijk/spark-15672-R-guide-update.
    
    (cherry picked from commit 43b04b7ecb313a2cee6121dd575de1f7dc785c11)
    Signed-off-by: Joseph K. Bradley <jo...@databricks.com>

commit 282a3cd02389464d6adbf02921281c963da29b00
Author: Davies Liu <da...@databricks.com>
Date:   2016-06-22T20:40:24Z

    [SPARK-16078][SQL] from_utc_timestamp/to_utc_timestamp should not depends on local timezone
    
    ## What changes were proposed in this pull request?
    
    Currently, we use local timezone to parse or format a timestamp (TimestampType), then use Long as the microseconds since epoch UTC.
    
    In from_utc_timestamp() and to_utc_timestamp(), we did not consider the local timezone, they could return different results with different local timezone.
    
    This PR will do the conversion based on human time (in local timezone), it should return same result in whatever timezone. But because the mapping from absolute timestamp to human time is not exactly one-to-one mapping, it will still return wrong result in some timezone (also in the begging or ending of DST).
    
    This PR is kind of the best effort fix. In long term, we should make the TimestampType be timezone aware to fix this totally.
    
    ## How was this patch tested?
    
    Tested these function in all timezone.
    
    Author: Davies Liu <da...@databricks.com>
    
    Closes #13784 from davies/convert_tz.
    
    (cherry picked from commit 20d411bc5d05dd099f6d5234a24e10a519a39bdf)
    Signed-off-by: Herman van Hovell <hv...@databricks.com>

commit 02435acf3bf84f77bb3c70a2fd548af8bad4c28e
Author: Xiangrui Meng <me...@databricks.com>
Date:   2016-06-22T22:50:21Z

    [SPARK-16153][MLLIB] switch to multi-line doc to avoid a genjavadoc bug
    
    ## What changes were proposed in this pull request?
    
    We recently deprecated setLabelCol in ChiSqSelectorModel (#13823):
    
    ~~~scala
      /** group setParam */
      Since("1.6.0")
      deprecated("labelCol is not used by ChiSqSelectorModel.", "2.0.0")
      def setLabelCol(value: String): this.type = set(labelCol, value)
    ~~~
    
    This unfortunately hit a genjavadoc bug and broken doc generation. This is the generated Java code:
    
    ~~~java
      /** group setParam */
      public  org.apache.spark.ml.feature.ChiSqSelectorModel setOutputCol (java.lang.String value)  { throw new RuntimeException(); }
       *
       * deprecated labelCol is not used by ChiSqSelectorModel. Since 2.0.0.
      */
      public  org.apache.spark.ml.feature.ChiSqSelectorModel setLabelCol (java.lang.String value)  { throw new RuntimeException(); }
    ~~~
    
    Switching to multiline is a workaround.
    
    Author: Xiangrui Meng <me...@databricks.com>
    
    Closes #13855 from mengxr/SPARK-16153.
    
    (cherry picked from commit 00cc5cca4522297b63b1522a2b8643b1a098e2b3)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 1d3c56e778b28ad4587d07765896814bfc1201f4
Author: Xiangrui Meng <me...@databricks.com>
Date:   2016-06-22T22:52:37Z

    [SPARK-16155][DOC] remove package grouping in Java docs
    
    ## What changes were proposed in this pull request?
    
    In 1.4 and earlier releases, we have package grouping in the generated Java API docs. See http://spark.apache.org/docs/1.4.0/api/java/index.html. However, this disappeared in 1.5.0: http://spark.apache.org/docs/1.5.0/api/java/index.html.
    
    Rather than fixing it, I'd suggest removing grouping. Because it might take some time to fix and it is a manual process to update the grouping in `SparkBuild.scala`. I didn't find anyone complaining about missing groups since 1.5.0 on Google.
    
    Manually checked the generated Java API docs and confirmed that they are the same as in master.
    
    Author: Xiangrui Meng <me...@databricks.com>
    
    Closes #13856 from mengxr/SPARK-16155.
    
    (cherry picked from commit 857ecff1d8268b28bb287e47cda370c87afe9d41)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit e2eb8e002acb19fd266d2237baec31f74aa02ef8
Author: Prajwal Tuladhar <pr...@infynyxx.com>
Date:   2016-06-22T23:30:10Z

    [SPARK-16131] initialize internal logger lazily in Scala preferred way
    
    ## What changes were proposed in this pull request?
    
    Initialize logger instance lazily in Scala preferred way
    
    ## How was this patch tested?
    
    By running `./build/mvn clean test` locally
    
    Author: Prajwal Tuladhar <pr...@infynyxx.com>
    
    Closes #13842 from infynyxx/spark_internal_logger.
    
    (cherry picked from commit 044971eca0ff3c2ce62afa665dbd3072d52cbbec)
    Signed-off-by: Shixiong Zhu <sh...@databricks.com>

commit e0a43235d9d59736ceb0d703c653ef1350e143ec
Author: gatorsmile <ga...@gmail.com>
Date:   2016-06-23T01:12:20Z

    [SPARK-16024][SQL][TEST] Verify Column Comment for Data Source Tables
    
    #### What changes were proposed in this pull request?
    This PR is to improve test coverage. It verifies whether `Comment` of `Column` can be appropriate handled.
    
    The test cases verify the related parts in Parser, both SQL and DataFrameWriter interface, and both Hive Metastore catalog and In-memory catalog.
    
    #### How was this patch tested?
    N/A
    
    Author: gatorsmile <ga...@gmail.com>
    
    Closes #13764 from gatorsmile/dataSourceComment.
    
    (cherry picked from commit 9f990fa3f9e0b798d8018cf4132b93a3468f33bb)
    Signed-off-by: Wenchen Fan <we...@databricks.com>

commit 5b4a9a4c37822cd7528c6bb933da3454fd3bcd37
Author: bomeng <bm...@us.ibm.com>
Date:   2016-06-23T03:06:19Z

    [SPARK-15230][SQL] distinct() does not handle column name with dot properly
    
    ## What changes were proposed in this pull request?
    
    When table is created with column name containing dot, distinct() will fail to run. For example,
    ```scala
    val rowRDD = sparkContext.parallelize(Seq(Row(1), Row(1), Row(2)))
    val schema = StructType(Array(StructField("column.with.dot", IntegerType, nullable = false)))
    val df = spark.createDataFrame(rowRDD, schema)
    ```
    running the following will have no problem:
    ```scala
    df.select(new Column("`column.with.dot`"))
    ```
    but running the query with additional distinct() will cause exception:
    ```scala
    df.select(new Column("`column.with.dot`")).distinct()
    ```
    
    The issue is that distinct() will try to resolve the column name, but the column name in the schema does not have backtick with it. So the solution is to add the backtick before passing the column name to resolve().
    
    ## How was this patch tested?
    
    Added a new test case.
    
    Author: bomeng <bm...@us.ibm.com>
    
    Closes #13140 from bomeng/SPARK-15230.
    
    (cherry picked from commit 925884a612dd88beaddf555c74d90856ab040ec7)
    Signed-off-by: Wenchen Fan <we...@databricks.com>

commit 4ad731ed6a963131f05c387c2f9536b56d228090
Author: Cheng Lian <li...@databricks.com>
Date:   2016-06-23T05:28:54Z

    [SQL][MINOR] Fix minor formatting issues in SHOW CREATE TABLE output
    
    ## What changes were proposed in this pull request?
    
    This PR fixes two minor formatting issues appearing in `SHOW CREATE TABLE` output.
    
    Before:
    
    ```
    CREATE EXTERNAL TABLE ...
    ...
    WITH SERDEPROPERTIES ('serialization.format' = '1'
    )
    ...
    TBLPROPERTIES ('avro.schema.url' = '/tmp/avro/test.avsc',
      'transient_lastDdlTime' = '1466638180')
    ```
    
    After:
    
    ```
    CREATE EXTERNAL TABLE ...
    ...
    WITH SERDEPROPERTIES (
      'serialization.format' = '1'
    )
    ...
    TBLPROPERTIES (
      'avro.schema.url' = '/tmp/avro/test.avsc',
      'transient_lastDdlTime' = '1466638180'
    )
    ```
    
    ## How was this patch tested?
    
    Manually tested.
    
    Author: Cheng Lian <li...@databricks.com>
    
    Closes #13864 from liancheng/show-create-table-format-fix.
    
    (cherry picked from commit f34b5c62b2da3fe0ea989acea46fff949d349afc)
    Signed-off-by: Reynold Xin <rx...@databricks.com>

commit 567093596057eb77d940d53c88b82da128acfd9b
Author: Xiangrui Meng <me...@databricks.com>
Date:   2016-06-23T15:26:17Z

    [SPARK-16154][MLLIB] Update spark.ml and spark.mllib package docs
    
    ## What changes were proposed in this pull request?
    
    Since we decided to switch spark.mllib package into maintenance mode in 2.0, it would be nice to update the package docs to reflect this change.
    
    ## How was this patch tested?
    
    Manually checked generated APIs.
    
    Author: Xiangrui Meng <me...@databricks.com>
    
    Closes #13859 from mengxr/SPARK-16154.
    
    (cherry picked from commit 65d1f0f716f50dd14b5dfe1e7fac772f1b4d2be0)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 9f18c8f386af558ed72b88ad372835f25e807e79
Author: Felix Cheung <fe...@hotmail.com>
Date:   2016-06-23T16:45:01Z

    [SPARK-16088][SPARKR] update setJobGroup, cancelJobGroup, clearJobGroup
    
    ## What changes were proposed in this pull request?
    
    Updated setJobGroup, cancelJobGroup, clearJobGroup to not require sc/SparkContext as parameter.
    Also updated roxygen2 doc and R programming guide on deprecations.
    
    ## How was this patch tested?
    
    unit tests
    
    Author: Felix Cheung <fe...@hotmail.com>
    
    Closes #13838 from felixcheung/rjobgroup.
    
    (cherry picked from commit b5a997667f4c0e514217da6df5af37b8b849dfdf)
    Signed-off-by: Shivaram Venkataraman <sh...@cs.berkeley.edu>

commit 63fd3301c5f341d616b314a0a44bfd7d2cf4fbae
Author: Shixiong Zhu <sh...@databricks.com>
Date:   2016-06-23T17:46:20Z

    [SPARK-16116][SQL] ConsoleSink should not require checkpointLocation
    
    ## What changes were proposed in this pull request?
    
    When the user uses `ConsoleSink`, we should use a temp location if `checkpointLocation` is not specified.
    
    ## How was this patch tested?
    
    The added unit test.
    
    Author: Shixiong Zhu <sh...@databricks.com>
    
    Closes #13817 from zsxwing/console-checkpoint.
    
    (cherry picked from commit d85bb10ce49926b8b661bd2cb97392205742fc14)
    Signed-off-by: Shixiong Zhu <sh...@databricks.com>

commit dff3d75db4c2848a43ed8a3084c75f38c93138af
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-06-23T18:00:00Z

    [SPARK-16130][ML] model loading backward compatibility for ml.classfication.LogisticRegression
    
    ## What changes were proposed in this pull request?
    jira: https://issues.apache.org/jira/browse/SPARK-16130
    model loading backward compatibility for ml.classfication.LogisticRegression
    
    ## How was this patch tested?
    existing ut and manual test for loading old models.
    
    Author: Yuhao Yang <hh...@gmail.com>
    
    Closes #13841 from hhbyyh/lrcomp.
    
    (cherry picked from commit 60398dabc50d402bbab4190fbe94ebed6d3a48dc)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 214676d29d3d66c3e37ab6ff9fae70adb056b8b2
Author: Davies Liu <da...@databricks.com>
Date:   2016-06-23T18:48:48Z

    [SPARK-16163] [SQL] Cache the statistics for logical plans
    
    ## What changes were proposed in this pull request?
    
    This calculation of statistics is not trivial anymore, it could be very slow on large query (for example, TPC-DS Q64 took several minutes to plan).
    
    During the planning of a query, the statistics of any logical plan should not change (even InMemoryRelation), so we should use `lazy val` to cache the statistics.
    
    For InMemoryRelation, the statistics could be updated after materialization, it's only useful when used in another query (before planning), because once we finished the planning, the statistics will not be used anymore.
    
    ## How was this patch tested?
    
    Testsed with TPC-DS Q64, it could be planned in a second after the patch.
    
    Author: Davies Liu <da...@databricks.com>
    
    Closes #13871 from davies/fix_statistics.
    
    (cherry picked from commit 10396d9505c752cc18b6424f415d4ff0f460ad65)
    Signed-off-by: Davies Liu <da...@gmail.com>

commit b8818d892a4f62ee4e8a1c16146b5ee4c7de9eab
Author: Ryan Blue <bl...@apache.org>
Date:   2016-06-23T18:54:37Z

    [SPARK-15725][YARN] Ensure ApplicationMaster sleeps for the min interval.
    
    ## What changes were proposed in this pull request?
    
    Update `ApplicationMaster` to sleep for at least the minimum allocation interval before calling `allocateResources`. This prevents overloading the `YarnAllocator` that is happening because the thread is triggered when an executor is killed and its connections die. In YARN, this prevents the app from overloading the allocator and becoming unstable.
    
    ## How was this patch tested?
    
    Tested that this allows the an app to recover instead of hanging. It is still possible for the YarnAllocator to be overwhelmed by requests, but this prevents the issue for the most common cause.
    
    Author: Ryan Blue <bl...@apache.org>
    
    Closes #13482 from rdblue/SPARK-15725-am-sleep-work-around.
    
    (cherry picked from commit a410814c87b120cb5cfbf095b1bd94b1de862844)
    Signed-off-by: Tom Graves <tg...@yahoo-inc.com>

commit 2ce240cfe0cbcb944d225b2455a9cb2e806699f0
Author: Ryan Blue <bl...@apache.org>
Date:   2016-06-23T19:03:46Z

    [SPARK-13723][YARN] Change behavior of --num-executors with dynamic allocation.
    
    ## What changes were proposed in this pull request?
    
    This changes the behavior of --num-executors and spark.executor.instances when using dynamic allocation. Instead of turning dynamic allocation off, it uses the value for the initial number of executors.
    
    This changes was discussed on [SPARK-13723](https://issues.apache.org/jira/browse/SPARK-13723). I highly recommend using it while we can change the behavior for 2.0.0. In practice, the 1.x behavior causes unexpected behavior for users (it is not clear that it disables dynamic allocation) and wastes cluster resources because users rarely notice the log message.
    
    ## How was this patch tested?
    
    This patch updates tests and adds a test for Utils.getDynamicAllocationInitialExecutors.
    
    Author: Ryan Blue <bl...@apache.org>
    
    Closes #13338 from rdblue/SPARK-13723-num-executors-with-dynamic-allocation.
    
    (cherry picked from commit 738f134bf4bf07bafb17e7066cf1a36e315872c2)
    Signed-off-by: Tom Graves <tg...@yahoo-inc.com>

commit 6cb24de99e011ce97fb7d3513a2760b0d1a85a45
Author: Dongjoon Hyun <do...@apache.org>
Date:   2016-06-23T22:27:43Z

    [SPARK-16164][SQL] Update `CombineFilters` to try to construct predicates with child predicate first
    
    ## What changes were proposed in this pull request?
    
    This PR changes `CombineFilters` to compose the final predicate condition by using (`child predicate` AND `parent predicate`) instead of (`parent predicate` AND `child predicate`). This is a best effort approach. Some other optimization rules may destroy this order by reorganizing conjunctive predicates.
    
    **Reported Error Scenario**
    Chris McCubbin reported a bug when he used StringIndexer in an ML pipeline with additional filters. It seems that during filter pushdown, we changed the ordering in the logical plan.
    ```scala
    import org.apache.spark.ml.feature._
    val df1 = (0 until 3).map(_.toString).toDF
    val indexer = new StringIndexer()
      .setInputCol("value")
      .setOutputCol("idx")
      .setHandleInvalid("skip")
      .fit(df1)
    val df2 = (0 until 5).map(_.toString).toDF
    val predictions = indexer.transform(df2)
    predictions.show() // this is okay
    predictions.where('idx > 2).show() // this will throw an exception
    ```
    
    Please see the notebook at https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/1233855/2159162931615821/588180/latest.html for error messages.
    
    ## How was this patch tested?
    
    Pass the Jenkins tests (including a new testcase).
    
    Author: Dongjoon Hyun <do...@apache.org>
    
    Closes #13872 from dongjoon-hyun/SPARK-16164.
    
    (cherry picked from commit 91b1ef28d134313d7b6faaffa1c390f3ca4455d0)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 05677bb5a1fed91711a0e1be466dbc86d15bbf8e
Author: Shixiong Zhu <sh...@databricks.com>
Date:   2016-06-23T23:04:16Z

    [SPARK-15443][SQL] Fix 'explain' for streaming Dataset
    
    ## What changes were proposed in this pull request?
    
    - Fix the `explain` command for streaming Dataset/DataFrame. E.g.,
    ```
    == Parsed Logical Plan ==
    'SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- 'MapElements <function1>, obj#6: java.lang.String
       +- 'DeserializeToObject unresolveddeserializer(createexternalrow(getcolumnbyordinal(0, StringType).toString, StructField(value,StringType,true))), obj#5: org.apache.spark.sql.Row
          +- Filter <function1>.apply
             +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]
    
    == Analyzed Logical Plan ==
    value: string
    SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- MapElements <function1>, obj#6: java.lang.String
       +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
          +- Filter <function1>.apply
             +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]
    
    == Optimized Logical Plan ==
    SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- MapElements <function1>, obj#6: java.lang.String
       +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
          +- Filter <function1>.apply
             +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]
    
    == Physical Plan ==
    *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- *MapElements <function1>, obj#6: java.lang.String
       +- *DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
          +- *Filter <function1>.apply
             +- StreamingRelation FileSource[/Users/zsx/stream], [value#0]
    ```
    
    - Add `StreamingQuery.explain` to display the last execution plan. E.g.,
    ```
    == Parsed Logical Plan ==
    SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- MapElements <function1>, obj#6: java.lang.String
       +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
          +- Filter <function1>.apply
             +- Relation[value#12] text
    
    == Analyzed Logical Plan ==
    value: string
    SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- MapElements <function1>, obj#6: java.lang.String
       +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
          +- Filter <function1>.apply
             +- Relation[value#12] text
    
    == Optimized Logical Plan ==
    SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- MapElements <function1>, obj#6: java.lang.String
       +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
          +- Filter <function1>.apply
             +- Relation[value#12] text
    
    == Physical Plan ==
    *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7]
    +- *MapElements <function1>, obj#6: java.lang.String
       +- *DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row
          +- *Filter <function1>.apply
             +- *Scan text [value#12] Format: org.apache.spark.sql.execution.datasources.text.TextFileFormat1836ab91, InputPaths: file:/Users/zsx/stream/a.txt, file:/Users/zsx/stream/b.txt, file:/Users/zsx/stream/c.txt, PushedFilters: [], ReadSchema: struct<value:string>
    ```
    
    ## How was this patch tested?
    
    The added unit tests.
    
    Author: Shixiong Zhu <sh...@databricks.com>
    
    Closes #13815 from zsxwing/sdf-explain.
    
    (cherry picked from commit 0e4bdebece892edb126fa443f67c846e44e7367e)
    Signed-off-by: Yin Huai <yh...@databricks.com>

commit e8d1bf60a45c6662c955b5a3618ff7299713b6d8
Author: Dongjoon Hyun <do...@apache.org>
Date:   2016-06-23T23:19:20Z

    [SPARK-16165][SQL] Fix the update logic for InMemoryTableScanExec.readBatches
    
    ## What changes were proposed in this pull request?
    
    Currently, `readBatches` accumulator of `InMemoryTableScanExec` is updated only when `spark.sql.inMemoryColumnarStorage.partitionPruning` is true. Although this metric is used for only testing purpose, we had better have correct metric without considering SQL options.
    
    ## How was this patch tested?
    
    Pass the Jenkins tests (including a new testcase).
    
    Author: Dongjoon Hyun <do...@apache.org>
    
    Closes #13870 from dongjoon-hyun/SPARK-16165.
    
    (cherry picked from commit 264bc63623b20529abcf84abcb333e7c16ad1ef9)
    Signed-off-by: Cheng Lian <li...@databricks.com>

commit edd5dd24a56f1316d14ca23261cea5b49b16adf2
Author: Sameer Agarwal <sa...@databricks.com>
Date:   2016-06-24T01:21:41Z

    [SPARK-16123] Avoid NegativeArraySizeException while reserving additional capacity in VectorizedColumnReader
    
    ## What changes were proposed in this pull request?
    
    This patch fixes an overflow bug in vectorized parquet reader where both off-heap and on-heap variants of `ColumnVector.reserve()` can unfortunately overflow while reserving additional capacity during reads.
    
    ## How was this patch tested?
    
    Manual Tests
    
    Author: Sameer Agarwal <sa...@databricks.com>
    
    Closes #13832 from sameeragarwal/negative-array.
    
    (cherry picked from commit cc71d4fa372f6eb187c68dbd8358de4003ace3fe)
    Signed-off-by: Herman van Hovell <hv...@databricks.com>

commit a6edec2c5c1d217a9fc16ee6696a558a6c88a34e
Author: Wenchen Fan <we...@databricks.com>
Date:   2016-06-24T03:20:55Z

    [SQL][MINOR] ParserUtils.operationNotAllowed should throw exception directly
    
    ## What changes were proposed in this pull request?
    
    It's weird that `ParserUtils.operationNotAllowed` returns an exception and the caller throw it.
    
    ## How was this patch tested?
    
    N/A
    
    Author: Wenchen Fan <we...@databricks.com>
    
    Closes #13874 from cloud-fan/style.
    
    (cherry picked from commit 6a3c6276f5cef26b0a4fef44c8ad99bbecfe006d)
    Signed-off-by: Herman van Hovell <hv...@databricks.com>

commit ea0cf93d3969845e9df8305c0ce54326cdfb2bbd
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-06-24T03:43:19Z

    [SPARK-16177][ML] model loading backward compatibility for ml.regression
    
    ## What changes were proposed in this pull request?
    jira: https://issues.apache.org/jira/browse/SPARK-16177
    model loading backward compatibility for ml.regression
    
    ## How was this patch tested?
    
    existing ut and manual test for loading 1.6 models.
    
    Author: Yuhao Yang <hh...@gmail.com>
    
    Closes #13879 from hhbyyh/regreComp.
    
    (cherry picked from commit 14bc5a7f36bed19cd714a4c725a83feaccac3468)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 557eee5b6d07f8a17257cd9aae5d7830b4de4690
Author: Xiangrui Meng <me...@databricks.com>
Date:   2016-06-24T04:43:13Z

    [SPARK-16142][R] group naiveBayes method docs in a single Rd
    
    ## What changes were proposed in this pull request?
    
    This PR groups `spark.naiveBayes`, `summary(NB)`, `predict(NB)`, and `write.ml(NB)` into a single Rd.
    
    ## How was this patch tested?
    
    Manually checked generated HTML doc. See attached screenshots.
    
    ![screen shot 2016-06-23 at 2 11 00 pm](https://cloud.githubusercontent.com/assets/829644/16320452/a5885e92-394c-11e6-994f-2ab5cddad86f.png)
    
    ![screen shot 2016-06-23 at 2 11 15 pm](https://cloud.githubusercontent.com/assets/829644/16320455/aad1f6d8-394c-11e6-8ef4-13bee989f52f.png)
    
    Author: Xiangrui Meng <me...@databricks.com>
    
    Closes #13877 from mengxr/SPARK-16142.
    
    (cherry picked from commit 4a40d43bb29704734b8128bf2a3f27802ae34e17)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 3d8d956448fd3b7ae8d380e655bfa245b11c4ea0
Author: Yuhao Yang <yu...@intel.com>
Date:   2016-06-24T04:50:25Z

    [SPARK-16133][ML] model loading backward compatibility for ml.feature
    
    ## What changes were proposed in this pull request?
    
    model loading backward compatibility for ml.feature,
    
    ## How was this patch tested?
    
    existing ut and manual test for loading 1.6 models.
    
    Author: Yuhao Yang <yu...@intel.com>
    Author: Yuhao Yang <hh...@gmail.com>
    
    Closes #13844 from hhbyyh/featureComp.
    
    (cherry picked from commit cc6778ee0bf4fa7a78abd30542c4a6f80ea371c5)
    Signed-off-by: Xiangrui Meng <me...@databricks.com>

commit 3ccdd6b9c6e63c7498771dcd8673914c46f6794a
Author: Cheng Lian <li...@databricks.com>
Date:   2016-06-24T06:11:46Z

    [SPARK-13709][SQL] Initialize deserializer with both table and partition properties when reading partitioned tables
    
    ## What changes were proposed in this pull request?
    
    When reading partitions of a partitioned Hive SerDe table, we only initializes the deserializer using partition properties. However, for SerDes like `AvroSerDe`, essential properties (e.g. Avro schema information) may be defined in table properties. We should merge both table properties and partition properties before initializing the deserializer.
    
    Note that an individual partition may have different properties than the one defined in the table properties (e.g. partitions within a table can have different SerDes). Thus, for any property key defined in both partition and table properties, the value set in partition properties wins.
    
    ## How was this patch tested?
    
    New test case added in `QueryPartitionSuite`.
    
    Author: Cheng Lian <li...@databricks.com>
    
    Closes #13865 from liancheng/spark-13709-partitioned-avro-table.
    
    (cherry picked from commit 2d2f607bfae97f2681df24f48bb8b1b483c6b309)
    Signed-off-by: Yin Huai <yh...@databricks.com>

commit b6420db9ebc59c453a6a523aba68addf5762bb2c
Author: peng.zhang <pe...@xiaomi.com>
Date:   2016-06-24T07:28:32Z

    [SPARK-16125][YARN] Fix not test yarn cluster mode correctly in YarnClusterSuite
    
    ## What changes were proposed in this pull request?
    
    Since SPARK-13220(Deprecate "yarn-client" and "yarn-cluster"), YarnClusterSuite doesn't test "yarn cluster" mode correctly.
    This pull request fixes it.
    
    ## How was this patch tested?
    Unit test
    
    (If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
    
    Author: peng.zhang <pe...@xiaomi.com>
    
    Closes #13836 from renozhang/SPARK-16125-test-yarn-cluster-mode.
    
    (cherry picked from commit f4fd7432fb9cf7b197ccada1378c4f2a6d427522)
    Signed-off-by: Sean Owen <so...@cloudera.com>

commit 201d5e8db3fd29898a6cd69e015ca491e5721b08
Author: Sean Owen <so...@cloudera.com>
Date:   2016-06-24T09:35:54Z

    [SPARK-16129][CORE][SQL] Eliminate direct use of commons-lang classes in favor of commons-lang3
    
    ## What changes were proposed in this pull request?
    
    Replace use of `commons-lang` in favor of `commons-lang3` and forbid the former via scalastyle; remove `NotImplementedException` from `comons-lang` in favor of JDK `UnsupportedOperationException`
    
    ## How was this patch tested?
    
    Jenkins tests
    
    Author: Sean Owen <so...@cloudera.com>
    
    Closes #13843 from srowen/SPARK-16129.
    
    (cherry picked from commit 158af162eac7348464c6751c8acd48fc6c117688)
    Signed-off-by: Sean Owen <so...@cloudera.com>

commit 76741b570e20eb7957ada28ad3c5babc0abb738f
Author: GayathriMurali <ga...@intel.com>
Date:   2016-06-24T11:25:40Z

    [SPARK-15997][DOC][ML] Update user guide for HashingTF, QuantileVectorizer and CountVectorizer
    
    ## What changes were proposed in this pull request?
    
    Made changes to HashingTF,QuantileVectorizer and CountVectorizer
    
    Author: GayathriMurali <ga...@intel.com>
    
    Closes #13745 from GayathriMurali/SPARK-15997.
    
    (cherry picked from commit be88383e15a86d094963de5f7e8792510bc990de)
    Signed-off-by: Nick Pentreath <ni...@za.ibm.com>

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14305: Spark-16669:Adding partition prunning to Metastore stati...

Posted by Parth-Brahmbhatt <gi...@git.apache.org>.
Github user Parth-Brahmbhatt commented on the issue:

    https://github.com/apache/spark/pull/14305
  
    @jaceklaskowski created https://github.com/apache/spark/pull/14306. intended to open it against 2.0 branch.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14305: Spark-16669:Adding partition prunning to Metastore stati...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14305
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14305: Spark-16669:Adding partition prunning to Metastore stati...

Posted by jaceklaskowski <gi...@git.apache.org>.
Github user jaceklaskowski commented on the issue:

    https://github.com/apache/spark/pull/14305
  
    I think the PR is completely broken and should be recreated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14305: Spark-16669:Adding partition prunning to Metastor...

Posted by Parth-Brahmbhatt <gi...@git.apache.org>.
Github user Parth-Brahmbhatt closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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