[SPARK-12046][DOC] Fixes various ScalaDoc/JavaDoc issues

This PR backports PR #10039 to master

Author: Cheng Lian <lian@databricks.com>

Closes #10063 from liancheng/spark-12046.doc-fix.master.
This commit is contained in:
Cheng Lian 2015-12-01 10:21:31 -08:00 committed by Michael Armbrust
parent 1401166576
commit 69dbe6b40d
25 changed files with 152 additions and 133 deletions

View file

@ -23,5 +23,5 @@ import java.io.Serializable;
* A four-argument function that takes arguments of type T1, T2, T3 and T4 and returns an R.
*/
public interface Function4<T1, T2, T3, T4, R> extends Serializable {
public R call(T1 v1, T2 v2, T3 v3, T4 v4) throws Exception;
R call(T1 v1, T2 v2, T3 v3, T4 v4) throws Exception;
}

View file

@ -23,5 +23,5 @@ import java.io.Serializable;
* A function with no return value.
*/
public interface VoidFunction<T> extends Serializable {
public void call(T t) throws Exception;
void call(T t) throws Exception;
}

View file

@ -23,5 +23,5 @@ import java.io.Serializable;
* A two-argument function that takes arguments of type T1 and T2 with no return value.
*/
public interface VoidFunction2<T1, T2> extends Serializable {
public void call(T1 v1, T2 v2) throws Exception;
void call(T1 v1, T2 v2) throws Exception;
}

View file

@ -215,13 +215,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
/**
* Generic function to combine the elements for each key using a custom set of aggregation
* functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a
* "combined type" C * Note that V and C can be different -- for example, one might group an
* "combined type" C. Note that V and C can be different -- for example, one might group an
* RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three
* functions:
*
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
*
* In addition, users can control the partitioning of the output RDD, the serializer that is use
* for the shuffle, and whether to perform map-side aggregation (if a mapper can produce multiple
@ -247,13 +247,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
/**
* Generic function to combine the elements for each key using a custom set of aggregation
* functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a
* "combined type" C * Note that V and C can be different -- for example, one might group an
* "combined type" C. Note that V and C can be different -- for example, one might group an
* RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three
* functions:
*
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
*
* In addition, users can control the partitioning of the output RDD. This method automatically
* uses map-side aggregation in shuffling the RDD.

View file

@ -21,13 +21,13 @@ package org.apache.spark
* This package implements Spark's memory management system. This system consists of two main
* components, a JVM-wide memory manager and a per-task manager:
*
* - [[org.apache.spark.memory.MemoryManager]] manages Spark's overall memory usage within a JVM.
* This component implements the policies for dividing the available memory across tasks and for
* allocating memory between storage (memory used caching and data transfer) and execution (memory
* used by computations, such as shuffles, joins, sorts, and aggregations).
* - [[org.apache.spark.memory.TaskMemoryManager]] manages the memory allocated by individual tasks.
* Tasks interact with TaskMemoryManager and never directly interact with the JVM-wide
* MemoryManager.
* - [[org.apache.spark.memory.MemoryManager]] manages Spark's overall memory usage within a JVM.
* This component implements the policies for dividing the available memory across tasks and for
* allocating memory between storage (memory used caching and data transfer) and execution
* (memory used by computations, such as shuffles, joins, sorts, and aggregations).
* - [[org.apache.spark.memory.TaskMemoryManager]] manages the memory allocated by individual
* tasks. Tasks interact with TaskMemoryManager and never directly interact with the JVM-wide
* MemoryManager.
*
* Internally, each of these components have additional abstractions for memory bookkeeping:
*

View file

@ -70,7 +70,7 @@ private[spark] class CoGroupPartition(
*
* Note: This is an internal API. We recommend users use RDD.cogroup(...) instead of
* instantiating this directly.
*
* @param rdds parent RDDs.
* @param part partitioner used to partition the shuffle output
*/

View file

@ -65,9 +65,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Note that V and C can be different -- for example, one might group an RDD of type
* (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions:
*
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
*
* In addition, users can control the partitioning of the output RDD, and whether to perform
* map-side aggregation (if a mapper can produce multiple items with the same key).

View file

@ -86,7 +86,7 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag](
Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i))
}
override def getPreferredLocations(partition: Partition): Seq[String] = {
override protected def getPreferredLocations(partition: Partition): Seq[String] = {
val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]]
tracker.getPreferredLocationsForShuffle(dep, partition.index)

View file

