Github user steveloughran commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12004#discussion_r89340299
  
    --- Diff: docs/cloud-integration.md ---
    @@ -0,0 +1,953 @@
    +---
    +layout: global
    +displayTitle: Integration with Cloud Infrastructures
    +title: Integration with Cloud Infrastructures
    +description: Introduction to cloud storage support in Apache Spark 
SPARK_VERSION_SHORT
    +---
    +<!---
    +  Licensed under the Apache License, Version 2.0 (the "License");
    +  you may not use this file except in compliance with the License.
    +  You may obtain a copy of the License at
    +
    +   http://www.apache.org/licenses/LICENSE-2.0
    +
    +  Unless required by applicable law or agreed to in writing, software
    +  distributed under the License is distributed on an "AS IS" BASIS,
    +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +  See the License for the specific language governing permissions and
    +  limitations under the License. See accompanying LICENSE file.
    +-->
    +
    +* This will become a table of contents (this text will be scraped).
    +{:toc}
    +
    +## <a name="introduction"></a>Introduction
    +
    +
    +All the public cloud infrastructures, Amazon AWS, Microsoft Azure, Google 
GCS and others offer
    +persistent data storage systems, "object stores". These are not quite the 
same as classic file
    +systems: in order to scale to hundreds of Petabytes, without any single 
points of failure
    +or size limits, object stores, "blobstores", have a simpler model of `name 
=> data`.
    +
    +Apache Spark can read or write data in object stores for data access.
    +through filesystem connectors implemented in Apache Hadoop or provided by 
third-parties.
    +These libraries make the object stores look *almost* like filesystems, 
with directories and
    +operations on files (rename) and directories (create, rename, delete) 
which mimic
    +those of a classic filesystem. Because of this, Spark and Spark-based 
applications
    +can work with object stores, generally treating them as as if they were 
slower-but-larger filesystems.
    +
    +With these connectors, Apache Spark supports object stores as the source
    +of data for analysis, including Spark Streaming and DataFrames.
    +
    +
    +## <a name="quick_start"></a>Quick Start
    +
    +Provided the relevant libraries are on the classpath, and Spark is 
configured with your credentials,
    +objects in an object store can be can be read or written through URLs 
which uses the name of the
    +object store client as the schema and the bucket/container as the hostname.
    +
    +
    +### Dependencies
    +
    +The Spark application neeeds the relevant Hadoop libraries, which can
    +be done by including the `spark-cloud` module for the specific version of 
spark used.
    +
    +The Spark application should include <code>hadoop-openstack</code> 
dependency, which can
    +be done by including the `spark-cloud` module for the specific version of 
spark used.
    +For example, for Maven support, add the following to the 
<code>pom.xml</code> file:
    +
    +{% highlight xml %}
    +<dependencyManagement>
    +  ...
    +  <dependency>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-cloud_2.11</artifactId>
    +    <version>${spark.version}</version>
    +  </dependency>
    +  ...
    +</dependencyManagement>
    +{% endhighlight %}
    +
    +If using the Scala 2.10-compatible version of Spark, the artifact is of 
course `spark-cloud_2.10`.
    +
    +### Basic Use
    +
    +
    +
    +To refer to a path in Amazon S3, use `s3a://` as the scheme (Hadoop 2.7+) 
or `s3n://` on older versions.
    +
    +{% highlight scala %}
    +sparkContext.textFile("s3a://landsat-pds/scene_list.gz").count()
    +{% endhighlight %}
    +
    +Similarly, an RDD can be saved to an object store via `saveAsTextFile()`
    +
    +
    +{% highlight scala %}
    +val numbers = sparkContext.parallelize(1 to 1000)
    +
    +// save to Amazon S3 (or compatible implementation)
    +numbers.saveAsTextFile("s3a://testbucket/counts")
    +
    +// Save to Azure Object store
    
+numbers.saveAsTextFile("wasb://testbuc...@example.blob.core.windows.net/counts")
    +
    +// save to an OpenStack Swift implementation
    +numbers.saveAsTextFile("swift://testbucket.openstack1/counts")
    +{% endhighlight %}
    +
    +That's essentially it: object stores can act as a source and destination 
of data, using exactly
    +the same APIs to load and save data as one uses to work with data in HDFS 
