Skip to content
Snippets Groups Projects
  1. Jul 29, 2015
    • Alexander Ulanov's avatar
      [SPARK-9436] [GRAPHX] Pregel simplification patch · b715933f
      Alexander Ulanov authored
      Pregel code contains two consecutive joins:
      ```
      g.vertices.innerJoin(messages)(vprog)
      ...
      g = g.outerJoinVertices(newVerts)
      { (vid, old, newOpt) => newOpt.getOrElse(old) }
      ```
      This can be simplified with one join. ankurdave proposed a patch based on our discussion in the mailing list: https://www.mail-archive.com/devspark.apache.org/msg10316.html
      
      Author: Alexander Ulanov <nashb@yandex.ru>
      
      Closes #7749 from avulanov/SPARK-9436-pregel and squashes the following commits:
      
      8568e06 [Alexander Ulanov] Pregel simplification patch
      b715933f
    • Reynold Xin's avatar
      [SPARK-9430][SQL] Rename IntervalType to CalendarIntervalType. · 5340dfaf
      Reynold Xin authored
      We want to introduce a new IntervalType in 1.6 that is based on only the number of microseoncds,
      so interval can be compared.
      
      Renaming the existing IntervalType to CalendarIntervalType so we can do that in the future.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7745 from rxin/calendarintervaltype and squashes the following commits:
      
      99f64e8 [Reynold Xin] One more line ...
      13466c8 [Reynold Xin] Fixed tests.
      e20f24e [Reynold Xin] [SPARK-9430][SQL] Rename IntervalType to CalendarIntervalType.
      5340dfaf
    • Iulian Dragos's avatar
      [SPARK-8977] [STREAMING] Defines the RateEstimator interface, and impements the RateController · 819be46e
      Iulian Dragos authored
      Based on #7471.
      
      - [x] add a test that exercises the publish path from driver to receiver
      - [ ] remove Serializable from `RateController` and `RateEstimator`
      
      Author: Iulian Dragos <jaguarul@gmail.com>
      Author: François Garillot <francois@garillot.net>
      
      Closes #7600 from dragos/topic/streaming-bp/rate-controller and squashes the following commits:
      
      f168c94 [Iulian Dragos] Latest review round.
      5125e60 [Iulian Dragos] Fix style.
      a2eb3b9 [Iulian Dragos] Merge remote-tracking branch 'upstream/master' into topic/streaming-bp/rate-controller
      475e346 [Iulian Dragos] Latest round of reviews.
      e9fb45e [Iulian Dragos] - Add a test for checkpointing - fixed serialization for RateController.executionContext
      715437a [Iulian Dragos] Review comments and added a `reset` call in ReceiverTrackerTest.
      e57c66b [Iulian Dragos] Added a couple of tests for the full scenario from driver to receivers, with several rate updates.
      b425d32 [Iulian Dragos] Removed DeveloperAPI, removed rateEstimator field, removed Noop rate estimator, changed logic for initialising rate estimator.
      238cfc6 [Iulian Dragos] Merge remote-tracking branch 'upstream/master' into topic/streaming-bp/rate-controller
      34a389d [Iulian Dragos] Various style changes and a first test for the rate controller.
      d32ca36 [François Garillot] [SPARK-8977][Streaming] Defines the RateEstimator interface, and implements the ReceiverRateController
      8941cf9 [Iulian Dragos] Renames and other nitpicks.
      162d9e5 [Iulian Dragos] Use Reflection for accessing truly private `executor` method and use the listener bus to know when receivers have registered (`onStart` is called before receivers have registered, leading to flaky behavior).
      210f495 [Iulian Dragos] Revert "Added a few tests that measure the receiver’s rate."
      0c51959 [Iulian Dragos] Added a few tests that measure the receiver’s rate.
      261a051 [Iulian Dragos] - removed field to hold the current rate limit in rate limiter - made rate limit a Long and default to Long.MaxValue (consequence of the above) - removed custom `waitUntil` and replaced it by `eventually`
      cd1397d [Iulian Dragos] Add a test for the propagation of a new rate limit from driver to receivers.
      6369b30 [Iulian Dragos] Merge pull request #15 from huitseeker/SPARK-8975
      d15de42 [François Garillot] [SPARK-8975][Streaming] Adds Ratelimiter unit tests w.r.t. spark.streaming.receiver.maxRate
      4721c7d [François Garillot] [SPARK-8975][Streaming] Add a mechanism to send a new rate from the driver to the block generator
      819be46e
    • Joseph Batchik's avatar
      [SPARK-746] [CORE] Added Avro Serialization to Kryo · 069a4c41
      Joseph Batchik authored
      Added a custom Kryo serializer for generic Avro records to reduce the network IO
      involved during a shuffle. This compresses the schema and allows for users to
      register their schemas ahead of time to further reduce traffic.
      
      Currently Kryo tries to use its default serializer for generic Records, which will include
      a lot of unneeded data in each record.
      
      Author: Joseph Batchik <joseph.batchik@cloudera.com>
      Author: Joseph Batchik <josephbatchik@gmail.com>
      
      Closes #7004 from JDrit/Avro_serialization and squashes the following commits:
      
      8158d51 [Joseph Batchik] updated per feedback
      c0cf329 [Joseph Batchik] implemented @squito suggestion for SparkEnv
      dd71efe [Joseph Batchik] fixed bug with serializing
      1183a48 [Joseph Batchik] updated codec settings
      fa9298b [Joseph Batchik] forgot a couple of fixes
      c5fe794 [Joseph Batchik] implemented @squito suggestion
      0f5471a [Joseph Batchik] implemented @squito suggestion to use a codec that is already in spark
      6d1925c [Joseph Batchik] fixed to changes suggested by @squito
      d421bf5 [Joseph Batchik] updated pom to removed versions
      ab46d10 [Joseph Batchik] Changed Avro dependency to be similar to parent
      f4ae251 [Joseph Batchik] fixed serialization error in that SparkConf cannot be serialized
      2b545cc [Joseph Batchik] started working on fixes for pr
      97fba62 [Joseph Batchik] Added a custom Kryo serializer for generic Avro records to reduce the network IO involved during a shuffle. This compresses the schema and allows for users to register their schemas ahead of time to further reduce traffic.
      069a4c41
    • Reynold Xin's avatar
      [SPARK-9127][SQL] Rand/Randn codegen fails with long seed. · 97906944
      Reynold Xin authored
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7747 from rxin/SPARK-9127 and squashes the following commits:
      
      e851418 [Reynold Xin] [SPARK-9127][SQL] Rand/Randn codegen fails with long seed.
      97906944
    • Wenchen Fan's avatar
      [SPARK-9251][SQL] do not order by expressions which still need evaluation · 708794e8
      Wenchen Fan authored
      as an offline discussion with rxin , it's weird to be computing stuff while doing sorting, we should only order by bound reference during execution.
      
      Author: Wenchen Fan <cloud0fan@outlook.com>
      
      Closes #7593 from cloud-fan/sort and squashes the following commits:
      
      7b1bef7 [Wenchen Fan] add test
      daf206d [Wenchen Fan] add more comments
      289bee0 [Wenchen Fan] do not order by expressions which still need evaluation
      708794e8
    • Davies Liu's avatar
      [SPARK-9281] [SQL] use decimal or double when parsing SQL · 15667a0a
      Davies Liu authored
      Right now, we use double to parse all the float number in SQL. When it's used in expression together with DecimalType, it will turn the decimal into double as well. Also it will loss some precision when using double.
      
      This PR change to parse float number to decimal or double, based on it's  using scientific notation or not, see https://msdn.microsoft.com/en-us/library/ms179899.aspx
      
      This is a break change, should we doc it somewhere?
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #7642 from davies/parse_decimal and squashes the following commits:
      
      1f576d9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into parse_decimal
      5e142b6 [Davies Liu] fix scala style
      eca99de [Davies Liu] fix tests
      2afe702 [Davies Liu] Merge branch 'master' of github.com:apache/spark into parse_decimal
      f4a320b [Davies Liu] Update SqlParser.scala
      1c48e34 [Davies Liu] use decimal or double when parsing SQL
      15667a0a
    • Yijie Shen's avatar
      [SPARK-9398] [SQL] Datetime cleanup · 6309b934
      Yijie Shen authored
      JIRA: https://issues.apache.org/jira/browse/SPARK-9398
      
      Author: Yijie Shen <henry.yijieshen@gmail.com>
      
      Closes #7725 from yjshen/date_null_check and squashes the following commits:
      
      b4eade1 [Yijie Shen] inline daysToMonthEnd
      d09acc1 [Yijie Shen] implement getLastDayOfMonth to avoid repeated evaluation
      d857ec3 [Yijie Shen] add null check in DateExpressionSuite
      6309b934
  2. Jul 28, 2015
    • Josh Rosen's avatar
      [SPARK-9419] ShuffleMemoryManager and MemoryStore should track memory on a... · ea49705b
      Josh Rosen authored
      [SPARK-9419] ShuffleMemoryManager and MemoryStore should track memory on a per-task, not per-thread, basis
      
      Spark's ShuffleMemoryManager and MemoryStore track memory on a per-thread basis, which causes problems in the handful of cases where we have tasks that use multiple threads. In PythonRDD, RRDD, ScriptTransformation, and PipedRDD we consume the input iterator in a separate thread in order to write it to an external process.  As a result, these RDD's input iterators are consumed in a different thread than the thread that created them, which can cause problems in our memory allocation tracking. For example, if allocations are performed in one thread but deallocations are performed in a separate thread then memory may be leaked or we may get errors complaining that more memory was allocated than was freed.
      
      I think that the right way to fix this is to change our accounting to be performed on a per-task instead of per-thread basis.  Note that the current per-thread tracking has caused problems in the past; SPARK-3731 (#2668) fixes a memory leak in PythonRDD that was caused by this issue (that fix is no longer necessary as of this patch).
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #7734 from JoshRosen/memory-tracking-fixes and squashes the following commits:
      
      b4b1702 [Josh Rosen] Propagate TaskContext to writer threads.
      57c9b4e [Josh Rosen] Merge remote-tracking branch 'origin/master' into memory-tracking-fixes
      ed25d3b [Josh Rosen] Address minor PR review comments
      44f6497 [Josh Rosen] Fix long line.
      7b0f04b [Josh Rosen] Fix ShuffleMemoryManagerSuite
      f57f3f2 [Josh Rosen] More thread -> task changes
      fa78ee8 [Josh Rosen] Move Executor's cleanup into Task so that TaskContext is defined when cleanup is performed
      5e2f01e [Josh Rosen] Fix capitalization
      1b0083b [Josh Rosen] Roll back fix in PySpark, which is no longer necessary
      2e1e0f8 [Josh Rosen] Use TaskAttemptIds to track shuffle memory
      c9e8e54 [Josh Rosen] Use TaskAttemptIds to track unroll memory
      ea49705b
    • Wenchen Fan's avatar
      [SPARK-8608][SPARK-8609][SPARK-9083][SQL] reset mutable states of... · 429b2f0d
      Wenchen Fan authored
      [SPARK-8608][SPARK-8609][SPARK-9083][SQL] reset mutable states of nondeterministic expression before evaluation and fix PullOutNondeterministic
      
      We will do local projection for LocalRelation, and thus reuse the same Expression object among multiply evaluations. We should reset the mutable states of Expression before evaluate it.
      
      Fix `PullOutNondeterministic` rule to make it work for `Sort`.
      
      Also got a chance to cleanup the dataframe test suite.
      
      Author: Wenchen Fan <cloud0fan@outlook.com>
      
      Closes #7674 from cloud-fan/show and squashes the following commits:
      
      888934f [Wenchen Fan] fix sort
      c0e93e8 [Wenchen Fan] local DataFrame with random columns should return same value when call `show`
      429b2f0d
    • Yin Huai's avatar
      [SPARK-9422] [SQL] Remove the placeholder attributes used in the aggregation buffers · 3744b7fd
      Yin Huai authored
      https://issues.apache.org/jira/browse/SPARK-9422
      
      Author: Yin Huai <yhuai@databricks.com>
      
      Closes #7737 from yhuai/removePlaceHolder and squashes the following commits:
      
      ec29b44 [Yin Huai]  Remove placeholder attributes.
      3744b7fd
    • Josh Rosen's avatar
      [SPARK-9421] Fix null-handling bugs in UnsafeRow.getDouble, getFloat(), and get(ordinal, dataType) · e78ec1a8
      Josh Rosen authored
      UnsafeRow.getDouble and getFloat() return NaN when called on columns that are null, which is inconsistent with the behavior of other row classes (which is to return 0.0).
      
      In addition, the generic get(ordinal, dataType) method should always return null for a null literal, but currently it handles nulls by calling the type-specific accessors.
      
      This patch addresses both of these issues and adds a regression test.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #7736 from JoshRosen/unsafe-row-null-fixes and squashes the following commits:
      
      c8eb2ee [Josh Rosen] Fix test in UnsafeRowConverterSuite
      6214682 [Josh Rosen] Fixes to null handling in UnsafeRow
      e78ec1a8
    • Reynold Xin's avatar
      [SPARK-9418][SQL] Use sort-merge join as the default shuffle join. · 6662ee21
      Reynold Xin authored
      Sort-merge join is more robust in Spark since sorting can be made using the Tungsten sort operator.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7733 from rxin/smj and squashes the following commits:
      
      61e4d34 [Reynold Xin] Fixed test case.
      5ffd731 [Reynold Xin] Fixed JoinSuite.
      a137dc0 [Reynold Xin] [SPARK-9418][SQL] Use sort-merge join as the default shuffle join.
      6662ee21
    • Reynold Xin's avatar
      [SPARK-9420][SQL] Move expressions in sql/core package to catalyst. · b7f54119
      Reynold Xin authored
      Since catalyst package already depends on Spark core, we can move those expressions
      into catalyst, and simplify function registry.
      
      This is a followup of #7478.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7735 from rxin/SPARK-8003 and squashes the following commits:
      
      2ffbdc3 [Reynold Xin] [SPARK-8003][SQL] Move expressions in sql/core package to catalyst.
      b7f54119
    • Tathagata Das's avatar
      [STREAMING] [HOTFIX] Ignore ReceiverTrackerSuite flaky test · c5ed3695
      Tathagata Das authored
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      
      Closes #7738 from tdas/ReceiverTrackerSuite-hotfix and squashes the following commits:
      
      00f0ee1 [Tathagata Das] ignore flaky test
      c5ed3695
    • Josh Rosen's avatar
      [SPARK-9393] [SQL] Fix several error-handling bugs in ScriptTransform operator · 59b92add
      Josh Rosen authored
      SparkSQL's ScriptTransform operator has several serious bugs which make debugging fairly difficult:
      
      - If exceptions are thrown in the writing thread then the child process will not be killed, leading to a deadlock because the reader thread will block while waiting for input that will never arrive.
      - TaskContext is not propagated to the writer thread, which may cause errors in upstream pipelined operators.
      - Exceptions which occur in the writer thread are not propagated to the main reader thread, which may cause upstream errors to be silently ignored instead of killing the job.  This can lead to silently incorrect query results.
      - The writer thread is not a daemon thread, but it should be.
      
      In addition, the code in this file is extremely messy:
      
      - Lots of fields are nullable but the nullability isn't clearly explained.
      - Many confusing variable names: for instance, there are variables named `ite` and `iterator` that are defined in the same scope.
      - Some code was misindented.
      - The `*serdeClass` variables are actually expected to be single-quoted strings, which is really confusing: I feel that this parsing / extraction should be performed in the analyzer, not in the operator itself.
      - There were no unit tests for the operator itself, only end-to-end tests.
      
      This pull request addresses these issues, borrowing some error-handling techniques from PySpark's PythonRDD.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #7710 from JoshRosen/script-transform and squashes the following commits:
      
      16c44e2 [Josh Rosen] Update some comments
      983f200 [Josh Rosen] Use unescapeSQLString instead of stripQuotes
      6a06a8c [Josh Rosen] Clean up handling of quotes in serde class name
      494cde0 [Josh Rosen] Propagate TaskContext to writer thread
      323bb2b [Josh Rosen] Fix error-swallowing bug
      b31258d [Josh Rosen] Rename iterator variables to disambiguate.
      88278de [Josh Rosen] Split ScriptTransformation writer thread into own class.
      8b162b6 [Josh Rosen] Add failing test which demonstrates exception masking issue
      4ee36a2 [Josh Rosen] Kill script transform subprocess when error occurs in input writer.
      bd4c948 [Josh Rosen] Skip launching of external command for empty partitions.
      b43e4ec [Josh Rosen] Clean up nullability in ScriptTransformation
      fa18d26 [Josh Rosen] Add basic unit test for script transform with 'cat' command.
      59b92add
    • Davies Liu's avatar
      [SPARK-9247] [SQL] Use BytesToBytesMap for broadcast join · 21825529
      Davies Liu authored
      This PR introduce BytesToBytesMap to UnsafeHashedRelation, use it in executor for better performance.
      
      It serialize all the key and values from java HashMap, put them into a BytesToBytesMap while deserializing. All the values for a same key are stored continuous to have better memory locality.
      
      This PR also address the comments for #7480 , do some clean up.
      
      Author: Davies Liu <davies@databricks.com>
      
      Closes #7592 from davies/unsafe_map2 and squashes the following commits:
      
      42c578a [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_map2
      fd09528 [Davies Liu] remove thread local cache and update docs
      1c5ad8d [Davies Liu] fix test
      5eb1b5a [Davies Liu] address comments in #7480
      46f1f22 [Davies Liu] fix style
      fc221e0 [Davies Liu] use BytesToBytesMap for broadcast join
      21825529
    • MechCoder's avatar
      [SPARK-7105] [PYSPARK] [MLLIB] Support model save/load in GMM · 198d181d
      MechCoder authored
      This PR introduces save / load for GMM's in python API.
      
      Also I refactored `GaussianMixtureModel` and inherited it from `JavaModelWrapper` with model being `GaussianMixtureModelWrapper`, a wrapper which provides convenience methods to `GaussianMixtureModel` (due to serialization and deserialization issues) and I moved the creation of gaussians to the scala backend.
      
      Author: MechCoder <manojkumarsivaraj334@gmail.com>
      
      Closes #7617 from MechCoder/python_gmm_save_load and squashes the following commits:
      
      9c305aa [MechCoder] [SPARK-7105] [PySpark] [MLlib] Support model save/load in GMM
      198d181d
    • Joseph Batchik's avatar
      [SPARK-8003][SQL] Added virtual column support to Spark · b88b868e
      Joseph Batchik authored
      Added virtual column support by adding a new resolution role to the query analyzer. Additional virtual columns can be added by adding case expressions to [the new rule](https://github.com/JDrit/spark/blob/virt_columns/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala#L1026) and my modifying the [logical plan](https://github.com/JDrit/spark/blob/virt_columns/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala#L216) to resolve them.
      
      This also solves [SPARK-8003](https://issues.apache.org/jira/browse/SPARK-8003)
      
      This allows you to perform queries such as:
      ```sql
      select spark__partition__id, count(*) as c from table group by spark__partition__id;
      ```
      
      Author: Joseph Batchik <josephbatchik@gmail.com>
      Author: JD <jd@csh.rit.edu>
      
      Closes #7478 from JDrit/virt_columns and squashes the following commits:
      
      7932bf0 [Joseph Batchik] adding spark__partition__id to hive as well
      f8a9c6c [Joseph Batchik] merging in master
      e49da48 [JD] fixes for @rxin's suggestions
      60e120b [JD] fixing test in merge
      4bf8554 [JD] merging in master
      c68bc0f [Joseph Batchik] Adding function register ability to SQLContext and adding a function for spark__partition__id()
      b88b868e
    • Eric Liang's avatar
      [SPARK-9391] [ML] Support minus, dot, and intercept operators in SparkR RFormula · 8d5bb528
      Eric Liang authored
      Adds '.', '-', and intercept parsing to RFormula. Also splits RFormulaParser into a separate file.
      
      Umbrella design doc here: https://docs.google.com/document/d/10NZNSEurN2EdWM31uFYsgayIPfCFHiuIu3pCWrUmP_c/edit?usp=sharing
      
      mengxr
      
      Author: Eric Liang <ekl@databricks.com>
      
      Closes #7707 from ericl/string-features-2 and squashes the following commits:
      
      8588625 [Eric Liang] exclude complex types for .
      8106ffe [Eric Liang] comments
      a9350bb [Eric Liang] s/var/val
      9c50d4d [Eric Liang] Merge branch 'string-features' into string-features-2
      581afb2 [Eric Liang] Merge branch 'master' into string-features
      08ae539 [Eric Liang] Merge branch 'string-features' into string-features-2
      f99131a [Eric Liang] comments
      cecec43 [Eric Liang] Merge branch 'string-features' into string-features-2
      0bf3c26 [Eric Liang] update docs
      4592df2 [Eric Liang] intercept supports
      7412a2e [Eric Liang] Fri Jul 24 14:56:51 PDT 2015
      3cf848e [Eric Liang] fix the parser
      0556c2b [Eric Liang] Merge branch 'string-features' into string-features-2
      c302a2c [Eric Liang] fix tests
      9d1ac82 [Eric Liang] Merge remote-tracking branch 'upstream/master' into string-features
      e713da3 [Eric Liang] comments
      cd231a9 [Eric Liang] Wed Jul 22 17:18:44 PDT 2015
      4d79193 [Eric Liang] revert to seq + distinct
      169a085 [Eric Liang] tweak functional test
      a230a47 [Eric Liang] Merge branch 'master' into string-features
      72bd6f3 [Eric Liang] fix merge
      d841cec [Eric Liang] Merge branch 'master' into string-features
      5b2c4a2 [Eric Liang] Mon Jul 20 18:45:33 PDT 2015
      b01c7c5 [Eric Liang] add test
      8a637db [Eric Liang] encoder wip
      a1d03f4 [Eric Liang] refactor into estimator
      8d5bb528
    • Yin Huai's avatar
      [SPARK-9196] [SQL] Ignore test DatetimeExpressionsSuite: function current_timestamp. · 6cdcc21f
      Yin Huai authored
      This test is flaky. https://issues.apache.org/jira/browse/SPARK-9196 will track the fix of it. For now, let's disable this test.
      
      Author: Yin Huai <yhuai@databricks.com>
      
      Closes #7727 from yhuai/SPARK-9196-ignore and squashes the following commits:
      
      f92bded [Yin Huai] Ignore current_timestamp.
      6cdcc21f
    • Marcelo Vanzin's avatar
      [SPARK-9327] [DOCS] Fix documentation about classpath config options. · 31ec6a87
      Marcelo Vanzin authored
      Author: Marcelo Vanzin <vanzin@cloudera.com>
      
      Closes #7651 from vanzin/SPARK-9327 and squashes the following commits:
      
      2923e23 [Marcelo Vanzin] [SPARK-9327] [docs] Fix documentation about classpath config options.
      31ec6a87
    • trestletech's avatar
      Use vector-friendly comparison for packages argument. · 61432340
      trestletech authored
      Otherwise, `sparkR.init()` with multiple `sparkPackages` results in this warning:
      
      ```
      Warning message:
      In if (packages != "") { :
        the condition has length > 1 and only the first element will be used
      ```
      
      Author: trestletech <jeff.allen@trestletechnology.net>
      
      Closes #7701 from trestletech/compare-packages and squashes the following commits:
      
      72c8b36 [trestletech] Correct function name.
      c52db0e [trestletech] Added test for multiple packages.
      3aab1a7 [trestletech] Use vector-friendly comparison for packages argument.
      61432340
    • Aaron Davidson's avatar
      [SPARK-9397] DataFrame should provide an API to find source data files if applicable · 35ef853b
      Aaron Davidson authored
      Certain applications would benefit from being able to inspect DataFrames that are straightforwardly produced by data sources that stem from files, and find out their source data. For example, one might want to display to a user the size of the data underlying a table, or to copy or mutate it.
      
      This PR exposes an `inputFiles` method on DataFrame which attempts to discover the source data in a best-effort manner, by inspecting HadoopFsRelations and JSONRelations.
      
      Author: Aaron Davidson <aaron@databricks.com>
      
      Closes #7717 from aarondav/paths and squashes the following commits:
      
      ff67430 [Aaron Davidson] inputFiles
      0acd3ad [Aaron Davidson] [SPARK-9397] DataFrame should provide an API to find source data files if applicable
      35ef853b
    • Reynold Xin's avatar
      [SPARK-8196][SQL] Fix null handling & documentation for next_day. · 9bbe0171
      Reynold Xin authored
      The original patch didn't handle nulls correctly for next_day.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7718 from rxin/next_day and squashes the following commits:
      
      616a425 [Reynold Xin] Merged DatetimeExpressionsSuite into DateFunctionsSuite.
      faa78cf [Reynold Xin] Merged DatetimeFunctionsSuite into DateExpressionsSuite.
      6c4fb6a [Reynold Xin] [SPARK-8196][SQL] Fix null handling & documentation for next_day.
      9bbe0171
    • Reynold Xin's avatar
      [SPARK-9373][SQL] follow up for StructType support in Tungsten projection. · c740bed1
      Reynold Xin authored
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7720 from rxin/struct-followup and squashes the following commits:
      
      d9757f5 [Reynold Xin] [SPARK-9373][SQL] follow up for StructType support in Tungsten projection.
      c740bed1
    • Reynold Xin's avatar
      [SPARK-9402][SQL] Remove CodegenFallback from Abs / FormatNumber. · 5a2330e5
      Reynold Xin authored
      Both expressions already implement code generation.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7723 from rxin/abs-formatnum and squashes the following commits:
      
      31ed765 [Reynold Xin] [SPARK-9402][SQL] Remove CodegenFallback from Abs / FormatNumber.
      5a2330e5
    • vinodkc's avatar
      [SPARK-8919] [DOCUMENTATION, MLLIB] Added @since tags to mllib.recommendation · 4af622c8
      vinodkc authored
      Author: vinodkc <vinod.kc.in@gmail.com>
      
      Closes #7325 from vinodkc/add_since_mllib.recommendation and squashes the following commits:
      
      93156f2 [vinodkc] Changed 0.8.0 to 0.9.1
      c413350 [vinodkc] Added @since
      4af622c8
    • Kenichi Maehashi's avatar
      [EC2] Cosmetic fix for usage of spark-ec2 --ebs-vol-num option · ac8c549e
      Kenichi Maehashi authored
      The last line of the usage seems ugly.
      
      ```
      $ spark-ec2 --help
      <snip>
        --ebs-vol-num=EBS_VOL_NUM
                              Number of EBS volumes to attach to each node as
                              /vol[x]. The volumes will be deleted when the
                              instances terminate. Only possible on EBS-backed AMIs.
                              EBS volumes are only attached if --ebs-vol-size >
                              0.Only support up to 8 EBS volumes.
      ```
      
      After applying this patch:
      
      ```
      $ spark-ec2 --help
      <snip>
        --ebs-vol-num=EBS_VOL_NUM
                              Number of EBS volumes to attach to each node as
                              /vol[x]. The volumes will be deleted when the
                              instances terminate. Only possible on EBS-backed AMIs.
                              EBS volumes are only attached if --ebs-vol-size > 0.
                              Only support up to 8 EBS volumes.
      ```
      
      As this is a trivial thing I didn't create JIRA for this.
      
      Author: Kenichi Maehashi <webmaster@kenichimaehashi.com>
      
      Closes #7632 from kmaehashi/spark-ec2-cosmetic-fix and squashes the following commits:
      
      526c118 [Kenichi Maehashi] cosmetic fix for spark-ec2 --ebs-vol-num option usage
      ac8c549e
    • Reynold Xin's avatar
      [SPARK-9394][SQL] Handle parentheses in CodeFormatter. · 15724fac
      Reynold Xin authored
      Our CodeFormatter currently does not handle parentheses, and as a result in code dump, we see code formatted this way:
      
      ```
      foo(
      a,
      b,
      c)
      ```
      
      With this patch, it is formatted this way:
      ```
      foo(
        a,
        b,
        c)
      ```
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7712 from rxin/codeformat-parentheses and squashes the following commits:
      
      c2b1c5f [Reynold Xin] Took square bracket out
      3cfb174 [Reynold Xin] Code review feedback.
      91f5bb1 [Reynold Xin] [SPARK-9394][SQL] Handle parentheses in CodeFormatter.
      15724fac
    • Reynold Xin's avatar
      fc3bd96b
    • zsxwing's avatar
      [SPARK-9335] [STREAMING] [TESTS] Make sure the test stream is deleted in KinesisBackedBlockRDDSuite · d93ab93d
      zsxwing authored
      KinesisBackedBlockRDDSuite should make sure delete the stream.
      
      Author: zsxwing <zsxwing@gmail.com>
      
      Closes #7663 from zsxwing/fix-SPARK-9335 and squashes the following commits:
      
      f0e9154 [zsxwing] Revert "[HOTFIX] - Disable Kinesis tests due to rate limits"
      71a4552 [zsxwing] Make sure the test stream is deleted
      d93ab93d
    • Cheng Hao's avatar
      [MINOR] [SQL] Support mutable expression unit test with codegen projection · 9c5612f4
      Cheng Hao authored
      This is actually contains 3 minor issues:
      1) Enable the unit test(codegen) for mutable expressions (FormatNumber, Regexp_Replace/Regexp_Extract)
      2) Use the `PlatformDependent.copyMemory` instead of the `System.arrayCopy`
      
      Author: Cheng Hao <hao.cheng@intel.com>
      
      Closes #7566 from chenghao-intel/codegen_ut and squashes the following commits:
      
      24f43ea [Cheng Hao] enable codegen for mutable expression & UTF8String performance
      9c5612f4
    • Reynold Xin's avatar
      [SPARK-9373][SQL] Support StructType in Tungsten projection · 60f08c7c
      Reynold Xin authored
      This pull request updates GenerateUnsafeProjection to support StructType. If an input struct type is backed already by an UnsafeRow, GenerateUnsafeProjection copies the bytes directly into its buffer space without any conversion. However, if the input is not an UnsafeRow, GenerateUnsafeProjection runs the code generated recursively to convert the input into an UnsafeRow and then copies it into the buffer space.
      
      Also create a TungstenProject operator that projects data directly into UnsafeRow. Note that I'm not sure if this is the way we want to structure Unsafe+codegen operators, but we can defer that decision to follow-up pull requests.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7689 from rxin/tungsten-struct-type and squashes the following commits:
      
      9162f42 [Reynold Xin] Support IntervalType in UnsafeRow's getter.
      be9f377 [Reynold Xin] Fixed tests.
      10c4b7c [Reynold Xin] Format generated code.
      77e8d0e [Reynold Xin] Fixed NondeterministicSuite.
      ac4951d [Reynold Xin] Yay.
      ac203bf [Reynold Xin] More comments.
      9f36216 [Reynold Xin] Updated comment.
      6b781fe [Reynold Xin] Reset the change in DataFrameSuite.
      525b95b [Reynold Xin] Merged with master, more documentation & test cases.
      321859a [Reynold Xin] [SPARK-9373][SQL] Support StructType in Tungsten projection [WIP]
      60f08c7c
    • Yijie Shen's avatar
      [SPARK-8828] [SQL] Revert SPARK-5680 · 63a492b9
      Yijie Shen authored
      JIRA: https://issues.apache.org/jira/browse/SPARK-8828
      
      Author: Yijie Shen <henry.yijieshen@gmail.com>
      
      Closes #7667 from yjshen/revert_combinesum_2 and squashes the following commits:
      
      c37ccb1 [Yijie Shen] add test case
      8377214 [Yijie Shen] revert spark.sql.useAggregate2 to its default value
      e2305ac [Yijie Shen] fix bug - avg on decimal column
      7cb0e95 [Yijie Shen] [wip] resolving bugs
      1fadb5a [Yijie Shen] remove occurance
      17c6248 [Yijie Shen] revert SPARK-5680
      63a492b9
    • Reynold Xin's avatar
      Fixed a test failure. · 3bc7055e
      Reynold Xin authored
      3bc7055e
  3. Jul 27, 2015
    • Reynold Xin's avatar
      [SPARK-9395][SQL] Create a SpecializedGetters interface to track all the specialized getters. · 84da8792
      Reynold Xin authored
      As we are adding more and more specialized getters to more classes (coming soon ArrayData), this interface can help us prevent missing a method in some interfaces.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #7713 from rxin/SpecializedGetters and squashes the following commits:
      
      3b39be1 [Reynold Xin] Added override modifier.
      567ba9c [Reynold Xin] [SPARK-9395][SQL] Create a SpecializedGetters interface to track all the specialized getters.
      84da8792
    • Daoyuan Wang's avatar
      [SPARK-8195] [SPARK-8196] [SQL] udf next_day last_day · 2e7f99a0
      Daoyuan Wang authored
      next_day, returns next certain dayofweek.
      last_day, returns the last day of the month which given date belongs to.
      
      Author: Daoyuan Wang <daoyuan.wang@intel.com>
      
      Closes #6986 from adrian-wang/udfnlday and squashes the following commits:
      
      ef7e3da [Daoyuan Wang] fix
      02b3426 [Daoyuan Wang] address 2 comments
      dc69630 [Daoyuan Wang] address comments from rxin
      8846086 [Daoyuan Wang] address comments from rxin
      d09bcce [Daoyuan Wang] multi fix
      1a9de3d [Daoyuan Wang] function next_day and last_day
      2e7f99a0
    • zsxwing's avatar
      [SPARK-8882] [STREAMING] Add a new Receiver scheduling mechanism · daa1964b
      zsxwing authored
      The design doc: https://docs.google.com/document/d/1ZsoRvHjpISPrDmSjsGzuSu8UjwgbtmoCTzmhgTurHJw/edit?usp=sharing
      
      Author: zsxwing <zsxwing@gmail.com>
      
      Closes #7276 from zsxwing/receiver-scheduling and squashes the following commits:
      
      137b257 [zsxwing] Add preferredNumExecutors to rescheduleReceiver
      61a6c3f [zsxwing] Set state to ReceiverState.INACTIVE in deregisterReceiver
      5e1fa48 [zsxwing] Fix the code style
      7451498 [zsxwing] Move DummyReceiver back to ReceiverTrackerSuite
      715ef9c [zsxwing] Rename: scheduledLocations -> scheduledExecutors; locations -> executors
      05daf9c [zsxwing] Use receiverTrackingInfo.toReceiverInfo
      1d6d7c8 [zsxwing] Merge branch 'master' into receiver-scheduling
      8f93c8d [zsxwing] Use hostPort as the receiver location rather than host; fix comments and unit tests
      59f8887 [zsxwing] Schedule all receivers at the same time when launching them
      075e0a3 [zsxwing] Add receiver RDD name; use '!isTrackerStarted' instead
      276a4ac [zsxwing] Remove "ReceiverLauncher" and move codes to "launchReceivers"
      fab9a01 [zsxwing] Move methods back to the outer class
      4e639c4 [zsxwing] Fix unintentional changes
      f60d021 [zsxwing] Reorganize ReceiverTracker to use an event loop for lock free
      105037e [zsxwing] Merge branch 'master' into receiver-scheduling
      5fee132 [zsxwing] Update tha scheduling algorithm to avoid to keep restarting Receiver
      9e242c8 [zsxwing] Remove the ScheduleReceiver message because we can refuse it when receiving RegisterReceiver
      a9acfbf [zsxwing] Merge branch 'squash-pr-6294' into receiver-scheduling
      881edb9 [zsxwing] ReceiverScheduler -> ReceiverSchedulingPolicy
      e530bcc [zsxwing] [SPARK-5681][Streaming] Use a lock to eliminate the race condition when stopping receivers and registering receivers happen at the same time #6294
      3b87e4a [zsxwing] Revert SparkContext.scala
      a86850c [zsxwing] Remove submitAsyncJob and revert JobWaiter
      f549595 [zsxwing] Add comments for the scheduling approach
      9ecc08e [zsxwing] Fix comments and code style
      28d1bee [zsxwing] Make 'host' protected; rescheduleReceiver -> getAllowedLocations
      2c86a9e [zsxwing] Use tryFailure to support calling jobFailed multiple times
      ca6fe35 [zsxwing] Add a test for Receiver.restart
      27acd45 [zsxwing] Add unit tests for LoadBalanceReceiverSchedulerImplSuite
      cc76142 [zsxwing] Add JobWaiter.toFuture to avoid blocking threads
      d9a3e72 [zsxwing] Add a new Receiver scheduling mechanism
      daa1964b
    • Michael Armbrust's avatar
      [SPARK-9386] [SQL] Feature flag for metastore partition pruning · ce89ff47
      Michael Armbrust authored
      Since we have been seeing a lot of failures related to this new feature, lets put it behind a flag and turn it off by default.
      
      Author: Michael Armbrust <michael@databricks.com>
      
      Closes #7703 from marmbrus/optionalMetastorePruning and squashes the following commits:
      
      6ad128c [Michael Armbrust] style
      8447835 [Michael Armbrust] [SPARK-9386][SQL] Feature flag for metastore partition pruning
      fd37b87 [Michael Armbrust] add config flag
      ce89ff47
Loading