Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 13 additions & 0 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -1444,6 +1444,19 @@ Apart from these, the following properties are also available, and may be useful
#### Spark Streaming
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
<td><code>spark.streaming.backpressure.enabled</code></td>
<td>false</td>
<td>
Enables or disables Spark Streaming's internal backpressure mechanism (since 1.5).
This enables the Spark Streaming to control the receiving rate based on the
current batch scheduling delays and processing times so that the system receives
only as fast as the system can process. Internally, this dynamically sets the
maximum receiving rate of receivers. This rate is upper bounded by the values
`spark.streaming.receiver.maxRate` and `spark.streaming.kafka.maxRatePerPartition`
if they are set (see below).
</td>
</tr>
<tr>
<td><code>spark.streaming.blockInterval</code></td>
<td>200ms</td>
Expand Down
13 changes: 12 additions & 1 deletion docs/streaming-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -1813,7 +1813,7 @@ To run a Spark Streaming applications, you need to have the following.
+ *Mesos* - [Marathon](https://github.com/mesosphere/marathon) has been used to achieve this
with Mesos.

- *[Since Spark 1.2] Configuring write ahead logs* - Since Spark 1.2,
- *Configuring write ahead logs* - Since Spark 1.2,
we have introduced _write ahead logs_ for achieving strong
fault-tolerance guarantees. If enabled, all the data received from a receiver gets written into
a write ahead log in the configuration checkpoint directory. This prevents data loss on driver
Expand All @@ -1828,6 +1828,17 @@ To run a Spark Streaming applications, you need to have the following.
stored in a replicated storage system. This can be done by setting the storage level for the
input stream to `StorageLevel.MEMORY_AND_DISK_SER`.

- *Setting the max receiving rate* - If the cluster resources is not large enough for the streaming
application to process data as fast as it is being received, the receivers can be rate limited
by setting a maximum rate limit in terms of records / sec.
See the [configuration parameters](configuration.html#spark-streaming)
`spark.streaming.receiver.maxRate` for receivers and `spark.streaming.kafka.maxRatePerPartition`
for Direct Kafka approach. In Spark 1.5, we have introduced a feature called *backpressure* that
eliminate the need to set this rate limit, as Spark Streaming automatically figures out the
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: eliminates

rate limits and dynamically adjusts them if the processing conditions change. This backpressure
can be enabled by setting the [configuration parameter](configuration.html#spark-streaming)
`spark.streaming.backpressure.enabled` to `true`.

### Upgrading Application Code
{:.no_toc}

Expand Down