aboutsummaryrefslogtreecommitdiff
path: root/streaming
Commit message (Collapse)AuthorAgeFilesLines
* Improved build configurationwitgo2014-04-281-14/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | 1, Fix SPARK-1441: compile spark core error with hadoop 0.23.x 2, Fix SPARK-1491: maven hadoop-provided profile fails to build 3, Fix org.scala-lang: * ,org.apache.avro:* inconsistent versions dependency 4, A modified on the sql/catalyst/pom.xml,sql/hive/pom.xml,sql/core/pom.xml (Four spaces formatted into two spaces) Author: witgo <witgo@qq.com> Closes #480 from witgo/format_pom and squashes the following commits: 03f652f [witgo] review commit b452680 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom bee920d [witgo] revert fix SPARK-1629: Spark Core missing commons-lang dependence 7382a07 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 6902c91 [witgo] fix SPARK-1629: Spark Core missing commons-lang dependence 0da4bc3 [witgo] merge master d1718ed [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom e345919 [witgo] add avro dependency to yarn-alpha 77fad08 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 62d0862 [witgo] Fix org.scala-lang: * inconsistent versions dependency 1a162d7 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom 934f24d [witgo] review commit cf46edc [witgo] exclude jruby 06e7328 [witgo] Merge branch 'SparkBuild' into format_pom 99464d2 [witgo] fix maven hadoop-provided profile fails to build 0c6c1fc [witgo] Fix compile spark core error with hadoop 0.23.x 6851bec [witgo] Maintain consistent SparkBuild.scala, pom.xml
* [SPARK-1633][Streaming] Java API unit test and example for custom streaming ↵Tathagata Das2014-04-287-31/+215
| | | | | | | | | | | | | | | | | | | | | | receiver in Java Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #558 from tdas/more-fixes and squashes the following commits: c0c84e6 [Tathagata Das] Removing extra println() d8a8cf4 [Tathagata Das] More tweaks to make unit test work in Jenkins. b7caa98 [Tathagata Das] More tweaks. d337367 [Tathagata Das] More tweaks 22d6f2d [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes 40a961b [Tathagata Das] Modified java test to reduce flakiness. 9410ca6 [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes 86d9147 [Tathagata Das] scala style fix 2f3d7b1 [Tathagata Das] Added Scala custom receiver example. d677611 [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes bec3fc2 [Tathagata Das] Added license. 51d6514 [Tathagata Das] Fixed docs on receiver. 81aafa0 [Tathagata Das] Added Java test for Receiver API, and added JavaCustomReceiver example.
* [Spark-1382] Fix NPE in DStream.slice (updated version of #365)zsxwing2014-04-252-11/+23
| | | | | | | | | | | | | | @zsxwing I cherry-picked your changes and merged the master. #365 had some conflicts once again! Author: zsxwing <zsxwing@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #562 from tdas/SPARK-1382 and squashes the following commits: e2962c1 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-1382 20968d9 [zsxwing] Replace Exception with SparkException in DStream e476651 [zsxwing] Merge remote-tracking branch 'origin/master' into SPARK-1382 35ba56a [zsxwing] SPARK-1382: Fix NPE in DStream.slice
* [SPARK-1617] and [SPARK-1618] Improvements to streaming ui and bug fix to ↵Tathagata Das2014-04-2414-102/+216
| | | | | | | | | | | | | | | | | | | socket receiver 1617: These changes expose the receiver state (active or inactive) and last error in the UI 1618: If the socket receiver cannot connect in the first attempt, it should try to restart after a delay. That was broken, as the thread that restarts (hence, stops) the receiver waited on Thread.join on itself! Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #540 from tdas/streaming-ui-fix and squashes the following commits: e469434 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-ui-fix dbddf75 [Tathagata Das] Style fix. 66df1a5 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-ui-fix ad98bc9 [Tathagata Das] Refactored streaming listener to use ReceiverInfo. d7f849c [Tathagata Das] Revert "Moved BatchInfo from streaming.scheduler to streaming.ui" 5c80919 [Tathagata Das] Moved BatchInfo from streaming.scheduler to streaming.ui da244f6 [Tathagata Das] Fixed socket receiver as well as made receiver state and error visible in the streamign UI.
* [SPARK-1510] Spark Streaming metrics source for metrics systemjerryshao2014-04-243-1/+79
| | | | | | | | | | | | | | This pulls in changes made by @jerryshao in https://github.com/apache/spark/pull/424 and merges with the master. Author: jerryshao <saisai.shao@intel.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #545 from tdas/streaming-metrics and squashes the following commits: 034b443 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-metrics fb3b0a5 [jerryshao] Modify according master update 21939f5 [jerryshao] Style changes according to style check error 976116b [jerryshao] Add StreamSource in StreamingContext for better monitoring through metrics system
* [SPARK-1592][streaming] Automatically remove streaming input blocksTathagata Das2014-04-245-18/+90
| | | | | | | | | | | | | | | The raw input data is stored as blocks in BlockManagers. Earlier they were cleared by cleaner ttl. Now since streaming does not require cleaner TTL to be set, the block would not get cleared. This increases up the Spark's memory usage, which is not even accounted and shown in the Spark storage UI. It may cause the data blocks to spill over to disk, which eventually slows down the receiving of data (persisting to memory become bottlenecked by writing to disk). The solution in this PR is to automatically remove those blocks. The mechanism to keep track of which BlockRDDs (which has presents the raw data blocks as a RDD) can be safely cleared already exists. Just use it to explicitly remove blocks from BlockRDDs. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #512 from tdas/block-rdd-unpersist and squashes the following commits: d25e610 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 5f46d69 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 2c320cd [Tathagata Das] Updated configuration with spark.streaming.unpersist setting. 2d4b2fd [Tathagata Das] Automatically removed input blocks
* Fix Scala StyleSandeep2014-04-241-2/+1
| | | | | | | | | | | | | | Any comments are welcome Author: Sandeep <sandeep@techaddict.me> Closes #531 from techaddict/stylefix-1 and squashes the following commits: 7492730 [Sandeep] Pass 4 98b2428 [Sandeep] fix rxin suggestions b5e2e6f [Sandeep] Pass 3 05932d7 [Sandeep] fix if else styling 2 08690e5 [Sandeep] fix if else styling
* SPARK-1488. Squash more language feature warnings in new commits by ↵Sean Owen2014-04-246-0/+7
| | | | | | | | | | | | | importing implicitConversion A recent commit reintroduced some of the same warnings that SPARK-1488 resolved. These are just a few more of the same changes to remove these warnings. Author: Sean Owen <sowen@cloudera.com> Closes #528 from srowen/SPARK-1488.2 and squashes the following commits: 62d592c [Sean Owen] More feature warnings in tests 4e2e94b [Sean Owen] Squash more language feature warnings in new commits by importing implicitConversion
* [SPARK-1540] Add an optional Ordering parameter to PairRDDFunctions.Matei Zaharia2014-04-233-14/+19
| | | | | | | | | | | | | | | | | | | | In https://issues.apache.org/jira/browse/SPARK-1540 we'd like to look at Spark's API to see if we can take advantage of Comparable keys in more places, which will make external spilling more efficient. This PR is a first step towards that that shows how to pass an Ordering when available and still continue functioning otherwise. It does this using a new implicit parameter with a default value of null. The API is currently only in Scala -- in Java we'd have to add new versions of mapToPair and such that take a Comparator, or a new method to add a "type hint" to an RDD. We can address those later though. Unfortunately requiring all keys to be Comparable would not work without requiring RDDs in general to contain only Comparable types. The reason is that methods such as distinct() and intersection() do a shuffle, but should be usable on RDDs of any type. So ordering will have to remain an optimization for the types that can be ordered. I think this isn't a horrible outcome though because one of the nice things about Spark's API is that it works on objects of *any* type, without requiring you to specify a schema or implement Writable or stuff like that. Author: Matei Zaharia <matei@databricks.com> This patch had conflicts when merged, resolved by Committer: Reynold Xin <rxin@apache.org> Closes #487 from mateiz/ordered-keys and squashes the following commits: bd565f6 [Matei Zaharia] Pass an Ordering to only one version of groupBy because the Scala language spec doesn't allow having an optional parameter on all of them (this was only compiling in Scala 2.10 due to a bug). 4629965 [Matei Zaharia] Add tests for other versions of groupBy 3beae85 [Matei Zaharia] Added a test for implicit orderings 80b7a3b [Matei Zaharia] Add an optional Ordering parameter to PairRDDFunctions.
* [streaming][SPARK-1578] Removed requirement for TTL in StreamingContext.Tathagata Das2014-04-224-49/+14
| | | | | | | | | | Since shuffles and RDDs that are out of context are automatically cleaned by Spark core (using ContextCleaner) there is no need for setting the cleaner TTL while creating a StreamingContext. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #491 from tdas/ttl-fix and squashes the following commits: cf01dc7 [Tathagata Das] Removed requirement for TTL in StreamingContext.
* SPARK-1496: Have jarOfClass return Option[String]Patrick Wendell2014-04-221-1/+1
| | | | | | | | | | A simple change, mostly had to change a bunch of example code. Author: Patrick Wendell <pwendell@gmail.com> Closes #438 from pwendell/jar-of-class and squashes the following commits: aa010ff [Patrick Wendell] SPARK-1496: Have jarOfClass return Option[String]
* [SPARK-1332] Improve Spark Streaming's Network Receiver and InputDStream API ↵Tathagata Das2014-04-2133-585/+1670
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | [WIP] The current Network Receiver API makes it slightly complicated to right a new receiver as one needs to create an instance of BlockGenerator as shown in SocketReceiver https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala#L51 Exposing the BlockGenerator interface has made it harder to improve the receiving process. The API of NetworkReceiver (which was not a very stable API anyways) needs to be change if we are to ensure future stability. Additionally, the functions like streamingContext.socketStream that create input streams, return DStream objects. That makes it hard to expose functionality (say, rate limits) unique to input dstreams. They should return InputDStream or NetworkInputDStream. This is still not yet implemented. This PR is blocked on the graceful shutdown PR #247 Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #300 from tdas/network-receiver-api and squashes the following commits: ea27b38 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3a4777c [Tathagata Das] Renamed NetworkInputDStream to ReceiverInputDStream, and ActorReceiver related stuff. 838dd39 [Tathagata Das] Added more events to the StreamingListener to report errors and stopped receivers. a75c7a6 [Tathagata Das] Address some PR comments and fixed other issues. 91bfa72 [Tathagata Das] Fixed bugs. 8533094 [Tathagata Das] Scala style fixes. 028bde6 [Tathagata Das] Further refactored receiver to allow restarting of a receiver. 43f5290 [Tathagata Das] Made functions that create input streams return InputDStream and NetworkInputDStream, for both Scala and Java. 2c94579 [Tathagata Das] Fixed graceful shutdown by removing interrupts on receiving thread. 9e37a0b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3223e95 [Tathagata Das] Refactored the code that runs the NetworkReceiver into further classes and traits to make them more testable. a36cc48 [Tathagata Das] Refactored the NetworkReceiver API for future stability.
* HOTFIX: Ignore streaming UI testPatrick Wendell2014-04-171-1/+2
| | | | | | | | | | | | This is currently causing many builds to hang. https://issues.apache.org/jira/browse/SPARK-1530 Author: Patrick Wendell <pwendell@gmail.com> Closes #440 from pwendell/uitest-fix and squashes the following commits: 9a143dc [Patrick Wendell] Ignore streaming UI test
* Update ReducedWindowedDStream.scalabaishuo(白硕)2014-04-161-1/+1
| | | | | | | | | | change _slideDuration to _windowDuration Author: baishuo(白硕) <vc_java@hotmail.com> Closes #425 from baishuo/master and squashes the following commits: 6f09ea1 [baishuo(白硕)] Update ReducedWindowedDStream.scala
* SPARK-1488. Resolve scalac feature warnings during buildSean Owen2014-04-148-0/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | For your consideration: scalac currently notes a number of feature warnings during compilation: ``` [warn] there were 65 feature warning(s); re-run with -feature for details ``` Warnings are like: ``` [warn] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SparkContext.scala:1261: implicit conversion method rddToPairRDDFunctions should be enabled [warn] by making the implicit value scala.language.implicitConversions visible. [warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions' [warn] or by setting the compiler option -language:implicitConversions. [warn] See the Scala docs for value scala.language.implicitConversions for a discussion [warn] why the feature should be explicitly enabled. [warn] implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = [warn] ^ ``` scalac is suggesting that it's just best practice to explicitly enable certain language features by importing them where used. This PR simply adds the imports it suggests (and squashes one other Java warning along the way). This leaves just deprecation warnings in the build. Author: Sean Owen <sowen@cloudera.com> Closes #404 from srowen/SPARK-1488 and squashes the following commits: 8598980 [Sean Owen] Quiet scalac warnings about language features by explicitly importing language features. 39bc831 [Sean Owen] Enable -feature in scalac to emit language feature warnings
* [SPARK-1386] Web UI for Spark StreamingTathagata Das2014-04-1116-88/+576
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | When debugging Spark Streaming applications it is necessary to monitor certain metrics that are not shown in the Spark application UI. For example, what is average processing time of batches? What is the scheduling delay? Is the system able to process as fast as it is receiving data? How many records I am receiving through my receivers? While the StreamingListener interface introduced in the 0.9 provided some of this information, it could only be accessed programmatically. A UI that shows information specific to the streaming applications is necessary for easier debugging. This PR introduces such a UI. It shows various statistics related to the streaming application. Here is a screenshot of the UI running on my local machine. http://i.imgur.com/1ooDGhm.png This UI is integrated into the Spark UI running at 4040. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Andrew Or <andrewor14@gmail.com> Closes #290 from tdas/streaming-web-ui and squashes the following commits: fc73ca5 [Tathagata Das] Merge pull request #9 from andrewor14/ui-refactor 642dd88 [Andrew Or] Merge SparkUISuite.scala into UISuite.scala eb30517 [Andrew Or] Merge github.com:apache/spark into ui-refactor f4f4cbe [Tathagata Das] More minor fixes. 34bb364 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 252c566 [Tathagata Das] Merge pull request #8 from andrewor14/ui-refactor e038b4b [Tathagata Das] Addressed Patrick's comments. 125a054 [Andrew Or] Disable serving static resources with gzip 90feb8d [Andrew Or] Address Patrick's comments 89dae36 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 72fe256 [Tathagata Das] Merge pull request #6 from andrewor14/ui-refactor 2fc09c8 [Tathagata Das] Added binary check exclusions aa396d4 [Andrew Or] Rename tabs and pages (No more IndexPage.scala) f8e1053 [Tathagata Das] Added Spark and Streaming UI unit tests. caa5e05 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 585cd65 [Tathagata Das] Merge pull request #5 from andrewor14/ui-refactor 914b8ff [Tathagata Das] Moved utils functions to UIUtils. 548c98c [Andrew Or] Wide refactoring of WebUI, UITab, and UIPage (see commit message) 6de06b0 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui ee6543f [Tathagata Das] Minor changes based on Andrew's comments. fa760fe [Tathagata Das] Fixed long line. 1c0bcef [Tathagata Das] Refactored streaming UI into two files. 1af239b [Tathagata Das] Changed streaming UI to attach itself as a tab with the Spark UI. 827e81a [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 168fe86 [Tathagata Das] Merge pull request #2 from andrewor14/ui-refactor 3e986f8 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui c78c92d [Andrew Or] Remove outdated comment 8f7323b [Andrew Or] End of file new lines, indentation, and imports (minor) 0d61ee8 [Andrew Or] Merge branch 'streaming-web-ui' of github.com:tdas/spark into ui-refactor 9a48fa1 [Andrew Or] Allow adding tabs to SparkUI dynamically + add example 61358e3 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-web-ui 53be2c5 [Tathagata Das] Minor style updates. ed25dfc [Andrew Or] Generalize SparkUI header to display tabs dynamically a37ad4f [Andrew Or] Comments, imports and formatting (minor) cd000b0 [Andrew Or] Merge github.com:apache/spark into ui-refactor 7d57444 [Andrew Or] Refactoring the UI interface to add flexibility aef4dd5 [Tathagata Das] Added Apache licenses. db27bad [Tathagata Das] Added last batch processing time to StreamingUI. 4d86e98 [Tathagata Das] Added basic stats to the StreamingUI and refactored the UI to a Page to make it easier to transition to using SparkUI later. 93f1c69 [Tathagata Das] Added network receiver information to the Streaming UI. 56cc7fb [Tathagata Das] First cut implementation of Streaming UI.
* SPARK-1057 (alternative) Remove fastutilSean Owen2014-04-112-14/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | (This is for discussion at this point -- I'm not suggesting this should be committed.) This is what removing fastutil looks like. Much of it is straightforward, like using `java.io` buffered stream classes, and Guava for murmurhash3. Uses of the `FastByteArrayOutputStream` were a little trickier. In only one case though do I think the change to use `java.io` actually entails an extra array copy. The rest is using `OpenHashMap` and `OpenHashSet`. These are now written in terms of more scala-like operations. `OpenHashMap` is where I made three non-trivial changes to make it work, and they need review: - It is no longer private - The key must be a `ClassTag` - Unless a lot of other code changes, the key type can't enforce being a supertype of `Null` It all works and tests pass, and I think there is reason to believe it's OK from a speed perspective. But what about those last changes? Author: Sean Owen <sowen@cloudera.com> Closes #266 from srowen/SPARK-1057-alternate and squashes the following commits: 2601129 [Sean Owen] Fix Map return type error not previously caught ec65502 [Sean Owen] Updates from matei's review 00bc81e [Sean Owen] Remove use of fastutil and replace with use of java.io, spark.util and Guava classes
* Update WindowedDStream.scalabaishuo(白硕)2014-04-111-1/+1
| | | | | | | | | | update the content of Exception when windowDuration is not multiple of parent.slideDuration Author: baishuo(白硕) <vc_java@hotmail.com> Closes #390 from baishuo/windowdstream and squashes the following commits: 533c968 [baishuo(白硕)] Update WindowedDStream.scala
* Remove Unnecessary Whitespace'sSandeep2014-04-1011-47/+47
| | | | | | | | | | stack these together in a commit else they show up chunk by chunk in different commits. Author: Sandeep <sandeep@techaddict.me> Closes #380 from techaddict/white_space and squashes the following commits: b58f294 [Sandeep] Remove Unnecessary Whitespace's
* Revert "SPARK-729: Closures not always serialized at capture time"Patrick Wendell2014-04-101-4/+4
| | | | This reverts commit 8ca3b2bc90a63b23a03f339e390174cd7a672b40.
* SPARK-729: Closures not always serialized at capture timeWilliam Benton2014-04-091-4/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | [SPARK-729](https://spark-project.atlassian.net/browse/SPARK-729) concerns when free variables in closure arguments to transformations are captured. Currently, it is possible for closures to get the environment in which they are serialized (not the environment in which they are created). There are a few possible approaches to solving this problem and this PR will discuss some of them. The approach I took has the advantage of being simple, obviously correct, and minimally-invasive, but it preserves something that has been bothering me about Spark's closure handling, so I'd like to discuss an alternative and get some feedback on whether or not it is worth pursuing. ## What I did The basic approach I took depends on the work I did for #143, and so this PR is based atop that. Specifically: #143 modifies `ClosureCleaner.clean` to preemptively determine whether or not closures are serializable immediately upon closure cleaning (rather than waiting for an job involving that closure to be scheduled). Thus non-serializable closure exceptions will be triggered by the line defining the closure rather than triggered where the closure is used. Since the easiest way to determine whether or not a closure is serializable is to attempt to serialize it, the code in #143 is creating a serialized closure as part of `ClosureCleaner.clean`. `clean` currently modifies its argument, but the method in `SparkContext` that wraps it to return a value (a reference to the modified-in-place argument). This branch modifies `ClosureCleaner.clean` so that it returns a value: if it is cleaning a serializable closure, it returns the result of deserializing its serialized argument; therefore it is returning a closure with an environment captured at cleaning time. `SparkContext.clean` then returns the result of `ClosureCleaner.clean`, rather than a reference to its modified-in-place argument. I've added tests for this behavior (777a1bc). The pull request as it stands, given the changes in #143, is nearly trivial. There is some overhead from deserializing the closure, but it is minimal and the benefit of obvious operational correctness (vs. a more sophisticated but harder-to-validate transformation in `ClosureCleaner`) seems pretty important. I think this is a fine way to solve this problem, but it's not perfect. ## What we might want to do The thing that has been bothering me about Spark's handling of closures is that it seems like we should be able to statically ensure that cleaning and serialization happen exactly once for a given closure. If we serialize a closure in order to determine whether or not it is serializable, we should be able to hang on to the generated byte buffer and use it instead of re-serializing the closure later. By replacing closures with instances of a sum type that encodes whether or not a closure has been cleaned or serialized, we could handle clean, to-be-cleaned, and serialized closures separately with case matches. Here's a somewhat-concrete sketch (taken from my git stash) of what this might look like: ```scala package org.apache.spark.util import java.nio.ByteBuffer import scala.reflect.ClassManifest sealed abstract class ClosureBox[T] { def func: T } final case class RawClosure[T](func: T) extends ClosureBox[T] {} final case class CleanedClosure[T](func: T) extends ClosureBox[T] {} final case class SerializedClosure[T](func: T, bytebuf: ByteBuffer) extends ClosureBox[T] {} object ClosureBoxImplicits { implicit def closureBoxFromFunc[T <: AnyRef](fun: T) = new RawClosure[T](fun) } ``` With these types declared, we'd be able to change `ClosureCleaner.clean` to take a `ClosureBox[T=>U]` (possibly generated by implicit conversion) and return a `ClosureBox[T=>U]` (either a `CleanedClosure[T=>U]` or a `SerializedClosure[T=>U]`, depending on whether or not serializability-checking was enabled) instead of a `T=>U`. A case match could thus short-circuit cleaning or serializing closures that had already been cleaned or serialized (both in `ClosureCleaner` and in the closure serializer). Cleaned-and-serialized closures would be represented by a boxed tuple of the original closure and a serialized copy (complete with an environment quiesced at transformation time). Additional implicit conversions could convert from `ClosureBox` instances to the underlying function type where appropriate. Tracking this sort of state in the type system seems like the right thing to do to me. ### Why we might not want to do that _It's pretty invasive._ Every function type used by every `RDD` subclass would have to change to reflect that they expected a `ClosureBox[T=>U]` instead of a `T=>U`. This obscures what's going on and is not a little ugly. Although I really like the idea of using the type system to enforce the clean-or-serialize once discipline, it might not be worth adding another layer of types (even if we could hide some of the extra boilerplate with judicious application of implicit conversions). _It statically guarantees a property whose absence is unlikely to cause any serious problems as it stands._ It appears that all closures are currently dynamically cleaned once and it's not obvious that repeated closure-cleaning is likely to be a problem in the future. Furthermore, serializing closures is relatively cheap, so doing it once to check for serialization and once again to actually ship them across the wire doesn't seem like a big deal. Taken together, these seem like a high price to pay for statically guaranteeing that closures are operated upon only once. ## Other possibilities I felt like the serialize-and-deserialize approach was best due to its obvious simplicity. But it would be possible to do a more sophisticated transformation within `ClosureCleaner.clean`. It might also be possible for `clean` to modify its argument in a way so that whether or not a given closure had been cleaned would be apparent upon inspection; this would buy us some of the operational benefits of the `ClosureBox` approach but not the static cleanliness. I'm interested in any feedback or discussion on whether or not the problems with the type-based approach indeed outweigh the advantage, as well as of approaches to this issue and to closure handling in general. Author: William Benton <willb@redhat.com> Closes #189 from willb/spark-729 and squashes the following commits: f4cafa0 [William Benton] Stylistic changes and cleanups b3d9c86 [William Benton] Fixed style issues in tests 9b56ce0 [William Benton] Added array-element capture test 97e9d91 [William Benton] Split closure-serializability failure tests 12ef6e3 [William Benton] Skip proactive closure capture for runJob 8ee3ee7 [William Benton] Predictable closure environment capture 12c63a7 [William Benton] Added tests for variable capture in closures d6e8dd6 [William Benton] Don't check serializability of DStream transforms. 4ecf841 [William Benton] Make proactive serializability checking optional. d8df3db [William Benton] Adds proactive closure-serializablilty checking 21b4b06 [William Benton] Test cases for SPARK-897. d5947b3 [William Benton] Ensure assertions in Graph.apply are asserted.
* Spark 1271: Co-Group and Group-By should pass Iterable[X]Holden Karau2014-04-085-50/+96
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Author: Holden Karau <holden@pigscanfly.ca> Closes #242 from holdenk/spark-1320-cogroupandgroupshouldpassiterator and squashes the following commits: f289536 [Holden Karau] Fix bad merge, should have been Iterable rather than Iterator 77048f8 [Holden Karau] Fix merge up to master d3fe909 [Holden Karau] use toSeq instead 7a092a3 [Holden Karau] switch resultitr to resultiterable eb06216 [Holden Karau] maybe I should have had a coffee first. use correct import for guava iterables c5075aa [Holden Karau] If guava 14 had iterables 2d06e10 [Holden Karau] Fix Java 8 cogroup tests for the new API 11e730c [Holden Karau] Fix streaming tests 66b583d [Holden Karau] Fix the core test suite to compile 4ed579b [Holden Karau] Refactor from iterator to iterable d052c07 [Holden Karau] Python tests now pass with iterator pandas 3bcd81d [Holden Karau] Revert "Try and make pickling list iterators work" cd1e81c [Holden Karau] Try and make pickling list iterators work c60233a [Holden Karau] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well 88a5cef [Holden Karau] Fix cogroup test in JavaAPISuite for streaming a5ee714 [Holden Karau] oops, was checking wrong iterator e687f21 [Holden Karau] Fix groupbykey test in JavaAPISuite of streaming ec8cc3e [Holden Karau] Fix test issues\! 4b0eeb9 [Holden Karau] Switch cast in PairDStreamFunctions fa395c9 [Holden Karau] Revert "Add a join based on the problem in SVD" ec99e32 [Holden Karau] Revert "Revert this but for now put things in list pandas" b692868 [Holden Karau] Revert 7e533f7 [Holden Karau] Fix the bug 8a5153a [Holden Karau] Revert me, but we have some stuff to debug b4e86a9 [Holden Karau] Add a join based on the problem in SVD c4510e2 [Holden Karau] Revert this but for now put things in list pandas b4e0b1d [Holden Karau] Fix style issues 71e8b9f [Holden Karau] I really need to stop calling size on iterators, it is the path of sadness. b1ae51a [Holden Karau] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work 37888ec [Holden Karau] core/tests now pass 249abde [Holden Karau] org.apache.spark.rdd.PairRDDFunctionsSuite passes 6698186 [Holden Karau] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy" fe992fe [Holden Karau] hmmm try and fix up basic operation suite 172705c [Holden Karau] Fix Java API suite caafa63 [Holden Karau] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy 88b3329 [Holden Karau] Fix groupbykey to actually give back an iterator 4991af6 [Holden Karau] Fix some tests be50246 [Holden Karau] Calling size on an iterator is not so good if we want to use it after 687ffbc [Holden Karau] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures
* [SPARK-1331] Added graceful shutdown to Spark StreamingTathagata Das2014-04-0814-204/+539
| | | | | | | | | | | | | | | | | | | | | Current version of StreamingContext.stop() directly kills all the data receivers (NetworkReceiver) without waiting for the data already received to be persisted and processed. This PR provides the fix. Now, when the StreamingContext.stop() is called, the following sequence of steps will happen. 1. The driver will send a stop signal to all the active receivers. 2. Each receiver, when it gets a stop signal from the driver, first stop receiving more data, then waits for the thread that persists data blocks to BlockManager to finish persisting all receive data, and finally quits. 3. After all the receivers have stopped, the driver will wait for the Job Generator and Job Scheduler to finish processing all the received data. It also fixes the semantics of StreamingContext.start and stop. It will throw appropriate errors and warnings if stop() is called before start(), stop() is called twice, etc. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #247 from tdas/graceful-shutdown and squashes the following commits: 61c0016 [Tathagata Das] Updated MIMA binary check excludes. ae1d39b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into graceful-shutdown 6b59cfc [Tathagata Das] Minor changes based on Andrew's comment on PR. d0b8d65 [Tathagata Das] Reduced time taken by graceful shutdown unit test. f55bc67 [Tathagata Das] Fix scalastyle c69b3a7 [Tathagata Das] Updates based on Patrick's comments. c43b8ae [Tathagata Das] Added graceful shutdown to Spark Streaming.
* [SPARK-1103] Automatic garbage collection of RDD, shuffle and broadcast dataTathagata Das2014-04-071-1/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR allows Spark to automatically cleanup metadata and data related to persisted RDDs, shuffles and broadcast variables when the corresponding RDDs, shuffles and broadcast variables fall out of scope from the driver program. This is still a work in progress as broadcast cleanup has not been implemented. **Implementation Details** A new class `ContextCleaner` is responsible cleaning all the state. It is instantiated as part of a `SparkContext`. RDD and ShuffleDependency classes have overridden `finalize()` function that gets called whenever their instances go out of scope. The `finalize()` function enqueues the object’s identifier (i.e. RDD ID, shuffle ID, etc.) with the `ContextCleaner`, which is a very short and cheap operation and should not significantly affect the garbage collection mechanism. The `ContextCleaner`, on a different thread, performs the cleanup, whose details are given below. *RDD cleanup:* `ContextCleaner` calls `RDD.unpersist()` is used to cleanup persisted RDDs. Regarding metadata, the DAGScheduler automatically cleans up all metadata related to a RDD after all jobs have completed. Only the `SparkContext.persistentRDDs` keeps strong references to persisted RDDs. The `TimeStampedHashMap` used for that has been replaced by `TimeStampedWeakValueHashMap` that keeps only weak references to the RDDs, allowing them to be garbage collected. *Shuffle cleanup:* New BlockManager message `RemoveShuffle(<shuffle ID>)` asks the `BlockManagerMaster` and currently active `BlockManager`s to delete all the disk blocks related to the shuffle ID. `ContextCleaner` cleans up shuffle data using this message and also cleans up the metadata in the `MapOutputTracker` of the driver. The `MapOutputTracker` at the workers, that caches the shuffle metadata, maintains a `BoundedHashMap` to limit the shuffle information it caches. Refetching the shuffle information from the driver is not too costly. *Broadcast cleanup:* To be done. [This PR](https://github.com/apache/incubator-spark/pull/543/) adds mechanism for explicit cleanup of broadcast variables. `Broadcast.finalize()` will enqueue its own ID with ContextCleaner and the PRs mechanism will be used to unpersist the Broadcast data. *Other cleanup:* `ShuffleMapTask` and `ResultTask` caches tasks and used TTL based cleanup (using `TimeStampedHashMap`), so nothing got cleaned up if TTL was not set. Instead, they now use `BoundedHashMap` to keep a limited number of map output information. Cost of repopulating the cache if necessary is very small. **Current state of implementation** Implemented RDD and shuffle cleanup. Things left to be done are. - Cleaning up for broadcast variable still to be done. - Automatic cleaning up keys with empty weak refs as values in `TimeStampedWeakValueHashMap` Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Andrew Or <andrewor14@gmail.com> Author: Roman Pastukhov <ignatich@mail.ru> Closes #126 from tdas/state-cleanup and squashes the following commits: 61b8d6e [Tathagata Das] Fixed issue with Tachyon + new BlockManager methods. f489fdc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup d25a86e [Tathagata Das] Fixed stupid typo. cff023c [Tathagata Das] Fixed issues based on Andrew's comments. 4d05314 [Tathagata Das] Scala style fix. 2b95b5e [Tathagata Das] Added more documentation on Broadcast implementations, specially which blocks are told about to the driver. Also, fixed Broadcast API to hide destroy functionality. 41c9ece [Tathagata Das] Added more unit tests for BlockManager, DiskBlockManager, and ContextCleaner. 6222697 [Tathagata Das] Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite. 104a89a [Tathagata Das] Fixed failing BroadcastSuite unit tests by introducing blocking for removeShuffle and removeBroadcast in BlockManager* a430f06 [Tathagata Das] Fixed compilation errors. b27f8e8 [Tathagata Das] Merge pull request #3 from andrewor14/cleanup cd72d19 [Andrew Or] Make automatic cleanup configurable (not documented) ada45f0 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup a2cc8bc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup c5b1d98 [Andrew Or] Address Patrick's comments a6460d4 [Andrew Or] Merge github.com:apache/spark into cleanup 762a4d8 [Tathagata Das] Merge pull request #1 from andrewor14/cleanup f0aabb1 [Andrew Or] Correct semantics for TimeStampedWeakValueHashMap + add tests 5016375 [Andrew Or] Address TD's comments 7ed72fb [Andrew Or] Fix style test fail + remove verbose test message regarding broadcast 634a097 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup 7edbc98 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into state-cleanup 8557c12 [Andrew Or] Merge github.com:apache/spark into cleanup e442246 [Andrew Or] Merge github.com:apache/spark into cleanup 88904a3 [Andrew Or] Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap fbfeec8 [Andrew Or] Add functionality to query executors for their local BlockStatuses 34f436f [Andrew Or] Generalize BroadcastBlockId to remove BroadcastHelperBlockId 0d17060 [Andrew Or] Import, comments, and style fixes (minor) c92e4d9 [Andrew Or] Merge github.com:apache/spark into cleanup f201a8d [Andrew Or] Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap e95479c [Andrew Or] Add tests for unpersisting broadcast 544ac86 [Andrew Or] Clean up broadcast blocks through BlockManager* d0edef3 [Andrew Or] Add framework for broadcast cleanup ba52e00 [Andrew Or] Refactor broadcast classes c7ccef1 [Andrew Or] Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-spark into cleanup 6c9dcf6 [Tathagata Das] Added missing Apache license d2f8b97 [Tathagata Das] Removed duplicate unpersistRDD. a007307 [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup 620eca3 [Tathagata Das] Changes based on PR comments. f2881fd [Tathagata Das] Changed ContextCleaner to use ReferenceQueue instead of finalizer e1fba5f [Tathagata Das] Style fix 892b952 [Tathagata Das] Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanup shuffle metadata in MapOutputTrackerWorker. a7260d3 [Tathagata Das] Added try-catch in context cleaner and null value cleaning in TimeStampedWeakValueHashMap. e61daa0 [Tathagata Das] Modifications based on the comments on PR 126. ae9da88 [Tathagata Das] Removed unncessary TimeStampedHashMap from DAGScheduler, added try-catches in finalize() methods, and replaced ArrayBlockingQueue to LinkedBlockingQueue to avoid blocking in Java's finalizing thread. cb0a5a6 [Tathagata Das] Fixed docs and styles. a24fefc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup 8512612 [Tathagata Das] Changed TimeStampedHashMap to use WrappedJavaHashMap. e427a9e [Tathagata Das] Added ContextCleaner to automatically clean RDDs and shuffles when they fall out of scope. Also replaced TimeStampedHashMap to BoundedHashMaps and TimeStampedWeakValueHashMap for the necessary hashmap behavior. 80dd977 [Roman Pastukhov] Fix for Broadcast unpersist patch. 1e752f1 [Roman Pastukhov] Added unpersist method to Broadcast.
* HOTFIX: Disable actor input stream test.Patrick Wendell2014-04-071-2/+2
| | | | | | | | | | This test makes incorrect assumptions about the behavior of Thread.sleep(). Author: Patrick Wendell <pwendell@gmail.com> Closes #347 from pwendell/stream-tests and squashes the following commits: 10e09e0 [Patrick Wendell] HOTFIX: Disable actor input stream.
* SPARK-1387. Update build plugins, avoid plugin version warning, centralize ↵Sean Owen2014-04-061-1/+0
| | | | | | | | | | | | | | | | | | | | versions Another handful of small build changes to organize and standardize a bit, and avoid warnings: - Update Maven plugin versions for good measure - Since plugins need maven 3.0.4 already, require it explicitly (<3.0.4 had some bugs anyway) - Use variables to define versions across dependencies where they should move in lock step - ... and make this consistent between Maven/SBT OK, I also updated the JIRA URL while I was at it here. Author: Sean Owen <sowen@cloudera.com> Closes #291 from srowen/SPARK-1387 and squashes the following commits: 461eca1 [Sean Owen] Couldn't resist also updating JIRA location to new one c2d5cc5 [Sean Owen] Update plugins and Maven version; use variables consistently across Maven/SBT to define dependency versions that should stay in step.
* SPARK-1365 [HOTFIX] Fix RateLimitedOutputStream testPatrick Wendell2014-03-311-2/+3
| | | | | | | | | | | This test needs to be fixed. It currently depends on Thread.sleep() having exact-timing semantics, which is not a valid assumption. Author: Patrick Wendell <pwendell@gmail.com> Closes #277 from pwendell/rate-limited-stream and squashes the following commits: 6c0ff81 [Patrick Wendell] SPARK-1365: Fix RateLimitedOutputStream test
* SPARK-1096, a space after comment start style checker.Prashant Sharma2014-03-287-11/+10
| | | | | | | | | | | | | Author: Prashant Sharma <prashant.s@imaginea.com> Closes #124 from ScrapCodes/SPARK-1096/scalastyle-comment-check and squashes the following commits: 214135a [Prashant Sharma] Review feedback. 5eba88c [Prashant Sharma] Fixed style checks for ///+ comments. e54b2f8 [Prashant Sharma] improved message, work around. 83e7144 [Prashant Sharma] removed dependency on scalastyle in plugin, since scalastyle sbt plugin already depends on the right version. Incase we update the plugin we will have to adjust our spark-style project to depend on right scalastyle version. 810a1d6 [Prashant Sharma] SPARK-1096, a space after comment style checker. ba33193 [Prashant Sharma] scala style as a project
* Spark 1095 : Adding explicit return types to all public methodsNirmalReddy2014-03-265-15/+33
| | | | | | | | | | | | | | Excluded those that are self-evident and the cases that are discussed in the mailing list. Author: NirmalReddy <nirmal_reddy2000@yahoo.com> Author: NirmalReddy <nirmal.reddy@imaginea.com> Closes #168 from NirmalReddy/Spark-1095 and squashes the following commits: ac54b29 [NirmalReddy] import misplaced 8c5ff3e [NirmalReddy] Changed syntax of unit returning methods 02d0778 [NirmalReddy] fixed explicit types in all the other packages 1c17773 [NirmalReddy] fixed explicit types in core package
* SPARK-1316. Remove use of Commons IOSean Owen2014-03-255-35/+27
| | | | | | | | | | | | | | (This follows from a side point on SPARK-1133, in discussion of the PR: https://github.com/apache/spark/pull/164 ) Commons IO is barely used in the project, and can easily be replaced with equivalent calls to Guava or the existing Spark `Utils.scala` class. Removing a dependency feels good, and this one in particular can get a little problematic since Hadoop uses it too. Author: Sean Owen <sowen@cloudera.com> Closes #226 from srowen/SPARK-1316 and squashes the following commits: 21efef3 [Sean Owen] Remove use of Commons IO
* SPARK-1251 Support for optimizing and executing structured queriesMichael Armbrust2014-03-202-3/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This pull request adds support to Spark for working with structured data using a simple SQL dialect, HiveQL and a Scala Query DSL. *This is being contributed as a new __alpha component__ to Spark and does not modify Spark core or other components.* The code is broken into three primary components: - Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions. - Execution (sql/core) - A query planner / execution engine for translating Catalyst’s logical query plans into Spark RDDs. This component also includes a new public interface, SqlContext, that allows users to execute SQL or structured scala queries against existing RDDs and Parquet files. - Hive Metastore Support (sql/hive) - An extension of SqlContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs. A more complete design of this new component can be found in [the associated JIRA](https://spark-project.atlassian.net/browse/SPARK-1251). [An updated version of the Spark documentation, including API Docs for all three sub-components,](http://people.apache.org/~pwendell/catalyst-docs/sql-programming-guide.html) is also available for review. With this PR comes support for inferring the schema of existing RDDs that contain case classes. Using this information, developers can now express structured queries that are automatically compiled into RDD operations. ```scala // Define the schema using a case class. case class Person(name: String, age: Int) val people: RDD[Person] = sc.textFile("people.txt").map(_.split(",")).map(p => Person(p(0), p(1).toInt)) // The following is the same as 'SELECT name FROM people WHERE age >= 10 && age <= 19' val teenagers = people.where('age >= 10).where('age <= 19).select('name).toRdd ``` RDDs can also be registered as Tables, allowing SQL queries to be written over them. ```scala people.registerAsTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 10 && age <= 19") ``` The results of queries are themselves RDDs and support standard RDD operations: ```scala teenagers.map(t => "Name: " + t(0)).collect().foreach(println) ``` Finally, with the optional Hive support, users can read and write data located in existing Apache Hive deployments using HiveQL. ```scala sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL sql("SELECT key, value FROM src").collect().foreach(println) ``` ## Relationship to Shark Unlike Shark, Spark SQL does not act as a drop in replacement for Hive or the HiveServer. Instead this new feature is intended to make it easier for Spark developers to run queries over structured data, using either SQL or the query DSL. After this sub-project graduates from Alpha status it will likely become a new optimizer/backend for the Shark project. Author: Michael Armbrust <michael@databricks.com> Author: Yin Huai <huaiyin.thu@gmail.com> Author: Reynold Xin <rxin@apache.org> Author: Lian, Cheng <rhythm.mail@gmail.com> Author: Andre Schumacher <andre.schumacher@iki.fi> Author: Yin Huai <huai@cse.ohio-state.edu> Author: Timothy Chen <tnachen@gmail.com> Author: Cheng Lian <lian.cs.zju@gmail.com> Author: Timothy Chen <tnachen@apache.org> Author: Henry Cook <henry.m.cook+github@gmail.com> Author: Mark Hamstra <markhamstra@gmail.com> Closes #146 from marmbrus/catalyst and squashes the following commits: 458bd1b [Michael Armbrust] Update people.txt 0d638c3 [Michael Armbrust] Typo fix from @ash211. bdab185 [Michael Armbrust] Address another round of comments: * Doc examples can now copy/paste into spark-shell. * SQLContext is serializable * Minor parser bugs fixed * Self-joins of RDDs now handled correctly. * Removed deprecated examples * Removed deprecated parquet docs * Made more of the API private * Copied all the DSLQuery tests and rewrote them as SQLQueryTests 778299a [Michael Armbrust] Fix some old links to spark-project.org fead0b6 [Michael Armbrust] Create a new RDD type, SchemaRDD, that is now the return type for all SQL operations. This improves the old API by reducing the number of implicits that are required, and avoids throwing away schema information when returning an RDD to the user. This change also makes it slightly less verbose to run language integrated queries. fee847b [Michael Armbrust] Merge remote-tracking branch 'origin/master' into catalyst, integrating changes to serialization for ShuffledRDD. 48a99bc [Michael Armbrust] Address first round of feedback. 461581c [Michael Armbrust] Blacklist test that depends on JVM specific rounding behaviour adcf1a4 [Henry Cook] Update sql-programming-guide.md 9dffbfa [Michael Armbrust] Style fixes. Add downloading of test cases to jenkins. 6978dd8 [Michael Armbrust] update docs, add apache license 1d0eb63 [Michael Armbrust] update changes with spark core e5e1d6b [Michael Armbrust] Remove travis configuration. c2efad6 [Michael Armbrust] First draft of SQL documentation. 013f62a [Michael Armbrust] Fix documentation / code style. c01470f [Michael Armbrust] Clean up example 2f22454 [Michael Armbrust] WIP: Parquet example. ce8073b [Michael Armbrust] clean up implicits. f7d992d [Michael Armbrust] Naming / spelling. 9eb0294 [Michael Armbrust] Bring expressions implicits into SqlContext. d2d9678 [Michael Armbrust] Make sure hive isn't in the assembly jar. Create a separate, optional Hive assembly that is used when present. 8b35e0a [Michael Armbrust] address feedback, work on DSL 5d71074 [Michael Armbrust] Merge pull request #62 from AndreSchumacher/parquet_file_fixes f93aa39 [Andre Schumacher] Better handling of path names in ParquetRelation 1a4bbd9 [Michael Armbrust] Merge pull request #60 from marmbrus/maven 3386e4f [Michael Armbrust] Merge pull request #58 from AndreSchumacher/parquet_fixes 3447c3e [Michael Armbrust] Don't override the metastore / warehouse in non-local/test hive context. 7233a74 [Michael Armbrust] initial support for maven builds f0ba39e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into maven 7386a9f [Michael Armbrust] Initial example programs using spark sql. aeaef54 [Andre Schumacher] Removing unnecessary Row copying and reverting some changes to MutableRow 7ca4b4e [Andre Schumacher] Improving checks in Parquet tests 5bacdc0 [Andre Schumacher] Moving towards mutable rows inside ParquetRowSupport 54637ec [Andre Schumacher] First part of second round of code review feedback c2a658d [Michael Armbrust] Merge pull request #55 from marmbrus/mutableRows ba28849 [Michael Armbrust] code review comments. d994333 [Michael Armbrust] Remove copies before shuffle, this required changing the default shuffle serialization. 9049cf0 [Michael Armbrust] Extend MutablePair interface to support easy syntax for in-place updates. Also add a constructor so that it can be serialized out-of-the-box. 959bdf0 [Michael Armbrust] Don't silently swallow all KryoExceptions, only the one that indicates the end of a stream. d371393 [Michael Armbrust] Add a framework for dealing with mutable rows to reduce the number of object allocations that occur in the critical path. c9f8fb3 [Michael Armbrust] Merge pull request #53 from AndreSchumacher/parquet_support 3c3f962 [Michael Armbrust] Fix a bug due to array reuse. This will need to be revisited after we merge the mutable row PR. 7d0f13e [Michael Armbrust] Update parquet support with master. 9d419a6 [Michael Armbrust] Merge remote-tracking branch 'catalyst/catalystIntegration' into parquet_support 0040ae6 [Andre Schumacher] Feedback from code review 1ce01c7 [Michael Armbrust] Merge pull request #56 from liancheng/unapplySeqForRow 70e489d [Cheng Lian] Fixed a spelling typo 6d315bb [Cheng Lian] Added Row.unapplySeq to extract fields from a Row object. 8d5da5e [Michael Armbrust] modify compute-classpath.sh to include datanucleus jars explicitly 99e61fb [Michael Armbrust] Merge pull request #51 from marmbrus/expressionEval 7b9d142 [Michael Armbrust] Update travis to increase permgen size. da9afbd [Michael Armbrust] Add byte wrappers for hive UDFS. 6fdefe6 [Michael Armbrust] Port sbt improvements from master. 296fe50 [Michael Armbrust] Address review feedback. d7fbc3a [Michael Armbrust] Several performance enhancements and simplifications of the expression evaluation framework. 3bda72d [Andre Schumacher] Adding license banner to new files 3ac9eb0 [Andre Schumacher] Rebasing to new main branch c863bed [Andre Schumacher] Codestyle checks 61e3bfb [Andre Schumacher] Adding WriteToFile operator and rewriting ParquetQuerySuite 3321195 [Andre Schumacher] Fixing one import in ParquetQueryTests.scala 3a0a552 [Andre Schumacher] Reorganizing Parquet table operations 18fdc44 [Andre Schumacher] Reworking Parquet metadata in relation and adding CREATE TABLE AS for Parquet tables 75262ee [Andre Schumacher] Integrating operations on Parquet files into SharkStrategies f347273 [Andre Schumacher] Adding ParquetMetaData extraction, fixing schema projection 6a6bf98 [Andre Schumacher] Added column projections to ParquetTableScan 0f17d7b [Andre Schumacher] Rewriting ParquetRelation tests with RowWriteSupport a11e364 [Andre Schumacher] Adding Parquet RowWriteSupport 6ad05b3 [Andre Schumacher] Moving ParquetRelation to spark.sql core eb0e521 [Andre Schumacher] Fixing package names and other problems that came up after the rebase 99a9209 [Andre Schumacher] Expanding ParquetQueryTests to cover all primitive types b33e47e [Andre Schumacher] First commit of Parquet import of primitive column types c334386 [Michael Armbrust] Initial support for generating schema's based on case classes. 608a29e [Michael Armbrust] Add hive as a repl dependency 7413ac2 [Michael Armbrust] make test downloading quieter. 4d57d0e [Michael Armbrust] Fix test execution on travis. 5f2963c [Michael Armbrust] naming and continuous compilation fixes. f5e7492 [Michael Armbrust] Add Apache license. Make naming more consistent. 3ac9416 [Michael Armbrust] Merge support for working with schema-ed RDDs using catalyst in as a spark subproject. 2225431 [Michael Armbrust] Merge pull request #48 from marmbrus/minorFixes d393d2a [Michael Armbrust] Review Comments: Add comment to map that adds a sub query. 24eaa79 [Michael Armbrust] fix > 100 chars 6e04e5b [Michael Armbrust] Add insertIntoTable to the DSL. df88f01 [Michael Armbrust] add a simple test for aggregation 18a861b [Michael Armbrust] Correctly convert nested products into nested rows when turning scala data into catalyst data. b922511 [Michael Armbrust] Fix insertion of nested types into hive tables. 5fe7de4 [Michael Armbrust] Move table creation out of rule into a separate function. a430895 [Michael Armbrust] Planning for logical Repartition operators. 532dd37 [Michael Armbrust] Allow the local warehouse path to be specified. 4905b2b [Michael Armbrust] Add more efficient TopK that avoids global sort for logical Sort => StopAfter. 8c01c24 [Michael Armbrust] Move definition of Row out of execution to top level sql package. c9116a6 [Michael Armbrust] Add combiner to avoid NPE when spark performs external aggregation. 29effad [Michael Armbrust] Include alias in attributes that are produced by overridden tables. 9990ec7 [Michael Armbrust] Merge pull request #28 from liancheng/columnPruning f22df3a [Michael Armbrust] Merge pull request #37 from yhuai/SerDe cf4db59 [Lian, Cheng] Added golden answers for PruningSuite 54f165b [Lian, Cheng] Fixed spelling typo in two golden answer file names 2682f72 [Lian, Cheng] Merge remote-tracking branch 'origin/master' into columnPruning c5a4fab [Lian, Cheng] Merge branch 'master' into columnPruning f670c8c [Yin Huai] Throw a NotImplementedError for not supported clauses in a CTAS query. 128a9f8 [Yin Huai] Minor changes. 017872c [Yin Huai] Remove stats20 from whitelist. a1a4776 [Yin Huai] Update comments. feb022c [Yin Huai] Partitioning key should be case insensitive. 555fb1d [Yin Huai] Correctly set the extension for a text file. d00260b [Yin Huai] Strips backticks from partition keys. 334aace [Yin Huai] New golden files. a40d6d6 [Yin Huai] Loading the static partition specified in a INSERT INTO/OVERWRITE query. 428aff5 [Yin Huai] Distinguish `INSERT INTO` and `INSERT OVERWRITE`. eea75c5 [Yin Huai] Correctly set codec. 45ffb86 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew e089627 [Yin Huai] Code style. 563bb22 [Yin Huai] Set compression info in FileSinkDesc. 35c9a8a [Michael Armbrust] Merge pull request #46 from marmbrus/reviewFeedback bdab5ed [Yin Huai] Add a TODO for loading data into partitioned tables. 5495fab [Yin Huai] Remove cloneRecords which is no longer needed. 1596e1b [Yin Huai] Cleanup imports to make IntelliJ happy. 3bb272d [Michael Armbrust] move org.apache.spark.sql package.scala to the correct location. 8506c17 [Michael Armbrust] Address review feedback. 3cb4f2e [Michael Armbrust] Merge pull request #45 from tnachen/master 9ad474d [Michael Armbrust] Merge pull request #44 from marmbrus/sampling 566fd66 [Timothy Chen] Whitelist tests and add support for Binary type 69adf72 [Yin Huai] Set cloneRecords to false. a9c3188 [Timothy Chen] Fix udaf struct return 346f828 [Yin Huai] Move SharkHadoopWriter to the correct location. 59e37a3 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew ed3a1d1 [Yin Huai] Load data directly into Hive. 7f206b5 [Michael Armbrust] Add support for hive TABLESAMPLE PERCENT. b6de691 [Michael Armbrust] Merge pull request #43 from liancheng/fixMakefile 1f6260d [Lian, Cheng] Fixed package name and test suite name in Makefile 5ae010f [Michael Armbrust] Merge pull request #42 from markhamstra/non-ascii 678341a [Mark Hamstra] Replaced non-ascii text 887f928 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew 1f7d00a [Reynold Xin] Merge pull request #41 from marmbrus/splitComponents 7588a57 [Michael Armbrust] Break into 3 major components and move everything into the org.apache.spark.sql package. bc9a12c [Michael Armbrust] Move hive test files. 5720d2b [Lian, Cheng] Fixed comment typo f0c3742 [Lian, Cheng] Refactored PhysicalOperation f235914 [Lian, Cheng] Test case udf_regex and udf_like need BooleanWritable registered cf691df [Lian, Cheng] Added the PhysicalOperation to generalize ColumnPrunings 2407a21 [Lian, Cheng] Added optimized logical plan to debugging output a7ad058 [Michael Armbrust] Merge pull request #40 from marmbrus/includeGoldens 9329820 [Michael Armbrust] add golden answer files to repository dce0593 [Michael Armbrust] move golden answer to the source code directory. 964368f [Michael Armbrust] Merge pull request #39 from marmbrus/lateralView 7785ee6 [Michael Armbrust] Tighten visibility based on comments. 341116c [Michael Armbrust] address comments. 0e6c1d7 [Reynold Xin] Merge pull request #38 from yhuai/parseDBNameInCTAS 2897deb [Michael Armbrust] fix scaladoc 7123225 [Yin Huai] Correctly parse the db name and table name in INSERT queries. b376d15 [Michael Armbrust] fix newlines at EOF 5cc367c [Michael Armbrust] use berkeley instead of cloudbees ff5ea3f [Michael Armbrust] new golden db92adc [Michael Armbrust] more tests passing. clean up logging. 740febb [Michael Armbrust] Tests for tgfs. 0ce61b0 [Michael Armbrust] Docs for GenericHiveUdtf. ba8897f [Michael Armbrust] Merge remote-tracking branch 'yin/parseDBNameInCTAS' into lateralView dd00b7e [Michael Armbrust] initial implementation of generators. ea76cf9 [Michael Armbrust] Add NoRelation to planner. bea4b7f [Michael Armbrust] Add SumDistinct. 016b489 [Michael Armbrust] fix typo. acb9566 [Michael Armbrust] Correctly type attributes of CTAS. 8841eb8 [Michael Armbrust] Rename Transform -> ScriptTransformation. 02ff8e4 [Yin Huai] Correctly parse the db name and table name in a CTAS query. 5e4d9b4 [Michael Armbrust] Merge pull request #35 from marmbrus/smallFixes 5479066 [Reynold Xin] Merge pull request #36 from marmbrus/partialAgg 8017afb [Michael Armbrust] fix copy paste error. dc6353b [Michael Armbrust] turn off deprecation cab1a84 [Michael Armbrust] Fix PartialAggregate inheritance. 883006d [Michael Armbrust] improve tests. 32b615b [Michael Armbrust] add override to asPartial. e1999f9 [Yin Huai] Use Deserializer and Serializer instead of AbstractSerDe. f94345c [Michael Armbrust] fix doc link d8cb805 [Michael Armbrust] Implement partial aggregation. ccdb07a [Michael Armbrust] Fix bug where averages of strings are turned into sums of strings. Remove a blank line. b4be6a5 [Michael Armbrust] better logging when applying rules. 67128b8 [Reynold Xin] Merge pull request #30 from marmbrus/complex cb57459 [Michael Armbrust] blacklist machine specific test. 2f27604 [Michael Armbrust] Address comments / style errors. 389525d [Michael Armbrust] update golden, blacklist mr. e3c10bd [Michael Armbrust] update whitelist. 44d343c [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into complex 42ec4af [Michael Armbrust] improve complex type support in hive udfs/udafs. ab5bff3 [Michael Armbrust] Support for get item of map types. 1679554 [Michael Armbrust] add toString for if and IS NOT NULL. ab9a131 [Michael Armbrust] when UDFs fail they should return null. 25288d0 [Michael Armbrust] Implement [] for arrays and maps. e7933e9 [Michael Armbrust] fix casting bug when working with fractional expressions. 010accb [Michael Armbrust] add tinyint to metastore type parser. 7a0f543 [Michael Armbrust] Avoid propagating types from unresolved nodes. ac9d7de [Michael Armbrust] Resolve *s in Transform clauses. 692a477 [Michael Armbrust] Support for wrapping arrays to be written into hive tables. 92e4158 [Reynold Xin] Merge pull request #32 from marmbrus/tooManyProjects 9c06778 [Michael Armbrust] fix serialization issues, add JavaStringObjectInspector. 72a003d [Michael Armbrust] revert regex change 7661b6c [Michael Armbrust] blacklist machines specific tests aa430e7 [Michael Armbrust] Update .travis.yml e4def6b [Michael Armbrust] set dataType for HiveGenericUdfs. 5e54aa6 [Michael Armbrust] quotes for struct field names. bbec500 [Michael Armbrust] update test coverage, new golden 3734a94 [Michael Armbrust] only quote string types. 3f9e519 [Michael Armbrust] use names w/ boolean args 5b3d2c8 [Michael Armbrust] implement distinct. 5b33216 [Michael Armbrust] work on decimal support. 2c6deb3 [Michael Armbrust] improve printing compatibility. 35a70fb [Michael Armbrust] multi-letter field names. a9388fb [Michael Armbrust] printing for map types. c3feda7 [Michael Armbrust] use toArray. c654f19 [Michael Armbrust] Support for list and maps in hive table scan. cf8d992 [Michael Armbrust] Use built in functions for creating temp directory. 1579eec [Michael Armbrust] Only cast unresolved inserts. 6420c7c [Michael Armbrust] Memoize the ordinal in the GetField expression. da7ae9d [Michael Armbrust] Add boolean writable that was breaking udf_regexp test. Not sure how this was passing before... 6709441 [Michael Armbrust] Evaluation for accessing nested fields. dc6463a [Michael Armbrust] Support for resolving access to nested fields using "." notation. d670e41 [Michael Armbrust] Print nested fields like hive does. efa7217 [Michael Armbrust] Support for reading structs in HiveTableScan. 9c22b4e [Michael Armbrust] Support for parsing nested types. 82163e3 [Michael Armbrust] special case handling of partitionKeys when casting insert into tables ea6f37f [Michael Armbrust] fix style. 7845364 [Michael Armbrust] deactivate concurrent test. b649c20 [Michael Armbrust] fix test logging / caching. 1590568 [Michael Armbrust] add log4j.properties 19bfd74 [Michael Armbrust] store hive output in circular buffer dfb67aa [Michael Armbrust] add test case cb775ac [Michael Armbrust] get rid of SharkContext singleton 2de89d0 [Michael Armbrust] Merge pull request #13 from tnachen/master 63003e9 [Michael Armbrust] Fix spacing. 41b41f3 [Michael Armbrust] Only cast unresolved inserts. 6eb5960 [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into udafs 5b7afd8 [Michael Armbrust] Merge pull request #10 from yhuai/exchangeOperator b1151a8 [Timothy Chen] Fix load data regex 8e0931f [Michael Armbrust] Cast to avoid using deprecated hive API. e079f2b [Timothy Chen] Add GenericUDAF wrapper and HiveUDAFFunction 45b334b [Yin Huai] fix comments 235cbb4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator fc67b50 [Yin Huai] Check for a Sort operator with the global flag set instead of an Exchange operator with a RangePartitioning. 6015f93 [Michael Armbrust] Merge pull request #29 from rxin/style 271e483 [Michael Armbrust] Update build status icon. d3a3d48 [Michael Armbrust] add testing to travis 807b2d7 [Michael Armbrust] check style and publish docs with travis d20b565 [Michael Armbrust] fix if style bce024d [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into style Disable if brace checking as it errors in single line functional cases unlike the style guide. d91e276 [Michael Armbrust] Remove dependence on HIVE_HOME for running tests. This was done by moving all the hive query test (from branch-0.12) and data files into src/test/hive. These are used by default when HIVE_HOME is not set. f47c2f6 [Yin Huai] set outputPartitioning in BroadcastNestedLoopJoin 41bbee6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator 7e24436 [Reynold Xin] Removed dependency on JDK 7 (nio.file). 5c1e600 [Reynold Xin] Added hash code implementation for AttributeReference 7213a2c [Reynold Xin] style fix for Hive.scala. 08e4d05 [Reynold Xin] First round of style cleanup. 605255e [Reynold Xin] Added scalastyle checker. 61e729c [Lian, Cheng] Added ColumnPrunings strategy and test cases 2486fb7 [Lian, Cheng] Fixed spelling 8ee41be [Lian, Cheng] Minor refactoring ebb56fa [Michael Armbrust] add travis config 4c89d6e [Reynold Xin] Merge pull request #27 from marmbrus/moreTests d4f539a [Michael Armbrust] blacklist mr and user specific tests. 677eb07 [Michael Armbrust] Update test whitelist. 5dab0bc [Michael Armbrust] Merge pull request #26 from liancheng/serdeAndPartitionPruning c263c84 [Michael Armbrust] Only push predicates into partitioned table scans. ab77882 [Michael Armbrust] upgrade spark to RC5. c98ede5 [Lian, Cheng] Response to comments from @marmbrus 83d4520 [Yin Huai] marmbrus's comments 70994a3 [Lian, Cheng] Revert unnecessary Scaladoc changes 9ebff47 [Yin Huai] remove unnecessary .toSeq e811d1a [Yin Huai] markhamstra's comments 4802f69 [Yin Huai] The outputPartitioning of a UnaryNode inherits its child's outputPartitioning by default. Also, update the logic in AddExchange to avoid unnecessary shuffling operations. 040fbdf [Yin Huai] AddExchange is the only place to add Exchange operators. 9fb357a [Yin Huai] use getSpecifiedDistribution to create Distribution. ClusteredDistribution and OrderedDistribution do not take Nil as inptu expressions. e9347fc [Michael Armbrust] Remove broken scaladoc links. 99c6707 [Michael Armbrust] upgrade spark 57799ad [Lian, Cheng] Added special treat for HiveVarchar in InsertIntoHiveTable cb49af0 [Lian, Cheng] Fixed Scaladoc links 4e5e4d4 [Lian, Cheng] Added PreInsertionCasts to do necessary casting before insertion 111ffdc [Lian, Cheng] More comments and minor reformatting 9e0d840 [Lian, Cheng] Added partition pruning optimization 761bbb8 [Lian, Cheng] Generalized BindReferences to run against any query plan 04eb5da [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator 9dd3b26 [Michael Armbrust] Fix scaladoc. 6f44cac [Lian, Cheng] Made TableReader & HadoopTableReader private to catalyst 7c92a41 [Lian, Cheng] Added Hive SerDe support ce5fdd6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator 2957f31 [Yin Huai] addressed comments on PR 907db68 [Michael Armbrust] Space after while. 04573a0 [Reynold Xin] Merge pull request #24 from marmbrus/binaryCasts 4e50679 [Reynold Xin] Merge pull request #25 from marmbrus/rowOrderingWhile 5bc1dc2 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator be1fff7 [Michael Armbrust] Replace foreach with while in RowOrdering. Fixes #23 fd084a4 [Michael Armbrust] implement casts binary <=> string. 0b31176 [Michael Armbrust] Merge pull request #22 from rxin/type 548e479 [Yin Huai] merge master into exchangeOperator and fix code style 5b11db0 [Reynold Xin] Added Void to Boolean type widening. 9e3d989 [Reynold Xin] Made HiveTypeCoercion.WidenTypes more clear. 9bb1979 [Reynold Xin] Merge pull request #19 from marmbrus/variadicUnion a2beb38 [Michael Armbrust] Merge pull request #21 from liancheng/fixIssue20 b20a4d4 [Lian, Cheng] Fix issue #20 6d6cb58 [Michael Armbrust] add source links that point to github to the scala doc. 4285962 [Michael Armbrust] Remove temporary test cases 167162f [Michael Armbrust] more merge errors, cleanup. e170ccf [Michael Armbrust] Improve documentation and remove some spurious changes that were introduced by the merge. 6377d0b [Michael Armbrust] Drop empty files, fix if (). c0b0e60 [Michael Armbrust] cleanup broken doc links. 330a88b [Michael Armbrust] Fix bugs in AddExchange. 4f345f2 [Michael Armbrust] Remove SortKey, use RowOrdering. 043e296 [Michael Armbrust] Make physical union nodes variadic. ece15e1 [Michael Armbrust] update unit tests 5c89d2e [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into exchangeOperator Fix deprecated use of combineValuesByKey. Get rid of test where the answer is dependent on the plan execution width. 9804eb5 [Michael Armbrust] upgrade spark 053a371 [Michael Armbrust] Merge pull request #15 from marmbrus/orderedRow 5ab18be [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into orderedRow ca2ff68 [Michael Armbrust] Merge pull request #17 from marmbrus/unionTypes bf9161c [Michael Armbrust] Merge pull request #18 from marmbrus/noSparkAgg 563053f [Michael Armbrust] Address @rxin's comments. 6537c66 [Michael Armbrust] Address @rxin's comments. 2a76fc6 [Michael Armbrust] add notes from @rxin. 685bfa1 [Michael Armbrust] fix spelling 69ed98f [Michael Armbrust] Output a single row for empty Aggregations with no grouping expressions. 7859a86 [Michael Armbrust] Remove SparkAggregate. Its kinda broken and breaks RDD lineage. fc22e01 [Michael Armbrust] whitelist newly passing union test. 3f547b8 [Michael Armbrust] Add support for widening types in unions. 53b95f8 [Michael Armbrust] coercion should not occur until children are resolved. b892e32 [Michael Armbrust] Union is not resolved until the types match up. 95ab382 [Michael Armbrust] Use resolved instead of custom function. This is better because some nodes override the notion of resolved. 81a109d [Michael Armbrust] fix link. f143f61 [Michael Armbrust] Implement sampling. Fixes a flaky test where the JVM notices that RAND as a Comparison method "violates its general contract!" 6cd442b [Michael Armbrust] Use numPartitions variable, fix grammar. c800798 [Michael Armbrust] Add build status icon. 0cf5a75 [Michael Armbrust] Merge pull request #16 from marmbrus/filterPushDown 05d3a0d [Michael Armbrust] Refactor to avoid serializing ordering details with every row. f2fdd77 [Michael Armbrust] fix required distribtion for aggregate. 658866e [Michael Armbrust] Pull back in changes made by @yhuai eliminating CoGroupedLocallyRDD.scala 583a337 [Michael Armbrust] break apart distribution and partitioning. e8d41a9 [Michael Armbrust] Merge remote-tracking branch 'yin/exchangeOperator' into exchangeOperator 0ff8be7 [Michael Armbrust] Cleanup spurious changes and fix doc links. 73c70de [Yin Huai] add a first set of unit tests for data properties. fbfa437 [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into filterPushDown Minor doc improvements. 2b9d80f [Yin Huai] initial commit of adding exchange operators to physical plans. fcbc03b [Michael Armbrust] Fix if (). 7b9080c [Michael Armbrust] Create OrderedRow class to allow ordering to be used by multiple operators. b4adb0f [Michael Armbrust] Merge pull request #14 from marmbrus/castingAndTypes b2a1ec5 [Michael Armbrust] add comment on how using numeric implicitly complicates spark serialization. e286d20 [Michael Armbrust] address code review comments. 80d0681 [Michael Armbrust] fix scaladoc links. de0c248 [Michael Armbrust] Print the executed plan in SharkQuery toString. 3413e61 [Michael Armbrust] Add mapChildren and withNewChildren methods to TreeNode. 404d552 [Michael Armbrust] Better exception when unbound attributes make it to evaluation. fb84ae4 [Michael Armbrust] Refactor DataProperty into Distribution. 2abb0bc [Michael Armbrust] better debug messages, use exists. 098dfc4 [Michael Armbrust] Implement Long sorting again. 60f3a9a [Michael Armbrust] More aggregate functions out of the aggregate class to make things more readable. a1ef62e [Michael Armbrust] Print the executed plan in SharkQuery toString. dfce426 [Michael Armbrust] Add mapChildren and withNewChildren methods to TreeNode. 037a2ed [Michael Armbrust] Better exception when unbound attributes make it to evaluation. ec90620 [Michael Armbrust] Support for Sets as arguments to TreeNode classes. b21f803 [Michael Armbrust] Merge pull request #11 from marmbrus/goldenGen 83adb9d [Yin Huai] add DataProperty 5a26292 [Michael Armbrust] Rules to bring casting more inline with Hive semantics. f0e0161 [Michael Armbrust] Move numeric types into DataTypes simplifying evaluator. This can probably also be use for codegen... 6d2924d [Michael Armbrust] add support for If. Not integrated in HiveQL yet. ccc4dbf [Michael Armbrust] Add optimization rule to simplify casts. 058ec15 [Michael Armbrust] handle more writeables. ffa9f25 [Michael Armbrust] blacklist some more MR tests. aa2239c [Michael Armbrust] filter test lines containing Owner: f71a325 [Michael Armbrust] Update golden jar. a3003ae [Michael Armbrust] Update makefile to use better sharding support. 568d150 [Michael Armbrust] Updates to white/blacklist. 8351f25 [Michael Armbrust] Add an ignored test to remind us we don't do empty aggregations right. c4104ec [Michael Armbrust] Numerous improvements to testing infrastructure. See comments for details. 09c6300 [Michael Armbrust] Add nullability information to StructFields. 5460b2d [Michael Armbrust] load srcpart by default. 3695141 [Michael Armbrust] Lots of parser improvements. 965ac9a [Michael Armbrust] Add expressions that allow access into complex types. 3ba53c9 [Michael Armbrust] Output type suffixes on AttributeReferences. 8777489 [Michael Armbrust] Initial support for operators that allow the user to specify partitioning. e57f97a [Michael Armbrust] more decimal/null support. e1440ed [Michael Armbrust] Initial support for function specific type conversions. 1814ed3 [Michael Armbrust] use childrenResolved function. f2ec57e [Michael Armbrust] Begin supporting decimal. 6924e6e [Michael Armbrust] Handle NullTypes when resolving HiveUDFs 7fcfa8a [Michael Armbrust] Initial support for parsing unspecified partition parameters. d0124f3 [Michael Armbrust] Correctly type null literals. b65626e [Michael Armbrust] Initial support for parsing BigDecimal. a90efda [Michael Armbrust] utility function for outputing string stacktraces. 7102f33 [Michael Armbrust] methods with side-effects should use (). 3ccaef7 [Michael Armbrust] add renaming TODO. bc282c7 [Michael Armbrust] fix bug in getNodeNumbered c8e89d5 [Michael Armbrust] memoize inputSet calculation. 6aefa46 [Michael Armbrust] Skip folding literals. a72e540 [Michael Armbrust] Add IN operator. 04f885b [Michael Armbrust] literals are only non-nullable if they are not null. 35d2948 [Michael Armbrust] correctly order partition and normal attributes in hive relation output. 12fd52d [Michael Armbrust] support for sorting longs. 0606520 [Michael Armbrust] drop old comment. 859200a [Michael Armbrust] support for reading more types from the metastore. 1fedd18 [Michael Armbrust] coercion from null to numeric types 71e902d [Michael Armbrust] fix test cases. cc06b6c [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into interviewAnswer 8a8b521 [Reynold Xin] Merge pull request #8 from marmbrus/testImprovment 86355a6 [Michael Armbrust] throw error if there are unexpected join clauses. c5842d2 [Michael Armbrust] don't throw an error when a select clause outputs multiple copies of the same attribute. 0e975ea [Michael Armbrust] parse bucket sampling as percentage sampling a92919d [Michael Armbrust] add alter view as to native commands f58d5a5 [Michael Armbrust] support for parsing SELECT DISTINCT f0faa26 [Michael Armbrust] add sample and distinct operators. ef7b943 [Michael Armbrust] add metastore support for float e9f4588 [Michael Armbrust] fix > 100 char. 755b229 [Michael Armbrust] blacklist some ddl tests. 9ae740a [Michael Armbrust] blacklist more tests that require MR. 4cfc11a [Michael Armbrust] more test coverage. 0d9d56a [Michael Armbrust] add more native commands to parser 78d730d [Michael Armbrust] Load src test table on RESET. 8364ec2 [Michael Armbrust] whitelist all possible partition values. b01468d [Michael Armbrust] support path rewrites when the query begins with a comment. 4c6b454 [Michael Armbrust] add option for recomputing the cached golden answer when tests fail. 4c5fb0f [Michael Armbrust] makefile target for building new whitelist. 4b6fed8 [Michael Armbrust] support for parsing both DESTINATION and INSERT_INTO. 516481c [Michael Armbrust] Ignore requests to explain native commands. 68aa2e6 [Michael Armbrust] Stronger type for Token extractor. ca4ea26 [Michael Armbrust] Support for parsing UDF(*). 1aafea3 [Michael Armbrust] Configure partition whitelist in TestShark reset. 9627616 [Michael Armbrust] Use current database as default database. 9b02b44 [Michael Armbrust] Fix spelling error. Add failFast mode. 6f64cee [Michael Armbrust] don't line wrap string literal eafaeed [Michael Armbrust] add type documentation f54c94c [Michael Armbrust] make golden answers file a test dependency 5362365 [Michael Armbrust] push conditions into join 0d2388b [Michael Armbrust] Point at databricks hosted scaladoc. 73b29cd [Michael Armbrust] fix bad casting 9aa06c5 [Michael Armbrust] Merge pull request #7 from marmbrus/docFixes 7eff191 [Michael Armbrust] link all the expression names. 83227e4 [Michael Armbrust] fix scaladoc list syntax, add docs for some rules 9de6b74 [Michael Armbrust] fix language feature and deprecation warnings. 0b1960a [Michael Armbrust] Fix broken scala doc links / warnings. b1acb36 [Michael Armbrust] Merge pull request #3 from yhuai/evalauteLiteralsInExpressions 01c00c2 [Michael Armbrust] new golden 5c14857 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions b749b51 [Michael Armbrust] Merge pull request #5 from marmbrus/testCaching 66adceb [Michael Armbrust] Merge pull request #6 from marmbrus/joinWork 1a393da [Yin Huai] folded -> foldable 1e964ea [Yin Huai] update a43d41c [Michael Armbrust] more tests passing! 8ca38d0 [Michael Armbrust] begin support for varchar / binary types. ab8bbd1 [Michael Armbrust] parsing % operator c16c8b5 [Michael Armbrust] case insensitive checking for hooks in tests. 3a90a5f [Michael Armbrust] simpler output when running a single test from the commandline. 5332fee [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 367fb9e [Yin Huai] update 0cd5cc6 [Michael Armbrust] add BIGINT cast parsing 61b266f [Michael Armbrust] comment for eliminate subqueries. d72a5a2 [Michael Armbrust] add long to literal factory object. b3bd15f [Michael Armbrust] blacklist more mr requiring tests. e06fd38 [Michael Armbrust] black list map reduce tests. 8e7ce30 [Michael Armbrust] blacklist some env specific tests. 6250cbd [Michael Armbrust] Do not exit on test failure b22b220 [Michael Armbrust] also look for cached hive test answers on the classpath. b6e4899 [Yin Huai] formatting e75c90d [Reynold Xin] Merge pull request #4 from marmbrus/hive12 5fabbec [Michael Armbrust] ignore partitioned scan test. scan seems to be working but there is some error about the table already existing? 9e190f5 [Michael Armbrust] drop unneeded () 68b58c1 [Michael Armbrust] drop a few more tests. b0aa400 [Michael Armbrust] update whitelist. c99012c [Michael Armbrust] skip tests with hooks db00ebf [Michael Armbrust] more types for hive udfs dbc3678 [Michael Armbrust] update ghpages repo 138f53d [Yin Huai] addressed comments and added a space after a space after the defining keyword of every control structure. 6f954ee [Michael Armbrust] export the hadoop classpath when starting sbt, required to invoke hive during tests. 46bf41b [Michael Armbrust] add a makefile for priming the test answer cache in parallel. usage: "make -j 8 -i" 8d47ed4 [Yin Huai] comment 2795f05 [Yin Huai] comment e003728 [Yin Huai] move OptimizerSuite to the package of catalyst.optimizer 2941d3a [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 0bd1688 [Yin Huai] update 6a7bd75 [Michael Armbrust] fix partition column delimiter configuration. e942da1 [Michael Armbrust] Begin upgrade to Hive 0.12.0. b8cd7e3 [Michael Armbrust] Merge pull request #7 from rxin/moreclean 52864da [Reynold Xin] Added executeCollect method to SharkPlan. f0e1cbf [Reynold Xin] Added resolved lazy val to LogicalPlan. b367e36 [Reynold Xin] Replaced the use of ??? with UnsupportedOperationException. 38124bd [Yin Huai] formatting 2924468 [Yin Huai] add two tests for testing pre-order and post-order tree traversal, respectively 555d839 [Reynold Xin] More cleaning ... d48d0e1 [Reynold Xin] Code review feedback. aa2e694 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 5c421ac [Reynold Xin] Imported SharkEnv, SharkContext, and HadoopTableReader to remove Shark dependency. 479e055 [Reynold Xin] A set of minor changes, including: - import order - limit some lines to 100 character wide - inline code comment - more scaladocs - minor spacing (i.e. add a space after if) da16e45 [Reynold Xin] Merge pull request #3 from rxin/packagename e36caf5 [Reynold Xin] Renamed Rule.name to Rule.ruleName since name is used too frequently in the code base and is shadowed often by local scope. 72426ed [Reynold Xin] Rename shark2 package to execution. 0892153 [Reynold Xin] Merge pull request #2 from rxin/packagename e58304a [Reynold Xin] Merge pull request #1 from rxin/gitignore 3f9fee1 [Michael Armbrust] rewrite push filter through join optimization. c6527f5 [Reynold Xin] Moved the test src files into the catalyst directory. c9777d8 [Reynold Xin] Put all source files in a catalyst directory. 019ea74 [Reynold Xin] Updated .gitignore to include IntelliJ files. 80ca4be [Timothy Chen] Address comments 0079392 [Michael Armbrust] support for multiple insert commands in a single query 75b5a01 [Michael Armbrust] remove space. 4283400 [Timothy Chen] Add limited predicate push down e547e50 [Michael Armbrust] implement First. e77c9b6 [Michael Armbrust] more work on unique join. c795e06 [Michael Armbrust] improve star expansion a26494e [Michael Armbrust] allow aliases to have qualifiers d078333 [Michael Armbrust] remove extra space a75c023 [Michael Armbrust] implement Coalesce 3a018b6 [Michael Armbrust] fix up docs. ab6f67d [Michael Armbrust] import the string "null" as actual null. 5377c04 [Michael Armbrust] don't call dataType until checking if children are resolved. 191ce3e [Michael Armbrust] analyze rewrite test query. 60b1526 [Michael Armbrust] don't call dataType until checking if children are resolved. 2ab5a32 [Michael Armbrust] stop using uberjar as it has its own set of issues. e42f75a [Michael Armbrust] Merge remote-tracking branch 'origin/master' into HEAD c086a35 [Michael Armbrust] docs, spacing c4060e4 [Michael Armbrust] cleanup 3b85462 [Michael Armbrust] more tests passing bcfc8c5 [Michael Armbrust] start supporting partition attributes when inserting data. c944a95 [Michael Armbrust] First aggregate expression. 1e28311 [Michael Armbrust] make tests execute in alpha order again a287481 [Michael Armbrust] spelling 8492548 [Michael Armbrust] beginning of UNIQUEJOIN parsing. a6ab6c7 [Michael Armbrust] add != 4529594 [Michael Armbrust] draft of coalesce 70f253f [Michael Armbrust] more tests passing! 7349e7b [Michael Armbrust] initial support for test thrift table d3c9305 [Michael Armbrust] fix > 100 char line 93b64b0 [Michael Armbrust] load test tables that are args to "DESCRIBE" 06b2aba [Michael Armbrust] don't be case sensitive when fixing load paths 6355d0e [Michael Armbrust] match actual return type of count with expected cda43ab [Michael Armbrust] don't throw an exception when one of the join tables is empty. fd4b096 [Michael Armbrust] fix casing of null strings as well. 4632695 [Michael Armbrust] support for megastore bigint 67b88cf [Michael Armbrust] more verbose debugging of evaluation return types c680e0d [Michael Armbrust] Failed string => number conversion should return null. 2326be1 [Michael Armbrust] make getClauses case insensitive. dac2786 [Michael Armbrust] correctly handle null values when going from string to numeric types. 045ac4b [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions fb5ddfd [Michael Armbrust] move ViewExamples to examples/ 83833e8 [Michael Armbrust] more tests passing! 47c98d6 [Michael Armbrust] add query tests for like and hash. 1724c16 [Michael Armbrust] clear lines that contain last updated times. cfd6bbc [Michael Armbrust] Quick skipping of tests that we can't even parse. 9b2642b [Michael Armbrust] make the blacklist support regexes 1d50af6 [Michael Armbrust] more datatypes, fix nonserializable instance variables in udfs 910e33e [Michael Armbrust] basic support for building an assembly jar. d55bb52 [Michael Armbrust] add local warehouse/metastore to gitignore. 495d9dc [Michael Armbrust] Add an expression for when we decide to support LIKE natively instead of using the HIVE udf. 65f4e69 [Michael Armbrust] remove incorrect comments 0831a3c [Michael Armbrust] support for parsing some operator udfs. 6c27aa7 [Michael Armbrust] more cast parsing. 43db061 [Michael Armbrust] significant generalization of hive udf functionality. 3fe24ec [Michael Armbrust] better implementation of 3vl in Evaluate, fix some > 100 char lines. e5690a6 [Michael Armbrust] add BinaryType adab892 [Michael Armbrust] Clear out functions that are created during tests when reset is called. d408021 [Michael Armbrust] support for printing out arrays in the output in the same form as hive (e.g., [e1, e1]). 8d5f504 [Michael Armbrust] Example of schema RDD using scala's dynamic trait, resulting in a more standard ORM style of usage. 21f0d91 [Michael Armbrust] Simple example of schemaRdd with scala filter function. 0daaa0e [Michael Armbrust] Promote booleans that appear in comparisons. 2b70abf [Michael Armbrust] true and false literals. ef8b0a5 [Michael Armbrust] more tests. 14d070f [Michael Armbrust] add support for correctly extracting partition keys. 0afbe73 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 69a0bd4 [Michael Armbrust] promote strings in predicates with number too. 3946e31 [Michael Armbrust] don't build strings unless assertion fails. 90c453d [Michael Armbrust] more tests passing! 6e6417a [Michael Armbrust] correct handling of nulls in boolean logic and sorting. 8000504 [Michael Armbrust] Improve type coercion. 9087152 [Michael Armbrust] fix toString of Not. 58b111c [Michael Armbrust] fix bad scaladoc tag. d5c05c6 [Michael Armbrust] For now, ignore the big data benchmark tests when the data isn't there. ac6376d [Michael Armbrust] Split out general shark query execution driver from test harness. 1d0ae1e [Michael Armbrust] Switch from IndexSeq[Any] to Row interface that will allow us unboxed access to primitive types. d873b2b [Yin Huai] Remove numbers associated with test cases. 8545675 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions b34a9eb [Michael Armbrust] Merge branch 'master' into filterPushDown d1e7b8e [Michael Armbrust] Update README.md c8b1553 [Michael Armbrust] Update README.md 9307ef9 [Michael Armbrust] update list of passing tests. 934c18c [Michael Armbrust] Filter out non-deterministic lines when comparing test answers. a045c9c [Michael Armbrust] SparkAggregate doesn't actually support sum right now. ae0024a [Yin Huai] update cf80545 [Yin Huai] Merge remote-tracking branch 'origin/evalauteLiteralsInExpressions' into evalauteLiteralsInExpressions 21976ae [Yin Huai] update b4999fe [Yin Huai] Merge remote-tracking branch 'upstream/filterPushDown' into evalauteLiteralsInExpressions dedbf0c [Yin Huai] support Boolean literals eaac9e2 [Yin Huai] explain the limitation of the current EvaluateLiterals 37817b5 [Yin Huai] add a comment to EvaluateLiterals. 468667f [Yin Huai] First draft of literal evaluation in the optimization phase. TreeNode has been extended to support transform in the post order. So, for an expression, we can evaluate literal from the leaf nodes of this expression tree. For an attribute reference in the expression node, we just leave it as is. b1d1843 [Michael Armbrust] more work on big data benchmark tests. cc9a957 [Michael Armbrust] support for creating test tables outside of TestShark 7d7fa9f [Michael Armbrust] support for create table as 5f54f03 [Michael Armbrust] parsing for ASC d42b725 [Michael Armbrust] Sum of strings requires cast 34b30fa [Michael Armbrust] not all attributes need to be bound (e.g. output attributes that are contained in non-leaf operators.) 81659cb [Michael Armbrust] implement transform operator. 5cd76d6 [Michael Armbrust] break up the file based test case code for reuse 1031b65 [Michael Armbrust] support for case insensitive resolution. 320df04 [Michael Armbrust] add snapshot repo for databricks (has shark/spark snapshots) b6f083e [Michael Armbrust] support for publishing scala doc to github from sbt d9d18b4 [Michael Armbrust] debug logging implicit. 669089c [Yin Huai] support Boolean literals ef3321e [Yin Huai] explain the limitation of the current EvaluateLiterals 73a05fd [Yin Huai] add a comment to EvaluateLiterals. 191eb7d [Yin Huai] First draft of literal evaluation in the optimization phase. TreeNode has been extended to support transform in the post order. So, for an expression, we can evaluate literal from the leaf nodes of this expression tree. For an attribute reference in the expression node, we just leave it as is. 80039cc [Yin Huai] Merge pull request #1 from yhuai/master cbe1ca1 [Yin Huai] add explicit result type to the overloaded sideBySide 5c518e4 [Michael Armbrust] fix bug in test. b50dd0e [Michael Armbrust] fix return type of overloaded method 05679b7 [Michael Armbrust] download assembly jar for easy compiling during interview. 8c60cc0 [Michael Armbrust] Update README.md 03b9526 [Michael Armbrust] First draft of optimizer tests. f392755 [Michael Armbrust] Add flatMap to TreeNode 6cbe8d1 [Michael Armbrust] fix bug in side by side, add support for working with unsplit strings 15a53fc [Michael Armbrust] more generic sum calculation and better binding of grouping expressions. 06749d0 [Michael Armbrust] add expression enumerations for query plan operators and recursive version of transform expression. 4b0a888 [Michael Armbrust] implement string comparison and more casts. 356b321 [Michael Armbrust] Update README.md 3776395 [Michael Armbrust] Update README.md 304d17d [Michael Armbrust] Create README.md b7d8be0 [Michael Armbrust] more tests passing. b82481f [Michael Armbrust] add todo comment. 02e6dee [Michael Armbrust] add another test that breaks the harness to the blacklist. cc5efe3 [Michael Armbrust] First draft of broadcast nested loop join with full outer support. c43a259 [Michael Armbrust] comments 15ff448 [Michael Armbrust] better error message when a dsl test throws an exception 76ec650 [Michael Armbrust] fix join conditions e10df99 [Michael Armbrust] Create new expr ids for local relations that exist more than once in a query plan. 91573a4 [Michael Armbrust] initial type promotion e2ef4a5 [Michael Armbrust] logging e43dc1e [Michael Armbrust] add string => int cast evaluation f1f7e96 [Michael Armbrust] fix incorrect generation of join keys 2b27230 [Michael Armbrust] add depth based subtree access 0f6279f [Michael Armbrust] broken tests. 389bc0b [Michael Armbrust] support for partitioned columns in output. 12584f4 [Michael Armbrust] better errors for missing clauses. support for matching multiple clauses with the same name. b67a225 [Michael Armbrust] better errors when types don't match up. 9e74808 [Michael Armbrust] add children resolved. 6d03ce9 [Michael Armbrust] defaults for unresolved relation 2469b00 [Michael Armbrust] skip nodes with unresolved children when doing coersions be5ae2c [Michael Armbrust] better resolution logging cb7b5af [Michael Armbrust] views example 420e05b [Michael Armbrust] more tests passing! 6916c63 [Michael Armbrust] Reading from partitioned hive tables. a1245f9 [Michael Armbrust] more tests passing 956e760 [Michael Armbrust] extended explain 5f14c35 [Michael Armbrust] more test tables supported 175c43e [Michael Armbrust] better errors for parse exceptions 480ade5 [Michael Armbrust] don't use partial cached results. 8a9d21c [Michael Armbrust] fix evaluation 7aee69c [Michael Armbrust] parsing for joins, boolean logic 7fcf480 [Michael Armbrust] test for and logic 3ea9b00 [Michael Armbrust] don't use simpleString if there are no new lines. 6902490 [Michael Armbrust] fix boolean logic evaluation 4d5eba7 [Michael Armbrust] add more dsl for expression arithmetic and boolean logic 8b2a2ee [Michael Armbrust] more tests passing! ad1f3b4 [Michael Armbrust] toString for null literals a5c0a1b [Michael Armbrust] more test harness improvements: * regex whitelist * side by side answer comparison (still needs formatting work) 60ec19d [Michael Armbrust] initial support for udfs c45b440 [Michael Armbrust] support for is (not) null and boolean logic 7f4a1dc [Michael Armbrust] add NoRelation logical operator 72e183b [Michael Armbrust] support for null values in tree node args. ad596d2 [Michael Armbrust] add sc to Union's otherCopyArgs e5c9d1a [Michael Armbrust] use nonEmpty dcc4fe1 [Michael Armbrust] support for src1 test table. c78b587 [Michael Armbrust] casting. 75c3f3f [Michael Armbrust] add support for logging with scalalogging. da2c011 [Michael Armbrust] make it more obvious when results are being truncated. 96b73ba [Michael Armbrust] more docs in TestShark 18524fd [Michael Armbrust] add method to SharkSqlQuery for directly executing the same query on hive. e6d063b [Michael Armbrust] more join tests. 664c1c3 [Michael Armbrust] make parsing of function names case insensitive. 0967d4e [Michael Armbrust] fix hardcoded path to hiveDevHome. 1a6db68 [Michael Armbrust] spelling 7638cb4 [Michael Armbrust] simple join execution with dsl tests. no hive tests yes. 859d4c9 [Michael Armbrust] better argString printing of nested trees. fc53615 [Michael Armbrust] add same instance comparisons for tree nodes. a026e6b [Michael Armbrust] move out hive specific operators fff4d1c [Michael Armbrust] add simple query execution debugging e2120ab [Michael Armbrust] sorting for strings da06eb6 [Michael Armbrust] Parsing for sortby and joins 9eb5c5e [Michael Armbrust] override equality in Attribute references to compare exprId. 8eb2460 [Michael Armbrust] add system property to override whitelist. 88124bb [Michael Armbrust] make strategy evaluation lazy. 74a3a21 [Michael Armbrust] implement outputSet d25b171 [Michael Armbrust] Add AND and OR expressions 67f0a4a [Michael Armbrust] dsl improvements: string to attribute, subquery, unionAll 12acf0a [Michael Armbrust] add .DS_Store for macs f7da6ce [Michael Armbrust] add agg with grouping expr in select test 36805b3 [Michael Armbrust] pull out and improve aggregation 75613e1 [Michael Armbrust] better evaluations failure messages. 4789a35 [Michael Armbrust] weaken type since its hard to create pure references. e89dd36 [Michael Armbrust] no newline for online trees d0590d4 [Michael Armbrust] include stack trace for catalyst failures. 081c0d9 [Michael Armbrust] more generic computation of agg functions. 31af3a0 [Michael Armbrust] fail when clauses are unhandeled in the parser ecd45b2 [Michael Armbrust] Add more passing tests. 97d5419 [Michael Armbrust] fix alignment. 565cc13 [Michael Armbrust] make the canary query optional. a95e65c [Michael Armbrust] support for resolving qualified attribute references. e1dfa0c [Michael Armbrust] better error reporting for comparison tests when hive works but catalyst fails. 4640a0b [Michael Armbrust] handle test tables when database is specified. bef12e3 [Michael Armbrust] Add Subquery node and trivial optimizer to remove it after analysis. fec5158 [Michael Armbrust] add hive / idea files to .gitignore 3f97ffe [Michael Armbrust] Rename Hive => HiveQl 656b836 [Michael Armbrust] Support for parsing select clause aliases. 3ca7414 [Michael Armbrust] StopAfter needs otherCopyArgs. 3ffde66 [Michael Armbrust] When the child of an alias is unresolved it should return an unresolved attribute instead of throwing an exception. 8cbef8a [Michael Armbrust] spelling aa8c37c [Michael Armbrust] Better toString for SortOrder 1bb8b45 [Michael Armbrust] fix error message for UnresolvedExceptions a2e0327 [Michael Armbrust] add a bunch of tests. 4a3e1ea [Michael Armbrust] docs and use shark for data loading. 339bb8f [Michael Armbrust] better docs, Not support 1d7b2d9 [Michael Armbrust] Add NaN conversions. 46a2534 [Michael Armbrust] only run canary query on failure. 8996066 [Michael Armbrust] remove protected from makeCopy 53bcf41 [Michael Armbrust] testing improvements: * reset hive vars * delete indexes and tables * delete database * reset to use default database * record tests that pass 04a372a [Michael Armbrust] add a flag for running all tests. 3b2235b [Michael Armbrust] More general implementation of arithmetic. edd7795 [Michael Armbrust] More testing improvements: * Check that results match for native commands * Ensure explain commands can be planned * Cache hive "golden" results da6c577 [Michael Armbrust] add string <==> file utility functions. 3adf5ca [Michael Armbrust] Initial support for groupBy and count. 7bcd8a4 [Michael Armbrust] Improvements to comparison tests: * Sort answer when query doesn't contain an order by. * Display null values the same as Hive. * Print full query results in easy to read format when they differ. a52e7c9 [Michael Armbrust] Transform children that are present in sequences of the product. d66ba7e [Michael Armbrust] drop printlns. 88f2efd [Michael Armbrust] Add sum / count distinct expressions. 05adedc [Michael Armbrust] rewrite relative paths when loading data in TestShark 07784b3 [Michael Armbrust] add support for rewriting paths and running 'set' commands. b8a9910 [Michael Armbrust] quote tests passing. 8e5e267 [Michael Armbrust] handle aliased select expressions. 4286a96 [Michael Armbrust] drop debugging println ac34aeb [Michael Armbrust] proof of concept for hive ast transformations. 2238b00 [Michael Armbrust] better error when makeCopy functions fails due to incorrect arguments ff1eab8 [Michael Armbrust] start trying to make insert into hive table more general. 74a6337 [Michael Armbrust] use fastEquals when doing transformations. 1184a23 [Michael Armbrust] add native test for escapes. b972b18 [Michael Armbrust] create BaseRelation class fa6bce9 [Michael Armbrust] implement union 6391a87 [Michael Armbrust] count aggregate. d47c317 [Michael Armbrust] add unary minus, more tests passing. c7114e4 [Michael Armbrust] first draft of star expansion. 044c43d [Michael Armbrust] better support for numeric literal parsing. 1d0f072 [Michael Armbrust] use native drop table as it doesn't appear to fail when the "table" is actually a view. 61503c5 [Michael Armbrust] add cached toRdd 2036883 [Michael Armbrust] skip explain queries when testing. ebac4b1 [Michael Armbrust] fix bug in sort reference calculation ca0dee0 [Michael Armbrust] docs. 1ee0471 [Michael Armbrust] string literal parsing. 357278b [Michael Armbrust] add limit support 9b3e479 [Michael Armbrust] creation of string literals. 02efa30 [Michael Armbrust] alias evaluation cb68b33 [Michael Armbrust] parsing for random sample in hive ql. 126dd36 [Michael Armbrust] include query plans in failure output bb59ae9 [Michael Armbrust] doc fixes 7e68286 [Michael Armbrust] fix confusing naming 768bb25 [Michael Armbrust] handle errors in shark query toString 829c3ce [Michael Armbrust] Auto loading of test data on demand. Add reset method to test shark. Make test shark a singleton to avoid weirdness with the hive megastore. ad02e41 [Michael Armbrust] comment jdo dependency 7bc89fe [Michael Armbrust] add collect to TreeNode. 438cf74 [Michael Armbrust] create explicit treeString function in addition to toString override. docs. 09679ee [Michael Armbrust] fix bug in TreeNode foreach 2930b27 [Michael Armbrust] more specific name for del query tests. 8842549 [Michael Armbrust] docs. da81f81 [Michael Armbrust] Implementation and tests for simple AVG query in Hive SQL. a8969b9 [Michael Armbrust] Factor out hive query comparison test framework. 1a7efb0 [Michael Armbrust] specialize spark aggregate for global aggregations. a36dd9a [Michael Armbrust] evaluation for other > data types. cae729b [Michael Armbrust] remove unnecessary lazy vals. d8e12af [Michael Armbrust] docs 3a60d67 [Michael Armbrust] implement average, placeholder for count f05c106 [Michael Armbrust] checkAnswer handles single row results. 2730534 [Michael Armbrust] implement inputSet a9aa79d [Michael Armbrust] debugging for sort exec 8bec3c9 [Michael Armbrust] better tree makeCopy when there are two constructors. 554b4b2 [Michael Armbrust] BoundAttribute pretty printing. 754f5fa [Michael Armbrust] dsl for setting nullability a206d7a [Michael Armbrust] clean up query tests. 84ad6ef [Michael Armbrust] better sort implementation and tests. de24923 [Michael Armbrust] add double type. 9611a2c [Michael Armbrust] literal creation for doubles. 7358313 [Michael Armbrust] sort order returns child type. b544715 [Michael Armbrust] implement eval for rand, and > for doubles 7013bad [Michael Armbrust] asc, desc should work for expressions and unresolved attributes (symbols) 1c1a35e [Michael Armbrust] add simple Rand expression. 3ca51de [Michael Armbrust] add orderBy to dsl 7ae41ab [Michael Armbrust] more literal implicit conversions b18b675 [Michael Armbrust] First cut at native query tests for shark. d392e29 [Michael Armbrust] add toRdd implicit conversion for logical plans in TestShark. 5eac895 [Michael Armbrust] better error when descending is specified. 2b16f86 [Michael Armbrust] add todo e527bb8 [Michael Armbrust] remove arguments to binary predicate constructor as they seem to break serialization 9dde3c8 [Michael Armbrust] add project and filter operations. ad9037b [Michael Armbrust] Add support for local relations. 6227143 [Michael Armbrust] evaluation of Equals. 7526290 [Michael Armbrust] BoundReference should also be an Attribute. bd33e26 [Michael Armbrust] more documentation 5de0ea3 [Michael Armbrust] Move all shark specific into a separate package. Lots of documentation improvements. 0ae292b [Michael Armbrust] implement calculation of sort expressions. 9fd5011 [Michael Armbrust] First cut at expression evaluation. 6259e3a [Michael Armbrust] cleanup 787e5a2 [Michael Armbrust] use fastEquals f90da36 [Michael Armbrust] better printing of optimization exceptions b05dd67 [Michael Armbrust] Application of rules to fixed point. bb2e0db [Michael Armbrust] pretty print for literals. 1ec3287 [Michael Armbrust] Add extractor for IntegerLiterals. d3a3687 [Michael Armbrust] add fastEquals 2b4935b [Michael Armbrust] set sbt.version explicitly 46dfd7f [Michael Armbrust] first cut at checking answer for HiveCompatability tests. c79f2fd [Michael Armbrust] insert operator should return an empty rdd. 14c22ec [Michael Armbrust] implement sorting when the sort expression is the first attribute of the input. ae7b4c3 [Michael Armbrust] remove implicit dependencies. now compiles without copying things into lib/ manually. 84082f9 [Michael Armbrust] add sbt binaries and scripts 15371a8 [Michael Armbrust] First draft of simple Hive DDL parser. 063bf44 [Michael Armbrust] Periods should end all comments. e1f7f4c [Michael Armbrust] Remove "NativePlaceholder" hack. ed3633e [Michael Armbrust] start consolidating Hive/Shark specific code. first hive compatibility test case passing! b34a770 [Michael Armbrust] Add data sink strategy, make strategy application a little more robust. e7174ec [Michael Armbrust] fix schema, add docs, make helper method protected. 26f410a [Michael Armbrust] physical traits should extend PhysicalPlan. dc72469 [Michael Armbrust] beginning of hive compatibility testing framework. 0763490 [Michael Armbrust] support for hive native command pass-through. d8a924f [Michael Armbrust] scaladoc 29a7163 [Michael Armbrust] Insert into hive table physical operator. 633cebc [Michael Armbrust] better error message when there is no appropriate planning strategy. 59ac444 [Michael Armbrust] add unary expression 3aa1b28 [Michael Armbrust] support for table names in the form 'database.tableName' 665f7d0 [Michael Armbrust] add logical nodes for hive data sinks. 64d2923 [Michael Armbrust] Add classes for representing sorts. f72b7ce [Michael Armbrust] first trivial end to end query execution. 5c7d244 [Michael Armbrust] first draft of references implementation. 7bff274 [Michael Armbrust] point at new shark. c7cd57f [Michael Armbrust] docs for util function. 910811c [Michael Armbrust] check each item of the sequence ef21a0b [Michael Armbrust] line up comments. 4b765d5 [Michael Armbrust] docs, drop println 6f9bafd [Michael Armbrust] empty output for unresolved relation to avoid exception in resolution. a703c49 [Michael Armbrust] this order works better until fixed point is implemented. ec1d7c0 [Michael Armbrust] Simple attribute resolution. 069df02 [Michael Armbrust] parsing binary predicates a1cf754 [Michael Armbrust] add joins and equality. 3f5bc98 [Michael Armbrust] add optiq to sbt. 54f3460 [Michael Armbrust] initial optiq parsing. d9161ce [Michael Armbrust] add join operator 1e423eb [Michael Armbrust] placeholders in LogicalPlan, docs 24ef6fb [Michael Armbrust] toString for alias. ae7d776 [Michael Armbrust] add nullability changing function d49dc02 [Michael Armbrust] scaladoc for named exprs 7c45dd7 [Michael Armbrust] pretty printing of trees. 78e34bf [Michael Armbrust] simple git ignore. 7ba19be [Michael Armbrust] First draft of interface to hive metastore. 7e7acf0 [Michael Armbrust] physical placeholder. 1c11136 [Michael Armbrust] first draft of error handling / plans for debugging. 3766a41 [Michael Armbrust] rearrange utility functions. 7fb3d5e [Michael Armbrust] docs and equality improvements. 45da47b [Michael Armbrust] flesh out plans and expressions a little. first cut at named expressions. 002d4d4 [Michael Armbrust] default to no alias. be25003 [Michael Armbrust] add repl initialization to sbt. 0608a00 [Michael Armbrust] tighten public interface a1a8b38 [Michael Armbrust] test that ids don't change for no-op transforms. daa71ca [Michael Armbrust] foreach, maps, and scaladoc 6a158cb [Michael Armbrust] simple transform working. db0299f [Michael Armbrust] basic analysis of relations minus transform function. f74c4ee [Michael Armbrust] parsing a simple query. 08e4f57 [Michael Armbrust] upgrade scala include shark. d3c6404 [Michael Armbrust] initial commit
* SPARK-1254. Consolidate, order, and harmonize repository declarations in ↵Sean Owen2014-03-151-14/+0
| | | | | | | | | | | | | | | | | | | | Maven/SBT builds This suggestion addresses a few minor suboptimalities with how repositories are handled. 1) Use HTTPS consistently to access repos, instead of HTTP 2) Consolidate repository declarations in the parent POM file, in the case of the Maven build, so that their ordering can be controlled to put the fully optional Cloudera repo at the end, after required repos. (This was prompted by the untimely failure of the Cloudera repo this week, which made the Spark build fail. #2 would have prevented that.) 3) Update SBT build to match Maven build in this regard 4) Update SBT build to not refer to Sonatype snapshot repos. This wasn't in Maven, and a build generally would not refer to external snapshots, but I'm not 100% sure on this one. Author: Sean Owen <sowen@cloudera.com> Closes #145 from srowen/SPARK-1254 and squashes the following commits: 42f9bfc [Sean Owen] Use HTTPS for repos; consolidate repos in parent in order to put optional Cloudera repo last; harmonize SBT build repos with Maven; remove snapshot repos from SBT build which weren't in Maven
* SPARK-1193. Fix indentation in pom.xmlsSandy Ryza2014-03-071-4/+4
| | | | | | | | Author: Sandy Ryza <sandy@cloudera.com> Closes #91 from sryza/sandy-spark-1193 and squashes the following commits: a878124 [Sandy Ryza] SPARK-1193. Fix indentation in pom.xmls
* [java8API] SPARK-964 Investigate the potential for using JDK 8 lambda ↵Prashant Sharma2014-03-035-158/+130
| | | | | | | | | | | | | | | | | | | expressions for the Java/Scala APIs Author: Prashant Sharma <prashant.s@imaginea.com> Author: Patrick Wendell <pwendell@gmail.com> Closes #17 from ScrapCodes/java8-lambdas and squashes the following commits: 95850e6 [Patrick Wendell] Some doc improvements and build changes to the Java 8 patch. 85a954e [Prashant Sharma] Nit. import orderings. 673f7ac [Prashant Sharma] Added support for -java-home as well 80a13e8 [Prashant Sharma] Used fake class tag syntax 26eb3f6 [Prashant Sharma] Patrick's comments on PR. 35d8d79 [Prashant Sharma] Specified java 8 building in the docs 31d4cd6 [Prashant Sharma] Maven build to support -Pjava8-tests flag. 4ab87d3 [Prashant Sharma] Review feedback on the pr c33dc2c [Prashant Sharma] SPARK-964, Java 8 API Support.
* SPARK-1158: Fix flaky RateLimitedOutputStreamSuite.Reynold Xin2014-03-032-20/+32
| | | | | | | | | | | | There was actually a problem with the RateLimitedOutputStream implementation where the first second doesn't write anything because of integer rounding. So RateLimitedOutputStream was overly aggressive in throttling. Author: Reynold Xin <rxin@apache.org> Closes #55 from rxin/ratelimitest and squashes the following commits: 52ce1b7 [Reynold Xin] SPARK-1158: Fix flaky RateLimitedOutputStreamSuite.
* SPARK-1121: Include avro for yarn-alpha buildsPatrick Wendell2014-03-021-0/+14
| | | | | | | | | | | | | | | | | This lets us explicitly include Avro based on a profile for 0.23.X builds. It makes me sad how convoluted it is to express this logic in Maven. @tgraves and @sryza curious if this works for you. I'm also considering just reverting to how it was before. The only real problem was that Spark advertised a dependency on Avro even though it only really depends transitively on Avro through other deps. Author: Patrick Wendell <pwendell@gmail.com> Closes #49 from pwendell/avro-build-fix and squashes the following commits: 8d6ee92 [Patrick Wendell] SPARK-1121: Add avro to yarn-alpha profile
* SPARK-1084.2 (resubmitted)Sean Owen2014-03-021-5/+0
| | | | | | | | | | | | | (Ported from https://github.com/apache/incubator-spark/pull/650 ) This adds one more change though, to fix the scala version warning introduced by json4s recently. Author: Sean Owen <sowen@cloudera.com> Closes #32 from srowen/SPARK-1084.2 and squashes the following commits: 9240abd [Sean Owen] Avoid scala version conflict in scalap induced by json4s dependency 1561cec [Sean Owen] Remove "exclude *" dependencies that are causing Maven warnings, and that are apparently unneeded anyway
* Ignore RateLimitedOutputStreamSuite for now.Reynold Xin2014-03-021-1/+1
| | | | | | | | | | This test has been flaky. We can re-enable it after @tdas has a chance to look at it. Author: Reynold Xin <rxin@apache.org> Closes #54 from rxin/ratelimit and squashes the following commits: 1a12198 [Reynold Xin] Ignore RateLimitedOutputStreamSuite for now.
* Remove remaining references to incubationPatrick Wendell2014-03-021-2/+2
| | | | | | | | | | This removes some loose ends not caught by the other (incubating -> tlp) patches. @markhamstra this updates the version as you mentioned earlier. Author: Patrick Wendell <pwendell@gmail.com> Closes #51 from pwendell/tlp and squashes the following commits: d553b1b [Patrick Wendell] Remove remaining references to incubation
* SPARK 1084.1 (resubmitted)Sean Owen2014-02-274-55/+105
| | | | | | | | | | | | | | | (Ported from https://github.com/apache/incubator-spark/pull/637 ) Author: Sean Owen <sowen@cloudera.com> Closes #31 from srowen/SPARK-1084.1 and squashes the following commits: 6c4a32c [Sean Owen] Suppress warnings about legitimate unchecked array creations, or change code to avoid it f35b833 [Sean Owen] Fix two misc javadoc problems 254e8ef [Sean Owen] Fix one new style error introduced in scaladoc warning commit 5b2fce2 [Sean Owen] Fix scaladoc invocation warning, and enable javac warnings properly, with plugin config updates 007762b [Sean Owen] Remove dead scaladoc links b8ff8cb [Sean Owen] Replace deprecated Ant <tasks> with <target>
* SPARK-1071: Tidy logging strategy and use of log4jSean Owen2014-02-231-5/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Prompted by a recent thread on the mailing list, I tried and failed to see if Spark can be made independent of log4j. There are a few cases where control of the underlying logging is pretty useful, and to do that, you have to bind to a specific logger. Instead I propose some tidying that leaves Spark's use of log4j, but gets rid of warnings and should still enable downstream users to switch. The idea is to pipe everything (except log4j) through SLF4J, and have Spark use SLF4J directly when logging, and where Spark needs to output info (REPL and tests), bind from SLF4J to log4j. This leaves the same behavior in Spark. It means that downstream users who want to use something except log4j should: - Exclude dependencies on log4j, slf4j-log4j12 from Spark - Include dependency on log4j-over-slf4j - Include dependency on another logger X, and another slf4j-X - Recreate any log config that Spark does, that is needed, in the other logger's config That sounds about right. Here are the key changes: - Include the jcl-over-slf4j shim everywhere by depending on it in core. - Exclude dependencies on commons-logging from third-party libraries. - Include the jul-to-slf4j shim everywhere by depending on it in core. - Exclude slf4j-* dependencies from third-party libraries to prevent collision or warnings - Added missing slf4j-log4j12 binding to GraphX, Bagel module tests And minor/incidental changes: - Update to SLF4J 1.7.5, which happily matches Hadoop 2’s version and is a recommended update over 1.7.2 - (Remove a duplicate HBase dependency declaration in SparkBuild.scala) - (Remove a duplicate mockito dependency declaration that was causing warnings and bugging me) Author: Sean Owen <sowen@cloudera.com> Closes #570 from srowen/SPARK-1071 and squashes the following commits: 52eac9f [Sean Owen] Add slf4j-over-log4j12 dependency to core (non-test) and remove it from things that depend on core. 77a7fa9 [Sean Owen] SPARK-1071: Tidy logging strategy and use of log4j
* Merge pull request #567 from ScrapCodes/style2.Prashant Sharma2014-02-0918-65/+108
| | | | | | | | | | | | | | | | SPARK-1058, Fix Style Errors and Add Scala Style to Spark Build. Pt 2 Continuation of PR #557 With this all scala style errors are fixed across the code base !! The reason for creating a separate PR was to not interrupt an already reviewed and ready to merge PR. Hope this gets reviewed soon and merged too. Author: Prashant Sharma <prashant.s@imaginea.com> Closes #567 and squashes the following commits: 3b1ec30 [Prashant Sharma] scala style fixes
* Merge pull request #557 from ScrapCodes/style. Closes #557.Patrick Wendell2014-02-091-7/+10
| | | | | | | | | | | | | | | | | | | | | SPARK-1058, Fix Style Errors and Add Scala Style to Spark Build. Author: Patrick Wendell <pwendell@gmail.com> Author: Prashant Sharma <scrapcodes@gmail.com> == Merge branch commits == commit 1a8bd1c059b842cb95cc246aaea74a79fec684f4 Author: Prashant Sharma <scrapcodes@gmail.com> Date: Sun Feb 9 17:39:07 2014 +0530 scala style fixes commit f91709887a8e0b608c5c2b282db19b8a44d53a43 Author: Patrick Wendell <pwendell@gmail.com> Date: Fri Jan 24 11:22:53 2014 -0800 Adding scalastyle snapshot
* Merge pull request #542 from markhamstra/versionBump. Closes #542.Mark Hamstra2014-02-081-1/+1
| | | | | | | | | | | | | | | | | | Version number to 1.0.0-SNAPSHOT Since 0.9.0-incubating is done and out the door, we shouldn't be building 0.9.0-incubating-SNAPSHOT anymore. @pwendell Author: Mark Hamstra <markhamstra@gmail.com> == Merge branch commits == commit 1b00a8a7c1a7f251b4bb3774b84b9e64758eaa71 Author: Mark Hamstra <markhamstra@gmail.com> Date: Wed Feb 5 09:30:32 2014 -0800 Version number to 1.0.0-SNAPSHOT
* Merge pull request #529 from hsaputra/cleanup_right_arrowop_scalaHenry Saputra2014-02-021-8/+8
| | | | | | | | | | | | | | | | | | | | | Change the ⇒ character (maybe from scalariform) to => in Scala code for style consistency Looks like there are some ⇒ Unicode character (maybe from scalariform) in Scala code. This PR is to change it to => to get some consistency on the Scala code. If we want to use ⇒ as default we could use sbt plugin scalariform to make sure all Scala code has ⇒ instead of => And remove unused imports found in TwitterInputDStream.scala while I was there =) Author: Henry Saputra <hsaputra@apache.org> == Merge branch commits == commit 29c1771d346dff901b0b778f764e6b4409900234 Author: Henry Saputra <hsaputra@apache.org> Date: Sat Feb 1 22:05:16 2014 -0800 Change the ⇒ character (maybe from scalariform) to => in Scala code for style consistency.
* Fix ClassCastException in JavaPairRDD.collectAsMap() (SPARK-1040)Josh Rosen2014-01-252-3/+3
| | | | | | | | | | | | | | | | | | | | This fixes an issue where collectAsMap() could fail when called on a JavaPairRDD that was derived by transforming a non-JavaPairRDD. The root problem was that we were creating the JavaPairRDD's ClassTag by casting a ClassTag[AnyRef] to a ClassTag[Tuple2[K2, V2]]. To fix this, I cast a ClassTag[Tuple2[_, _]] instead, since this actually produces a ClassTag of the appropriate type because ClassTags don't capture type parameters: scala> implicitly[ClassTag[Tuple2[_, _]]] == implicitly[ClassTag[Tuple2[Int, Int]]] res8: Boolean = true scala> implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[Int, Int]]] == implicitly[ClassTag[Tuple2[Int, Int]]] res9: Boolean = false
* Updated java API docs for streaming, along with very minor changes in the ↵Tathagata Das2014-01-167-73/+77
| | | | code examples.
* Made some classes private[stremaing] and deprecated a method in ↵Tathagata Das2014-01-155-4/+11
| | | | JavaStreamingContext.
* Merge remote-tracking branch 'apache/master' into filestream-fixTathagata Das2014-01-141-0/+17
|\
| * Add missing header filesPatrick Wendell2014-01-141-0/+17
| |