Skip to content
Snippets Groups Projects
  1. Jan 24, 2017
    • Ilya Matiach's avatar
      [SPARK-18036][ML][MLLIB] Fixing decision trees handling edge cases · d9783380
      Ilya Matiach authored
      ## What changes were proposed in this pull request?
      
      Decision trees/GBT/RF do not handle edge cases such as constant features or empty features.
      In the case of constant features we choose any arbitrary split instead of failing with a cryptic error message.
      In the case of empty features we fail with a better error message stating:
      DecisionTree requires number of features > 0, but was given an empty features vector
      Instead of the cryptic error message:
      java.lang.UnsupportedOperationException: empty.max
      
      ## How was this patch tested?
      
      Unit tests are added in the patch for:
      DecisionTreeRegressor
      GBTRegressor
      Random Forest Regressor
      
      Author: Ilya Matiach <ilmat@microsoft.com>
      
      Closes #16377 from imatiach-msft/ilmat/fix-decision-tree.
      d9783380
    • Wenchen Fan's avatar
      [SPARK-17913][SQL] compare atomic and string type column may return confusing result · 59c184e0
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      Spark SQL follows MySQL to do the implicit type conversion for binary comparison: http://dev.mysql.com/doc/refman/5.7/en/type-conversion.html
      
      However, this may return confusing result, e.g. `1 = 'true'` will return true, `19157170390056973L = '19157170390056971'` will return true.
      
      I think it's more reasonable to follow postgres in this case, i.e. cast string to the type of the other side, but return null if the string is not castable to keep hive compatibility.
      
      ## How was this patch tested?
      
      newly added tests.
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #15880 from cloud-fan/compare.
      59c184e0
    • Parag Chaudhari's avatar
      [SPARK-14049][CORE] Add functionality in spark history sever API to query applications by end time · 0ff67a1c
      Parag Chaudhari authored
      ## What changes were proposed in this pull request?
      
      Currently, spark history server REST API provides functionality to query applications by application start time range based on minDate and maxDate query parameters, but it  lacks support to query applications by their end time. In this pull request we are proposing optional minEndDate and maxEndDate query parameters and filtering capability based on these parameters to spark history server REST API. This functionality can be used for following queries,
      1. Applications finished in last 'x' minutes
      2. Applications finished before 'y' time
      3. Applications finished between 'x' time to 'y' time
      4. Applications started from 'x' time and finished before 'y' time.
      
      For backward compatibility, we can keep existing minDate and maxDate query parameters as they are and they can continue support filtering based on start time range.
      ## How was this patch tested?
      
      Existing unit tests and 4 new unit tests.
      
      Author: Parag Chaudhari <paragpc@amazon.com>
      
      Closes #11867 from paragpc/master-SHS-query-by-endtime_2.
      0ff67a1c
    • windpiger's avatar
      [SPARK-19246][SQL] CataLogTable's partitionSchema order and exist check · 752502be
      windpiger authored
      ## What changes were proposed in this pull request?
      
      CataLogTable's partitionSchema should check if each column name in partitionColumnNames must match one and only one field in schema, if not we should throw an exception
      
      and CataLogTable's partitionSchema should keep order with partitionColumnNames
      
      ## How was this patch tested?
      N/A
      
      Author: windpiger <songjun@outlook.com>
      
      Closes #16606 from windpiger/checkPartionColNameWithSchema.
      752502be
    • windpiger's avatar
      [SPARK-19152][SQL] DataFrameWriter.saveAsTable support hive append · 3c86fddd
      windpiger authored
      ## What changes were proposed in this pull request?
      
      After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.
      
      This PR implement:
      DataFrameWriter.saveAsTable work with hive format with append mode
      
      ## How was this patch tested?
      unit test added
      
      Author: windpiger <songjun@outlook.com>
      
      Closes #16552 from windpiger/saveAsTableWithHiveAppend.
      3c86fddd
    • Souljoy Zhuo's avatar
      delete useless var “j” · cca86800
      Souljoy Zhuo authored
      the var “j” defined in "var j = 0" is useless for “def compress”
      
      Author: Souljoy Zhuo <zhuoshoujie@126.com>
      
      Closes #16676 from xiaoyesoso/patch-1.
      Unverified
      cca86800
    • uncleGen's avatar
      [DOCS] Fix typo in docs · 7c61c2a1
      uncleGen authored
      ## What changes were proposed in this pull request?
      
      Fix typo in docs
      
      ## How was this patch tested?
      
      Author: uncleGen <hustyugm@gmail.com>
      
      Closes #16658 from uncleGen/typo-issue.
      Unverified
      7c61c2a1
    • Felix Cheung's avatar
      [SPARK-18823][SPARKR] add support for assigning to column · f27e0247
      Felix Cheung authored
      ## What changes were proposed in this pull request?
      
      Support for
      ```
      df[[myname]] <- 1
      df[[2]] <- df$eruptions
      ```
      
      ## How was this patch tested?
      
      manual tests, unit tests
      
      Author: Felix Cheung <felixcheung_m@hotmail.com>
      
      Closes #16663 from felixcheung/rcolset.
      f27e0247
    • hyukjinkwon's avatar
      [SPARK-16101][HOTFIX] Fix the build with Scala 2.10 by explicit typed argument · ec9493b4
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      I goofed in https://github.com/apache/spark/pull/16669 which introduces the break in scala 2.10.
      
      This fixes
      
      ```bash
      [error] /home/jenkins/workspace/spark-master-compile-sbt-scala-2.10/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala:65: polymorphic expression cannot be instantiated to expected type;
      [error]  found   : [B >: org.apache.spark.sql.types.StructField](B, Int) => Int
      [error]  required: org.apache.spark.sql.types.StructField => ?
      [error]     fields.map(schema.indexOf).toArray
      [error]                       ^
      [error] one error found
      [error] (sql/compile:compileIncremental) Compilation failed
      ```
      
      ## How was this patch tested?
      
      Manually via
      
      ```bash
      ./dev/change-scala-version.sh 2.10
      ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
      ```
      ```
      [INFO] ------------------------------------------------------------------------
      [INFO] Reactor Summary:
      [INFO]
      [INFO] Spark Project Parent POM ........................... SUCCESS [  2.719 s]
      [INFO] Spark Project Tags ................................. SUCCESS [  3.441 s]
      [INFO] Spark Project Sketch ............................... SUCCESS [  3.411 s]
      [INFO] Spark Project Networking ........................... SUCCESS [  5.088 s]
      [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [  5.131 s]
      [INFO] Spark Project Unsafe ............................... SUCCESS [  5.813 s]
      [INFO] Spark Project Launcher ............................. SUCCESS [  6.567 s]
      [INFO] Spark Project Core ................................. SUCCESS [01:39 min]
      [INFO] Spark Project ML Local Library ..................... SUCCESS [  6.644 s]
      [INFO] Spark Project GraphX ............................... SUCCESS [ 11.304 s]
      [INFO] Spark Project Streaming ............................ SUCCESS [ 26.275 s]
      [INFO] Spark Project Catalyst ............................. SUCCESS [01:04 min]
      [INFO] Spark Project SQL .................................. SUCCESS [02:07 min]
      [INFO] Spark Project ML Library ........................... SUCCESS [01:20 min]
      [INFO] Spark Project Tools ................................ SUCCESS [  8.755 s]
      [INFO] Spark Project Hive ................................. SUCCESS [ 51.141 s]
      [INFO] Spark Project REPL ................................. SUCCESS [ 13.688 s]
      [INFO] Spark Project YARN Shuffle Service ................. SUCCESS [  7.211 s]
      [INFO] Spark Project YARN ................................. SUCCESS [ 10.908 s]
      [INFO] Spark Project Assembly ............................. SUCCESS [  2.940 s]
      [INFO] Spark Project External Flume Sink .................. SUCCESS [  4.386 s]
      [INFO] Spark Project External Flume ....................... SUCCESS [  8.589 s]
      [INFO] Spark Project External Flume Assembly .............. SUCCESS [  1.891 s]
      [INFO] Spark Integration for Kafka 0.8 .................... SUCCESS [  8.458 s]
      [INFO] Spark Project Examples ............................. SUCCESS [ 17.706 s]
      [INFO] Spark Project External Kafka Assembly .............. SUCCESS [  3.070 s]
      [INFO] Spark Integration for Kafka 0.10 ................... SUCCESS [ 11.227 s]
      [INFO] Spark Integration for Kafka 0.10 Assembly .......... SUCCESS [  2.982 s]
      [INFO] Kafka 0.10 Source for Structured Streaming ......... SUCCESS [  7.494 s]
      [INFO] Spark Project Java 8 Tests ......................... SUCCESS [  3.748 s]
      [INFO] ------------------------------------------------------------------------
      [INFO] BUILD SUCCESS
      [INFO] ------------------------------------------------------------------------
      ```
      
      and manual test `CSVSuite` with Scala 2.11 with my IDE.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #16684 from HyukjinKwon/hot-fix-type-ensurance.
      ec9493b4
    • Shixiong Zhu's avatar
      [SPARK-19268][SS] Disallow adaptive query execution for streaming queries · 60bd91a3
      Shixiong Zhu authored
      ## What changes were proposed in this pull request?
      
      As adaptive query execution may change the number of partitions in different batches, it may break streaming queries. Hence, we should disallow this feature in Structured Streaming.
      
      ## How was this patch tested?
      
      `test("SPARK-19268: Adaptive query execution should be disallowed")`.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #16683 from zsxwing/SPARK-19268.
      60bd91a3
    • hyukjinkwon's avatar
      [SPARK-9435][SQL] Reuse function in Java UDF to correctly support expressions... · e576c1ed
      hyukjinkwon authored
      [SPARK-9435][SQL] Reuse function in Java UDF to correctly support expressions that require equality comparison between ScalaUDF
      
      ## What changes were proposed in this pull request?
      
      Currently, running the codes in Java
      
      ```java
      spark.udf().register("inc", new UDF1<Long, Long>() {
        Override
        public Long call(Long i) {
          return i + 1;
        }
      }, DataTypes.LongType);
      
      spark.range(10).toDF("x").createOrReplaceTempView("tmp");
      Row result = spark.sql("SELECT inc(x) FROM tmp GROUP BY inc(x)").head();
      Assert.assertEquals(7, result.getLong(0));
      ```
      
      fails as below:
      
      ```
      org.apache.spark.sql.AnalysisException: expression 'tmp.`x`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
      Aggregate [UDF(x#19L)], [UDF(x#19L) AS UDF(x)#23L]
      +- SubqueryAlias tmp, `tmp`
         +- Project [id#16L AS x#19L]
            +- Range (0, 10, step=1, splits=Some(8))
      
      	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40)
      	at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:57)
      ```
      
      The root cause is because we were creating the function every time when it needs to build as below:
      
      ```scala
      scala> def inc(i: Int) = i + 1
      inc: (i: Int)Int
      
      scala> (inc(_: Int)).hashCode
      res15: Int = 1231799381
      
      scala> (inc(_: Int)).hashCode
      res16: Int = 2109839984
      
      scala> (inc(_: Int)) == (inc(_: Int))
      res17: Boolean = false
      ```
      
      This seems leading to the comparison failure between `ScalaUDF`s created from Java UDF API, for example, in `Expression.semanticEquals`.
      
      In case of Scala one, it seems already fine.
      
      Both can be tested easily as below if any reviewer is more comfortable with Scala:
      
      ```scala
      val df = Seq((1, 10), (2, 11), (3, 12)).toDF("x", "y")
      val javaUDF = new UDF1[Int, Int]  {
        override def call(i: Int): Int = i + 1
      }
      // spark.udf.register("inc", javaUDF, IntegerType) // Uncomment this for Java API
      // spark.udf.register("inc", (i: Int) => i + 1)    // Uncomment this for Scala API
      df.createOrReplaceTempView("tmp")
      spark.sql("SELECT inc(y) FROM tmp GROUP BY inc(y)").show()
      ```
      
      ## How was this patch tested?
      
      Unit test in `JavaUDFSuite.java` and `./dev/lint-java`.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #16553 from HyukjinKwon/SPARK-9435.
      e576c1ed
  2. Jan 23, 2017
    • jiangxingbo's avatar
      [SPARK-19272][SQL] Remove the param `viewOriginalText` from `CatalogTable` · 3bdf3ee8
      jiangxingbo authored
      ## What changes were proposed in this pull request?
      
      Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable.
      
      This PR brought in the following changes:
      1. Remove the param `viewOriginalText` from `CatalogTable`;
      2. Update the output of command `DescribeTableCommand`.
      
      ## How was this patch tested?
      
      Tested by exsiting test cases, also updated the failed test cases.
      
      Author: jiangxingbo <jiangxb1987@gmail.com>
      
      Closes #16679 from jiangxb1987/catalogTable.
      3bdf3ee8
    • Wenchen Fan's avatar
      [SPARK-19290][SQL] add a new extending interface in Analyzer for post-hoc resolution · fcfd5d0b
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      To implement DDL commands, we added several analyzer rules in sql/hive module to analyze DDL related plans. However, our `Analyzer` currently only have one extending interface: `extendedResolutionRules`, which defines extra rules that will be run together with other rules in the resolution batch, and doesn't fit DDL rules well, because:
      
      1. DDL rules may do some checking and normalization, but we may do it many times as the resolution batch will run rules again and again, until fixed point, and it's hard to tell if a DDL rule has already done its checking and normalization. It's fine because DDL rules are idempotent, but it's bad for analysis performance
      2. some DDL rules may depend on others, and it's pretty hard to write `if` conditions to guarantee the dependencies. It will be good if we have a batch which run rules in one pass, so that we can guarantee the dependencies by rules order.
      
      This PR adds a new extending interface in `Analyzer`: `postHocResolutionRules`, which defines rules that will be run only once in a batch runs right after the resolution batch.
      
      ## How was this patch tested?
      
      existing tests
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #16645 from cloud-fan/analyzer.
      fcfd5d0b
    • Zheng RuiFeng's avatar
      [SPARK-17747][ML] WeightCol support non-double numeric datatypes · 49f5b0ae
      Zheng RuiFeng authored
      ## What changes were proposed in this pull request?
      
      1, add test for `WeightCol` in `MLTestingUtils.checkNumericTypes`
      2, move datatype cast to `Predict.fit`, and supply algos' `train()` with casted dataframe
      ## How was this patch tested?
      
      local tests in spark-shell and unit tests
      
      Author: Zheng RuiFeng <ruifengz@foxmail.com>
      
      Closes #15314 from zhengruifeng/weightCol_support_int.
      49f5b0ae
    • jerryshao's avatar
      [SPARK-19306][CORE] Fix inconsistent state in DiskBlockObject when expection occurred · e4974721
      jerryshao authored
      ## What changes were proposed in this pull request?
      
      In `DiskBlockObjectWriter`, when some errors happened during writing, it will call `revertPartialWritesAndClose`, if this method again failed due to some issues like out of disk, it will throw exception without resetting the state of this writer, also skipping the revert. So here propose to fix this issue to offer user a chance to recover from such issue.
      
      ## How was this patch tested?
      
      Existing test.
      
      Author: jerryshao <sshao@hortonworks.com>
      
      Closes #16657 from jerryshao/SPARK-19306.
      e4974721
    • Ilya Matiach's avatar
      [SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case · 5b258b8b
      Ilya Matiach authored
      [SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case where no children exist in updateAssignments
      
      ## What changes were proposed in this pull request?
      
      Fix a bug in which BisectingKMeans fails with error:
      java.util.NoSuchElementException: key not found: 166
              at scala.collection.MapLike$class.default(MapLike.scala:228)
              at scala.collection.AbstractMap.default(Map.scala:58)
              at scala.collection.MapLike$class.apply(MapLike.scala:141)
              at scala.collection.AbstractMap.apply(Map.scala:58)
              at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply$mcDJ$sp(BisectingKMeans.scala:338)
              at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
              at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
              at scala.collection.TraversableOnce$$anonfun$minBy$1.apply(TraversableOnce.scala:231)
              at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:111)
              at scala.collection.immutable.List.foldLeft(List.scala:84)
              at scala.collection.LinearSeqOptimized$class.reduceLeft(LinearSeqOptimized.scala:125)
              at scala.collection.immutable.List.reduceLeft(List.scala:84)
              at scala.collection.TraversableOnce$class.minBy(TraversableOnce.scala:231)
              at scala.collection.AbstractTraversable.minBy(Traversable.scala:105)
              at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:337)
              at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:334)
              at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
              at scala.collection.Iterator$$anon$14.hasNext(Iterator.scala:389)
      
      ## How was this patch tested?
      
      The dataset was run against the code change to verify that the code works.  I will try to add unit tests to the code.
      
      (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: Ilya Matiach <ilmat@microsoft.com>
      
      Closes #16355 from imatiach-msft/ilmat/fix-kmeans.
      5b258b8b
    • z001qdp's avatar
      [SPARK-17455][MLLIB] Improve PAVA implementation in IsotonicRegression · c8aea744
      z001qdp authored
      ## What changes were proposed in this pull request?
      
      New implementation of the Pool Adjacent Violators Algorithm (PAVA) in mllib.IsotonicRegression, which used under the hood by ml.regression.IsotonicRegression. The previous implementation could have factorial complexity in the worst case. This implementation, which closely follows those in scikit-learn and the R `iso` package, runs in quadratic time in the worst case.
      ## How was this patch tested?
      
      Existing unit tests in both `mllib` and `ml` passed before and after this patch. Scaling properties were tested by running the `poolAdjacentViolators` method in [scala-benchmarking-template](https://github.com/sirthias/scala-benchmarking-template) with the input generated by
      
      ``` scala
      val x = (1 to length).toArray.map(_.toDouble)
      val y = x.reverse.zipWithIndex.map{ case (yi, i) => if (i % 2 == 1) yi - 1.5 else yi}
      val w = Array.fill(length)(1d)
      
      val input: Array[(Double, Double, Double)] = (y zip x zip w) map{ case ((y, x), w) => (y, x, w)}
      ```
      
      Before this patch:
      
      | Input Length | Time (us) |
      | --: | --: |
      | 100 | 1.35 |
      | 200 | 3.14 |
      | 400 | 116.10 |
      | 800 | 2134225.90 |
      
      After this patch:
      
      | Input Length | Time (us) |
      | --: | --: |
      | 100 | 1.25 |
      | 200 | 2.53 |
      | 400 | 5.86 |
      | 800 | 10.55 |
      
      Benchmarking was also performed with randomly-generated y values, with similar results.
      
      Author: z001qdp <Nicholas.Eggert@target.com>
      
      Closes #15018 from neggert/SPARK-17455-isoreg-algo.
      c8aea744
    • Yuhao's avatar
      [SPARK-14709][ML] spark.ml API for linear SVM · 4a11d029
      Yuhao authored
      ## What changes were proposed in this pull request?
      
      jira: https://issues.apache.org/jira/browse/SPARK-14709
      
      Provide API for SVM algorithm for DataFrames. As discussed in jira, the initial implementation uses OWL-QN with Hinge loss function.
      The API should mimic existing spark.ml.classification APIs.
      Currently only Binary Classification is supported. Multinomial support can be added in this or following release.
      ## How was this patch tested?
      
      new unit tests and simple manual test
      
      Author: Yuhao <yuhao.yang@intel.com>
      Author: Yuhao Yang <hhbyyh@gmail.com>
      
      Closes #15211 from hhbyyh/mlsvm.
      4a11d029
    • windpiger's avatar
      [SPARK-19284][SQL] append to partitioned datasource table should without custom partition location · 0ef1421a
      windpiger authored
      ## What changes were proposed in this pull request?
      
      when we append data to a existed partitioned datasource table, the InsertIntoHadoopFsRelationCommand.getCustomPartitionLocations currently
      return the same location with Hive default, it should return None.
      
      ## How was this patch tested?
      
      Author: windpiger <songjun@outlook.com>
      
      Closes #16642 from windpiger/appendSchema.
      0ef1421a
    • Yuming Wang's avatar
      [SPARK-19146][CORE] Drop more elements when stageData.taskData.size > retainedTasks · c9949214
      Yuming Wang authored
      ## What changes were proposed in this pull request?
      
      Drop more elements when `stageData.taskData.size > retainedTasks` to reduce the number of times on call drop function.
      
      ## How was this patch tested?
      
      Jenkins
      
      Author: Yuming Wang <wgyumg@gmail.com>
      
      Closes #16527 from wangyum/SPARK-19146.
      Unverified
      c9949214
    • Dongjoon Hyun's avatar
      [SPARK-19218][SQL] Fix SET command to show a result correctly and in a sorted order · c4a6519c
      Dongjoon Hyun authored
      ## What changes were proposed in this pull request?
      
      This PR aims to fix the following two things.
      
      1. `sql("SET -v").collect()` or `sql("SET -v").show()` raises the following exceptions for String configuration with default value, `null`. For the test, please see [Jenkins result](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71539/testReport/) and https://github.com/apache/spark/commit/60953bf1f1ba144e709fdae3903a390ff9479fd0 in #16624 .
      
      ```
      sbt.ForkMain$ForkError: java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
      createexternalrow(input[0, string, false].toString, input[1, string, false].toString, input[2, string, false].toString, StructField(key,StringType,false), StructField(value,StringType,false), StructField(meaning,StringType,false))
      :- input[0, string, false].toString
      :  +- input[0, string, false]
      :- input[1, string, false].toString
      :  +- input[1, string, false]
      +- input[2, string, false].toString
         +- input[2, string, false]
      ```
      
      2. Currently, `SET` and `SET -v` commands show unsorted result.
          We had better show a sorted result for UX. Also, this is compatible with Hive.
      
      **BEFORE**
      ```
      scala> sql("set").show(false)
      ...
      |spark.driver.host              |10.22.16.140                                                                                                                                 |
      |spark.driver.port              |63893                                                                                                                                        |
      |spark.repl.class.uri           |spark://10.22.16.140:63893/classes                                                                                                           |
      ...
      |spark.app.name                 |Spark shell                                                                                                                                  |
      |spark.driver.memory            |4G                                                                                                                                           |
      |spark.executor.id              |driver                                                                                                                                       |
      |spark.submit.deployMode        |client                                                                                                                                       |
      |spark.master                   |local[*]                                                                                                                                     |
      |spark.home                     |/Users/dhyun/spark                                                                                                                           |
      |spark.sql.catalogImplementation|hive                                                                                                                                         |
      |spark.app.id                   |local-1484333618945                                                                                                                          |
      ```
      
      **AFTER**
      
      ```
      scala> sql("set").show(false)
      ...
      |spark.app.id                   |local-1484333925649                                                                                                                          |
      |spark.app.name                 |Spark shell                                                                                                                                  |
      |spark.driver.host              |10.22.16.140                                                                                                                                 |
      |spark.driver.memory            |4G                                                                                                                                           |
      |spark.driver.port              |64994                                                                                                                                        |
      |spark.executor.id              |driver                                                                                                                                       |
      |spark.jars                     |                                                                                                                                             |
      |spark.master                   |local[*]                                                                                                                                     |
      |spark.repl.class.uri           |spark://10.22.16.140:64994/classes                                                                                                           |
      |spark.sql.catalogImplementation|hive                                                                                                                                         |
      |spark.submit.deployMode        |client                                                                                                                                       |
      ```
      
      ## How was this patch tested?
      
      Jenkins with a new test case.
      
      Author: Dongjoon Hyun <dongjoon@apache.org>
      
      Closes #16579 from dongjoon-hyun/SPARK-19218.
      c4a6519c
    • actuaryzhang's avatar
      [SPARK-19155][ML] Make family case insensitive in GLM · f067acef
      actuaryzhang authored
      ## What changes were proposed in this pull request?
      This is a supplement to PR #16516 which did not make the value from `getFamily` case insensitive. Current tests of poisson/binomial glm with weight fail when specifying 'Poisson' or 'Binomial', because the calculation of `dispersion` and `pValue` checks the value of family retrieved from `getFamily`
      ```
      model.getFamily == Binomial.name || model.getFamily == Poisson.name
      ```
      
      ## How was this patch tested?
      Update existing tests for 'Poisson' and 'Binomial'.
      
      yanboliang felixcheung imatiach-msft
      
      Author: actuaryzhang <actuaryzhang10@gmail.com>
      
      Closes #16675 from actuaryzhang/family.
      f067acef
  3. Jan 22, 2017
    • Wenchen Fan's avatar
      [SPARK-19309][SQL] disable common subexpression elimination for conditional expressions · de6ad3df
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed.
      
      Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression.
      
      This PR simply stops common subexpression elimination for conditional expressions, with some cleanup.
      
      ## How was this patch tested?
      
      regression test
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #16659 from cloud-fan/codegen.
      de6ad3df
    • gatorsmile's avatar
      [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is Not Enabled · 772035e7
      gatorsmile authored
      ### What changes were proposed in this pull request?
      It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.
      
      ### How was this patch tested?
      Fixed the test cases
      
      Author: gatorsmile <gatorsmile@gmail.com>
      
      Closes #16587 from gatorsmile/blockHiveTable.
      772035e7
    • hyukjinkwon's avatar
      [SPARK-16101][SQL] Refactoring CSV read path to be consistent with JSON data source · 74e65cb7
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      This PR refactors CSV read path to be consistent with JSON data source. It makes the methods in classes have consistent arguments with JSON ones.
      
      `UnivocityParser` and `JacksonParser`
      
      ``` scala
      private[csv] class UnivocityParser(
          schema: StructType,
          requiredSchema: StructType,
          options: CSVOptions) extends Logging {
        ...
      
      def parse(input: String): Seq[InternalRow] = {
        ...
      ```
      
      ``` scala
      class JacksonParser(
          schema: StructType,
          columnNameOfCorruptRecord: String,
          options: JSONOptions) extends Logging {
        ...
      
      def parse(input: String): Option[InternalRow] = {
        ...
      ```
      
      These allow parsing an iterator (`String` to `InternalRow`) as below for both JSON and CSV:
      
      ```scala
      iter.flatMap(parser.parse)
      ```
      
      ## How was this patch tested?
      
      Existing tests should cover this.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #16669 from HyukjinKwon/SPARK-16101-read.
      74e65cb7
  4. Jan 21, 2017
    • Yanbo Liang's avatar
      [SPARK-19291][SPARKR][ML] spark.gaussianMixture supports output log-likelihood. · 0c589e37
      Yanbo Liang authored
      ## What changes were proposed in this pull request?
      ```spark.gaussianMixture``` supports output total log-likelihood for the model like R ```mvnormalmixEM```.
      
      ## How was this patch tested?
      R unit test.
      
      Author: Yanbo Liang <ybliang8@gmail.com>
      
      Closes #16646 from yanboliang/spark-19291.
      0c589e37
    • Yanbo Liang's avatar
      [SPARK-19155][ML] MLlib GeneralizedLinearRegression family and link should case insensitive · 3dcad9fa
      Yanbo Liang authored
      ## What changes were proposed in this pull request?
      MLlib ```GeneralizedLinearRegression``` ```family``` and ```link``` should be case insensitive. This is consistent with some other MLlib params such as [```featureSubsetStrategy```](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala#L415).
      
      ## How was this patch tested?
      Update corresponding tests.
      
      Author: Yanbo Liang <ybliang8@gmail.com>
      
      Closes #16516 from yanboliang/spark-19133.
      3dcad9fa
    • windpiger's avatar
      [SPARK-19153][SQL] DataFrameWriter.saveAsTable work with create partitioned table · aa014eb7
      windpiger authored
      ## What changes were proposed in this pull request?
      
      After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19153), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.
      
      this PR provide DataFrameWriter.saveAsTable work with hive format to create partitioned table.
      
      ## How was this patch tested?
      unit test added
      
      Author: windpiger <songjun@outlook.com>
      
      Closes #16593 from windpiger/saveAsTableWithPartitionedTable.
      aa014eb7
    • hyukjinkwon's avatar
      [SPARK-19117][SPARK-18922][TESTS] Fix the rest of flaky, newly introduced and... · 6113fe78
      hyukjinkwon authored
      [SPARK-19117][SPARK-18922][TESTS] Fix the rest of flaky, newly introduced and missed test failures on Windows
      
      ## What changes were proposed in this pull request?
      
      **Failed tests**
      
      ```
      org.apache.spark.sql.hive.execution.HiveQuerySuite:
       - transform with SerDe3 *** FAILED ***
       - transform with SerDe4 *** FAILED ***
      ```
      
      ```
      org.apache.spark.sql.hive.execution.HiveDDLSuite:
       - create hive serde table with new syntax *** FAILED ***
       - add/drop partition with location - managed table *** FAILED ***
      ```
      
      ```
      org.apache.spark.sql.hive.ParquetMetastoreSuite:
       - Explicitly added partitions should be readable after load *** FAILED ***
       - Non-partitioned table readable after load *** FAILED ***
      ```
      
      **Aborted tests**
      
      ```
      Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.HiveSerDeSuite *** ABORTED *** (157 milliseconds)
         org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive   argetscala-2.11   est-classesdatafilessales.txt;
      ```
      
      **Flaky tests(failed 9ish out of 10)**
      
      ```
      org.apache.spark.scheduler.SparkListenerSuite:
       - local metrics *** FAILED ***
      ```
      
      ## How was this patch tested?
      
      Manually tested via AppVeyor.
      
      **Failed tests**
      
      ```
      org.apache.spark.sql.hive.execution.HiveQuerySuite:
       - transform with SerDe3 !!! CANCELED !!! (0 milliseconds)
       - transform with SerDe4 !!! CANCELED !!! (0 milliseconds)
      ```
      
      ```
      org.apache.spark.sql.hive.execution.HiveDDLSuite:
       - create hive serde table with new syntax (1 second, 672 milliseconds)
       - add/drop partition with location - managed table (2 seconds, 391 milliseconds)
      ```
      
      ```
      org.apache.spark.sql.hive.ParquetMetastoreSuite:
       - Explicitly added partitions should be readable after load (609 milliseconds)
       - Non-partitioned table readable after load (344 milliseconds)
      ```
      
      **Aborted tests**
      
      ```
      spark.sql.hive.execution.HiveSerDeSuite:
       - Read with RegexSerDe (2 seconds, 142 milliseconds)
       - Read and write with LazySimpleSerDe (tab separated) (2 seconds)
       - Read with AvroSerDe (1 second, 47 milliseconds)
       - Read Partitioned with AvroSerDe (1 second, 422 milliseconds)
      ```
      
      **Flaky tests (failed 9ish out of 10)**
      
      ```
      org.apache.spark.scheduler.SparkListenerSuite:
       - local metrics (4 seconds, 562 milliseconds)
      ```
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #16586 from HyukjinKwon/set-path-appveyor.
      Unverified
      6113fe78
    • Xin Ren's avatar
      [SPARK-17724][STREAMING][WEBUI] Unevaluated new lines in tooltip in DAG Visualization of a job · bcdabaac
      Xin Ren authored
      https://issues.apache.org/jira/browse/SPARK-17724
      
      ## What changes were proposed in this pull request?
      For unevaluated `\n`, evaluate it and enable line break, for Streaming WebUI `stages` page and `job` page.
      (I didn't change Scala source file, since Jetty server has to somehow indicate line break and js to code display it.)
      (This PR is a continue from previous PR https://github.com/apache/spark/pull/15353 for the same issue, sorry being so long time)
      
      Two changes:
      
      1. RDD Node tooltipText is actually showing the `<circle>`  `title` property, so I set extra attribute in `spark-dag-viz.js`: `.attr("data-html", "true")`
      
      `<circle x="-5" y="-5" r="5" data-toggle="tooltip" data-placement="bottom" title="" data-original-title="ParallelCollectionRDD [9]\nmakeRDD at QueueStream.scala:49"></circle>`
      
      2. Static `<tspan>` text of each stage, split by `/n`, and append an extra `<tspan>` element to its parentNode
      
      `<text><tspan xml:space="preserve" dy="1em" x="1">reduceByKey</tspan><tspan xml:space="preserve" dy="1em" x="1">reduceByKey/n 23:34:49</tspan></text>
      `
      
      ## UI changes
      Screenshot **before fix**, `\n` is not evaluated in both circle tooltipText and static text:
      ![screen shot 2017-01-19 at 12 21 54 am](https://cloud.githubusercontent.com/assets/3925641/22098829/53c7f49c-dddd-11e6-9daa-b3ddb6044114.png)
      
      Screenshot **after fix**:
      ![screen shot 2017-01-19 at 12 20 30 am](https://cloud.githubusercontent.com/assets/3925641/22098806/294910d4-dddd-11e6-9948-d942e09f545e.png)
      
      ## How was this patch tested?
      Tested locally. For Streaming WebUI `stages` page and `job` page, on multiple browsers:
      - Chrome
      - Firefox
      - Safari
      
      Author: Xin Ren <renxin.ubc@gmail.com>
      
      Closes #16643 from keypointt/SPARK-17724-2nd.
      Unverified
      bcdabaac
  5. Jan 20, 2017
    • Wenchen Fan's avatar
      [SPARK-19305][SQL] partitioned table should always put partition columns at the end of table schema · 3c2ba9fc
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      For data source tables, we will always reorder the specified table schema, or the query in CTAS, to put partition columns at the end. e.g. `CREATE TABLE t(a int, b int, c int, d int) USING parquet PARTITIONED BY (d, b)` will create a table with schema `<a, c, d, b>`
      
      Hive serde tables don't have this problem before, because its CREATE TABLE syntax specifies data schema and partition schema individually.
      
      However, after we unifed the CREATE TABLE syntax, Hive serde table also need to do the reorder. This PR puts the reorder logic in a analyzer rule,  which works with both data source tables and Hive serde tables.
      
      ## How was this patch tested?
      
      new regression test
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #16655 from cloud-fan/schema.
      3c2ba9fc
    • sureshthalamati's avatar
      [SPARK-14536][SQL] fix to handle null value in array type column for postgres. · f174cdc7
      sureshthalamati authored
      ## What changes were proposed in this pull request?
      
      JDBC  read  is failing with  NPE due to missing null value check for array data type if the source table has null values in the array type column.  For null values Resultset.getArray()  returns null.
      This PR adds null safe check to the Resultset.getArray() value before invoking method on the Array object.
      ## How was this patch tested?
      
      Updated the PostgresIntegration test suite to test null values. Ran docker integration tests on my laptop.
      
      Author: sureshthalamati <suresh.thalamati@gmail.com>
      
      Closes #15192 from sureshthalamati/jdbc_array_null_fix-SPARK-14536.
      f174cdc7
    • hyukjinkwon's avatar
      [SPARK-16101][SQL] Refactoring CSV write path to be consistent with JSON data source · 54268b42
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      This PR refactors CSV write path to be consistent with JSON data source.
      
      This PR makes the methods in classes have consistent arguments with JSON ones.
        - `UnivocityGenerator` and `JacksonGenerator`
      
          ``` scala
          private[csv] class UnivocityGenerator(
              schema: StructType,
              writer: Writer,
              options: CSVOptions = new CSVOptions(Map.empty[String, String])) {
          ...
      
          def write ...
          def close ...
          def flush ...
          ```
      
          ``` scala
          private[sql] class JacksonGenerator(
             schema: StructType,
             writer: Writer,
             options: JSONOptions = new JSONOptions(Map.empty[String, String])) {
          ...
      
          def write ...
          def close ...
          def flush ...
          ```
      
      - This PR also makes the classes put in together in a consistent manner with JSON.
        - `CsvFileFormat`
      
          ``` scala
          CsvFileFormat
          CsvOutputWriter
          ```
      
        - `JsonFileFormat`
      
          ``` scala
          JsonFileFormat
          JsonOutputWriter
          ```
      
      ## How was this patch tested?
      
      Existing tests should cover this.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #16496 from HyukjinKwon/SPARK-16101-write.
      54268b42
    • Shixiong Zhu's avatar
      [SPARK-19267][SS] Fix a race condition when stopping StateStore · ea31f92b
      Shixiong Zhu authored
      ## What changes were proposed in this pull request?
      
      There is a race condition when stopping StateStore which makes `StateStoreSuite.maintenance` flaky. `StateStore.stop` doesn't wait for the running task to finish, and an out-of-date task may fail `doMaintenance` and cancel the new task. Here is a reproducer: https://github.com/zsxwing/spark/commit/dde1b5b106ba034861cf19e16883cfe181faa6f3
      
      This PR adds MaintenanceTask to eliminate the race condition.
      
      ## How was this patch tested?
      
      Jenkins
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      
      Closes #16627 from zsxwing/SPARK-19267.
      ea31f92b
    • Davies Liu's avatar
      [SPARK-18589][SQL] Fix Python UDF accessing attributes from both side of join · 9b7a03f1
      Davies Liu authored
      ## What changes were proposed in this pull request?
      
      PythonUDF is unevaluable, which can not be used inside a join condition, currently the optimizer will push a PythonUDF which accessing both side of join into the join condition, then the query will fail to plan.
      
      This PR fix this issue by checking the expression is evaluable  or not before pushing it into Join.
      
      ## How was this patch tested?
      
      Add a regression test.
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #16581 from davies/pyudf_join.
      9b7a03f1
    • Tathagata Das's avatar
      [SPARK-19314][SS][CATALYST] Do not allow sort before aggregation in Structured Streaming plan · 552e5f08
      Tathagata Das authored
      ## What changes were proposed in this pull request?
      
      Sort in a streaming plan should be allowed only after a aggregation in complete mode. Currently it is incorrectly allowed when present anywhere in the plan. It gives unpredictable potentially incorrect results.
      
      ## How was this patch tested?
      New test
      
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      
      Closes #16662 from tdas/SPARK-19314.
      552e5f08
    • Parag Chaudhari's avatar
      [SPARK-19069][CORE] Expose task 'status' and 'duration' in spark history server REST API. · e20d9b15
      Parag Chaudhari authored
      ## What changes were proposed in this pull request?
      
      Although Spark history server UI shows task ‘status’ and ‘duration’ fields, it does not expose these fields in the REST API response. For the Spark history server API users, it is not possible to determine task status and duration. Spark history server has access to task status and duration from event log, but it is not exposing these in API. This patch is proposed to expose task ‘status’ and ‘duration’ fields in Spark history server REST API.
      
      ## How was this patch tested?
      
      Modified existing test cases in org.apache.spark.deploy.history.HistoryServerSuite.
      
      Author: Parag Chaudhari <paragpc@amazon.com>
      
      Closes #16473 from paragpc/expose_task_status.
      e20d9b15
    • sarutak's avatar
      [SPARK-19302][DOC][MINOR] Fix the wrong item format in security.md · d50d12b4
      sarutak authored
      ## What changes were proposed in this pull request?
      
      In docs/security.md, there is a description as follows.
      
      ```
       steps to configure the key-stores and the trust-store for the standalone deployment mode is as
       follows:
       * Generate a keys pair for each node
       * Export the public key of the key pair to a file on each node
       * Import all exported public keys into a single trust-store
      ```
      
      According to markdown format, the first item should follow a blank line.
      
      ## How was this patch tested?
      
      Manually tested.
      
      Following captures are rendered web page before and after fix.
      
      * before
      ![before](https://cloud.githubusercontent.com/assets/4736016/22136731/b358115c-df19-11e6-8f6c-2f7b65766265.png)
      
      * after
      ![after](https://cloud.githubusercontent.com/assets/4736016/22136745/c6366ff8-df19-11e6-840d-e7e894218f9c.png)
      
      Author: sarutak <sarutak@oss.nttdata.co.jp>
      
      Closes #16653 from sarutak/SPARK-19302.
      Unverified
      d50d12b4
    • wangzhenhua's avatar
      [SPARK-19271][SQL] Change non-cbo estimation of aggregate · 039ed9fe
      wangzhenhua authored
      ## What changes were proposed in this pull request?
      
      Change non-cbo estimation behavior of aggregate:
      - If groupExpression is empty, we can know row count (=1) and the corresponding size;
      - otherwise, estimation falls back to UnaryNode's computeStats method, which should not propagate rowCount and attributeStats in Statistics because they are not estimated in that method.
      
      ## How was this patch tested?
      
      Added test case
      
      Author: wangzhenhua <wangzhenhua@huawei.com>
      
      Closes #16631 from wzhfy/aggNoCbo.
      039ed9fe
  6. Jan 19, 2017
    • Wenchen Fan's avatar
      [SPARK-19292][SQL] filter with partition columns should be case-insensitive on Hive tables · 0bf605c2
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      When we query a table with a filter on partitioned columns, we will push the partition filter to the metastore to get matched partitions directly.
      
      In `HiveExternalCatalog.listPartitionsByFilter`, we assume the column names in partition filter are already normalized and we don't need to consider case sensitivity. However, `HiveTableScanExec` doesn't follow this assumption. This PR fixes it.
      
      ## How was this patch tested?
      
      new regression test
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #16647 from cloud-fan/bug.
      0bf605c2
Loading