or other filesystems.
    +
    +Because object stores are viewed by Spark as filesystems, object stores can
    +be used as the source or destination of any spark work —be it batch, 
SQL, DataFrame,
    +Streaming or something else.
    +
    +The steps to do so are as follows
    +
    +1. Use the full URI to refer to a bucket, including the prefix for the 
client-side library
    +to use. Example: `s3a://landsat-pds/scene_list.gz`
    +1. Have the Spark context configured with the authentication details of 
the object store.
    +In a YARN cluster, this may also be done in the `core-site.xml` file.
    +1. Have the JAR containing the filesystem classes on the classpath 
—along with all of its dependencies.
    +
    +### <a name="dataframes"></a>Example: DataFrames
    +
    +DataFrames can be created from and saved to object stores through the 
`read()` and `write()` methods.
    +
    +{% highlight scala %}
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.types.StringType
    +
    +val spark = SparkSession
    +    .builder
    +    .appName("DataFrames")
    +    .config(sparkConf)
    +    .getOrCreate()
    +import spark.implicits._
    +val numRows = 1000
    +
    +// generate test data
    +val sourceData = spark.range(0, numRows).select($"id".as("l"), 
$"id".cast(StringType).as("s"))
    +
    +// define the destination
    +val dest = 
"wasb://yourcontai...@youraccount.blob.core.windows.net/dataframes"
    +
    +// write the data
    +val orcFile = dest + "/data.orc"
    +sourceData.write.format("orc").save(orcFile)
    +
    +// now read it back
    +val orcData = spark.read.format("orc").load(orcFile)
    +
    +// finally, write the data as Parquet
    +orcData.write.format("parquet").save(dest + "/data.parquet")
    +spark.stop()
    +{% endhighlight %}
    +
    +### <a name="streaming"></a>Example: Spark Streaming and Cloud Storage
    +
    +Spark Streaming can monitor files added to object stores, by
    +creating a `FileInputDStream` DStream monitoring a path under a bucket.
    +
    +{% highlight scala %}
    +import org.apache.spark.SparkConf
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.streaming._
    +
    +val sparkConf = new SparkConf()
    +val ssc = new StreamingContext(sparkConf, Milliseconds(5000))
    +try {
    +  val lines = ssc.textFileStream("s3a://bucket/incoming")
    +  val matches = lines.filter(_.endsWith("3"))
    +  matches.print()
    +  ssc.start()
    +  ssc.awaitTermination()
    +} finally {
    +  ssc.stop(true)
    +}
    +{% endhighlight %}
    +
    +1. The time to scan for new files is proportional to the number of files
    +under the path —not the number of *new* files, and that it can become a 
slow operation.
    +The size of the window needs to be set to handle this.
    +
    +1. Files only appear in an object store once they are completely written; 
there
    +is no need for a worklow of write-then-rename to ensure that files aren't 
picked up
    +while they are still being written. Applications can write straight to the 
monitored directory.
    +
    +#### <a name="checkpointing"></a>Checkpointing Streams to object stores
    +
    +Streams should only be checkpointed to an object store considered 
compatible with
    +HDFS. As the checkpoint operation includes a `rename()` operation, 
checkpointing to
    +an object store can be so slow that streaming throughput collapses.
    +
    +
    +## <a name="output"></a>Object Stores as a substitute for HDFS
    +
    +As the examples show, you can write data to object stores. However, that 
