aboutsummaryrefslogtreecommitdiff
path: root/docs
diff options
context:
space:
mode:
Diffstat (limited to 'docs')
-rw-r--r--docs/configuration.md18
-rw-r--r--docs/streaming-programming-guide.md12
2 files changed, 23 insertions, 7 deletions
diff --git a/docs/configuration.md b/docs/configuration.md
index 6bd0658b3e..08392c3918 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1574,6 +1574,24 @@ Apart from these, the following properties are also available, and may be useful
How many batches the Spark Streaming UI and status APIs remember before garbage collecting.
</td>
</tr>
+<tr>
+ <td><code>spark.streaming.driver.writeAheadLog.closeFileAfterWrite</code></td>
+ <td>false</td>
+ <td>
+ Whether to close the file after writing a write ahead log record on the driver. Set this to 'true'
+ when you want to use S3 (or any file system that does not support flushing) for the metadata WAL
+ on the driver.
+ </td>
+</tr>
+<tr>
+ <td><code>spark.streaming.receiver.writeAheadLog.closeFileAfterWrite</code></td>
+ <td>false</td>
+ <td>
+ Whether to close the file after writing a write ahead log record on the receivers. Set this to 'true'
+ when you want to use S3 (or any file system that does not support flushing) for the data WAL
+ on the receivers.
+ </td>
+</tr>
</table>
#### SparkR
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 3b071c7da5..1edc0fe347 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -1985,7 +1985,11 @@ To run a Spark Streaming applications, you need to have the following.
to increase aggregate throughput. Additionally, it is recommended that the replication of the
received data within Spark be disabled when the write ahead log is enabled as the log is already
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`.
+ input stream to `StorageLevel.MEMORY_AND_DISK_SER`. While using S3 (or any file system that
+ does not support flushing) for _write ahead logs_, please remember to enable
+ `spark.streaming.driver.writeAheadLog.closeFileAfterWrite` and
+ `spark.streaming.receiver.writeAheadLog.closeFileAfterWrite`. See
+ [Spark Streaming Configuration](configuration.html#spark-streaming) for more details.
- *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
@@ -2023,12 +2027,6 @@ contains serialized Scala/Java/Python objects and trying to deserialize objects
modified classes may lead to errors. In this case, either start the upgraded app with a different
checkpoint directory, or delete the previous checkpoint directory.
-### Other Considerations
-{:.no_toc}
-If the data is being received by the receivers faster than what can be processed,
-you can limit the rate by setting the [configuration parameter](configuration.html#spark-streaming)
-`spark.streaming.receiver.maxRate`.
-
***
## Monitoring Applications