Skip to content
Snippets Groups Projects
  1. Aug 01, 2014
    • Aaron Davidson's avatar
      SPARK-2791: Fix committing, reverting and state tracking in shuffle file consolidation · 78f2af58
      Aaron Davidson authored
      All changes from this PR are by mridulm and are drawn from his work in #1609. This patch is intended to fix all major issues related to shuffle file consolidation that mridulm found, while minimizing changes to the code, with the hope that it may be more easily merged into 1.1.
      
      This patch is **not** intended as a replacement for #1609, which provides many additional benefits, including fixes to ExternalAppendOnlyMap, improvements to DiskBlockObjectWriter's API, and several new unit tests.
      
      If it is feasible to merge #1609 for the 1.1 deadline, that is a preferable option.
      
      Author: Aaron Davidson <aaron@databricks.com>
      
      Closes #1678 from aarondav/consol and squashes the following commits:
      
      53b3f6d [Aaron Davidson] Correct behavior when writing unopened file
      701d045 [Aaron Davidson] Rebase with sort-based shuffle
      9160149 [Aaron Davidson] SPARK-2532: Minimal shuffle consolidation fixes
      78f2af58
    • joyyoj's avatar
      [SPARK-2379] Fix the bug that streaming's receiver may fall into a dead loop · b270309d
      joyyoj authored
      Author: joyyoj <sunshch@gmail.com>
      
      Closes #1694 from joyyoj/SPARK-2379 and squashes the following commits:
      
      d73790d [joyyoj] SPARK-2379 Fix the bug that streaming's receiver may fall into a dead loop
      22e7821 [joyyoj] Merge remote-tracking branch 'apache/master'
      3f4a602 [joyyoj] Merge remote-tracking branch 'remotes/apache/master'
      f4660c5 [joyyoj] [SPARK-1998] SparkFlumeEvent with body bigger than 1020 bytes are not read properly
      b270309d
    • zsxwing's avatar
      SPARK-1612: Fix potential resource leaks · f5d9bea2
      zsxwing authored
      JIRA: https://issues.apache.org/jira/browse/SPARK-1612
      
      Move the "close" statements into a "finally" block.
      
      Author: zsxwing <zsxwing@gmail.com>
      
      Closes #535 from zsxwing/SPARK-1612 and squashes the following commits:
      
      ae52f50 [zsxwing] Update to follow the code style
      549ba13 [zsxwing] SPARK-1612: Fix potential resource leaks
      f5d9bea2
    • Liang-Chi Hsieh's avatar
      [SPARK-2490] Change recursive visiting on RDD dependencies to iterative approach · baf9ce1a
      Liang-Chi Hsieh authored
      When performing some transformations on RDDs after many iterations, the dependencies of RDDs could be very long. It can easily cause StackOverflowError when recursively visiting these dependencies in Spark core. For example:
      
          var rdd = sc.makeRDD(Array(1))
          for (i <- 1 to 1000) {
            rdd = rdd.coalesce(1).cache()
            rdd.collect()
          }
      
      This PR changes recursive visiting on rdd's dependencies to iterative approach to avoid StackOverflowError.
      
      In addition to the recursive visiting, since the Java serializer has a known [bug](http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4152790) that causes StackOverflowError too when serializing/deserializing a large graph of objects. So applying this PR only solves part of the problem. Using KryoSerializer to replace Java serializer might be helpful. However, since KryoSerializer is not supported for `spark.closure.serializer` now, I can not test if KryoSerializer can solve Java serializer's problem completely.
      
      Author: Liang-Chi Hsieh <viirya@gmail.com>
      
      Closes #1418 from viirya/remove_recursive_visit and squashes the following commits:
      
      6b2c615 [Liang-Chi Hsieh] change function name; comply with code style.
      5f072a7 [Liang-Chi Hsieh] add comments to explain Stack usage.
      8742dbb [Liang-Chi Hsieh] comply with code style.
      900538b [Liang-Chi Hsieh] change recursive visiting on rdd's dependencies to iterative approach to avoid stackoverflowerror.
      baf9ce1a
    • Aaron Staple's avatar
      [SPARK-695] In DAGScheduler's getPreferredLocs, track set of visited partitions. · eb5bdcaf
      Aaron Staple authored
      getPreferredLocs traverses a dependency graph of partitions using depth first search.  Given a complex dependency graph, the old implementation may explore a set of paths in the graph that is exponential in the number of nodes.  By maintaining a set of visited nodes the new implementation avoids revisiting nodes, preventing exponential blowup.
      
      Some comment and whitespace cleanups are also included.
      
      Author: Aaron Staple <aaron.staple@gmail.com>
      
      Closes #1362 from staple/SPARK-695 and squashes the following commits:
      
      ecea0f3 [Aaron Staple] address review comments
      751c661 [Aaron Staple] [SPARK-695] Add a unit test.
      5adf326 [Aaron Staple] Replace getPreferredLocsInternal's HashMap argument with a simpler HashSet.
      58e37d0 [Aaron Staple] Replace comment documenting NarrowDependency.
      6751ced [Aaron Staple] Revert "Remove unused variable."
      04c7097 [Aaron Staple] Fix indentation.
      0030884 [Aaron Staple] Remove unused variable.
      33f67c6 [Aaron Staple] Clarify comment.
      4e42b46 [Aaron Staple] Remove apparently incorrect comment describing NarrowDependency.
      65c2d3d [Aaron Staple] [SPARK-695] In DAGScheduler's getPreferredLocs, track set of visited partitions.
      eb5bdcaf
    • CrazyJvm's avatar
      [SQL] Documentation: Explain cacheTable command · c82fe478
      CrazyJvm authored
      add the `cacheTable` specification
      
      Author: CrazyJvm <crazyjvm@gmail.com>
      
      Closes #1681 from CrazyJvm/sql-programming-guide-cache and squashes the following commits:
      
      0a231e0 [CrazyJvm] grammar fixes
      a04020e [CrazyJvm] modify title to Cached tables
      18b6594 [CrazyJvm] fix format
      2cbbf58 [CrazyJvm] add cacheTable guide
      c82fe478
    • Cheng Hao's avatar
      [SPARK-2767] [SQL] SparkSQL CLI doens't output error message if query failed. · c0b47bad
      Cheng Hao authored
      Author: Cheng Hao <hao.cheng@intel.com>
      
      Closes #1686 from chenghao-intel/spark_sql_cli and squashes the following commits:
      
      eb664cc [Cheng Hao] Output detailed failure message in console
      93b0382 [Cheng Hao] Fix Bug of no output in cli if exception thrown internally
      c0b47bad
    • chutium's avatar
      [SPARK-2729] [SQL] Forgot to match Timestamp type in ColumnBuilder · 580c7011
      chutium authored
      just a match forgot, found after SPARK-2710 , TimestampType can be used by a SchemaRDD generated from JDBC ResultSet
      
      Author: chutium <teng.qiu@gmail.com>
      
      Closes #1636 from chutium/SPARK-2729 and squashes the following commits:
      
      71af77a [chutium] [SPARK-2729] [SQL] added Timestamp in NullableColumnAccessorSuite
      39cf9f8 [chutium] [SPARK-2729] add Timestamp Type into ColumnBuilder TestSuite, ref. #1636
      ab6ff97 [chutium] [SPARK-2729] Forgot to match Timestamp type in ColumnBuilder
      580c7011
    • Cheng Hao's avatar
      [SQL][SPARK-2212]Hash Outer Join · 4415722e
      Cheng Hao authored
      This patch is to support the hash based outer join. Currently, outer join for big relations are resort to `BoradcastNestedLoopJoin`, which is super slow. This PR will create 2 hash tables for both relations in the same partition, which greatly reduce the table scans.
      
      Here is the testing code that I used:
      ```
      package org.apache.spark.sql.hive
      
      import org.apache.spark.SparkContext
      import org.apache.spark.SparkConf
      import org.apache.spark.sql._
      
      case class Record(key: String, value: String)
      
      object JoinTablePrepare extends App {
        import TestHive2._
      
        val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"${i % 828193}", s"val_$i")))
      
        runSqlHive("SHOW TABLES")
        runSqlHive("DROP TABLE if exists a")
        runSqlHive("DROP TABLE if exists b")
        runSqlHive("DROP TABLE if exists result")
        rdd.registerAsTable("records")
      
        runSqlHive("""CREATE TABLE a (key STRING, value STRING)
                       | ROW FORMAT SERDE
                       | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'
                       | STORED AS RCFILE
                     """.stripMargin)
        runSqlHive("""CREATE TABLE b (key STRING, value STRING)
                       | ROW FORMAT SERDE
                       | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'
                       | STORED AS RCFILE
                     """.stripMargin)
        runSqlHive("""CREATE TABLE result (key STRING, value STRING)
                       | ROW FORMAT SERDE
                       | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'
                       | STORED AS RCFILE
                     """.stripMargin)
      
        hql(s"""from records
                   | insert into table a
                   | select key, value
                 """.stripMargin)
        hql(s"""from records
                   | insert into table b select key + 100000, value
                 """.stripMargin)
      }
      
      object JoinTablePerformanceTest extends App {
        import TestHive2._
      
        hql("SHOW TABLES")
        hql("set spark.sql.shuffle.partitions=20")
      
        val leftOuterJoin = "insert overwrite table result select a.key, b.value from a left outer join b on a.key=b.key"
        val rightOuterJoin = "insert overwrite table result select a.key, b.value from a right outer join b on a.key=b.key"
        val fullOuterJoin = "insert overwrite table result select a.key, b.value from a full outer join b on a.key=b.key"
      
        val results = ("LeftOuterJoin", benchmark(leftOuterJoin)) :: ("LeftOuterJoin", benchmark(leftOuterJoin)) ::
                      ("RightOuterJoin", benchmark(rightOuterJoin)) :: ("RightOuterJoin", benchmark(rightOuterJoin)) ::
                      ("FullOuterJoin", benchmark(fullOuterJoin)) :: ("FullOuterJoin", benchmark(fullOuterJoin)) :: Nil
        val explains = hql(s"explain $leftOuterJoin").collect ++ hql(s"explain $rightOuterJoin").collect ++ hql(s"explain $fullOuterJoin").collect
        println(explains.mkString(",\n"))
        results.foreach { case (prompt, result) => {
            println(s"$prompt: took ${result._1} ms (${result._2} records)")
          }
        }
      
        def benchmark(cmd: String) = {
          val begin = System.currentTimeMillis()
          val result = hql(cmd)
          val end = System.currentTimeMillis()
          val count = hql("select count(1) from result").collect.mkString("")
          ((end - begin), count)
        }
      }
      ```
      And the result as shown below:
      ```
      [Physical execution plan:],
      [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
      [ Project [key#95,value#98]],
      [  HashOuterJoin [key#95], [key#97], LeftOuter, None],
      [   Exchange (HashPartitioning [key#95], 20)],
      [    HiveTableScan [key#95], (MetastoreRelation default, a, None), None],
      [   Exchange (HashPartitioning [key#97], 20)],
      [    HiveTableScan [key#97,value#98], (MetastoreRelation default, b, None), None],
      [Physical execution plan:],
      [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
      [ Project [key#102,value#105]],
      [  HashOuterJoin [key#102], [key#104], RightOuter, None],
      [   Exchange (HashPartitioning [key#102], 20)],
      [    HiveTableScan [key#102], (MetastoreRelation default, a, None), None],
      [   Exchange (HashPartitioning [key#104], 20)],
      [    HiveTableScan [key#104,value#105], (MetastoreRelation default, b, None), None],
      [Physical execution plan:],
      [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
      [ Project [key#109,value#112]],
      [  HashOuterJoin [key#109], [key#111], FullOuter, None],
      [   Exchange (HashPartitioning [key#109], 20)],
      [    HiveTableScan [key#109], (MetastoreRelation default, a, None), None],
      [   Exchange (HashPartitioning [key#111], 20)],
      [    HiveTableScan [key#111,value#112], (MetastoreRelation default, b, None), None]
      LeftOuterJoin: took 16072 ms ([3000000] records)
      LeftOuterJoin: took 14394 ms ([3000000] records)
      RightOuterJoin: took 14802 ms ([3000000] records)
      RightOuterJoin: took 14747 ms ([3000000] records)
      FullOuterJoin: took 17715 ms ([6000000] records)
      FullOuterJoin: took 17629 ms ([6000000] records)
      ```
      
      Without this PR, the benchmark will run seems never end.
      
      Author: Cheng Hao <hao.cheng@intel.com>
      
      Closes #1147 from chenghao-intel/hash_based_outer_join and squashes the following commits:
      
      65c599e [Cheng Hao] Fix issues with the community comments
      72b1394 [Cheng Hao] Fix bug of stale value in joinedRow
      55baef7 [Cheng Hao] Add HashOuterJoin
      4415722e
    • Yin Huai's avatar
      [SPARK-2179][SQL] A minor refactoring Java data type APIs (2179 follow-up). · c41fdf04
      Yin Huai authored
      It is a follow-up PR of SPARK-2179 (https://issues.apache.org/jira/browse/SPARK-2179). It makes package names of data type APIs more consistent across languages (Scala: `org.apache.spark.sql`, Java: `org.apache.spark.sql.api.java`, Python: `pyspark.sql`).
      
      Author: Yin Huai <huai@cse.ohio-state.edu>
      
      Closes #1712 from yhuai/javaDataType and squashes the following commits:
      
      62eb705 [Yin Huai] Move package-info.
      add4bcb [Yin Huai] Make the package names of data type classes consistent across languages by moving all Java data type classes to package sql.api.java.
      c41fdf04
    • Sandy Ryza's avatar
      SPARK-2099. Report progress while task is running. · 8d338f64
      Sandy Ryza authored
      This is a sketch of a patch that allows the UI to show metrics for tasks that have not yet completed.  It adds a heartbeat every 2 seconds from the executors to the driver, reporting metrics for all of the executor's tasks.
      
      It still needs unit tests, polish, and cluster testing, but I wanted to put it up to get feedback on the approach.
      
      Author: Sandy Ryza <sandy@cloudera.com>
      
      Closes #1056 from sryza/sandy-spark-2099 and squashes the following commits:
      
      93b9fdb [Sandy Ryza] Up heartbeat interval to 10 seconds and other tidying
      132aec7 [Sandy Ryza] Heartbeat and HeartbeatResponse are already Serializable as case classes
      38dffde [Sandy Ryza] Additional review feedback and restore test that was removed in BlockManagerSuite
      51fa396 [Sandy Ryza] Remove hostname race, add better comments about threading, and some stylistic improvements
      3084f10 [Sandy Ryza] Make TaskUIData a case class again
      3bda974 [Sandy Ryza] Stylistic fixes
      0dae734 [Sandy Ryza] SPARK-2099. Report progress while task is running.
      8d338f64
    • Xiangrui Meng's avatar
      [HOTFIX] downgrade breeze version to 0.7 · 5328c0aa
      Xiangrui Meng authored
      breeze-0.8.1 causes dependency issues, as discussed in #940 .
      
      Author: Xiangrui Meng <meng@databricks.com>
      
      Closes #1718 from mengxr/revert-breeze and squashes the following commits:
      
      99c4681 [Xiangrui Meng] downgrade breeze version to 0.7
      5328c0aa
    • witgo's avatar
      [SPARK-1997] update breeze to version 0.8.1 · 0dacb1ad
      witgo authored
      `breeze 0.8.1`  dependent on  `scala-logging-slf4j 2.1.1` The relevant code on #1369
      
      Author: witgo <witgo@qq.com>
      
      Closes #940 from witgo/breeze-8.0.1 and squashes the following commits:
      
      65cc65e [witgo] update breeze  to version 0.8.1
      0dacb1ad
    • Sean Owen's avatar
      SPARK-2768 [MLLIB] Add product, user recommend method to MatrixFactorizationModel · 82d209d4
      Sean Owen authored
      Right now, `MatrixFactorizationModel` can only predict a score for one or more `(user,product)` tuples. As a comment in the file notes, it would be more useful to expose a recommend method, that computes top N scoring products for a user (or vice versa – users for a product).
      
      (This also corrects some long lines in the Java ALS test suite.)
      
      As you can see, it's a little messy to access the class from Java. Should there be a Java-friendly wrapper for it? with a pointer about where that should go, I could add that.
      
      Author: Sean Owen <srowen@gmail.com>
      
      Closes #1687 from srowen/SPARK-2768 and squashes the following commits:
      
      b349675 [Sean Owen] Additional review changes
      c9edb04 [Sean Owen] Updates from code review
      7bc35f9 [Sean Owen] Add recommend methods to MatrixFactorizationModel
      82d209d4
    • jerryshao's avatar
      [SPARK-2103][Streaming] Change to ClassTag for KafkaInputDStream and fix reflection issue · a32f0fb7
      jerryshao authored
      This PR updates previous Manifest for KafkaInputDStream's Decoder to ClassTag, also fix the problem addressed in [SPARK-2103](https://issues.apache.org/jira/browse/SPARK-2103).
      
      Previous Java interface cannot actually get the type of Decoder, so when using this Manifest to reconstruct the decode object will meet reflection exception.
      
      Also for other two Java interfaces, ClassTag[String] is useless because calling Scala API will get the right implicit ClassTag.
      
      Current Kafka unit test cannot actually verify the interface. I've tested these interfaces in my local and distribute settings.
      
      Author: jerryshao <saisai.shao@intel.com>
      
      Closes #1508 from jerryshao/SPARK-2103 and squashes the following commits:
      
      e90c37b [jerryshao] Add Mima excludes
      7529810 [jerryshao] Change Manifest to ClassTag for KafkaInputDStream's Decoder and fix Decoder construct issue when using Java API
      a32f0fb7
    • Ye Xianjin's avatar
      [Spark 2557] fix LOCAL_N_REGEX in createTaskScheduler and make local-n and... · 284771ef
      Ye Xianjin authored
      [Spark 2557] fix LOCAL_N_REGEX in createTaskScheduler and make local-n and local-n-failures consistent
      
      [SPARK-2557](https://issues.apache.org/jira/browse/SPARK-2557)
      
      Author: Ye Xianjin <advancedxy@gmail.com>
      
      Closes #1464 from advancedxy/SPARK-2557 and squashes the following commits:
      
      d844d67 [Ye Xianjin] add local-*-n-failures, bad-local-n, bad-local-n-failures test case
      3bbc668 [Ye Xianjin] fix LOCAL_N_REGEX regular expression and make local_n_failures accept * as all cores on the computer
      284771ef
    • Rahul Singhal's avatar
      SPARK-2134: Report metrics before application finishes · f1957e11
      Rahul Singhal authored
      Author: Rahul Singhal <rahul.singhal@guavus.com>
      
      Closes #1076 from rahulsinghaliitd/SPARK-2134 and squashes the following commits:
      
      15f18b6 [Rahul Singhal] SPARK-2134: Report metrics before application finishes
      f1957e11
    • Matei Zaharia's avatar
      SPARK-983. Support external sorting in sortByKey() · 72e33699
      Matei Zaharia authored
      This patch simply uses the ExternalSorter class from sort-based shuffle.
      
      Closes #931 and Closes #1090
      
      Author: Matei Zaharia <matei@databricks.com>
      
      Closes #1677 from mateiz/spark-983 and squashes the following commits:
      
      96b3fda [Matei Zaharia] SPARK-983. Support external sorting in sortByKey()
      72e33699
    • Kousuke Saruta's avatar
      [SPARK-2670] FetchFailedException should be thrown when local fetch has failed · 8ff4417f
      Kousuke Saruta authored
      Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
      
      Closes #1578 from sarutak/SPARK-2670 and squashes the following commits:
      
      85c8938 [Kousuke Saruta] Removed useless results.put for fail fast
      e8713cc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670
      d353984 [Kousuke Saruta] Refined assertion messages in BlockFetcherIteratorSuite.scala
      03bcb02 [Kousuke Saruta] Merge branch 'SPARK-2670' of github.com:sarutak/spark into SPARK-2670
      5d05855 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670
      4fca130 [Kousuke Saruta] Added test cases for BasicBlockFetcherIterator
      b7b8250 [Kousuke Saruta] Modified BasicBlockFetchIterator to fail fast when local fetch error has been occurred
      a3a9be1 [Kousuke Saruta] Modified BlockFetcherIterator for SPARK-2670
      460dc01 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670
      e310c0b [Kousuke Saruta] Modified BlockFetcherIterator to handle local fetch failure as fatch fail
      8ff4417f
    • Sandy Ryza's avatar
      SPARK-2738. Remove redundant imports in BlockManagerSuite · cb9e7d5a
      Sandy Ryza authored
      Author: Sandy Ryza <sandy@cloudera.com>
      
      Closes #1642 from sryza/sandy-spark-2738 and squashes the following commits:
      
      a923e4e [Sandy Ryza] SPARK-2738. Remove redundant imports in BlockManagerSuite
      cb9e7d5a
    • Prashant Sharma's avatar
      SPARK-2632, SPARK-2576. Fixed by only importing what is necessary during class definition. · 14991011
      Prashant Sharma authored
      Without this patch, it imports everything available in the scope.
      
      ```scala
      
      scala> val a = 10l
      val a = 10l
      a: Long = 10
      
      scala> import a._
      import a._
      import a._
      
      scala> case class A(a: Int) // show
      case class A(a: Int) // show
      class $read extends Serializable {
        def <init>() = {
          super.<init>;
          ()
        };
        class $iwC extends Serializable {
          def <init>() = {
            super.<init>;
            ()
          };
          class $iwC extends Serializable {
            def <init>() = {
              super.<init>;
              ()
            };
            import org.apache.spark.SparkContext._;
            class $iwC extends Serializable {
              def <init>() = {
                super.<init>;
                ()
              };
              val $VAL5 = $line5.$read.INSTANCE;
              import $VAL5.$iw.$iw.$iw.$iw.a;
              class $iwC extends Serializable {
                def <init>() = {
                  super.<init>;
                  ()
                };
                import a._;
                class $iwC extends Serializable {
                  def <init>() = {
                    super.<init>;
                    ()
                  };
                  class $iwC extends Serializable {
                    def <init>() = {
                      super.<init>;
                      ()
                    };
                    case class A extends scala.Product with scala.Serializable {
                      <caseaccessor> <paramaccessor> val a: Int = _;
                      def <init>(a: Int) = {
                        super.<init>;
                        ()
                      }
                    }
                  };
                  val $iw = new $iwC.<init>
                };
                val $iw = new $iwC.<init>
              };
              val $iw = new $iwC.<init>
            };
            val $iw = new $iwC.<init>
          };
          val $iw = new $iwC.<init>
        };
        val $iw = new $iwC.<init>
      }
      object $read extends scala.AnyRef {
        def <init>() = {
          super.<init>;
          ()
        };
        val INSTANCE = new $read.<init>
      }
      defined class A
      ```
      
      With this patch, it just imports  only the necessary.
      
      ```scala
      
      scala> val a = 10l
      val a = 10l
      a: Long = 10
      
      scala> import a._
      import a._
      import a._
      
      scala> case class A(a: Int) // show
      case class A(a: Int) // show
      class $read extends Serializable {
        def <init>() = {
          super.<init>;
          ()
        };
        class $iwC extends Serializable {
          def <init>() = {
            super.<init>;
            ()
          };
          class $iwC extends Serializable {
            def <init>() = {
              super.<init>;
              ()
            };
            case class A extends scala.Product with scala.Serializable {
              <caseaccessor> <paramaccessor> val a: Int = _;
              def <init>(a: Int) = {
                super.<init>;
                ()
              }
            }
          };
          val $iw = new $iwC.<init>
        };
        val $iw = new $iwC.<init>
      }
      object $read extends scala.AnyRef {
        def <init>() = {
          super.<init>;
          ()
        };
        val INSTANCE = new $read.<init>
      }
      defined class A
      
      scala>
      
      ```
      
      This patch also adds a `:fallback` mode on being enabled it will restore the spark-shell's 1.0.0 behaviour.
      
      Author: Prashant Sharma <scrapcodes@gmail.com>
      Author: Yin Huai <huai@cse.ohio-state.edu>
      Author: Prashant Sharma <prashant.s@imaginea.com>
      
      Closes #1635 from ScrapCodes/repl-fix-necessary-imports and squashes the following commits:
      
      b1968d2 [Prashant Sharma] Added toschemaRDD to test case.
      0b712bb [Yin Huai] Add a REPL test to test importing a method.
      02ad8ff [Yin Huai] Add a REPL test for importing SQLContext.createSchemaRDD.
      ed6d0c7 [Prashant Sharma] Added a fallback mode, incase users run into issues while using repl.
      b63d3b2 [Prashant Sharma] SPARK-2632, SPARK-2576. Fixed by only importing what is necessary during class definition.
      14991011
    • Haoyuan Li's avatar
      [SPARK-2702][Core] Upgrade Tachyon dependency to 0.5.0 · 2cdc3e5c
      Haoyuan Li authored
      Author: Haoyuan Li <haoyuan@cs.berkeley.edu>
      
      Closes #1651 from haoyuan/upgrade-tachyon and squashes the following commits:
      
      6f3f98f [Haoyuan Li] upgrade tachyon to 0.5.0
      2cdc3e5c
  2. Jul 31, 2014
    • Doris Xin's avatar
      [SPARK-2782][mllib] Bug fix for getRanks in SpearmanCorrelation · c4755403
      Doris Xin authored
      getRanks computes the wrong rank when numPartition >= size in the input RDDs before this patch. added units to address this bug.
      
      Author: Doris Xin <doris.s.xin@gmail.com>
      
      Closes #1710 from dorx/correlationBug and squashes the following commits:
      
      733def4 [Doris Xin] bugs and reviewer comments.
      31db920 [Doris Xin] revert unnecessary change
      043ff83 [Doris Xin] bug fix for spearman corner case
      c4755403
    • Xiangrui Meng's avatar
      [SPARK-2777][MLLIB] change ALS factors storage level to MEMORY_AND_DISK · b1900832
      Xiangrui Meng authored
      Now the factors are persisted in memory only. If they get kicked off by later jobs, we might have to start the computation from very beginning. A better solution is changing the storage level to `MEMORY_AND_DISK`.
      
      srowen
      
      Author: Xiangrui Meng <meng@databricks.com>
      
      Closes #1700 from mengxr/als-level and squashes the following commits:
      
      c103d76 [Xiangrui Meng] change ALS factors storage level to MEMORY_AND_DISK
      b1900832
    • GuoQiang Li's avatar
      SPARK-2766: ScalaReflectionSuite throw an llegalArgumentException in JDK 6 · 9998efab
      GuoQiang Li authored
      Author: GuoQiang Li <witgo@qq.com>
      
      Closes #1683 from witgo/SPARK-2766 and squashes the following commits:
      
      d0db00c [GuoQiang Li] ScalaReflectionSuite  throw an llegalArgumentException in JDK 6
      9998efab
    • Yin Huai's avatar
      [SPARK-2779] [SQL] asInstanceOf[Map[...]] should use scala.collection.Map... · 9632719c
      Yin Huai authored
      [SPARK-2779] [SQL] asInstanceOf[Map[...]] should use scala.collection.Map instead of scala.collection.immutable.Map
      
      Since we let users create Rows. It makes sense to accept mutable Maps as values of MapType columns.
      
      JIRA: https://issues.apache.org/jira/browse/SPARK-2779
      
      Author: Yin Huai <huai@cse.ohio-state.edu>
      
      Closes #1705 from yhuai/SPARK-2779 and squashes the following commits:
      
      00d72fd [Yin Huai] Use scala.collection.Map.
      9632719c
    • Joseph K. Bradley's avatar
      [SPARK-2756] [mllib] Decision tree bug fixes · b124de58
      Joseph K. Bradley authored
      (1) Inconsistent aggregate (agg) indexing for unordered features.
      (2) Fixed gain calculations for edge cases.
      (3) One-off error in choosing thresholds for continuous features for small datasets.
      (4) (not a bug) Changed meaning of tree depth by 1 to fit scikit-learn and rpart. (Depth 1 used to mean 1 leaf node; depth 0 now means 1 leaf node.)
      
      Other updates, to help with tests:
      * Updated DecisionTreeRunner to print more info.
      * Added utility functions to DecisionTreeModel: toString, depth, numNodes
      * Improved internal DecisionTree documentation
      
      Bug fix details:
      
      (1) Indexing was inconsistent for aggregate calculations for unordered features (in multiclass classification with categorical features, where the features had few enough values such that they could be considered unordered, i.e., isSpaceSufficientForAllCategoricalSplits=true).
      
      * updateBinForUnorderedFeature indexed agg as (node, feature, featureValue, binIndex), where
      ** featureValue was from arr (so it was a feature value)
      ** binIndex was in [0,…, 2^(maxFeatureValue-1)-1)
      * The rest of the code indexed agg as (node, feature, binIndex, label).
      * Corrected this bug by changing updateBinForUnorderedFeature to use the second indexing pattern.
      
      Unit tests in DecisionTreeSuite
      * Updated a few tests to train a model and test its training accuracy, which catches the indexing bug from updateBinForUnorderedFeature() discussed above.
      * Added new test (“stump with categorical variables for multiclass classification, with just enough bins”) to test bin extremes.
      
      (2) Bug fix: calculateGainForSplit (for classification):
      * It used to return dummy prediction values when either the right or left children had 0 weight.  These were incorrect for multiclass classification.  It has been corrected.
      
      Updated impurities to allow for count = 0.  This was related to the above bug fix for calculateGainForSplit (for classification).
      
      Small updates to documentation and coding style.
      
      (3) Bug fix: Off-by-1 when finding thresholds for splits for continuous features.
      
      * Exhibited bug in new test in DecisionTreeSuite: “stump with 1 continuous variable for binary classification, to check off-by-1 error”
      * Description: When finding thresholds for possible splits for continuous features in DecisionTree.findSplitsBins, the thresholds were set according to individual training examples’ feature values.
      * Fix: The threshold is set to be the average of 2 consecutive (sorted) examples’ feature values.  E.g.: If the old code set the threshold using example i, the new code sets the threshold using exam
      * Note: In 4 DecisionTreeSuite tests with all labels identical, removed check of threshold since it is somewhat arbitrary.
      
      CC: mengxr manishamde  Please let me know if I missed something!
      
      Author: Joseph K. Bradley <joseph.kurata.bradley@gmail.com>
      
      Closes #1673 from jkbradley/decisiontree-bugfix and squashes the following commits:
      
      2b20c61 [Joseph K. Bradley] Small doc and style updates
      dab0b67 [Joseph K. Bradley] Added documentation for DecisionTree internals
      8bb8aa0 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix
      978cfcf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix
      6eed482 [Joseph K. Bradley] In DecisionTree: Changed from using procedural syntax for functions returning Unit to explicitly writing Unit return type.
      376dca2 [Joseph K. Bradley] Updated meaning of maxDepth by 1 to fit scikit-learn and rpart. * In code, replaced usages of maxDepth <-- maxDepth + 1 * In params, replace settings of maxDepth <-- maxDepth - 1
      59750f8 [Joseph K. Bradley] * Updated Strategy to check numClassesForClassification only if algo=Classification. * Updates based on comments: ** DecisionTreeRunner *** Made dataFormat arg default to libsvm ** Small cleanups ** tree.Node: Made recursive helper methods private, and renamed them.
      52e17c5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix
      da50db7 [Joseph K. Bradley] Added one more test to DecisionTreeSuite: stump with 2 continuous variables for binary classification.  Caused problems in past, but fixed now.
      8ea8750 [Joseph K. Bradley] Bug fix: Off-by-1 when finding thresholds for splits for continuous features.
      2283df8 [Joseph K. Bradley] 2 bug fixes.
      73fbea2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix
      5f920a1 [Joseph K. Bradley] Demonstration of bug before submitting fix: Updated DecisionTreeSuite so that 3 tests fail.  Will describe bug in next commit.
      b124de58
    • Doris Xin's avatar
      [SPARK-2724] Python version of RandomRDDGenerators · d8430148
      Doris Xin authored
      RandomRDDGenerators but without support for randomRDD and randomVectorRDD, which take in arbitrary DistributionGenerator.
      
      `randomRDD.py` is named to avoid collision with the built-in Python `random` package.
      
      Author: Doris Xin <doris.s.xin@gmail.com>
      
      Closes #1628 from dorx/pythonRDD and squashes the following commits:
      
      55c6de8 [Doris Xin] review comments. all python units passed.
      f831d9b [Doris Xin] moved default args logic into PythonMLLibAPI
      2d73917 [Doris Xin] fix for linalg.py
      8663e6a [Doris Xin] reverting back to a single python file for random
      f47c481 [Doris Xin] docs update
      687aac0 [Doris Xin] add RandomRDDGenerators.py to run-tests
      4338f40 [Doris Xin] renamed randomRDD to rand and import as random
      29d205e [Doris Xin] created mllib.random package
      bd2df13 [Doris Xin] typos
      07ddff2 [Doris Xin] units passed.
      23b2ecd [Doris Xin] WIP
      d8430148
    • Zongheng Yang's avatar
      [SPARK-2531 & SPARK-2436] [SQL] Optimize the BuildSide when planning BroadcastNestedLoopJoin. · 8f51491e
      Zongheng Yang authored
      This PR resolves the following two tickets:
      
      - [SPARK-2531](https://issues.apache.org/jira/browse/SPARK-2531): BNLJ currently assumes the build side is the right relation. This patch refactors some of its logic to take into account a BuildSide properly.
      - [SPARK-2436](https://issues.apache.org/jira/browse/SPARK-2436): building on top of the above, we simply use the physical size statistics (if available) of both relations, and make the smaller relation the build side in the planner.
      
      Author: Zongheng Yang <zongheng.y@gmail.com>
      
      Closes #1448 from concretevitamin/bnlj-buildSide and squashes the following commits:
      
      1780351 [Zongheng Yang] Use size estimation to decide optimal build side of BNLJ.
      68e6c5b [Zongheng Yang] Consolidate two adjacent pattern matchings.
      96d312a [Zongheng Yang] Use a while loop instead of collection methods chaining.
      4bc525e [Zongheng Yang] Make BroadcastNestedLoopJoin take a BuildSide.
      8f51491e
    • Aaron Davidson's avatar
      SPARK-2282: Reuse Socket for sending accumulator updates to Pyspark · ef4ff00f
      Aaron Davidson authored
      Prior to this change, every PySpark task completion opened a new socket to the accumulator server, passed its updates through, and then quit. I'm not entirely sure why PySpark always sends accumulator updates, but regardless this causes a very rapid buildup of ephemeral TCP connections that remain in the TCP_WAIT state for around a minute before being cleaned up.
      
      Rather than trying to allow these sockets to be cleaned up faster, this patch simply reuses the connection between tasks completions (since they're fed updates in a single-threaded manner by the DAGScheduler anyway).
      
      The only tricky part here was making sure that the AccumulatorServer was able to shutdown in a timely manner (i.e., stop polling for new data), and this was accomplished via minor feats of magic.
      
      I have confirmed that this patch eliminates the buildup of ephemeral sockets due to the accumulator updates. However, I did note that there were still significant sockets being created against the PySpark daemon port, but my machine was not able to create enough sockets fast enough to fail. This may not be the last time we've seen this issue, though.
      
      Author: Aaron Davidson <aaron@databricks.com>
      
      Closes #1503 from aarondav/accum and squashes the following commits:
      
      b3e12f7 [Aaron Davidson] SPARK-2282: Reuse Socket for sending accumulator updates to Pyspark
      ef4ff00f
    • Rui Li's avatar
      SPARK-2740: allow user to specify ascending and numPartitions for sortBy... · 492a195c
      Rui Li authored
      It should be more convenient if user can specify ascending and numPartitions when calling sortByKey.
      
      Author: Rui Li <rui.li@intel.com>
      
      Closes #1645 from lirui-intel/spark-2740 and squashes the following commits:
      
      fb5d52e [Rui Li] SPARK-2740: allow user to specify ascending and numPartitions for sortByKey
      492a195c
    • kballou's avatar
      Docs: monitoring, streaming programming guide · cc820502
      kballou authored
      Fix several awkward wordings and grammatical issues in the following
      documents:
      
      *   docs/monitoring.md
      
      *   docs/streaming-programming-guide.md
      
      Author: kballou <kballou@devnulllabs.io>
      
      Closes #1662 from kennyballou/grammar_fixes and squashes the following commits:
      
      e1b8ad6 [kballou] Docs: monitoring, streaming programming guide
      cc820502
    • Josh Rosen's avatar
      Improvements to merge_spark_pr.py · e0213621
      Josh Rosen authored
      This commit fixes a couple of issues in the merge_spark_pr.py developer script:
      
      - Allow recovery from failed cherry-picks.
      - Fix detection of pull requests that have already been merged.
      
      Both of these fixes are useful when backporting changes.
      
      Author: Josh Rosen <joshrosen@apache.org>
      
      Closes #1668 from JoshRosen/pr-script-improvements and squashes the following commits:
      
      ff4f33a [Josh Rosen] Default SPARK_HOME to cwd(); detect missing JIRA credentials.
      ed5bc57 [Josh Rosen] Improvements for backporting using merge_spark_pr:
      e0213621
    • Yin Huai's avatar
      [SPARK-2523] [SQL] Hadoop table scan bug fixing (fix failing Jenkins maven test) · 49b36129
      Yin Huai authored
      This PR tries to resolve the broken Jenkins maven test issue introduced by #1439. Now, we create a single query test to run both the setup work and the test query.
      
      Author: Yin Huai <huai@cse.ohio-state.edu>
      
      Closes #1669 from yhuai/SPARK-2523-fixTest and squashes the following commits:
      
      358af1a [Yin Huai] Make partition_based_table_scan_with_different_serde run atomically.
      49b36129
    • Xiangrui Meng's avatar
      [SPARK-2511][MLLIB] add HashingTF and IDF · dc0865bc
      Xiangrui Meng authored
      This is roughly the TF-IDF implementation used in the Databricks Cloud Demo: http://databricks.com/cloud/ .
      
      Both `HashingTF` and `IDF` are implemented as transformers, similar to scikit-learn.
      
      Author: Xiangrui Meng <meng@databricks.com>
      
      Closes #1671 from mengxr/tfidf and squashes the following commits:
      
      7d65888 [Xiangrui Meng] use JavaConverters._
      5fe9ec4 [Xiangrui Meng] fix unit test
      6e214ec [Xiangrui Meng] add apache header
      cfd9aed [Xiangrui Meng] add Java-friendly methods move classes to mllib.feature
      3814440 [Xiangrui Meng] add HashingTF and IDF
      dc0865bc
    • Sean Owen's avatar
      SPARK-2646. log4j initialization not quite compatible with log4j 2.x · e5749a13
      Sean Owen authored
      The logging code that handles log4j initialization leads to an stack overflow error when used with log4j 2.x, which has just been released. This occurs even a downstream project has correctly adjusted SLF4J bindings, and that is the right thing to do for log4j 2.x, since it is effectively a separate project from 1.x.
      
      Here is the relevant bit of Logging.scala:
      
      ```
        private def initializeLogging() {
          // If Log4j is being used, but is not initialized, load a default properties file
          val binder = StaticLoggerBinder.getSingleton
          val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory")
          val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
          if (!log4jInitialized && usingLog4j) {
            val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
            Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
              case Some(url) =>
                PropertyConfigurator.configure(url)
                log.info(s"Using Spark's default log4j profile: $defaultLogProps")
              case None =>
                System.err.println(s"Spark was unable to load $defaultLogProps")
            }
          }
          Logging.initialized = true
      
          // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads
          // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
          log
        }
      ```
      
      The first minor issue is that there is a call to a logger inside this method, which is initializing logging. In this situation, it ends up causing the initialization to be called recursively until the stack overflow. It would be slightly tidier to log this only after Logging.initialized = true. Or not at all. But it's not the root problem, or else, it would not work at all now.
      
      The calls to log4j classes here always reference log4j 1.2 no matter what. For example, there is not getAllAppenders in log4j 2.x. That's fine. Really, "usingLog4j" means "using log4j 1.2" and "log4jInitialized" means "log4j 1.2 is initialized".
      
      usingLog4j should be false for log4j 2.x, because the initialization only matters for log4j 1.2. But, it's true, and that's the real issue. And log4jInitialized is always false, since calls to the log4j 1.2 API are stubs and no-ops in this setup, where the caller has swapped in log4j 2.x. Hence the loop.
      
      This is fixed, I believe, if "usingLog4j" can be false for log4j 2.x. The SLF4J static binding class has the same name for both versions, unfortunately, which causes the issue. However they're in different packages. For example, if the test included "... and begins with org.slf4j", it should work, as the SLF4J binding for log4j 2.x is provided by log4j 2.x at the moment, and is in package org.apache.logging.slf4j.
      
      Of course, I assume that SLF4J will eventually offer its own binding. I hope to goodness they at least name the binding class differently, or else this will again not work. But then some other check can probably be made.
      
      Author: Sean Owen <srowen@gmail.com>
      
      Closes #1547 from srowen/SPARK-2646 and squashes the following commits:
      
      92a9898 [Sean Owen] System.out -> System.err
      94be4c7 [Sean Owen] Add back log message as System.out, with informational comment
      a7f8876 [Sean Owen] Updates from review
      6f3c1d3 [Sean Owen] Remove log statement in logging initialization, and distinguish log4j 1.2 from 2.0, to avoid stack overflow in initialization
      e5749a13
    • Sean Owen's avatar
      SPARK-2749 [BUILD] Part 2. Fix a follow-on scalastyle error · 4dbabb39
      Sean Owen authored
      The test compile error is fixed, but the build still fails because of one scalastyle error.
      
      https://amplab.cs.berkeley.edu/jenkins/view/Spark/job/Spark-Master-Maven-pre-YARN/lastFailedBuild/hadoop.version=1.0.4,label=centos/console
      
      Author: Sean Owen <srowen@gmail.com>
      
      Closes #1690 from srowen/SPARK-2749 and squashes the following commits:
      
      1c9e7a6 [Sean Owen] Also: fix scalastyle error by wrapping a long line
      4dbabb39
    • Sandy Ryza's avatar
      SPARK-2664. Deal with `--conf` options in spark-submit that relate to fl... · f68105df
      Sandy Ryza authored
      ...ags
      
      Author: Sandy Ryza <sandy@cloudera.com>
      
      Closes #1665 from sryza/sandy-spark-2664 and squashes the following commits:
      
      0518c63 [Sandy Ryza] SPARK-2664. Deal with `--conf` options in spark-submit that relate to flags
      f68105df
    • Aaron Davidson's avatar
      SPARK-2028: Expose mapPartitionsWithInputSplit in HadoopRDD · f1933123
      Aaron Davidson authored
      This allows users to gain access to the InputSplit which backs each partition.
      
      An alternative solution would have been to have a .withInputSplit() method which returns a new RDD[(InputSplit, (K, V))], but this is confusing because you could not cache this RDD or shuffle it, as InputSplit is not inherently serializable.
      
      Author: Aaron Davidson <aaron@databricks.com>
      
      Closes #973 from aarondav/hadoop and squashes the following commits:
      
      9c9112b [Aaron Davidson] Add JavaAPISuite test
      9942cd7 [Aaron Davidson] Add Java API
      1284a3a [Aaron Davidson] SPARK-2028: Expose mapPartitionsWithInputSplit in HadoopRDD
      f1933123
    • Michael Armbrust's avatar
      [SPARK-2397][SQL] Deprecate LocalHiveContext · 72cfb139
      Michael Armbrust authored
      LocalHiveContext is redundant with HiveContext.  The only difference is it creates `./metastore` instead of `./metastore_db`.
      
      Author: Michael Armbrust <michael@databricks.com>
      
      Closes #1641 from marmbrus/localHiveContext and squashes the following commits:
      
      e5ec497 [Michael Armbrust] Add deprecation version
      626e056 [Michael Armbrust] Don't remove from imports yet
      905cc5f [Michael Armbrust] Merge remote-tracking branch 'apache/master' into localHiveContext
      1c2727e [Michael Armbrust] Deprecate LocalHiveContext
      72cfb139
Loading