does not mean
    +That they can be used as replacements for a cluster-wide filesystem.
    +
    +The full details are covered in [Cloud Object Stores are Not Real 
Filesystems](#cloud_stores_are_not_filesystems).
    +
    +The brief summary is:
    +
    +| Object Store Connector      |  Replace HDFS? |
    +|-----------------------------|--------------------|
    +| Apache `s3a://` `s3n://`    | No  |
    +| Amazon EMR `s3://`          | Yes |
    +| Microsoft Azure `wasb://`   | Yes |
    +| OpenStack `swift://`        | No  |
    +
    +It is possible to use any of the object stores as a destination of work, 
i.e. use
    +`saveAsTextFile()` or `save` to save data there, but the commit process 
may be slow
    +and, unreliable in the presence of failures.
    +
    +It is faster and safer to use the cluster filesystem as the destination of 
Spark jobs,
    +using that data as the data for follow-on work. The final results can
    +be persisted in to the object store using `distcp`.
    +
    +### Recommended settings for writing to object stores
    +
    +Here are the settings to use when writing to object stores. This uses the 
"version 2" algorithm
    +for committing files —which does less renaming than the v1 algorithm. 
Speculative execution is
    +disabled to avoid multiple writers corrupting the output.
    +
    +```
    +spark.speculation false
    +spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2
    +spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true
    +```
    +
    +There's also the option of skipping the cleanup of temporary files in the 
output directory.
    +Enabling this option eliminates a small delay caused by listing and 
deleting any such files.
    +
    +```
    +spark.hadoop.mapreduce.fileoutputcommitter.cleanup.skipped true
    +```
    +
    +Bear in mind that storing temporary files can run up charges; Delete
    +directories called `"_temporary"` on a regular basis to avoid this.
    +
    +
    +### YARN Scheduler settings
    +
    +When running Spark in a YARN cluster running in EC2, turning off locality 
avoids any delays
    +waiting for the scheduler to find a node close to the data.
    +
    +```xml
    +  <property>
    +    <name>yarn.scheduler.capacity.node-locality-delay</name>
    +    <value>0</value>
    +  </property>
    +```
    +
    +This has to be set in the YARN cluster configuration, not in the Spark 
configuration.
    +
    +### Parquet IO Settings
    +
    +For optimal performance when reading files saved in the Apache Parquet 
format,
    +read and write operations must be minimized, including generation of 
summary metadata,
    +and coalescing metadata from multiple files. The Predicate pushdown option
    +enables the Parquet library to skip un-needed columns, so saving bandwidth.
    +
    +    spark.hadoop.parquet.enable.summary-metadata false
    +    spark.sql.parquet.mergeSchema false
    +    spark.sql.parquet.filterPushdown true
    +    spark.sql.hive.metastorePartitionPruning true
    +
    +### ORC IO Settings
    +
    +For optimal performance when reading files saved in the Apache ORC format,
    +read and write operations must be minimized. Here are the options to 
achieve this.
    +
    +
    +    spark.sql.orc.filterPushdown true
    +    spark.sql.orc.splits.include.file.footer true
    +    spark.sql.orc.cache.stripe.details.size 10000
    +    spark.sql.hive.metastorePartitionPruning true
    +
    +The Predicate pushdown option enables the ORC library to skip un-needed 
columns, and use index
    +information to filter out parts of the file where it can be determined 
that no columns match the predicate.
    +
    +The `spark.sql.orc.splits.include.file.footer` option means that the ORC 
file footer information,
    +is passed around with the file information —so eliminating the need to 
reread this data.
    +
    +
    +## <a name="authenticating"></a>Authenticating with Object Stores
    +
    +Apart from the special case of public read-only data, all object stores
    +require callers to authenticate themselves.
    +To do this, the Spark context must be configured with the authentication
    +details of the object store.
    +
    +1. In a YARN cluster, this may be done automatically in the 
`core-site.xml` file.
    +1. When Spark is running in cloud infrastructure (for example, on Amazon 
EC2, Google Cloud or
    +Microsoft Azure), the authentication details may be automatically derived 
from information
    +available to the VM.
    +1. `spark-submit` automatically picks up the contents of `AWS_ACCESS_KEY` 
and `AWS_SECRET_KEY`
    +environment variables and sets the associated configuration parameters 
for`s3n` and `s3a`
    +to these values. This essentially propagates the values across the Spark 
cluster.
    +1. Authentication details may be manually added to the Spark configuration
    +1. Alternatively, they can be programmatically added. *Important: never 
put authentication
    +secrets in source code. They will be compromised*.
    +
    +It is critical that the credentials used to access object stores are kept 
secret. Not only can
    +they be abused to run up compute charges, they can be used to read and 
alter private data.
    +
    +1. If adding login details to a spark configuration file, do not share 
this file, including
    +attaching to bug reports or committing it to SCM repositories.
    +1. Have different accounts for access to the storage for each application,
    +each with access rights restricted to those object storage 
buckets/containers used by the
    +application.
    +1. If the object store supports any form of session credential (e.g. 
Amazon's STS), issue
    +session credentials for the expected lifetime of the application.
    +1. When using a version of Spark with with Hadoop 2.8+ libraries, consider 
using Hadoop
    +credential files to store secrets, referencing
    +these files in the relevant ID/secret properties of the XML configuration 
file.
    +
    +
    +## <a name="object_stores"></a>Object stores and Their Library Dependencies
    +
    +The different object stores supported by Spark depend on specific Hadoop 
versions,
    +and require specific Hadoop JARs and dependent Java libraries on the 
classpath.
    +
    +<table class="table">
    +  <tr><th>Schema</th><th>Store</th><th>Details</th></tr>
    +  <tr>
    +    <td><code>s3a://</code></td>
    +    <td>Amazon S3</a>
    +    <td>
    +    Recommended S3 client for Spark releases built on Apache Hadoop 2.7 or 
later.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td><code>s3n://</code></td>
    +    <td>Amazon S3</a>
    +    <td>
    +    Deprected S3 client; only use for Spark releases built on Apache 
Hadoop 2.6 or earlier.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td><code>s3://</code></td>
    +    <td>Amazon S3 on Amazon EMR</a>
    +    <td>
    +    Amazon's own S3 client; use only and exclusivley in Amazon EMR.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td><code>wasb://</code></td>
    +    <td>Azure Storage</a>
    +    <td>
    +    Client for Microsoft Azure Storage; since Hadoop 2.7.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td><code>swift://</code></td>
    +    <td>OpenStack Swift</a>
    +    <td>
    +    Client for OpenStack Swift object stores.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td><code>gs://</code></td>
    +    <td>Google Cloud Storage</a>
    +    <td>
    +    Google's client for their cloud object store.
    +    </td>
    +  </tr>
    +</table>
    +
    +
    +### <a name="working_with_amazon_s3"></a>Working with Amazon S3
    +
    +Amazon's S3 object store is probably the most widely used object store 
—it is also the one
    +with the most client libraries. This is due to the evolution of Hadoop's 
support, and Amazon
    +offering Hadoop and Spark as its EMR service, along with its own S3 client.
    +
    +The recommendations for which client to use depend upon the version of 
Hadoop on the Spark classpath.
    +
    +<table class="table">
    +  <tr><th>Hadoop Library Version</th><th>Client</th></tr>
    +  <tr>
    +    <td>Hadoop 2.7+ and commercial products based on it</a>
    +    <td><code>s3a://</code></td>
    +  </tr>
    +  <tr>
    +    <td>Hadoop 2.6 or earlier</a>
    +    <td><code>s3n://</code></td>
    +  </tr>
    +  <tr>
    +    <td>Amazon EMR</a>
    +    <td><code>s3://</code></td>
    +  </tr>
    +</table>
    +
    +Authentication is generally via properties set in the spark context or, in 
YARN clusters,
    +`core-site.xml`.
    +Versions of the S3A client also support short-lived session credentials 
and IAM authentication to
    +automatically pick up credentials on EC2 deployments. Consult the 
appropriate Hadoop documentation for specifics.
    +
    +`spark-submit` will automatically pick up and propagate `AWS_ACCESS_KEY` 
and `AWS_SECRET_KEY`
    +from the environment variables set in the environment of the user running 
`spark-submit`; these
    +will override any set in the configuration files.
    +
    +Be aware that while S3 buckets support complex access control 
declarations, Spark needs
    +full read/write access to any bucket to which it must write data. That is: 
it does not support writing
    +to buckets where the root paths are read only, or not readable at all.
    +
    +#### <a name="s3a"></a>S3A Filesystem Client: `s3a://`
    +
    +The ["S3A" filesystem 
client](https://hadoop.apache.org/docs/stable2/hadoop-aws/tools/hadoop-aws/index.html)
    +shipped with in Hadoop 2.6, and has been considered ready for production 
use since Hadoop 2.7.1
    +
    +*The S3A connector is the sole S3 connector undergoing active maintenance 
at the Apache, and
    +should be used wherever possible.*
    +
    +**Classpath**
    --- End diff --
    
    I'll tighten it all down, and cut the CP details.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to