Skip to content
Snippets Groups Projects
  1. May 11, 2017
  2. May 10, 2017
    • Josh Rosen's avatar
      [SPARK-20685] Fix BatchPythonEvaluation bug in case of single UDF w/ repeated arg. · 8ddbc431
      Josh Rosen authored
      ## What changes were proposed in this pull request?
      
      There's a latent corner-case bug in PySpark UDF evaluation where executing a `BatchPythonEvaluation` with a single multi-argument UDF where _at least one argument value is repeated_ will crash at execution with a confusing error.
      
      This problem was introduced in #12057: the code there has a fast path for handling a "batch UDF evaluation consisting of a single Python UDF", but that branch incorrectly assumes that a single UDF won't have repeated arguments and therefore skips the code for unpacking arguments from the input row (whose schema may not necessarily match the UDF inputs due to de-duplication of repeated arguments which occurred in the JVM before sending UDF inputs to Python).
      
      This fix here is simply to remove this special-casing: it turns out that the code in the "multiple UDFs" branch just so happens to work for the single-UDF case because Python treats `(x)` as equivalent to `x`, not as a single-argument tuple.
      
      ## How was this patch tested?
      
      New regression test in `pyspark.python.sql.tests` module (tested and confirmed that it fails before my fix).
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #17927 from JoshRosen/SPARK-20685.
      8ddbc431
    • Felix Cheung's avatar
      [SPARK-20689][PYSPARK] python doctest leaking bucketed table · af8b6cc8
      Felix Cheung authored
      ## What changes were proposed in this pull request?
      
      It turns out pyspark doctest is calling saveAsTable without ever dropping them. Since we have separate python tests for bucketed table, and there is no checking of results, there is really no need to run the doctest, other than leaving it as an example in the generated doc
      
      ## How was this patch tested?
      
      Jenkins
      
      Author: Felix Cheung <felixcheung_m@hotmail.com>
      
      Closes #17932 from felixcheung/pytablecleanup.
      af8b6cc8
    • Ala Luszczak's avatar
      [SPARK-19447] Remove remaining references to generated rows metric · 5c2c4dcc
      Ala Luszczak authored
      ## What changes were proposed in this pull request?
      
      https://github.com/apache/spark/commit/b486ffc86d8ad6c303321dcf8514afee723f61f8 left behind references to "number of generated rows" metrics, that should have been removed.
      
      ## How was this patch tested?
      
      Existing unit tests.
      
      Author: Ala Luszczak <ala@databricks.com>
      
      Closes #17939 from ala/SPARK-19447-fix.
      5c2c4dcc
    • Xianyang Liu's avatar
      [MINOR][BUILD] Fix lint-java breaks. · fcb88f92
      Xianyang Liu authored
      ## What changes were proposed in this pull request?
      
      This PR proposes to fix the lint-breaks as below:
      ```
      [ERROR] src/main/java/org/apache/spark/unsafe/Platform.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[45,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[62,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[78,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[92,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[102,25] (naming) MethodName: Method name 'Once' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.api.java.JavaDStream.
      ```
      
      after:
      ```
      dev/lint-java
      Checkstyle checks passed.
      ```
      [Test Result](https://travis-ci.org/ConeyLiu/spark/jobs/229666169)
      
      ## How was this patch tested?
      
      Travis CI
      
      Author: Xianyang Liu <xianyang.liu@intel.com>
      
      Closes #17890 from ConeyLiu/codestyle.
      fcb88f92
    • wangzhenhua's avatar
      [SPARK-20678][SQL] Ndv for columns not in filter condition should also be updated · 76e4a556
      wangzhenhua authored
      ## What changes were proposed in this pull request?
      
      In filter estimation, we update column stats for those columns in filter condition. However, if the number of rows decreases after the filter (i.e. the overall selectivity is less than 1), we need to update (scale down) the number of distinct values (NDV) for all columns, no matter they are in filter conditions or not.
      
      This pr also fixes the inconsistency of rounding mode for ndv and rowCount.
      
      ## How was this patch tested?
      
      Added new tests.
      
      Author: wangzhenhua <wangzhenhua@huawei.com>
      
      Closes #17918 from wzhfy/scaleDownNdvAfterFilter.
      76e4a556
    • Wenchen Fan's avatar
      [SPARK-20688][SQL] correctly check analysis for scalar sub-queries · 789bdbe3
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      In `CheckAnalysis`, we should call `checkAnalysis` for `ScalarSubquery` at the beginning, as later we will call `plan.output` which is invalid if `plan` is not resolved.
      
      ## How was this patch tested?
      
      new regression test
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #17930 from cloud-fan/tmp.
      789bdbe3
    • NICHOLAS T. MARION's avatar
      [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities · b512233a
      NICHOLAS T. MARION authored
      ## 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 <nmarion@us.ibm.com>
      
      Closes #17686 from n-marion/xss-fix.
      b512233a
    • Michael Mior's avatar
      [SPARK-20637][CORE] Remove mention of old RDD classes from comments · a4cbf26b
      Michael Mior authored
      ## What changes were proposed in this pull request?
      
      A few comments around the code mention RDD classes that do not exist anymore. I'm not sure of the best way to replace these, so I've just removed them here.
      
      ## How was this patch tested?
      
      Only changes code comments, no testing required
      
      Author: Michael Mior <mmior@uwaterloo.ca>
      
      Closes #17900 from michaelmior/remove-old-rdds.
      a4cbf26b
    • Alex Bozarth's avatar
      [SPARK-20630][WEB UI] Fixed column visibility in Executor Tab · ca4625e0
      Alex Bozarth authored
      ## What changes were proposed in this pull request?
      
      #14617 added new columns to the executor table causing the visibility checks for the logs and threadDump columns to toggle the wrong columns since they used hard-coded column numbers.
      
      I've updated the checks to use column names instead of numbers so future updates don't accidentally break this again.
      
      Note: This will also need to be back ported into 2.2 since #14617 was merged there
      
      ## How was this patch tested?
      
      Manually tested
      
      Author: Alex Bozarth <ajbozart@us.ibm.com>
      
      Closes #17904 from ajbozarth/spark20630.
      ca4625e0
    • zero323's avatar
      [SPARK-20631][PYTHON][ML] LogisticRegression._checkThresholdConsistency should... · 804949c6
      zero323 authored
      [SPARK-20631][PYTHON][ML] LogisticRegression._checkThresholdConsistency should use values not Params
      
      ## What changes were proposed in this pull request?
      
      - Replace `getParam` calls with `getOrDefault` calls.
      - Fix exception message to avoid unintended `TypeError`.
      - Add unit tests
      
      ## How was this patch tested?
      
      New unit tests.
      
      Author: zero323 <zero323@users.noreply.github.com>
      
      Closes #17891 from zero323/SPARK-20631.
      804949c6
    • Takuya UESHIN's avatar
      [SPARK-20668][SQL] Modify ScalaUDF to handle nullability. · 0ef16bd4
      Takuya UESHIN authored
      ## What changes were proposed in this pull request?
      
      When registering Scala UDF, we can know if the udf will return nullable value or not. `ScalaUDF` and related classes should handle the nullability.
      
      ## How was this patch tested?
      
      Existing tests.
      
      Author: Takuya UESHIN <ueshin@databricks.com>
      
      Closes #17911 from ueshin/issues/SPARK-20668.
      0ef16bd4
    • Yuhao Yang's avatar
      [SPARK-20670][ML] Simplify FPGrowth transform · a819dab6
      Yuhao Yang authored
      ## What changes were proposed in this pull request?
      
      jira: https://issues.apache.org/jira/browse/SPARK-20670
      As suggested by Sean Owen in https://github.com/apache/spark/pull/17130, the transform code in FPGrowthModel can be simplified.
      
      As I tested on some public dataset http://fimi.ua.ac.be/data/, the performance of the new transform code is even or better than the old implementation.
      
      ## How was this patch tested?
      
      Existing unit test.
      
      Author: Yuhao Yang <yuhao.yang@intel.com>
      
      Closes #17912 from hhbyyh/fpgrowthTransform.
      a819dab6
    • Josh Rosen's avatar
      [SPARK-20686][SQL] PropagateEmptyRelation incorrectly handles aggregate without grouping · a90c5cd8
      Josh Rosen authored
      ## What changes were proposed in this pull request?
      
      The query
      
      ```
      SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1
      ```
      
      should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows.
      
      This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead:
      
      An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows.
      
      If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation.
      
      The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be).
      
      This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation.
      
      ## How was this patch tested?
      
      - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file.
      - Updated unit tests in `PropagateEmptyRelationSuite`.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #17929 from JoshRosen/fix-PropagateEmptyRelation.
      a90c5cd8
    • hyukjinkwon's avatar
      [SPARK-20590][SQL] Use Spark internal datasource if multiples are found for the same shorten name · 3d2131ab
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      One of the common usability problems around reading data in spark (particularly CSV) is that there can often be a conflict between different readers in the classpath.
      
      As an example, if someone launches a 2.x spark shell with the spark-csv package in the classpath, Spark currently fails in an extremely unfriendly way (see databricks/spark-csv#367):
      
      ```bash
      ./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0
      scala> val df = spark.read.csv("/foo/bar.csv")
      java.lang.RuntimeException: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), please specify the fully qualified class name.
        at scala.sys.package$.error(package.scala:27)
        at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:574)
        at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:85)
        at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:85)
        at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:295)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
        at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:533)
        at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:412)
        ... 48 elided
      ```
      
      This PR proposes a simple way of fixing this error by picking up the internal datasource if there is single (the datasource that has "org.apache.spark" prefix).
      
      ```scala
      scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc")
      17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
      com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
      ```
      
      ```scala
      scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc")
      17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
      com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
      ```
      
      ## How was this patch tested?
      
      Manually tested as below:
      
      ```bash
      ./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0
      ```
      
      ```scala
      spark.sparkContext.setLogLevel("WARN")
      ```
      
      **positive cases**:
      
      ```scala
      scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc")
      17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
      com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
      ```
      
      ```scala
      scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc")
      17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
      com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
      ```
      
      (newlines were inserted for readability).
      
      ```scala
      scala> spark.range(1).write.format("com.databricks.spark.csv").mode("overwrite").save("/tmp/abc")
      ```
      
      ```scala
      scala> spark.range(1).write.format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat").mode("overwrite").save("/tmp/abc")
      ```
      
      **negative cases**:
      
      ```scala
      scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelation").save("/tmp/abc")
      java.lang.InstantiationException: com.databricks.spark.csv.CsvRelation
      ...
      ```
      
      ```scala
      scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelatio").save("/tmp/abc")
      java.lang.ClassNotFoundException: Failed to find data source: com.databricks.spark.csv.CsvRelatio. Please find packages at http://spark.apache.org/third-party-projects.html
      ...
      ```
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #17916 from HyukjinKwon/datasource-detect.
      3d2131ab
  3. May 09, 2017
    • Yuming Wang's avatar
      [SPARK-17685][SQL] Make SortMergeJoinExec's currentVars is null when calling createJoinKey · 771abeb4
      Yuming Wang authored
      ## What changes were proposed in this pull request?
      
      The following SQL query cause `IndexOutOfBoundsException` issue when `LIMIT > 1310720`:
      ```sql
      CREATE TABLE tab1(int int, int2 int, str string);
      CREATE TABLE tab2(int int, int2 int, str string);
      INSERT INTO tab1 values(1,1,'str');
      INSERT INTO tab1 values(2,2,'str');
      INSERT INTO tab2 values(1,1,'str');
      INSERT INTO tab2 values(2,3,'str');
      
      SELECT
        count(*)
      FROM
        (
          SELECT t1.int, t2.int2
          FROM (SELECT * FROM tab1 LIMIT 1310721) t1
          INNER JOIN (SELECT * FROM tab2 LIMIT 1310721) t2
          ON (t1.int = t2.int AND t1.int2 = t2.int2)
        ) t;
      ```
      
      This pull request fix this issue.
      
      ## How was this patch tested?
      
      unit tests
      
      Author: Yuming Wang <wgyumg@gmail.com>
      
      Closes #17920 from wangyum/SPARK-17685.
      771abeb4
    • uncleGen's avatar
      [SPARK-20373][SQL][SS] Batch queries with 'Dataset/DataFrame.withWatermark()` does not execute · c0189abc
      uncleGen authored
      ## What changes were proposed in this pull request?
      
      Any Dataset/DataFrame batch query with the operation `withWatermark` does not execute because the batch planner does not have any rule to explicitly handle the EventTimeWatermark logical plan.
      The right solution is to simply remove the plan node, as the watermark should not affect any batch query in any way.
      
      Changes:
      - In this PR, we add a new rule `EliminateEventTimeWatermark` to check if we need to ignore the event time watermark. We will ignore watermark in any batch query.
      
      Depends upon:
      - [SPARK-20672](https://issues.apache.org/jira/browse/SPARK-20672). We can not add this rule into analyzer directly, because streaming query will be copied to `triggerLogicalPlan ` in every trigger, and the rule will be applied to `triggerLogicalPlan` mistakenly.
      
      Others:
      - A typo fix in example.
      
      ## How was this patch tested?
      
      add new unit test.
      
      Author: uncleGen <hustyugm@gmail.com>
      
      Closes #17896 from uncleGen/SPARK-20373.
      c0189abc
    • Yin Huai's avatar
      Revert "[SPARK-20311][SQL] Support aliases for table value functions" · f79aa285
      Yin Huai authored
      This reverts commit 714811d0.
      f79aa285
    • Reynold Xin's avatar
      Revert "[SPARK-12297][SQL] Hive compatibility for Parquet Timestamps" · ac1ab6b9
      Reynold Xin authored
      This reverts commit 22691556.
      
      See JIRA ticket for more information.
      ac1ab6b9
    • Holden Karau's avatar
      [SPARK-20627][PYSPARK] Drop the hadoop distirbution name from the Python version · 1b85bcd9
      Holden Karau authored
      ## What changes were proposed in this pull request?
      
      Drop the hadoop distirbution name from the Python version (PEP440 - https://www.python.org/dev/peps/pep-0440/). We've been using the local version string to disambiguate between different hadoop versions packaged with PySpark, but PEP0440 states that local versions should not be used when publishing up-stream. Since we no longer make PySpark pip packages for different hadoop versions, we can simply drop the hadoop information. If at a later point we need to start publishing different hadoop versions we can look at make different packages or similar.
      
      ## How was this patch tested?
      
      Ran `make-distribution` locally
      
      Author: Holden Karau <holden@us.ibm.com>
      
      Closes #17885 from holdenk/SPARK-20627-remove-pip-local-version-string.
      1b85bcd9
    • Sean Owen's avatar
      [SPARK-19876][BUILD] Move Trigger.java to java source hierarchy · 25ee816e
      Sean Owen authored
      ## What changes were proposed in this pull request?
      
      Simply moves `Trigger.java` to `src/main/java` from `src/main/scala`
      See https://github.com/apache/spark/pull/17219
      
      ## How was this patch tested?
      
      Existing tests.
      
      Author: Sean Owen <sowen@cloudera.com>
      
      Closes #17921 from srowen/SPARK-19876.2.
      25ee816e
    • Reynold Xin's avatar
      [SPARK-20674][SQL] Support registering UserDefinedFunction as named UDF · d099f414
      Reynold Xin authored
      ## What changes were proposed in this pull request?
      For some reason we don't have an API to register UserDefinedFunction as named UDF. It is a no brainer to add one, in addition to the existing register functions we have.
      
      ## How was this patch tested?
      Added a test case in UDFSuite for the new API.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #17915 from rxin/SPARK-20674.
      d099f414
    • Wenchen Fan's avatar
      [SPARK-20548][FLAKY-TEST] share one REPL instance among REPL test cases · f561a76b
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      `ReplSuite.newProductSeqEncoder with REPL defined class` was flaky and throws OOM exception frequently. By analyzing the heap dump, we found the reason is that, in each test case of `ReplSuite`, we create a REPL instance, which creates a classloader and loads a lot of classes related to `SparkContext`. More details please see https://github.com/apache/spark/pull/17833#issuecomment-298711435.
      
      In this PR, we create a new test suite, `SingletonReplSuite`, which shares one REPL instances among all the test cases. Then we move most of the tests from `ReplSuite` to `SingletonReplSuite`, to avoid creating a lot of REPL instances and reduce memory footprint.
      
      ## How was this patch tested?
      
      test only change
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #17844 from cloud-fan/flaky-test.
      f561a76b
    • Sanket's avatar
      [SPARK-20355] Add per application spark version on the history server headerpage · 181261a8
      Sanket authored
      ## What changes were proposed in this pull request?
      
      Spark Version for a specific application is not displayed on the history page now. It should be nice to switch the spark version on the UI when we click on the specific application.
      Currently there seems to be way as SparkListenerLogStart records the application version. So, it should be trivial to listen to this event and provision this change on the UI.
      For Example
      <img width="1439" alt="screen shot 2017-04-06 at 3 23 41 pm" src="https://cloud.githubusercontent.com/assets/8295799/25092650/41f3970a-2354-11e7-9b0d-4646d0adeb61.png">
      <img width="1399" alt="screen shot 2017-04-17 at 9 59 33 am" src="https://cloud.githubusercontent.com/assets/8295799/25092743/9f9e2f28-2354-11e7-9605-f2f1c63f21fe.png">
      
      {"Event":"SparkListenerLogStart","Spark Version":"2.0.0"}
      (Please fill in changes proposed in this fix)
      Modified the SparkUI for History server to listen to SparkLogListenerStart event and extract the version and print it.
      
      ## How was this patch tested?
      Manual testing of UI page. Attaching the UI screenshot changes here
      
      (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
      (If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
      
      Please review http://spark.apache.org/contributing.html before opening a pull request.
      
      Author: Sanket <schintap@untilservice-lm>
      
      Closes #17658 from redsanket/SPARK-20355.
      181261a8
    • Takeshi Yamamuro's avatar
      [SPARK-20311][SQL] Support aliases for table value functions · 714811d0
      Takeshi Yamamuro authored
      ## What changes were proposed in this pull request?
      This pr added parsing rules to support aliases in table value functions.
      
      ## How was this patch tested?
      Added tests in `PlanParserSuite`.
      
      Author: Takeshi Yamamuro <yamamuro@apache.org>
      
      Closes #17666 from maropu/SPARK-20311.
      714811d0
    • Xiao Li's avatar
      [SPARK-20667][SQL][TESTS] Cleanup the cataloged metadata after completing the... · 0d00c768
      Xiao Li authored
      [SPARK-20667][SQL][TESTS] Cleanup the cataloged metadata after completing the package of sql/core and sql/hive
      
      ## What changes were proposed in this pull request?
      
      So far, we do not drop all the cataloged objects after each package. Sometimes, we might hit strange test case errors because the previous test suite did not drop the cataloged/temporary objects (tables/functions/database). At least, we can first clean up the environment when completing the package of `sql/core` and `sql/hive`.
      
      ## How was this patch tested?
      N/A
      
      Author: Xiao Li <gatorsmile@gmail.com>
      
      Closes #17908 from gatorsmile/reset.
      0d00c768
    • Yanbo Liang's avatar
      [SPARK-20606][ML] ML 2.2 QA: Remove deprecated methods for ML · b8733e0a
      Yanbo Liang authored
      ## What changes were proposed in this pull request?
      Remove ML methods we deprecated in 2.1.
      
      ## How was this patch tested?
      Existing tests.
      
      Author: Yanbo Liang <ybliang8@gmail.com>
      
      Closes #17867 from yanboliang/spark-20606.
      b8733e0a
    • Jon McLean's avatar
      [SPARK-20615][ML][TEST] SparseVector.argmax throws IndexOutOfBoundsException · be53a783
      Jon McLean authored
      ## What changes were proposed in this pull request?
      
      Added a check for for the number of defined values.  Previously the argmax function assumed that at least one value was defined if the vector size was greater than zero.
      
      ## How was this patch tested?
      
      Tests were added to the existing VectorsSuite to cover this case.
      
      Author: Jon McLean <jon.mclean@atsid.com>
      
      Closes #17877 from jonmclean/vectorArgmaxIndexBug.
      be53a783
    • Nick Pentreath's avatar
      [SPARK-20587][ML] Improve performance of ML ALS recommendForAll · 10b00aba
      Nick Pentreath authored
      This PR is a `DataFrame` version of #17742 for [SPARK-11968](https://issues.apache.org/jira/browse/SPARK-11968), for improving the performance of `recommendAll` methods.
      
      ## How was this patch tested?
      
      Existing unit tests.
      
      Author: Nick Pentreath <nickp@za.ibm.com>
      
      Closes #17845 from MLnick/ml-als-perf.
      10b00aba
    • Peng's avatar
      [SPARK-11968][MLLIB] Optimize MLLIB ALS recommendForAll · 80794247
      Peng authored
      The recommendForAll of MLLIB ALS is very slow.
      GC is a key problem of the current method.
      The task use the following code to keep temp result:
      val output = new Array[(Int, (Int, Double))](m*n)
      m = n = 4096 (default value, no method to set)
      so output is about 4k * 4k * (4 + 4 + 8) = 256M. This is a large memory and cause serious GC problem, and it is frequently OOM.
      
      Actually, we don't need to save all the temp result. Support we recommend topK (topK is about 10, or 20) product for each user, we only need 4k * topK * (4 + 4 + 8) memory to save the temp result.
      
      The Test Environment:
      3 workers: each work 10 core, each work 30G memory, each work 1 executor.
      The Data: User 480,000, and Item 17,000
      
      BlockSize:     1024  2048  4096  8192
      Old method:  245s  332s  488s  OOM
      This solution: 121s  118s   117s  120s
      
      The existing UT.
      
      Author: Peng <peng.meng@intel.com>
      Author: Peng Meng <peng.meng@intel.com>
      
      Closes #17742 from mpjlu/OptimizeAls.
      80794247
    • Felix Cheung's avatar
      [SPARK-20661][SPARKR][TEST][FOLLOWUP] SparkR tableNames() test fails · b952b44a
      Felix Cheung authored
      ## What changes were proposed in this pull request?
      
      Change it to check for relative count like in this test https://github.com/apache/spark/blame/master/R/pkg/inst/tests/testthat/test_sparkSQL.R#L3355 for catalog APIs
      
      ## How was this patch tested?
      
      unit tests, this needs to combine with another commit with SQL change to check
      
      Author: Felix Cheung <felixcheung_m@hotmail.com>
      
      Closes #17905 from felixcheung/rtabletests.
      b952b44a
  4. May 08, 2017
Loading