@ -33,8 +33,9 @@ import org.apache.spark.util.Utils
/**
* A unit of execution. We have two kinds of Task's in Spark:
* - [[org.apache.spark.scheduler.ShuffleMapTask]]
* - [[org.apache.spark.scheduler.ResultTask]]
*
* - [[org.apache.spark.scheduler.ShuffleMapTask]]
* - [[org.apache.spark.scheduler.ResultTask]]
*
* A Spark job consists of one or more stages. The very last stage in a job consists of multiple
* ResultTasks, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task

View file

@ -53,12 +53,13 @@ private[spark] object SerializationDebugger extends Logging {
/**
* Find the path leading to a not serializable object. This method is modeled after OpenJDK's
* serialization mechanism, and handles the following cases:
* - primitives
* - arrays of primitives
* - arrays of non-primitive objects
* - Serializable objects
* - Externalizable objects
* - writeReplace
*
* - primitives
* - arrays of primitives
* - arrays of non-primitive objects
* - Serializable objects
* - Externalizable objects
* - writeReplace
*
* It does not yet handle writeObject override, but that shouldn't be too hard to do either.
*/

View file

@ -122,6 +122,7 @@ class Vector(val elements: Array[Double]) extends Serializable {
override def toString: String = elements.mkString("(", ", ", ")")
}
@deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0")
object Vector {
def apply(elements: Array[Double]): Vector = new Vector(elements)

View file

@ -67,24 +67,24 @@ import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter}
*
* At a high level, this class works internally as follows:
*
* - We repeatedly fill up buffers of in-memory data, using either a PartitionedAppendOnlyMap if
* we want to combine by key, or a PartitionedPairBuffer if we don't.
* Inside these buffers, we sort elements by partition ID and then possibly also by key.
* To avoid calling the partitioner multiple times with each key, we store the partition ID
* alongside each record.
* - We repeatedly fill up buffers of in-memory data, using either a PartitionedAppendOnlyMap if
* we want to combine by key, or a PartitionedPairBuffer if we don't.
* Inside these buffers, we sort elements by partition ID and then possibly also by key.
* To avoid calling the partitioner multiple times with each key, we store the partition ID
* alongside each record.
*
* - When each buffer reaches our memory limit, we spill it to a file. This file is sorted first
* by partition ID and possibly second by key or by hash code of the key, if we want to do
* aggregation. For each file, we track how many objects were in each partition in memory, so we
* don't have to write out the partition ID for every element.
* - When each buffer reaches our memory limit, we spill it to a file. This file is sorted first
* by partition ID and possibly second by key or by hash code of the key, if we want to do
* aggregation. For each file, we track how many objects were in each partition in memory, so we
* don't have to write out the partition ID for every element.
*
* - When the user requests an iterator or file output, the spilled files are merged, along with
* any remaining in-memory data, using the same sort order defined above (unless both sorting
* and aggregation are disabled). If we need to aggregate by key, we either use a total ordering
* from the ordering parameter, or read the keys with the same hash code and compare them with
* each other for equality to merge values.
* - When the user requests an iterator or file output, the spilled files are merged, along with
* any remaining in-memory data, using the same sort order defined above (unless both sorting
* and aggregation are disabled). If we need to aggregate by key, we either use a total ordering
* from the ordering parameter, or read the keys with the same hash code and compare them with
* each other for equality to merge values.
*
* - Users are expected to call stop() at the end to delete all the intermediate files.
* - Users are expected to call stop() at the end to delete all the intermediate files.
*/
private[spark] class ExternalSorter[K, V, C](
context: TaskContext,

View file

@ -23,9 +23,10 @@ import org.apache.spark.storage.DiskBlockObjectWriter
/**
* A common interface for size-tracking collections of key-value pairs that
* - Have an associated partition for each key-value pair.
* - Support a memory-efficient sorted iterator
* - Support a WritablePartitionedIterator for writing the contents directly as bytes.
*
* - Have an associated partition for each key-value pair.
* - Support a memory-efficient sorted iterator
* - Support a WritablePartitionedIterator for writing the contents directly as bytes.
*/
private[spark] trait WritablePartitionedPairCollection[K, V] {
/**

View file

@ -46,17 +46,18 @@ case class SerializableAWSCredentials(accessKeyId: String, secretKey: String)
* https://github.com/awslabs/amazon-kinesis-client
*
* The way this Receiver works is as follows:
* - The receiver starts a KCL Worker, which is essentially runs a threadpool of multiple
* KinesisRecordProcessor
* - Each KinesisRecordProcessor receives data from a Kinesis shard in batches. Each batch is
* inserted into a Block Generator, and the corresponding range of sequence numbers is recorded.
* - When the block generator defines a block, then the recorded sequence number ranges that were
* inserted into the block are recorded separately for being used later.
* - When the block is ready to be pushed, the block is pushed and the ranges are reported as
* metadata of the block. In addition, the ranges are used to find out the latest sequence
* number for each shard that can be checkpointed through the DynamoDB.
* - Periodically, each KinesisRecordProcessor checkpoints the latest successfully stored sequence
* number for it own shard.
*
* - The receiver starts a KCL Worker, which is essentially runs a threadpool of multiple
* KinesisRecordProcessor
* - Each KinesisRecordProcessor receives data from a Kinesis shard in batches. Each batch is
* inserted into a Block Generator, and the corresponding range of sequence numbers is recorded.
* - When the block generator defines a block, then the recorded sequence number ranges that were
* inserted into the block are recorded separately for being used later.
* - When the block is ready to be pushed, the block is pushed and the ranges are reported as
* metadata of the block. In addition, the ranges are used to find out the latest sequence
* number for each shard that can be checkpointed through the DynamoDB.
* - Periodically, each KinesisRecordProcessor checkpoints the latest successfully stored sequence
* number for it own shard.
*
* @param streamName Kinesis stream name
* @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)

View file

@ -226,12 +226,13 @@ object KinesisUtils {
* This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
*
* Note:
* - The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
* on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
* gets AWS credentials.
* - The region of the `endpointUrl` will be used for DynamoDB and CloudWatch.
* - The Kinesis application name used by the Kinesis Client Library (KCL) will be the app name in
* [[org.apache.spark.SparkConf]].
*
* - The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
* on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
* gets AWS credentials.
* - The region of the `endpointUrl` will be used for DynamoDB and CloudWatch.
* - The Kinesis application name used by the Kinesis Client Library (KCL) will be the app name
* in [[org.apache.spark.SparkConf]].
*
* @param ssc StreamingContext object
* @param streamName Kinesis stream name

View file

@ -81,11 +81,13 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va
* Set the convergence tolerance. Default 0.001
* convergenceTol is a condition which decides iteration termination.
* The end of iteration is decided based on below logic.
* - If the norm of the new solution vector is >1, the diff of solution vectors
* is compared to relative tolerance which means normalizing by the norm of
* the new solution vector.
* - If the norm of the new solution vector is <=1, the diff of solution vectors
* is compared to absolute tolerance which is not normalizing.
*
* - If the norm of the new solution vector is >1, the diff of solution vectors
* is compared to relative tolerance which means normalizing by the norm of
* the new solution vector.
* - If the norm of the new solution vector is <=1, the diff of solution vectors
* is compared to absolute tolerance which is not normalizing.
*
* Must be between 0.0 and 1.0 inclusively.
*/
def setConvergenceTol(tolerance: Double): this.type = {

View file

@ -535,6 +535,8 @@ object Unidoc {
.map(_.filterNot(_.getName.contains("$")))
.map(_.filterNot(_.getCanonicalPath.contains("akka")))
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/deploy")))
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/examples")))
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/memory")))
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/network")))
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/shuffle")))
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/executor")))

View file

@ -136,11 +136,12 @@ class Column(protected[sql] val expr: Expression) extends Logging {
/**
* Extracts a value or values from a complex type.
* The following types of extraction are supported:
* - Given an Array, an integer ordinal can be used to retrieve a single value.
* - Given a Map, a key of the correct type can be used to retrieve an individual value.
* - Given a Struct, a string fieldName can be used to extract that field.
* - Given an Array of Structs, a string fieldName can be used to extract filed
* of every struct in that array, and return an Array of fields
*
* - Given an Array, an integer ordinal can be used to retrieve a single value.
* - Given a Map, a key of the correct type can be used to retrieve an individual value.
* - Given a Struct, a string fieldName can be used to extract that field.
* - Given an Array of Structs, a string fieldName can be used to extract filed
* of every struct in that array, and return an Array of fields
*
* @group expr_ops
* @since 1.4.0

View file

@ -574,11 +574,12 @@ class StreamingContext private[streaming] (
* :: DeveloperApi ::
*
* Return the current state of the context. The context can be in three possible states -
* - StreamingContextState.INTIALIZED - The context has been created, but not been started yet.
* Input DStreams, transformations and output operations can be created on the context.
* - StreamingContextState.ACTIVE - The context has been started, and been not stopped.
* Input DStreams, transformations and output operations cannot be created on the context.
* - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more.
*
* - StreamingContextState.INTIALIZED - The context has been created, but not been started yet.
* Input DStreams, transformations and output operations can be created on the context.
* - StreamingContextState.ACTIVE - The context has been started, and been not stopped.
* Input DStreams, transformations and output operations cannot be created on the context.
* - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more.
*/
@DeveloperApi
def getState(): StreamingContextState = synchronized {

View file

@ -42,6 +42,7 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti
* class remembers the information about the files selected in past batches for
* a certain duration (say, "remember window") as shown in the figure below.
*
* {{{
* |<----- remember window ----->|
* ignore threshold --->| |<--- current batch time
* |____.____.____.____.____.____|
@ -49,6 +50,7 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti
* ---------------------|----|----|----|----|----|----|-----------------------> Time
* |____|____|____|____|____|____|
* remembered batches
* }}}
*
* The trailing end of the window is the "ignore threshold" and all files whose mod times
* are less than this threshold are assumed to have already been selected and are therefore
@ -59,14 +61,15 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti
* `isNewFile` for more details.
*
* This makes some assumptions from the underlying file system that the system is monitoring.
* - The clock of the file system is assumed to synchronized with the clock of the machine running
* the streaming app.
* - If a file is to be visible in the directory listings, it must be visible within a certain
* duration of the mod time of the file. This duration is the "remember window", which is set to
* 1 minute (see `FileInputDStream.minRememberDuration`). Otherwise, the file will never be
* selected as the mod time will be less than the ignore threshold when it becomes visible.
* - Once a file is visible, the mod time cannot change. If it does due to appends, then the
* processing semantics are undefined.
*
* - The clock of the file system is assumed to synchronized with the clock of the machine running
* the streaming app.
* - If a file is to be visible in the directory listings, it must be visible within a certain
* duration of the mod time of the file. This duration is the "remember window", which is set to
* 1 minute (see `FileInputDStream.minRememberDuration`). Otherwise, the file will never be
* selected as the mod time will be less than the ignore threshold when it becomes visible.
* - Once a file is visible, the mod time cannot change. If it does due to appends, then the
* processing semantics are undefined.
*/
private[streaming]
class FileInputDStream[K, V, F <: NewInputFormat[K, V]](

View file

@ -84,13 +84,14 @@ private[streaming] class BlockGenerator(
/**
* The BlockGenerator can be in 5 possible states, in the order as follows.
* - Initialized: Nothing has been started
* - Active: start() has been called, and it is generating blocks on added data.
* - StoppedAddingData: stop() has been called, the adding of data has been stopped,
* but blocks are still being generated and pushed.
* - StoppedGeneratingBlocks: Generating of blocks has been stopped, but
* they are still being pushed.
* - StoppedAll: Everything has stopped, and the BlockGenerator object can be GCed.
*
* - Initialized: Nothing has been started
* - Active: start() has been called, and it is generating blocks on added data.
* - StoppedAddingData: stop() has been called, the adding of data has been stopped,
* but blocks are still being generated and pushed.
* - StoppedGeneratingBlocks: Generating of blocks has been stopped, but
* they are still being pushed.
* - StoppedAll: Everything has stopped, and the BlockGenerator object can be GCed.
*/
private object GeneratorState extends Enumeration {
type GeneratorState = Value
@ -125,9 +126,10 @@ private[streaming] class BlockGenerator(
/**
* Stop everything in the right order such that all the data added is pushed out correctly.
* - First, stop adding data to the current buffer.
* - Second, stop generating blocks.
* - Finally, wait for queue of to-be-pushed blocks to be drained.
*
* - First, stop adding data to the current buffer.
* - Second, stop generating blocks.
* - Finally, wait for queue of to-be-pushed blocks to be drained.
*/
def stop(): Unit = {
// Set the state to stop adding data

View file

@ -27,28 +27,29 @@ import org.apache.spark.streaming.receiver.Receiver
* A class that tries to schedule receivers with evenly distributed. There are two phases for
* scheduling receivers.
*
* - The first phase is global scheduling when ReceiverTracker is starting and we need to schedule
* all receivers at the same time. ReceiverTracker will call `scheduleReceivers` at this phase.
* It will try to schedule receivers such that they are evenly distributed. ReceiverTracker should
* update its `receiverTrackingInfoMap` according to the results of `scheduleReceivers`.
* `ReceiverTrackingInfo.scheduledLocations` for each receiver should be set to an location list
* that contains the scheduled locations. Then when a receiver is starting, it will send a
* register request and `ReceiverTracker.registerReceiver` will be called. In
* `ReceiverTracker.registerReceiver`, if a receiver's scheduled locations is set, it should check
* if the location of this receiver is one of the scheduled locations, if not, the register will
* be rejected.
* - The second phase is local scheduling when a receiver is restarting. There are two cases of
* receiver restarting:
* - If a receiver is restarting because it's rejected due to the real location and the scheduled
* locations mismatching, in other words, it fails to start in one of the locations that
* `scheduleReceivers` suggested, `ReceiverTracker` should firstly choose the executors that are
* still alive in the list of scheduled locations, then use them to launch the receiver job.
* - If a receiver is restarting without a scheduled locations list, or the executors in the list
* are dead, `ReceiverTracker` should call `rescheduleReceiver`. If so, `ReceiverTracker` should
* not set `ReceiverTrackingInfo.scheduledLocations` for this receiver, instead, it should clear
* it. Then when this receiver is registering, we can know this is a local scheduling, and
* `ReceiverTrackingInfo` should call `rescheduleReceiver` again to check if the launching
* location is matching.
* - The first phase is global scheduling when ReceiverTracker is starting and we need to schedule
* all receivers at the same time. ReceiverTracker will call `scheduleReceivers` at this phase.
* It will try to schedule receivers such that they are evenly distributed. ReceiverTracker
* should update its `receiverTrackingInfoMap` according to the results of `scheduleReceivers`.
* `ReceiverTrackingInfo.scheduledLocations` for each receiver should be set to an location list
* that contains the scheduled locations. Then when a receiver is starting, it will send a
* register request and `ReceiverTracker.registerReceiver` will be called. In
* `ReceiverTracker.registerReceiver`, if a receiver's scheduled locations is set, it should
* check if the location of this receiver is one of the scheduled locations, if not, the register
* will be rejected.
* - The second phase is local scheduling when a receiver is restarting. There are two cases of
* receiver restarting:
* - If a receiver is restarting because it's rejected due to the real location and the scheduled
* locations mismatching, in other words, it fails to start in one of the locations that
* `scheduleReceivers` suggested, `ReceiverTracker` should firstly choose the executors that
* are still alive in the list of scheduled locations, then use them to launch the receiver
* job.
* - If a receiver is restarting without a scheduled locations list, or the executors in the list
* are dead, `ReceiverTracker` should call `rescheduleReceiver`. If so, `ReceiverTracker`
* should not set `ReceiverTrackingInfo.scheduledLocations` for this receiver, instead, it
* should clear it. Then when this receiver is registering, we can know this is a local
* scheduling, and `ReceiverTrackingInfo` should call `rescheduleReceiver` again to check if
* the launching location is matching.
*
* In conclusion, we should make a global schedule, try to achieve that exactly as long as possible,
* otherwise do local scheduling.

View file

@ -34,9 +34,10 @@ import org.apache.spark.{Logging, SparkConf}
/**
* This class manages write ahead log files.
* - Writes records (bytebuffers) to periodically rotating log files.
* - Recovers the log files and the reads the recovered records upon failures.
* - Cleans up old log files.
*
* - Writes records (bytebuffers) to periodically rotating log files.
* - Recovers the log files and the reads the recovered records upon failures.
* - Cleans up old log files.
*
* Uses [[org.apache.spark.streaming.util.FileBasedWriteAheadLogWriter]] to write
* and [[org.apache.spark.streaming.util.FileBasedWriteAheadLogReader]] to read.

View file

@ -72,10 +72,10 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name:
/**
* Stop the timer, and return the last time the callback was made.
* - interruptTimer = true will interrupt the callback
* if it is in progress (not guaranteed to give correct time in this case).
* - interruptTimer = false guarantees that there will be at least one callback after `stop` has
* been called.
*
* @param interruptTimer True will interrupt the callback if it is in progress (not guaranteed to
* give correct time in this case). False guarantees that there will be at
* least one callback after `stop` has been called.
*/
def stop(interruptTimer: Boolean): Long = synchronized {
if (!stopped) {

View file

@ -1336,11 +1336,11 @@ object Client extends Logging {
*
* This method uses two configuration values:
*
* - spark.yarn.config.gatewayPath: a string that identifies a portion of the input path that may
* only be valid in the gateway node.
* - spark.yarn.config.replacementPath: a string with which to replace the gateway path. This may
* contain, for example, env variable references, which will be expanded by the NMs when
* starting containers.
* - spark.yarn.config.gatewayPath: a string that identifies a portion of the input path that may
* only be valid in the gateway node.
* - spark.yarn.config.replacementPath: a string with which to replace the gateway path. This may
* contain, for example, env variable references, which will be expanded by the NMs when
* starting containers.
*
* If either config is not available, the input path is returned.
*/