| Commit message (Collapse) | Author | Age | Files | Lines |
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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.
|
|
|
|
|
|
|
|
|
|
|
|
| |
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.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
(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
|
|
|
|
|
|
|
|
|
|
| |
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.
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
(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>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
| |
code examples.
|
|
|
|
| |
JavaStreamingContext.
|
|\ |
|
| | |
|
|/
|
|
| |
in log4j.properties of external modules.
|
| |
|
|\
| |
| |
| |
| | |
Conflicts:
streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
|
| |\
| | |
| | |
| | | |
Adjusted visibility of various components and documentation for 0.9.0 release.
|
| | | |
|
| |\ \
| | | |
| | | |
| | | |
| | | |
| | | |
| | | |
| | | |
| | | |
| | | | |
Automatically unpersisting RDDs that have been cleaned up from DStreams
Earlier RDDs generated by DStreams were forgotten but not unpersisted. The system relied on the natural BlockManager LRU to drop the data. The cleaner.ttl was a hammer to clean up RDDs but it is something that needs to be set separately and need to be set very conservatively (at best, few minutes). This automatic unpersisting allows the system to handle this automatically, which reduces memory usage. As a side effect it will also improve GC performance as there are less number of objects stored in memory. In fact, for some workloads, it may allow RDDs to be cached as deserialized, which speeds up processing without too much GC overheads.
This is disabled by default. To enable it set configuration spark.streaming.unpersist to true. In future release, this will be set to true by default.
Also, reduced sleep time in TaskSchedulerImpl.stop() from 5 second to 1 second. From my conversation with Matei, there does not seem to be any good reason for the sleep for letting messages be sent out be so long.
|
| | | | |
|
| |\ \ \
| | |_|/
| |/| |
| | | |
| | | |
| | | |
| | | |
| | | |
| | | |
| | | | |
Improved logic of finding new files in FileInputDStream
Earlier, if HDFS has a hiccup and reports a existence of a new file (mod time T sec) at time T + 1 sec, then fileStream could have missed that file. With this change, it should be able to find files that are delayed by up to <batch size> seconds. That is, even if file is reported at T + <batch time> sec, file stream should be able to catch it.
The new logic, at a high level, is as follows. It keeps track of the new files it found in the previous interval and mod time of the oldest of those files (lets call it X). Then in the current interval, it will ignore those files that were seen in the previous interval and those which have mod time older than X. So if a new file gets reported by HDFS that in the current interval, but has mod time in the previous interval, it will be considered. However, if the mod time earlier than the previous interval (that is, earlier than X), they will be ignored. This is the current limitation, and future version would improve this behavior further.
Also reduced line lengths in DStream to <=100 chars.
|
| | | |
| | | |
| | | |
| | | | |
were useless as InputDStream has been made to register itself. Also made DStream.register() private[streaming] - not useful to expose the confusing function. Updated a lot of documentation.
|
|\| | |
| |/ /
|/| | |
|
| |/
| |
| |
| | |
`sbt/sbt doc` used to fail. This fixed it.
|
|/ |
|
|\
| |
| |
| |
| |
| |
| |
| | |
Moved DStream and PairDSream to org.apache.spark.streaming.dstream
Similar to the package location of `org.apache.spark.rdd.RDD`, `DStream` has been moved from `org.apache.spark.streaming.DStream` to `org.apache.spark.streaming.dstream.DStream`. I know that the package name is a little long, but I think its better to keep it consistent with Spark's structure.
Also fixed persistence of windowed DStream. The RDDs generated generated by windowed DStream are essentially unions of underlying RDDs, and persistent these union RDDs would store numerous copies of the underlying data. Instead setting the persistence level on the windowed DStream is made to set the persistence level of the underlying DStream.
|
| |\
| | |
| | |
| | |
| | | |
Conflicts:
streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
|
| | |
| | |
| | |
| | | |
level of input streams.
|
| |\ \ |
|
| | | |
| | | |
| | | |
| | | | |
org.apache.spark.streaming to org.apache.spark.streaming.dstream.
|
|\ \ \ \
| | | | |
| | | | |
| | | | |
| | | | |
| | | | | |
Remove now un-needed hostPort option
I noticed this was logging some scary error messages in various places. After I looked into it, this is no longer really used. I removed the option and re-wrote the one remaining use case (it was unnecessary there anyways).
|
| | | | | |
|
|\ \ \ \ \
| |_|_|_|/
|/| | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | | |
Remove simple redundant return statements for Scala methods/functions
Remove simple redundant return statements for Scala methods/functions:
-) Only change simple return statements at the end of method
-) Ignore the complex if-else check
-) Ignore the ones inside synchronized
-) Add small changes to making var to val if possible and remove () for simple get
This hopefully makes the review simpler =)
Pass compile and tests.
|
| |\| | | |
|
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | | |
-) Only change simple return statements at the end of method
-) Ignore the complex if-else check
-) Ignore the ones inside synchronized
|
|\ \ \ \ \
| | |_|_|/
| |/| | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | |
| | | | | |
Better error handling in Spark Streaming and more API cleanup
Earlier errors in jobs generated by Spark Streaming (or in the generation of jobs) could not be caught from the main driver thread (i.e. the thread that called StreamingContext.start()) as it would be thrown in different threads. With this change, after `ssc.start`, one can call `ssc.awaitTermination()` which will be block until the ssc is closed, or there is an exception. This makes it easier to debug.
This change also adds ssc.stop(<stop-spark-context>) where you can stop StreamingContext without stopping the SparkContext.
Also fixes the bug that came up with PRs #393 and #381. MetadataCleaner default value has been changed from 3500 to -1 for normal SparkContext and 3600 when creating a StreamingContext. Also, updated StreamingListenerBus with changes similar to SparkListenerBus in #392.
And changed a lot of protected[streaming] to private[streaming].
|
| | | | | |
|
| | |_|/
| |/| |
| | | |
| | | | |
NetworkInputTracker upon close.
|
| | | | |
|
| |\ \ \
| | | |/
| | |/| |
|
| | | | |
|
| | | |
| | | |
| | | |
| | | | |
protected[streaming] to private[streaming] in StreamingContext and DStream. Added waitForStop to StreamingContext, and StreamingContextSuite.
|
| | | | |
|
| |/ /
|/| |
| | |
| | |
| | |
| | | |
`foreachRDD` makes it clear that the granularity of this operator is per-RDD.
As it stands, `foreach` is inconsistent with with `map`, `filter`, and the other
DStream operators which get pushed down to individual records within each RDD.
|
| | |
| | |
| | |
| | | |
This reverts commit 942c80b34c4642de3b0237761bc1aaeb8cbdd546.
|