Skip to content
Snippets Groups Projects
  1. Feb 04, 2016
    • Andrew Or's avatar
      [SPARK-13162] Standalone mode does not respect initial executors · 4120bcba
      Andrew Or authored
      Currently the Master would always set an application's initial executor limit to infinity. If the user specified `spark.dynamicAllocation.initialExecutors`, the config would not take effect. This is similar to #11047 but for standalone mode.
      
      Author: Andrew Or <andrew@databricks.com>
      
      Closes #11054 from andrewor14/standalone-da-initial.
      4120bcba
    • Holden Karau's avatar
      [SPARK-13164][CORE] Replace deprecated synchronized buffer in core · 62a7c283
      Holden Karau authored
      Building with scala 2.11 results in the warning trait SynchronizedBuffer in package mutable is deprecated: Synchronization via traits is deprecated as it is inherently unreliable. Consider java.util.concurrent.ConcurrentLinkedQueue as an alternative. Investigation shows we are already using ConcurrentLinkedQueue in other locations so switch our uses of SynchronizedBuffer to ConcurrentLinkedQueue.
      
      Author: Holden Karau <holden@us.ibm.com>
      
      Closes #11059 from holdenk/SPARK-13164-replace-deprecated-synchronized-buffer-in-core.
      62a7c283
    • Charles Allen's avatar
      [SPARK-12330][MESOS] Fix mesos coarse mode cleanup · 2eaeafe8
      Charles Allen authored
      In the current implementation the mesos coarse scheduler does not wait for the mesos tasks to complete before ending the driver. This causes a race where the task has to finish cleaning up before the mesos driver terminates it with a SIGINT (and SIGKILL after 3 seconds if the SIGINT doesn't work).
      
      This PR causes the mesos coarse scheduler to wait for the mesos tasks to finish (with a timeout defined by `spark.mesos.coarse.shutdown.ms`)
      
      This PR also fixes a regression caused by [SPARK-10987] whereby submitting a shutdown causes a race between the local shutdown procedure and the notification of the scheduler driver disconnection. If the scheduler driver disconnection wins the race, the coarse executor incorrectly exits with status 1 (instead of the proper status 0)
      
      With this patch the mesos coarse scheduler terminates properly, the executors clean up, and the tasks are reported as `FINISHED` in the Mesos console (as opposed to `KILLED` in < 1.6 or `FAILED` in 1.6 and later)
      
      Author: Charles Allen <charles@allen-net.com>
      
      Closes #10319 from drcrallen/SPARK-12330.
      2eaeafe8
    • Reynold Xin's avatar
      [SPARK-12828][SQL] Natural join follow-up · dee801ad
      Reynold Xin authored
      This is a small addendum to #10762 to make the code more robust again future changes.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #11070 from rxin/SPARK-12828-natural-join.
      dee801ad
    • Liang-Chi Hsieh's avatar
      [SPARK-13113] [CORE] Remove unnecessary bit operation when decoding page number · d3908714
      Liang-Chi Hsieh authored
      JIRA: https://issues.apache.org/jira/browse/SPARK-13113
      
      As we shift bits right, looks like the bitwise AND operation is unnecessary.
      
      Author: Liang-Chi Hsieh <viirya@gmail.com>
      
      Closes #11002 from viirya/improve-decodepagenumber.
      d3908714
  2. Feb 03, 2016
    • Yuhao Yang's avatar
      [ML][DOC] fix wrong api link in ml onevsrest · c2c956bc
      Yuhao Yang authored
      minor fix for api link in ml onevsrest
      
      Author: Yuhao Yang <hhbyyh@gmail.com>
      
      Closes #11068 from hhbyyh/onevsrestDoc.
      c2c956bc
    • Daoyuan Wang's avatar
      [SPARK-12828][SQL] add natural join support · 0f81318a
      Daoyuan Wang authored
      Jira:
      https://issues.apache.org/jira/browse/SPARK-12828
      
      Author: Daoyuan Wang <daoyuan.wang@intel.com>
      
      Closes #10762 from adrian-wang/naturaljoin.
      0f81318a
    • Andrew Or's avatar
      [SPARK-13079][SQL] Extend and implement InMemoryCatalog · a6483112
      Andrew Or authored
      This is a step towards consolidating `SQLContext` and `HiveContext`.
      
      This patch extends the existing Catalog API added in #10982 to include methods for handling table partitions. In particular, a partition is identified by `PartitionSpec`, which is just a `Map[String, String]`. The Catalog is still not used by anything yet, but its API is now more or less complete and an implementation is fully tested.
      
      About 200 lines are test code.
      
      Author: Andrew Or <andrew@databricks.com>
      
      Closes #11069 from andrewor14/catalog.
      a6483112
    • Holden Karau's avatar
      [SPARK-13152][CORE] Fix task metrics deprecation warning · a8e2ba77
      Holden Karau authored
      Make an internal non-deprecated version of incBytesRead and incRecordsRead so we don't have unecessary deprecation warnings in our build.
      
      Right now incBytesRead and incRecordsRead are marked as deprecated and for internal use only. We should make private[spark] versions which are not deprecated and switch to those internally so as to not clutter up the warning messages when building.
      
      cc andrewor14 who did the initial deprecation
      
      Author: Holden Karau <holden@us.ibm.com>
      
      Closes #11056 from holdenk/SPARK-13152-fix-task-metrics-deprecation-warnings.
      a8e2ba77
    • Davies Liu's avatar
      [SPARK-13131] [SQL] Use best and average time in benchmark · de091452
      Davies Liu authored
      Best time is stabler than average time, also added a column for nano seconds per row (which could be used to estimate contributions of each components in a query).
      
      Having best time and average time together for more information (we can see kind of variance).
      
      rate, time per row and relative are all calculated using best time.
      
      The result looks like this:
      ```
      Intel(R) Core(TM) i7-4558U CPU  2.80GHz
      rang/filter/sum:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
      -------------------------------------------------------------------------------------------
      rang/filter/sum codegen=false          14332 / 16646         36.0          27.8       1.0X
      rang/filter/sum codegen=true              845 /  940        620.0           1.6      17.0X
      ```
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #11018 from davies/gen_bench.
      de091452
    • Reynold Xin's avatar
      [SPARK-13166][SQL] Remove DataStreamReader/Writer · 915a7539
      Reynold Xin authored
      They seem redundant and we can simply use DataFrameReader/Writer. The new usage looks like:
      
      ```scala
      val df = sqlContext.read.stream("...")
      val handle = df.write.stream("...")
      handle.stop()
      ```
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #11062 from rxin/SPARK-13166.
      915a7539
    • Alex Bozarth's avatar
      [SPARK-3611][WEB UI] Show number of cores for each executor in application web UI · 3221eddb
      Alex Bozarth authored
      Added a Cores column in the Executors UI
      
      Author: Alex Bozarth <ajbozart@us.ibm.com>
      
      Closes #11039 from ajbozarth/spark3611.
      3221eddb
    • Herman van Hovell's avatar
      [SPARK-13157] [SQL] Support any kind of input for SQL commands. · 9dd2741e
      Herman van Hovell authored
      The ```SparkSqlLexer``` currently swallows characters which have not been defined in the grammar. This causes problems with SQL commands, such as: ```add jar file:///tmp/ab/TestUDTF.jar```. In this example the `````` is swallowed.
      
      This PR adds an extra Lexer rule to handle such input, and makes a tiny modification to the ```ASTNode```.
      
      cc davies liancheng
      
      Author: Herman van Hovell <hvanhovell@questtec.nl>
      
      Closes #11052 from hvanhovell/SPARK-13157.
      9dd2741e
    • Davies Liu's avatar
      [SPARK-12798] [SQL] generated BroadcastHashJoin · c4feec26
      Davies Liu authored
      A row from stream side could match multiple rows on build side, the loop for these matched rows should not be interrupted when emitting a row, so we buffer the output rows in a linked list, check the termination condition on producer loop (for example, Range or Aggregate).
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #10989 from davies/gen_join.
      c4feec26
    • Mario Briggs's avatar
      [SPARK-12739][STREAMING] Details of batch in Streaming tab uses two Duration columns · e9eb248e
      Mario Briggs authored
      I have clearly prefix the two 'Duration' columns in 'Details of Batch' Streaming tab as 'Output Op Duration' and 'Job Duration'
      
      Author: Mario Briggs <mario.briggs@in.ibm.com>
      Author: mariobriggs <mariobriggs@in.ibm.com>
      
      Closes #11022 from mariobriggs/spark-12739.
      e9eb248e
    • Sameer Agarwal's avatar
      [SPARK-12957][SQL] Initial support for constraint propagation in SparkSQL · 138c300f
      Sameer Agarwal authored
      Based on the semantics of a query, we can derive a number of data constraints on output of each (logical or physical) operator. For instance, if a filter defines `‘a > 10`, we know that the output data of this filter satisfies 2 constraints:
      
      1. `‘a > 10`
      2. `isNotNull(‘a)`
      
      This PR proposes a possible way of keeping track of these constraints and propagating them in the logical plan, which can then help us build more advanced optimizations (such as pruning redundant filters, optimizing joins, among others). We define constraints as a set of (implicitly conjunctive) expressions. For e.g., if a filter operator has constraints = `Set(‘a > 10, ‘b < 100)`, it’s implied that the outputs satisfy both individual constraints (i.e., `‘a > 10` AND `‘b < 100`).
      
      Design Document: https://docs.google.com/a/databricks.com/document/d/1WQRgDurUBV9Y6CWOBS75PQIqJwT-6WftVa18xzm7nCo/edit?usp=sharing
      
      Author: Sameer Agarwal <sameer@databricks.com>
      
      Closes #10844 from sameeragarwal/constraints.
      138c300f
    • Davies Liu's avatar
      [SPARK-13147] [SQL] improve readability of generated code · e86f8f63
      Davies Liu authored
      1. try to avoid the suffix (unique id)
      2. remove the comment if there is no code generated.
      3. re-arrange the order of functions
      4. trop the new line for inlined blocks.
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #11032 from davies/better_suffix.
      e86f8f63
  3. Feb 02, 2016
    • Shixiong Zhu's avatar
      [SPARK-7997][CORE] Add rpcEnv.awaitTermination() back to SparkEnv · 335f10ed
      Shixiong Zhu authored
      `rpcEnv.awaitTermination()` was not added in #10854 because some Streaming Python tests hung forever.
      
      This patch fixed the hung issue and added rpcEnv.awaitTermination() back to SparkEnv.
      
      Previously, Streaming Kafka Python tests shutdowns the zookeeper server before stopping StreamingContext. Then when stopping StreamingContext, KafkaReceiver may be hung due to https://issues.apache.org/jira/browse/KAFKA-601, hence, some thread of RpcEnv's Dispatcher cannot exit and rpcEnv.awaitTermination is hung.The patch just changed the shutdown order to fix it.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #11031 from zsxwing/awaitTermination.
      335f10ed
    • Imran Younus's avatar
      [SPARK-12732][ML] bug fix in linear regression train · 05571466
      Imran Younus authored
      Fixed the bug in linear regression train for the case when the target variable is constant. The two cases for `fitIntercept=true` or `fitIntercept=false` should be treated differently.
      
      Author: Imran Younus <iyounus@us.ibm.com>
      
      Closes #10702 from iyounus/SPARK-12732_bug_fix_in_linear_regression_train.
      05571466
    • Davies Liu's avatar
      [SPARK-12951] [SQL] support spilling in generated aggregate · 99a6e3c1
      Davies Liu authored
      This PR add spilling support for generated TungstenAggregate.
      
      If spilling happened, it's not that bad to do the iterator based sort-merge-aggregate (not generated).
      
      The changes will be covered by TungstenAggregationQueryWithControlledFallbackSuite
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #10998 from davies/gen_spilling.
      99a6e3c1
    • Adam Budde's avatar
      [SPARK-13122] Fix race condition in MemoryStore.unrollSafely() · ff71261b
      Adam Budde authored
      https://issues.apache.org/jira/browse/SPARK-13122
      
      A race condition can occur in MemoryStore's unrollSafely() method if two threads that
      return the same value for currentTaskAttemptId() execute this method concurrently. This
      change makes the operation of reading the initial amount of unroll memory used, performing
      the unroll, and updating the associated memory maps atomic in order to avoid this race
      condition.
      
      Initial proposed fix wraps all of unrollSafely() in a memoryManager.synchronized { } block. A cleaner approach might be introduce a mechanism that synchronizes based on task attempt ID. An alternative option might be to track unroll/pending unroll memory based on block ID rather than task attempt ID.
      
      Author: Adam Budde <budde@amazon.com>
      
      Closes #11012 from budde/master.
      ff71261b
    • Nong Li's avatar
      [SPARK-12992] [SQL] Update parquet reader to support more types when decoding to ColumnarBatch. · 21112e8a
      Nong Li authored
      This patch implements support for more types when doing the vectorized decode. There are
      a few more types remaining but they should be very straightforward after this. This code
      has a few copy and paste pieces but they are difficult to eliminate due to performance
      considerations.
      
      Specifically, this patch adds support for:
        - String, Long, Byte types
        - Dictionary encoding for those types.
      
      Author: Nong Li <nong@databricks.com>
      
      Closes #10908 from nongli/spark-12992.
      21112e8a
    • Wenchen Fan's avatar
      [SPARK-13020][SQL][TEST] fix random generator for map type · 672032d0
      Wenchen Fan authored
      when we generate map, we first randomly pick a length, then create a seq of key value pair with the expected length, and finally call `toMap`. However, `toMap` will remove all duplicated keys, which makes the actual map size much less than we expected.
      
      This PR fixes this problem by put keys in a set first, to guarantee we have enough keys to build a map with expected length.
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #10930 from cloud-fan/random-generator.
      672032d0
    • Davies Liu's avatar
      [SPARK-13150] [SQL] disable two flaky tests · 6de6a977
      Davies Liu authored
      Author: Davies Liu <davies@databricks.com>
      
      Closes #11037 from davies/disable_flaky.
      6de6a977
    • Kevin (Sangwoo) Kim's avatar
      [DOCS] Update StructType.scala · b377b035
      Kevin (Sangwoo) Kim authored
      The example will throw error like
      <console>:20: error: not found: value StructType
      
      Need to add this line:
      import org.apache.spark.sql.types._
      
      Author: Kevin (Sangwoo) Kim <sangwookim.me@gmail.com>
      
      Closes #10141 from swkimme/patch-1.
      b377b035
    • Gabriele Nizzoli's avatar
      [SPARK-13121][STREAMING] java mapWithState mishandles scala Option · d0df2ca4
      Gabriele Nizzoli authored
      Already merged into 1.6 branch, this PR is to commit to master the same change
      
      Author: Gabriele Nizzoli <mail@nizzoli.net>
      
      Closes #11028 from gabrielenizzoli/patch-1.
      d0df2ca4
    • Davies Liu's avatar
      [SPARK-12913] [SQL] Improve performance of stat functions · be5dd881
      Davies Liu authored
      As benchmarked and discussed here: https://github.com/apache/spark/pull/10786/files#r50038294, benefits from codegen, the declarative aggregate function could be much faster than imperative one.
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #10960 from davies/stddev.
      be5dd881
    • Reynold Xin's avatar
      [SPARK-13138][SQL] Add "logical" package prefix for ddl.scala · 7f6e3ec7
      Reynold Xin authored
      ddl.scala is defined in the execution package, and yet its reference of "UnaryNode" and "Command" are logical. This was fairly confusing when I was trying to understand the ddl code.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #11021 from rxin/SPARK-13138.
      7f6e3ec7
    • Grzegorz Chilkiewicz's avatar
      [SPARK-12711][ML] ML StopWordsRemover does not protect itself from column name duplication · b1835d72
      Grzegorz Chilkiewicz authored
      Fixes problem and verifies fix by test suite.
      Also - adds optional parameter: nullable (Boolean) to: SchemaUtils.appendColumn
      and deduplicates SchemaUtils.appendColumn functions.
      
      Author: Grzegorz Chilkiewicz <grzegorz.chilkiewicz@codilime.com>
      
      Closes #10741 from grzegorz-chilkiewicz/master.
      b1835d72
    • Daoyuan Wang's avatar
      [SPARK-13056][SQL] map column would throw NPE if value is null · 358300c7
      Daoyuan Wang authored
      Jira:
      https://issues.apache.org/jira/browse/SPARK-13056
      
      Create a map like
      { "a": "somestring", "b": null}
      Query like
      SELECT col["b"] FROM t1;
      NPE would be thrown.
      
      Author: Daoyuan Wang <daoyuan.wang@intel.com>
      
      Closes #10964 from adrian-wang/npewriter.
      358300c7
    • Bryan Cutler's avatar
      [SPARK-12631][PYSPARK][DOC] PySpark clustering parameter desc to consistent format · cba1d6b6
      Bryan Cutler authored
      Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent.  This is for the clustering module.
      
      Author: Bryan Cutler <cutlerb@gmail.com>
      
      Closes #10610 from BryanCutler/param-desc-consistent-cluster-SPARK-12631.
      cba1d6b6
    • hyukjinkwon's avatar
      [SPARK-13114][SQL] Add a test for tokens more than the fields in schema · b9383012
      hyukjinkwon authored
      https://issues.apache.org/jira/browse/SPARK-13114
      
      This PR adds a test for tokens more than the fields in schema.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #11020 from HyukjinKwon/SPARK-13114.
      b9383012
    • Michael Armbrust's avatar
      [SPARK-13094][SQL] Add encoders for seq/array of primitives · 29d92181
      Michael Armbrust authored
      Author: Michael Armbrust <michael@databricks.com>
      
      Closes #11014 from marmbrus/seqEncoders.
      29d92181
    • Michael Armbrust's avatar
      [SPARK-10820][SQL] Support for the continuous execution of structured queries · 12a20c14
      Michael Armbrust authored
      This is a follow up to 9aadcffa that extends Spark SQL to allow users to _repeatedly_ optimize and execute structured queries.  A `ContinuousQuery` can be expressed using SQL, DataFrames or Datasets.  The purpose of this PR is only to add some initial infrastructure which will be extended in subsequent PRs.
      
      ## User-facing API
      
      - `sqlContext.streamFrom` and `df.streamTo` return builder objects that are analogous to the `read/write` interfaces already available to executing queries in a batch-oriented fashion.
      - `ContinuousQuery` provides an interface for interacting with a query that is currently executing in the background.
      
      ## Internal Interfaces
       - `StreamExecution` - executes streaming queries in micro-batches
      
      The following are currently internal, but public APIs will be provided in a future release.
       - `Source` - an interface for providers of continually arriving data.  A source must have a notion of an `Offset` that monotonically tracks what data has arrived.  For fault tolerance, a source must be able to replay data given a start offset.
       - `Sink` - an interface that accepts the results of a continuously executing query.  Also responsible for tracking the offset that should be resumed from in the case of a failure.
      
      ## Testing
       - `MemoryStream` and `MemorySink` - simple implementations of source and sink that keep all data in memory and have methods for simulating durability failures
       - `StreamTest` - a framework for performing actions and checking invariants on a continuous query
      
      Author: Michael Armbrust <michael@databricks.com>
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      Author: Josh Rosen <rosenville@gmail.com>
      
      Closes #11006 from marmbrus/structured-streaming.
      12a20c14
    • Michael Armbrust's avatar
      [SPARK-13087][SQL] Fix group by function for sort based aggregation · 22ba2134
      Michael Armbrust authored
      It is not valid to call `toAttribute` on a `NamedExpression` unless we know for sure that the child produced that `NamedExpression`.  The current code worked fine when the grouping expressions were simple, but when they were a derived value this blew up at execution time.
      
      Author: Michael Armbrust <michael@databricks.com>
      
      Closes #11013 from marmbrus/groupByFunction-master.
      22ba2134
    • Reynold Xin's avatar
      Closes #10662. Closes #10661 · b8666fd0
      Reynold Xin authored
      b8666fd0
    • Reynold Xin's avatar
      [SPARK-13130][SQL] Make codegen variable names easier to read · 0fff5c6e
      Reynold Xin authored
      1. Use lower case
      2. Change long prefixes to something shorter (in this case I am changing only one: TungstenAggregate -> agg).
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #11017 from rxin/SPARK-13130.
      0fff5c6e
  4. Feb 01, 2016
Loading