-
Notifications
You must be signed in to change notification settings - Fork 29k
Spark 1095 : Adding explicit return types to all public methods #168
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -391,19 +391,24 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { | |
| /** | ||
| * Save this RDD as a text file, using string representations of elements. | ||
| */ | ||
| def saveAsTextFile(path: String) = rdd.saveAsTextFile(path) | ||
| def saveAsTextFile(path: String) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is an example of procedure syntax that should say |
||
| rdd.saveAsTextFile(path) | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * Save this RDD as a compressed text file, using string representations of elements. | ||
| */ | ||
| def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) = | ||
| def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { | ||
| rdd.saveAsTextFile(path, codec) | ||
| } | ||
|
|
||
| /** | ||
| * Save this RDD as a SequenceFile of serialized objects. | ||
| */ | ||
| def saveAsObjectFile(path: String) = rdd.saveAsObjectFile(path) | ||
| def saveAsObjectFile(path: String) { | ||
| rdd.saveAsObjectFile(path) | ||
| } | ||
|
|
||
| /** | ||
| * Creates tuples of the elements in this RDD by applying `f`. | ||
|
|
@@ -420,7 +425,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { | |
| * executed on this RDD. It is strongly recommended that this RDD is persisted in | ||
| * memory, otherwise saving it on a file will require recomputation. | ||
| */ | ||
| def checkpoint() = rdd.checkpoint() | ||
| def checkpoint() { | ||
| rdd.checkpoint() | ||
| } | ||
|
|
||
| /** | ||
| * Return whether this RDD has been checkpointed or not | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,16 +20,17 @@ package org.apache.spark.deploy.master | |
| import scala.collection.JavaConversions._ | ||
|
|
||
| import akka.serialization.Serialization | ||
| import org.apache.zookeeper.CreateMode | ||
|
|
||
| import org.apache.curator.framework.CuratorFramework | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These imports should be grouped above with the akka import. https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide#SparkCodeStyleGuide-Imports |
||
| import org.apache.spark.{Logging, SparkConf} | ||
| import org.apache.zookeeper.CreateMode | ||
|
|
||
| class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) | ||
| extends PersistenceEngine | ||
| with Logging | ||
| { | ||
| val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" | ||
| val zk = SparkCuratorUtil.newClient(conf) | ||
| val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) | ||
|
|
||
| SparkCuratorUtil.mkdir(zk, WORKING_DIR) | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Code style here is slightly wrong, should be