Skip to content
Snippets Groups Projects
  1. Nov 18, 2016
    • Reynold Xin's avatar
      [SPARK-18505][SQL] Simplify AnalyzeColumnCommand · 4b1df0e8
      Reynold Xin authored
      
      ## What changes were proposed in this pull request?
      I'm spending more time at the design & code level for cost-based optimizer now, and have found a number of issues related to maintainability and compatibility that I will like to address.
      
      This is a small pull request to clean up AnalyzeColumnCommand:
      
      1. Removed warning on duplicated columns. Warnings in log messages are useless since most users that run SQL don't see them.
      2. Removed the nested updateStats function, by just inlining the function.
      3. Renamed a few functions to better reflect what they do.
      4. Removed the factory apply method for ColumnStatStruct. It is a bad pattern to use a apply method that returns an instantiation of a class that is not of the same type (ColumnStatStruct.apply used to return CreateNamedStruct).
      5. Renamed ColumnStatStruct to just AnalyzeColumnCommand.
      6. Added more documentation explaining some of the non-obvious return types and code blocks.
      
      In follow-up pull requests, I'd like to address the following:
      
      1. Get rid of the Map[String, ColumnStat] map, since internally we should be using Attribute to reference columns, rather than strings.
      2. Decouple the fields exposed by ColumnStat and internals of Spark SQL's execution path. Currently the two are coupled because ColumnStat takes in an InternalRow.
      3. Correctness: Remove code path that stores statistics in the catalog using the base64 encoding of the UnsafeRow format, which is not stable across Spark versions.
      4. Clearly document the data representation stored in the catalog for statistics.
      
      ## How was this patch tested?
      Affected test cases have been updated.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #15933 from rxin/SPARK-18505.
      
      (cherry picked from commit 6f7ff750)
      Signed-off-by: default avatarReynold Xin <rxin@databricks.com>
      4b1df0e8
    • Shixiong Zhu's avatar
      [SPARK-18477][SS] Enable interrupts for HDFS in HDFSMetadataLog · 136f687c
      Shixiong Zhu authored
      
      ## What changes were proposed in this pull request?
      
      HDFS `write` may just hang until timeout if some network error happens. It's better to enable interrupts to allow stopping the query fast on HDFS.
      
      This PR just changes the logic to only disable interrupts for local file system, as HADOOP-10622 only happens for local file system.
      
      ## How was this patch tested?
      
      Jenkins
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #15911 from zsxwing/interrupt-on-dfs.
      
      (cherry picked from commit e5f5c29e)
      Signed-off-by: default avatarTathagata Das <tathagata.das1565@gmail.com>
      136f687c
    • hyukjinkwon's avatar
      [SPARK-18422][CORE] Fix wholeTextFiles test to pass on Windows in JavaAPISuite · 6717981e
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      This PR fixes the test `wholeTextFiles` in `JavaAPISuite.java`. This is failed due to the different path format on Windows.
      
      For example, the path in `container` was
      
      ```
      C:\projects\spark\target\tmp\1478967560189-0/part-00000
      ```
      
      whereas `new URI(res._1()).getPath()` was as below:
      
      ```
      /C:/projects/spark/target/tmp/1478967560189-0/part-00000
      ```
      
      ## How was this patch tested?
      
      Tests in `JavaAPISuite.java`.
      
      Tested via AppVeyor.
      
      **Before**
      Build: https://ci.appveyor.com/project/spark-test/spark/build/63-JavaAPISuite-1
      Diff: https://github.com/apache/spark/compare/master...spark-test:JavaAPISuite-1
      
      ```
      [info] Test org.apache.spark.JavaAPISuite.wholeTextFiles started
      [error] Test org.apache.spark.JavaAPISuite.wholeTextFiles failed: java.lang.AssertionError: expected:<spark is easy to use.
      [error] > but was:<null>, took 0.578 sec
      [error]     at org.apache.spark.JavaAPISuite.wholeTextFiles(JavaAPISuite.java:1089)
      ...
      ```
      
      **After**
      Build started: [CORE] `org.apache.spark.JavaAPISuite` [![PR-15866](https://ci.appveyor.com/api/projects/status/github/spark-test/spark?branch=198DDA52-F201-4D2B-BE2F-244E0C1725B2&svg=true)](https://ci.appveyor.com/project/spark-test/spark/branch/198DDA52-F201-4D2B-BE2F-244E0C1725B2)
      Diff: https://github.com/apache/spark/compare/master...spark-test:198DDA52-F201-4D2B-BE2F-244E0C1725B2
      
      
      
      ```
      [info] Test org.apache.spark.JavaAPISuite.wholeTextFiles started
      ...
      ```
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #15866 from HyukjinKwon/SPARK-18422.
      
      (cherry picked from commit 40d59ff5)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      6717981e
    • Andrew Ray's avatar
      [SPARK-18457][SQL] ORC and other columnar formats using HiveShim read all... · ec622eb7
      Andrew Ray authored
      [SPARK-18457][SQL] ORC and other columnar formats using HiveShim read all columns when doing a simple count
      
      ## What changes were proposed in this pull request?
      
      When reading zero columns (e.g., count(*)) from ORC or any other format that uses HiveShim, actually set the read column list to empty for Hive to use.
      
      ## How was this patch tested?
      
      Query correctness is handled by existing unit tests. I'm happy to add more if anyone can point out some case that is not covered.
      
      Reduction in data read can be verified in the UI when built with a recent version of Hadoop say:
      ```
      build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.0 -Phive -DskipTests clean package
      ```
      However the default Hadoop 2.2 that is used for unit tests does not report actual bytes read and instead just full file sizes (see FileScanRDD.scala line 80). Therefore I don't think there is a good way to add a unit test for this.
      
      I tested with the following setup using above build options
      ```
      case class OrcData(intField: Long, stringField: String)
      spark.range(1,1000000).map(i => OrcData(i, s"part-$i")).toDF().write.format("orc").save("orc_test")
      
      sql(
            s"""CREATE EXTERNAL TABLE orc_test(
               |  intField LONG,
               |  stringField STRING
               |)
               |STORED AS ORC
               |LOCATION '${System.getProperty("user.dir") + "/orc_test"}'
             """.stripMargin)
      ```
      
      ## Results
      
      query | Spark 2.0.2 | this PR
      ---|---|---
      `sql("select count(*) from orc_test").collect`|4.4 MB|199.4 KB
      `sql("select intField from orc_test").collect`|743.4 KB|743.4 KB
      `sql("select * from orc_test").collect`|4.4 MB|4.4 MB
      
      Author: Andrew Ray <ray.andrew@gmail.com>
      
      Closes #15898 from aray/sql-orc-no-col.
      
      (cherry picked from commit 795e9fc9)
      Signed-off-by: default avatarReynold Xin <rxin@databricks.com>
      ec622eb7
    • Tyson Condie's avatar
      [SPARK-18187][SQL] CompactibleFileStreamLog should not use "compactInterval"... · 5912c19e
      Tyson Condie authored
      [SPARK-18187][SQL] CompactibleFileStreamLog should not use "compactInterval" direcly with user setting.
      
      ## What changes were proposed in this pull request?
      CompactibleFileStreamLog relys on "compactInterval" to detect a compaction batch. If the "compactInterval" is reset by user, CompactibleFileStreamLog will return wrong answer, resulting data loss. This PR procides a way to check the validity of 'compactInterval', and calculate an appropriate value.
      
      ## How was this patch tested?
      When restart a stream, we change the 'spark.sql.streaming.fileSource.log.compactInterval' different with the former one.
      
      The primary solution to this issue was given by uncleGen
      Added extensions include an additional metadata field in OffsetSeq and CompactibleFileStreamLog APIs. zsxwing
      
      Author: Tyson Condie <tcondie@gmail.com>
      Author: genmao.ygm <genmao.ygm@genmaoygmdeMacBook-Air.local>
      
      Closes #15852 from tcondie/spark-18187.
      
      (cherry picked from commit 51baca22)
      Signed-off-by: default avatarShixiong Zhu <shixiong@databricks.com>
      5912c19e
  2. Nov 17, 2016
    • Josh Rosen's avatar
      [SPARK-18462] Fix ClassCastException in SparkListenerDriverAccumUpdates event · e8b1955e
      Josh Rosen authored
      ## What changes were proposed in this pull request?
      
      This patch fixes a `ClassCastException: java.lang.Integer cannot be cast to java.lang.Long` error which could occur in the HistoryServer while trying to process a deserialized `SparkListenerDriverAccumUpdates` event.
      
      The problem stems from how `jackson-module-scala` handles primitive type parameters (see https://github.com/FasterXML/jackson-module-scala/wiki/FAQ#deserializing-optionint-and-other-primitive-challenges
      
       for more details). This was causing a problem where our code expected a field to be deserialized as a `(Long, Long)` tuple but we got an `(Int, Int)` tuple instead.
      
      This patch hacks around this issue by registering a custom `Converter` with Jackson in order to deserialize the tuples as `(Object, Object)` and perform the appropriate casting.
      
      ## How was this patch tested?
      
      New regression tests in `SQLListenerSuite`.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #15922 from JoshRosen/SPARK-18462.
      
      (cherry picked from commit d9dd979d)
      Signed-off-by: default avatarReynold Xin <rxin@databricks.com>
      e8b1955e
    • Wenchen Fan's avatar
      [SPARK-18360][SQL] default table path of tables in default database should... · fc466be4
      Wenchen Fan authored
      [SPARK-18360][SQL] default table path of tables in default database should depend on the location of default database
      
      ## What changes were proposed in this pull request?
      
      The current semantic of the warehouse config:
      
      1. it's a static config, which means you can't change it once your spark application is launched.
      2. Once a database is created, its location won't change even the warehouse path config is changed.
      3. default database is a special case, although its location is fixed, but the locations of tables created in it are not. If a Spark app starts with warehouse path B(while the location of default database is A), then users create a table `tbl` in default database, its location will be `B/tbl` instead of `A/tbl`. If uses change the warehouse path config to C, and create another table `tbl2`, its location will still be `B/tbl2` instead of `C/tbl2`.
      
      rule 3 doesn't make sense and I think we made it by mistake, not intentionally. Data source tables don't follow rule 3 and treat default database like normal ones.
      
      This PR fixes hive serde tables to make it consistent with data source tables.
      
      ## How was this patch tested?
      
      HiveSparkSubmitSuite
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #15812 from cloud-fan/default-db.
      
      (cherry picked from commit ce13c267)
      Signed-off-by: default avatarYin Huai <yhuai@databricks.com>
      fc466be4
    • root's avatar
      [SPARK-18490][SQL] duplication nodename extrainfo for ShuffleExchange · 97879888
      root authored
      
      ## What changes were proposed in this pull request?
      
         In ShuffleExchange, the nodename's extraInfo are the same when exchangeCoordinator.isEstimated
       is true or false.
      
      Merge the two situation in the PR.
      
      Author: root <root@iZbp1gsnrlfzjxh82cz80vZ.(none)>
      
      Closes #15920 from windpiger/DupNodeNameShuffleExchange.
      
      (cherry picked from commit b0aa1aa1)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      97879888
    • Zheng RuiFeng's avatar
      [SPARK-18480][DOCS] Fix wrong links for ML guide docs · 536a2159
      Zheng RuiFeng authored
      
      ## What changes were proposed in this pull request?
      1, There are two `[Graph.partitionBy]` in `graphx-programming-guide.md`, the first one had no effert.
      2, `DataFrame`, `Transformer`, `Pipeline` and `Parameter`  in `ml-pipeline.md` were linked to `ml-guide.html` by mistake.
      3, `PythonMLLibAPI` in `mllib-linear-methods.md` was not accessable, because class `PythonMLLibAPI` is private.
      4, Other link updates.
      ## How was this patch tested?
       manual tests
      
      Author: Zheng RuiFeng <ruifengz@foxmail.com>
      
      Closes #15912 from zhengruifeng/md_fix.
      
      (cherry picked from commit cdaf4ce9)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      536a2159
    • VinceShieh's avatar
      [SPARK-17462][MLLIB]use VersionUtils to parse Spark version strings · 42777b1b
      VinceShieh authored
      
      ## What changes were proposed in this pull request?
      
      Several places in MLlib use custom regexes or other approaches to parse Spark versions.
      Those should be fixed to use the VersionUtils. This PR replaces custom regexes with
      VersionUtils to get Spark version numbers.
      ## How was this patch tested?
      
      Existing tests.
      
      Signed-off-by: VinceShieh vincent.xieintel.com
      
      Author: VinceShieh <vincent.xie@intel.com>
      
      Closes #15055 from VinceShieh/SPARK-17462.
      
      (cherry picked from commit de77c677)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      42777b1b
    • anabranch's avatar
      [SPARK-18365][DOCS] Improve Sample Method Documentation · 4fcecb4c
      anabranch authored
      ## What changes were proposed in this pull request?
      
      I found the documentation for the sample method to be confusing, this adds more clarification across all languages.
      
      - [x] Scala
      - [x] Python
      - [x] R
      - [x] RDD Scala
      - [ ] RDD Python with SEED
      - [X] RDD Java
      - [x] RDD Java with SEED
      - [x] RDD Python
      
      ## How was this patch tested?
      
      NA
      
      Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark
      
       before opening a pull request.
      
      Author: anabranch <wac.chambers@gmail.com>
      Author: Bill Chambers <bill@databricks.com>
      
      Closes #15815 from anabranch/SPARK-18365.
      
      (cherry picked from commit 49b6f456)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      4fcecb4c
    • Weiqing Yang's avatar
      [YARN][DOC] Remove non-Yarn specific configurations from running-on-yarn.md · 2ee4fc88
      Weiqing Yang authored
      
      ## What changes were proposed in this pull request?
      
      Remove `spark.driver.memory`, `spark.executor.memory`,  `spark.driver.cores`, and `spark.executor.cores` from `running-on-yarn.md` as they are not Yarn-specific, and they are also defined in`configuration.md`.
      
      ## How was this patch tested?
      Build passed & Manually check.
      
      Author: Weiqing Yang <yangweiqing001@gmail.com>
      
      Closes #15869 from weiqingy/yarnDoc.
      
      (cherry picked from commit a3cac7bd)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      2ee4fc88
    • Wenchen Fan's avatar
      [SPARK-18464][SQL] support old table which doesn't store schema in metastore · 014fceee
      Wenchen Fan authored
      
      ## What changes were proposed in this pull request?
      
      Before Spark 2.1, users can create an external data source table without schema, and we will infer the table schema at runtime. In Spark 2.1, we decided to infer the schema when the table was created, so that we don't need to infer it again and again at runtime.
      
      This is a good improvement, but we should still respect and support old tables which doesn't store table schema in metastore.
      
      ## How was this patch tested?
      
      regression test.
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #15900 from cloud-fan/hive-catalog.
      
      (cherry picked from commit 07b3f045)
      Signed-off-by: default avatarReynold Xin <rxin@databricks.com>
      014fceee
  3. Nov 16, 2016
    • Holden Karau's avatar
      [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed · 6a3cbbc0
      Holden Karau authored
      ## What changes were proposed in this pull request?
      
      This PR aims to provide a pip installable PySpark package. This does a bunch of work to copy the jars over and package them with the Python code (to prevent challenges from trying to use different versions of the Python code with different versions of the JAR). It does not currently publish to PyPI but that is the natural follow up (SPARK-18129).
      
      Done:
      - pip installable on conda [manual tested]
      - setup.py installed on a non-pip managed system (RHEL) with YARN [manual tested]
      - Automated testing of this (virtualenv)
      - packaging and signing with release-build*
      
      Possible follow up work:
      - release-build update to publish to PyPI (SPARK-18128)
      - figure out who owns the pyspark package name on prod PyPI (is it someone with in the project or should we ask PyPI or should we choose a different name to publish with like ApachePySpark?)
      - Windows support and or testing ( SPARK-18136 )
      - investigate details of wheel caching and see if we can avoid cleaning the wheel cache during our test
      - consider how we want to number our dev/snapshot versions
      
      Explicitly out of scope:
      - Using pip installed PySpark to start a standalone cluster
      - Using pip installed PySpark for non-Python Spark programs
      
      *I've done some work to test release-build locally but as a non-committer I've just done local testing.
      ## How was this patch tested?
      
      Automated testing with virtualenv, manual testing with conda, a system wide install, and YARN integration.
      
      release-build changes tested locally as a non-committer (no testing of upload artifacts to Apache staging websites)
      
      Author: Holden Karau <holden@us.ibm.com>
      Author: Juliet Hougland <juliet@cloudera.com>
      Author: Juliet Hougland <not@myemail.com>
      
      Closes #15659 from holdenk/SPARK-1267-pip-install-pyspark.
      6a3cbbc0
    • Takuya UESHIN's avatar
      [SPARK-18442][SQL] Fix nullability of WrapOption. · 95157938
      Takuya UESHIN authored
      
      ## What changes were proposed in this pull request?
      
      The nullability of `WrapOption` should be `false`.
      
      ## How was this patch tested?
      
      Existing tests.
      
      Author: Takuya UESHIN <ueshin@happy-camper.st>
      
      Closes #15887 from ueshin/issues/SPARK-18442.
      
      (cherry picked from commit 170eeb34)
      Signed-off-by: default avatarWenchen Fan <wenchen@databricks.com>
      95157938
    • Artur Sukhenko's avatar
      [YARN][DOC] Increasing NodeManager's heap size with External Shuffle Service · 523abfe1
      Artur Sukhenko authored
      ## What changes were proposed in this pull request?
      
      Suggest users to increase `NodeManager's` heap size if `External Shuffle Service` is enabled as
      `NM` can spend a lot of time doing GC resulting in  shuffle operations being a bottleneck due to `Shuffle Read blocked time` bumped up.
      Also because of GC  `NodeManager` can use an enormous amount of CPU and cluster performance will suffer.
      I have seen NodeManager using 5-13G RAM and up to 2700% CPU with `spark_shuffle` service on.
      
      ## How was this patch tested?
      
      #### Added step 5:
      ![shuffle_service](https://cloud.githubusercontent.com/assets/15244468/20355499/2fec0fde-ac2a-11e6-8f8b-1c80daf71be1.png
      
      )
      
      Author: Artur Sukhenko <artur.sukhenko@gmail.com>
      
      Closes #15906 from Devian-ua/nmHeapSize.
      
      (cherry picked from commit 55589987)
      Signed-off-by: default avatarReynold Xin <rxin@databricks.com>
      523abfe1
    • Tathagata Das's avatar
      [SPARK-18461][DOCS][STRUCTUREDSTREAMING] Added more information about monitoring streaming queries · 3d4756d5
      Tathagata Das authored
      ## What changes were proposed in this pull request?
      <img width="941" alt="screen shot 2016-11-15 at 6 27 32 pm" src="https://cloud.githubusercontent.com/assets/663212/20332521/4190b858-ab61-11e6-93a6-4bdc05105ed9.png">
      <img width="940" alt="screen shot 2016-11-15 at 6 27 45 pm" src="https://cloud.githubusercontent.com/assets/663212/20332525/44a0d01e-ab61-11e6-8668-47f925490d4f.png
      
      ">
      
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      
      Closes #15897 from tdas/SPARK-18461.
      
      (cherry picked from commit bb6cdfd9)
      Signed-off-by: default avatarMichael Armbrust <michael@databricks.com>
      3d4756d5
    • Tathagata Das's avatar
      [SPARK-18459][SPARK-18460][STRUCTUREDSTREAMING] Rename triggerId to batchId... · b86e962c
      Tathagata Das authored
      [SPARK-18459][SPARK-18460][STRUCTUREDSTREAMING] Rename triggerId to batchId and add triggerDetails to json in StreamingQueryStatus
      
      ## What changes were proposed in this pull request?
      
      SPARK-18459: triggerId seems like a number that should be increasing with each trigger, whether or not there is data in it. However, actually, triggerId increases only where there is a batch of data in a trigger. So its better to rename it to batchId.
      
      SPARK-18460: triggerDetails was missing from json representation. Fixed it.
      
      ## How was this patch tested?
      Updated existing unit tests.
      
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      
      Closes #15895 from tdas/SPARK-18459.
      
      (cherry picked from commit 0048ce7c)
      Signed-off-by: default avatarShixiong Zhu <shixiong@databricks.com>
      b86e962c
    • gatorsmile's avatar
      [SPARK-18415][SQL] Weird Plan Output when CTE used in RunnableCommand · c0dbe08d
      gatorsmile authored
      
      ### What changes were proposed in this pull request?
      Currently, when CTE is used in RunnableCommand, the Analyzer does not replace the logical node `With`. The child plan of RunnableCommand is not resolved. Thus, the output of the `With` plan node looks very confusing.
      For example,
      ```
      sql(
        """
          |CREATE VIEW cte_view AS
          |WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
          |SELECT n FROM w
        """.stripMargin).explain()
      ```
      The output is like
      ```
      ExecutedCommand
         +- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
      SELECT n FROM w, false, false, PersistedView
               +- 'With [(w,SubqueryAlias w
      +- Project [1 AS n#16]
         +- OneRowRelation$
      ), (cte1,'SubqueryAlias cte1
      +- 'Project [unresolvedalias(2, None)]
         +- OneRowRelation$
      ), (cte2,'SubqueryAlias cte2
      +- 'Project [unresolvedalias(3, None)]
         +- OneRowRelation$
      )]
                  +- 'Project ['n]
                     +- 'UnresolvedRelation `w`
      ```
      After the fix, the output is as shown below.
      ```
      ExecutedCommand
         +- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
      SELECT n FROM w, false, false, PersistedView
               +- CTE [w, cte1, cte2]
                  :  :- SubqueryAlias w
                  :  :  +- Project [1 AS n#16]
                  :  :     +- OneRowRelation$
                  :  :- 'SubqueryAlias cte1
                  :  :  +- 'Project [unresolvedalias(2, None)]
                  :  :     +- OneRowRelation$
                  :  +- 'SubqueryAlias cte2
                  :     +- 'Project [unresolvedalias(3, None)]
                  :        +- OneRowRelation$
                  +- 'Project ['n]
                     +- 'UnresolvedRelation `w`
      ```
      
      BTW, this PR also fixes the output of the view type.
      
      ### How was this patch tested?
      Manual
      
      Author: gatorsmile <gatorsmile@gmail.com>
      
      Closes #15854 from gatorsmile/cteName.
      
      (cherry picked from commit 608ecc51)
      Signed-off-by: default avatarHerman van Hovell <hvanhovell@databricks.com>
      c0dbe08d
    • Xianyang Liu's avatar
      [SPARK-18420][BUILD] Fix the errors caused by lint check in Java · b0ae8712
      Xianyang Liu authored
      
      Small fix, fix the errors caused by lint check in Java
      
      - Clear unused objects and `UnusedImports`.
      - Add comments around the method `finalize` of `NioBufferedFileInputStream`to turn off checkstyle.
      - Cut the line which is longer than 100 characters into two lines.
      
      Travis CI.
      ```
      $ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
      $ dev/lint-java
      ```
      Before:
      ```
      Checkstyle checks failed at following occurrences:
      [ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[21,8] (imports) UnusedImports: Unused import - org.apache.commons.crypto.cipher.CryptoCipherFactory.
      [ERROR] src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java:[516,5] (modifier) RedundantModifier: Redundant 'public' modifier.
      [ERROR] src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java:[133] (coding) NoFinalizer: Avoid using finalizer method.
      [ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java:[71] (sizes) LineLength: Line is longer than 100 characters (found 113).
      [ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java:[112] (sizes) LineLength: Line is longer than 100 characters (found 110).
      [ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[31,17] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
      [ERROR]src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103).
      [ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.ml.linalg.Vectors.
      [ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
      ```
      
      After:
      ```
      $ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
      $ dev/lint-java
      Using `mvn` from path: /home/travis/build/ConeyLiu/spark/build/apache-maven-3.3.9/bin/mvn
      Checkstyle checks passed.
      ```
      
      Author: Xianyang Liu <xyliu0530@icloud.com>
      
      Closes #15865 from ConeyLiu/master.
      
      (cherry picked from commit 7569cf6c)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      b0ae8712
    • Zheng RuiFeng's avatar
      [SPARK-18446][ML][DOCS] Add links to API docs for ML algos · 416bc3dd
      Zheng RuiFeng authored
      
      ## What changes were proposed in this pull request?
      Add links to API docs for ML algos
      ## How was this patch tested?
      Manual checking for the API links
      
      Author: Zheng RuiFeng <ruifengz@foxmail.com>
      
      Closes #15890 from zhengruifeng/algo_link.
      
      (cherry picked from commit a75e3fe9)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      416bc3dd
    • Zheng RuiFeng's avatar
      [SPARK-18434][ML] Add missing ParamValidations for ML algos · 6b6eb4e5
      Zheng RuiFeng authored
      
      ## What changes were proposed in this pull request?
      Add missing ParamValidations for ML algos
      ## How was this patch tested?
      existing tests
      
      Author: Zheng RuiFeng <ruifengz@foxmail.com>
      
      Closes #15881 from zhengruifeng/arg_checking.
      
      (cherry picked from commit c68f1a38)
      Signed-off-by: default avatarYanbo Liang <ybliang8@gmail.com>
      6b6eb4e5
    • Weiqing Yang's avatar
      [MINOR][DOC] Fix typos in the 'configuration', 'monitoring' and... · 82084700
      Weiqing Yang authored
      [MINOR][DOC] Fix typos in the 'configuration', 'monitoring' and 'sql-programming-guide' documentation
      
      ## What changes were proposed in this pull request?
      
      Fix typos in the 'configuration', 'monitoring' and 'sql-programming-guide' documentation.
      
      ## How was this patch tested?
      Manually.
      
      Author: Weiqing Yang <yangweiqing001@gmail.com>
      
      Closes #15886 from weiqingy/fixTypo.
      
      (cherry picked from commit 241e04bc)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      82084700
    • uncleGen's avatar
      [SPARK-18410][STREAMING] Add structured kafka example · 6b2301b8
      uncleGen authored
      
      ## What changes were proposed in this pull request?
      
      This PR provides structured kafka wordcount examples
      
      ## How was this patch tested?
      
      Author: uncleGen <hustyugm@gmail.com>
      
      Closes #15849 from uncleGen/SPARK-18410.
      
      (cherry picked from commit e6145772)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      6b2301b8
    • Sean Owen's avatar
      [SPARK-18400][STREAMING] NPE when resharding Kinesis Stream · a94659ce
      Sean Owen authored
      
      ## What changes were proposed in this pull request?
      
      Avoid NPE in KinesisRecordProcessor when shutdown happens without successful init
      
      ## How was this patch tested?
      
      Existing tests
      
      Author: Sean Owen <sowen@cloudera.com>
      
      Closes #15882 from srowen/SPARK-18400.
      
      (cherry picked from commit 43a26899)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      a94659ce
    • Liwei Lin's avatar
      [DOC][MINOR] Kafka doc: breakup into lines · 4567db9d
      Liwei Lin authored
      ## Before
      
      ![before](https://cloud.githubusercontent.com/assets/15843379/20340231/99b039fe-ac1b-11e6-9ba9-b44582427459.png)
      
      ## After
      
      ![after](https://cloud.githubusercontent.com/assets/15843379/20340236/9d5796e2-ac1b-11e6-92bb-6da40ba1a383.png
      
      )
      
      Author: Liwei Lin <lwlin7@gmail.com>
      
      Closes #15903 from lw-lin/kafka-doc-lines.
      
      (cherry picked from commit 3e01f128)
      Signed-off-by: default avatarSean Owen <sowen@cloudera.com>
      Unverified
      4567db9d
    • Dongjoon Hyun's avatar
      [SPARK-18433][SQL] Improve DataSource option keys to be more case-insensitive · b18c5a9b
      Dongjoon Hyun authored
      
      ## What changes were proposed in this pull request?
      
      This PR aims to improve DataSource option keys to be more case-insensitive
      
      DataSource partially use CaseInsensitiveMap in code-path. For example, the following fails to find url.
      
      ```scala
      val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2)
      df.write.format("jdbc")
          .option("UrL", url1)
          .option("dbtable", "TEST.SAVETEST")
          .options(properties.asScala)
          .save()
      ```
      
      This PR makes DataSource options to use CaseInsensitiveMap internally and also makes DataSource to use CaseInsensitiveMap generally except `InMemoryFileIndex` and `InsertIntoHadoopFsRelationCommand`. We can not pass them CaseInsensitiveMap because they creates new case-sensitive HadoopConfs by calling newHadoopConfWithOptions(options) inside.
      
      ## How was this patch tested?
      
      Pass the Jenkins test with newly added test cases.
      
      Author: Dongjoon Hyun <dongjoon@apache.org>
      
      Closes #15884 from dongjoon-hyun/SPARK-18433.
      
      (cherry picked from commit 74f5c217)
      Signed-off-by: default avatarWenchen Fan <wenchen@databricks.com>
      b18c5a9b
    • Yanbo Liang's avatar
      [SPARK-18438][SPARKR][ML] spark.mlp should support RFormula. · 7b57e480
      Yanbo Liang authored
      
      ## What changes were proposed in this pull request?
      ```spark.mlp``` should support ```RFormula``` like other ML algorithm wrappers.
      BTW, I did some cleanup and improvement for ```spark.mlp```.
      
      ## How was this patch tested?
      Unit tests.
      
      Author: Yanbo Liang <ybliang8@gmail.com>
      
      Closes #15883 from yanboliang/spark-18438.
      
      (cherry picked from commit 95eb06bd)
      Signed-off-by: default avatarYanbo Liang <ybliang8@gmail.com>
      7b57e480
  4. Nov 15, 2016
  5. Nov 14, 2016
    • gatorsmile's avatar
      [SPARK-18430][SQL] Fixed Exception Messages when Hitting an Invocation Exception of Function Lookup · a0125fd6
      gatorsmile authored
      ### What changes were proposed in this pull request?
      When the exception is an invocation exception during function lookup, we return a useless/confusing error message:
      
      For example,
      ```Scala
      df.selectExpr("concat_ws()")
      ```
      Below is the error message we got:
      ```
      null; line 1 pos 0
      org.apache.spark.sql.AnalysisException: null; line 1 pos 0
      ```
      
      To get the meaningful error message, we need to get the cause. The fix is exactly the same as what we did in https://github.com/apache/spark/pull/12136
      
      . After the fix, the message we got is the exception issued in the constuctor of function implementation:
      ```
      requirement failed: concat_ws requires at least one argument.; line 1 pos 0
      org.apache.spark.sql.AnalysisException: requirement failed: concat_ws requires at least one argument.; line 1 pos 0
      ```
      
      ### How was this patch tested?
      Added test cases.
      
      Author: gatorsmile <gatorsmile@gmail.com>
      
      Closes #15878 from gatorsmile/functionNotFound.
      
      (cherry picked from commit 86430cc4)
      Signed-off-by: default avatarReynold Xin <rxin@databricks.com>
      a0125fd6
    • Zheng RuiFeng's avatar
      [SPARK-18428][DOC] Update docs for GraphX · 649c15fa
      Zheng RuiFeng authored
      
      ## What changes were proposed in this pull request?
      1, Add link of `VertexRDD` and `EdgeRDD`
      2, Notify in `Vertex and Edge RDDs` that not all methods are listed
      3, `VertexID` -> `VertexId`
      
      ## How was this patch tested?
      No tests, only docs is modified
      
      Author: Zheng RuiFeng <ruifengz@foxmail.com>
      
      Closes #15875 from zhengruifeng/update_graphop_doc.
      
      (cherry picked from commit c31def1d)
      Signed-off-by: default avatarReynold Xin <rxin@databricks.com>
      649c15fa
    • Michael Armbrust's avatar
      [SPARK-18124] Observed delay based Event Time Watermarks · 27999b36
      Michael Armbrust authored
      
      This PR adds a new method `withWatermark` to the `Dataset` API, which can be used specify an _event time watermark_.  An event time watermark allows the streaming engine to reason about the point in time after which we no longer expect to see late data.  This PR also has augmented `StreamExecution` to use this watermark for several purposes:
        - To know when a given time window aggregation is finalized and thus results can be emitted when using output modes that do not allow updates (e.g. `Append` mode).
        - To minimize the amount of state that we need to keep for on-going aggregations, by evicting state for groups that are no longer expected to change.  Although, we do still maintain all state if the query requires (i.e. if the event time is not present in the `groupBy` or when running in `Complete` mode).
      
      An example that emits windowed counts of records, waiting up to 5 minutes for late data to arrive.
      ```scala
      df.withWatermark("eventTime", "5 minutes")
        .groupBy(window($"eventTime", "1 minute") as 'window)
        .count()
        .writeStream
        .format("console")
        .mode("append") // In append mode, we only output finalized aggregations.
        .start()
      ```
      
      ### Calculating the watermark.
      The current event time is computed by looking at the `MAX(eventTime)` seen this epoch across all of the partitions in the query minus some user defined _delayThreshold_.  An additional constraint is that the watermark must increase monotonically.
      
      Note that since we must coordinate this value across partitions occasionally, the actual watermark used is only guaranteed to be at least `delay` behind the actual event time.  In some cases we may still process records that arrive more than delay late.
      
      This mechanism was chosen for the initial implementation over processing time for two reasons:
        - it is robust to downtime that could affect processing delay
        - it does not require syncing of time or timezones between the producer and the processing engine.
      
      ### Other notable implementation details
       - A new trigger metric `eventTimeWatermark` outputs the current value of the watermark.
       - We mark the event time column in the `Attribute` metadata using the key `spark.watermarkDelay`.  This allows downstream operations to know which column holds the event time.  Operations like `window` propagate this metadata.
       - `explain()` marks the watermark with a suffix of `-T${delayMs}` to ease debugging of how this information is propagated.
       - Currently, we don't filter out late records, but instead rely on the state store to avoid emitting records that are both added and filtered in the same epoch.
      
      ### Remaining in this PR
       - [ ] The test for recovery is currently failing as we don't record the watermark used in the offset log.  We will need to do so to ensure determinism, but this is deferred until #15626 is merged.
      
      ### Other follow-ups
      There are some natural additional features that we should consider for future work:
       - Ability to write records that arrive too late to some external store in case any out-of-band remediation is required.
       - `Update` mode so you can get partial results before a group is evicted.
       - Other mechanisms for calculating the watermark.  In particular a watermark based on quantiles would be more robust to outliers.
      
      Author: Michael Armbrust <michael@databricks.com>
      
      Closes #15702 from marmbrus/watermarks.
      
      (cherry picked from commit c0718782)
      Signed-off-by: default avatarTathagata Das <tathagata.das1565@gmail.com>
      27999b36
Loading