From 1da9a3d181e5226a0ae9379c0c8905b319a4afe9 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 18 Nov 2016 15:50:15 +0000 Subject: [PATCH 01/28] [SPARK-7481] stripped down packaging only module --- assembly/pom.xml | 14 + cloud/pom.xml | 174 ++++++ dev/deps/spark-deps-hadoop-2.7 | 5 + docs/cloud-integration.md | 953 ++++++++++++++++++++++++++++++++ docs/rdd-programming-guide.md | 6 +- docs/storage-openstack-swift.md | 35 +- pom.xml | 25 + 7 files changed, 1185 insertions(+), 27 deletions(-) create mode 100644 cloud/pom.xml create mode 100644 docs/cloud-integration.md diff --git a/assembly/pom.xml b/assembly/pom.xml index 742a4a1531e71..ba34967037a0f 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -226,5 +226,19 @@ provided + + + + cloud + + + org.apache.spark + spark-cloud_${scala.binary.version} + ${project.version} + + + diff --git a/cloud/pom.xml b/cloud/pom.xml new file mode 100644 index 0000000000000..0a9602afbb4b5 --- /dev/null +++ b/cloud/pom.xml @@ -0,0 +1,174 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-SNAPSHOT + ../pom.xml + + + spark-cloud_2.11 + jar + Spark Project Cloud Integration + + Contains support for cloud infrastructures, specifically the Hadoop JARs and + transitive dependencies needed to interact with the infrastructures. + + Any project which explicitly depends upon the spark-cloud artifact will get the dependencies; + the exact versions of which will depend upon the hadoop version Spark was compiled against. + + Hadoop 2.7: + hadoop-aws + aws-java-sdk-s3 + hadoop-azure + azure-storage + hadoop-openstack + + WARNING: the signatures of methods in aws-java-sdk/aws-java-sdk-s3 can change between versions: + use the same version against which Hadoop was compiled. + + + + cloud + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-mllib_${scala.binary.version} + ${project.version} + provided + + + + + net.java.dev.jets3t + jets3t + + + + + com.google.guava + guava + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + + + + sbt + + + com.google.guava + guava + compile + + + + + + hadoop-2.7 + + + org.apache.hadoop + hadoop-aws + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-azure + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-openstack + ${hadoop.deps.scope} + + + + com.fasterxml.jackson.core + jackson-databind + ${hadoop.deps.scope} + + + com.fasterxml.jackson.core + jackson-annotations + ${hadoop.deps.scope} + + + + + + + + + diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 6bf0923a1d751..16edf80d40712 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -16,6 +16,8 @@ arpack_combined_all-0.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar +aws-java-sdk-1.7.4.jar +azure-storage-2.0.0.jar base64-2.3.8.jar bcprov-jdk15on-1.51.jar bonecp-0.8.0.RELEASE.jar @@ -61,6 +63,8 @@ guice-3.0.jar guice-servlet-3.0.jar hadoop-annotations-2.7.3.jar hadoop-auth-2.7.3.jar +hadoop-aws-2.7.3.jar +hadoop-azure-2.7.3.jar hadoop-client-2.7.3.jar hadoop-common-2.7.3.jar hadoop-hdfs-2.7.3.jar @@ -69,6 +73,7 @@ hadoop-mapreduce-client-common-2.7.3.jar hadoop-mapreduce-client-core-2.7.3.jar hadoop-mapreduce-client-jobclient-2.7.3.jar hadoop-mapreduce-client-shuffle-2.7.3.jar +hadoop-hadoop-openstack-2.7.3.jar hadoop-yarn-api-2.7.3.jar hadoop-yarn-client-2.7.3.jar hadoop-yarn-common-2.7.3.jar diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md new file mode 100644 index 0000000000000..ec51ecb78cf14 --- /dev/null +++ b/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 +--- + + +* This will become a table of contents (this text will be scraped). +{:toc} + +## 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. + + +## 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 hadoop-openstack 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 pom.xml file: + +{% highlight xml %} + + ... + + org.apache.spark + spark-cloud_2.11 + ${spark.version} + + ... + +{% 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://testbucket@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. + +### 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://yourcontainer@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 %} + +### 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. + +#### 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. + + +## 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 + + yarn.scheduler.capacity.node-locality-delay + 0 + +``` + +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. + + +## 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. + + +## 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. + + + + + + + + + + + + + + + + + + + + + + + + + + + +
SchemaStoreDetails
s3a://Amazon S3 + + Recommended S3 client for Spark releases built on Apache Hadoop 2.7 or later. +
s3n://Amazon S3 + + Deprected S3 client; only use for Spark releases built on Apache Hadoop 2.6 or earlier. +
s3://Amazon S3 on Amazon EMR + + Amazon's own S3 client; use only and exclusivley in Amazon EMR. +
wasb://Azure Storage + + Client for Microsoft Azure Storage; since Hadoop 2.7. +
swift://OpenStack Swift + + Client for OpenStack Swift object stores. +
gs://Google Cloud Storage + + Google's client for their cloud object store. +
+ + +### 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. + + + + + + + + + + + + +
Hadoop Library VersionClient
Hadoop 2.7+ and commercial products based on it + s3a://
Hadoop 2.6 or earlier + s3n://
Amazon EMR + s3://
+ +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. + +#### 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** + +1. The implementation is in `hadoop-aws`, which is included in `$SPARK_HOME/jars` when Spark +is built with cloud support. + +1. Dependencies: `amazon-aws-sdk` JAR (Hadoop 2.7); +`amazon-s3-sdk` and `amazon-core-sdk` in Hadoop 2.8+. + +1. The Amazon JARs have proven very brittle —the version of the Amazon +libraries *must* match that which the Hadoop binaries were built against. + +1. S3A has authentication problems on Java 8u60+ if there is an old version +of Joda Time on the classpath. +If authentication is failing, see if`joda-time.jar` needs upgrading to 2.8.1 or later. + +**Tuning for performance:** + +For recent Hadoop versions, *when working with binary formats* (Parquet, ORC) use + +``` +spark.hadoop.fs.s3a.experimental.input.fadvise random +``` + +This reads from the object in blocks, which is efficient when seeking backwards as well as +forwards in a file —at the expense of making full file reads slower. This option is ignored +on older S3A versions. + +When working with text formats (text, CSV), or any sequential read through an entire file, +this "random" IO policy should be disabled. This is actually the default, but can be done +explicitly: + +``` +spark.hadoop.fs.s3a.experimental.input.fadvise normal +spark.hadoop.fs.s3a.readahead.range 157810688 +``` + +This optimizes the object read for sequential input, and when there is a forward `seek()` call +up to that readahead range, will simply read the data in the current HTTPS request, rather than +abort it and start a new one. + + +#### S3 Native Client `s3n://` + +The ["S3N" filesystem client](https://hadoop.apache.org/docs/stable2/hadoop-aws/tools/hadoop-aws/index.html) +was implemented in 2008 and has been widely used. + +While stable, S3N is essentially unmaintained, and deprecated in favor of S3A. +As well as being slower and limited in authentication mechanisms, the +only maintenance it receives are critical security issues. + +**Classpath** + +Hadoop 2.5 and earlier: add `jets3t.jar` to the classpath + +Hadoop 2.6+: bBoth `hadoop-aws.jar` and `jets3t.jar` (version 0.9.0 or later) +must be on the classpath. + +#### Amazon EMR's S3 Client: `s3://` + + +In Amazon EMR, `s3://` is the URL schema used to refer to +[Amazon's own filesystem client](https://aws.amazon.com/premiumsupport/knowledge-center/emr-file-system-s3/), +one that is closed-source. + +As EMR also maps `s3n://` to the same filesystem, using URLs with the `s3n://` schema avoids +some confusion. Bear in mind, however, that Amazon's S3 client library is not the Apache one: +only Amazon can field bug reports related to it. + +To work with this data outside of EMR itself, use `s3a://` or `s3n://` instead. + + +#### Obsolete: Apache Hadoop's S3 client, `s3://` + +Apache's own Hadoop releases (i.e not EMR), uses URL `s3://` to refer to a +deprecated inode-based filesystem implemented on top of S3. +This filesystem is obsolete, deprecated and has been dropped from Hadoop 3.x. + +*Important: * Do not use `s3://` URLs with Apache Spark except on Amazon EMR* +It is not the same as the Amazon EMR one and incompatible with all other applications. + + +### Working with Microsoft Azure Storage + +Azure support comes with the [`wasb` filesystem client](https://hadoop.apache.org/docs/stable2/hadoop-azure/index.html). + +The Apache implementation is that used by Microsoft in Azure itself: it can be used +to access data in Azure as well as remotely. The object store itself is *consistent*, and +can be reliably used as the destination of queries. + +**Classpath** + +1. The `wasb` filesystem client is implemented in the`hadoop-azure` JAR available in Hadoop 2.7. +1. It also needs a matching `azure-storage` JAR. + + +### Working with OpenStack Swift + + +The OpenStack [`swift://` filesystem client](https://hadoop.apache.org/docs/stable2/hadoop-openstack/index.html) +works with Swift object stores in private OpenStack installations, public installations +including Rackspace Cloud and IBM Softlayer. + +**Classpath** + +1. `swift://` support comes from `hadoop-openstack`. +1. All other dependencies, including `httpclient`, `jackson`, and `commons-logging` are always +included in Spark distributions. + +### Working with Google Cloud Storage + +[Google Cloud Storage](https://cloud.google.com/storage) is supported via Google's own +[GCS filesystem client](https://cloud.google.com/hadoop/google-cloud-storage-connector). + +**Classpath** + +1. For use outside of Google cloud, `gcs-connector.jar` must be be manually downloaded then added +to `$SPARK_HOME/jars`. + + +## Important: Cloud Object Stores are Not Real Filesystems + +Object stores are not filesystems: they are not a hierarchical tree of directories and files. + +The Hadoop filesystem APIs offer a filesystem API to the object stores, but underneath +they are still object stores, [and the difference is significant](http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) + +While object stores can be used as the source and store +for persistent data, they cannot be used as a direct replacement for a cluster-wide filesystem such as HDFS. +This is important to know, as the fact they are accessed with the same APIs can be misleading. + +### Directory Operations May be Slow and Non-atomic + +Directory rename and delete may be performed as a series of operations. Specifically, recursive +directory deletion may be implemented as "list the objects, delete them singly or in batches". +File and directory renames may be implemented as "copy all the objects" followed by the delete operation. + +1. The time to delete a directory depends on the number of files in the directory. +1. Directory deletion may fail partway through, leaving a partially deleted directory. +1. Directory renaming may fail part way through, leaving the destination directory containing some of the files +being renamed, the source directory untouched. +1. The time to rename files and directories increases with the amount of data to rename. +1. If the rename is done on the client, the time to rename +each file will depend upon the bandwidth between client and the filesystem. The further away the client +is, the longer the rename will take. +1. Recursive directory listing can be very slow. This can slow down some parts of job submission +and execution. + +Because of these behaviours, committing of work by renaming directories is neither efficient nor +reliable. In Spark 1.6 and predecessors, there was a special output committer for Parquet, +the `org.apache.spark.sql.execution.datasources.parquet.DirectParquetOutputCommitter` +which bypasses the rename phase. However, as well as having major problems when used + with speculative execution enabled, it handled failures badly. For this reason, it +[was removed from Spark 2.0](https://issues.apache.org/jira/browse/SPARK-10063). + +*Critical* speculative execution does not work against object +stores which do not support atomic directory renames. Your output may get +corrupted. + +*Warning* even non-speculative execution is at risk of leaving the output of a job in an inconsistent +state if a "Direct" output committer is used and executors fail. + +### Data is Not Written Until the OutputStream's `close()` Operation. + +Data written to the object store is often buffered to a local file or stored in memory, +until one of the following conditions of met: + +1. When the output stream's `close()` operation is invoked. +1. Where supported and enabled, there is enough data to create a partition in a + multi-partitioned upload. + +Calls to `OutputStream.flush()` are usually a no-op, or limited to flushing to any local buffer +file. + +- Data is not visible in the object store until the entire output stream has been written. +- If the operation of writing the data does not complete, no data is saved to the object store. +(this includes transient network failures as well as failures of the process itself) +- There may not be an entry in the object store for the file (even a zero-byte one) until +the write is complete. Hence: no indication that a file is being written. +- The time to close a file is usually proportional to `filesize/bandwidth`. + +### An Object Store May Display Eventual Consistency + +Object stores are often *Eventually Consistent*. Objects are replicated across servers +for availability —changes to a replica takes time to propagate to the other replicas; +the store is `inconsistent` during this process. + +Places this can be visible include: + +- When listing "a directory"; newly created files may not yet be visible, deleted ones still present. +- After updating an object: opening and reading the object may still return the previous data. +- After deleting an obect: opening it may succeed, returning the data. +- While reading an object, if it is updated or deleted during the process. + +For many years, Amazon US East S3 lacked create consistency: attempting to open a newly created object +could return a 404 response, which Hadoop maps to a `FileNotFoundException`. This was fixed in August 2015 +—see [S3 Consistency Model](http://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel) +for the full details. + +### Read Operations May be Significantly Slower Than Normal Filesystem Operations. + +Object stores usually implement their APIs as HTTP operations; clients make HTTP(S) requests +and block for responses. Each of these calls can be expensive. For maximum performance + +1. Try to list filesystem paths in bulk. +1. Know that `FileSystem.getFileStatus()` is expensive: cache the results rather than repeat +the call. +1. Similarly, avoid wrapper methods such as `FileSystem.exists()`, `isDirectory()` or `isFile()`. +1. Try to forward `seek()` through a file, rather than backwards. +1. Avoid renaming files: This is slow and, if it fails, may fail leave the destination in a mess. +1. Use the local filesystem as the destination of output which you intend to reload in follow-on work. +Retain the object store as the final destination of persistent output, not as a replacement for +HDFS. + + +## Testing Spark's Cloud Integration + +The `spark-cloud` module contains tests which can run against the object stores. These verify +functionality integration and performance. + +### Example Configuration for Testing Cloud Data + + +The test runs need a configuration file to declare the (secret) bindings to the cloud infrastructure. +The configuration used is the Hadoop XML format, because it allows XInclude importing of +secrets kept out of any source tree. + +The secret properties are defined using the Hadoop configuration option names, such as +`fs.s3a.access.key` and `fs.s3a.secret.key` + +The file must be declared to the maven test run in the property `cloud.test.configuration.file`, +which can be done in the command line + +``` +mvn test --pl cloud -Dcloud.test.configuration.file=../cloud.xml +``` + +*Important*: keep all credentials out of SCM-managed repositories. Even if `.gitignore` +or equivalent is used to exclude the file, they may unintenally get bundled and released +with an application. It is safest to keep the `cloud.xml` files out of the tree, +and keep the authentication secrets themselves in a single location for all applications +tested. + +Here is an example XML file `/home/developer/aws/cloud.xml` for running the S3A and Azure tests, +referencing the secret credentials kept in the file `/home/hadoop/aws/auth-keys.xml`. + +```xml + + + + + s3a.tests.enabled + true + Flag to enable S3A tests + + + + s3a.test.uri + s3a://testplan1 + S3A path to a bucket which the test runs are free to write, read and delete + data. + + + + azure.tests.enabled + true + + + + azure.test.uri + wasb://MYCONTAINER@TESTACCOUNT.blob.core.windows.net + + + +``` + +The configuration uses XInclude to pull in the secret credentials for the account +from the user's `/home/developer/.ssh/auth-keys.xml` file: + +```xml + + + fs.s3a.access.key + USERKEY + + + fs.s3a.secret.key + SECRET_AWS_KEY + + + fs.azure.account.key.TESTACCOUNT.blob.core.windows.net + SECRET_AZURE_KEY + + +``` + +Splitting the secret values out of the other XML files allows for the other files to +be managed via SCM and/or shared, with reduced risk. + +Note that the configuration file is used to define the entire Hadoop configuration used +within the Spark Context created; all options for the specific test filesystems may be +defined, such as endpoints and timeouts. + +### S3A Options + + + + + + + + + + + + + + + + + + + + + + + +
OptionMeaningDefault
s3a.tests.enabled + Execute tests using the S3A filesystem. + false
s3a.test.uri + URI for S3A tests. Required if S3A tests are enabled. +
s3a.test.csvfile.path + Path to a (possibly encrypted) CSV file used in linecount tests. + s3a://landsat-pds/scene_list.gz
s3a.test.csvfile.endpoint + Endpoint URI for CSV test file. This allows a different S3 instance + to be set for tests reading or writing data than against public CSV + source files. + Example: s3.amazonaws.com + s3.amazonaws.com
+ +When testing against Amazon S3, their [public datasets](https://aws.amazon.com/public-data-sets/) +are used. + +The gzipped CSV file `s3a://landsat-pds/scene_list.gz` is used for testing line input and file IO; +the default is a 20+ MB file hosted by Amazon. This file is public and free for anyone to +access, making it convenient and cost effective. + +The size and number of lines in this file increases over time; +the current size of the file can be measured through `curl`: + +```bash +curl -I -X HEAD http://landsat-pds.s3.amazonaws.com/scene_list.gz +``` + +When testing against non-AWS infrastructure, an alternate file may be specified +in the option `s3a.test.csvfile.path`; with its endpoint set to that of the +S3 endpoint + + +```xml + + s3a.test.csvfile.path + s3a://testdata/landsat.gz + + + + fs.s3a.endpoint + s3server.example.org + + + + s3a.test.csvfile.endpoint + ${fs.s3a.endpoint} + + +``` + +When testing against an S3 instance which only supports the AWS V4 Authentication +API, such as Frankfurt and Seoul, the `fs.s3a.endpoint` property must be set to that of +the specific location. Because the public landsat dataset is hosted in AWS US-East, it must retain +the original S3 endpoint. This is done by default, though it can also be set explicitly: + + +```xml + + fs.s3a.endpoint + s3.eu-central-1.amazonaws.com + + + + s3a.test.csvfile.endpoint + s3.amazonaws.com + +``` + +Finally, the CSV file tests can be skipped entirely by declaring the URL to be "" + + +```xml + + s3a.test.csvfile.path + + +``` +## Azure Test Options + + + + + + + + + + + + + + +
OptionMeaningDefault
azure.tests.enabled + Execute tests using the Azure WASB filesystem + false
azure.test.uri + URI for Azure WASB tests. Required if Azure tests are enabled. +
+ + +## Running a Single Test Case + +Each cloud test takes time, especially if the tests are being run outside of the +infrastructure of the specific cloud infrastructure provider. +Accordingly, it is important to be able to work on a single test case at a time +when implementing or debugging a test. + +Tests in a cloud suite must be conditional on the specific filesystem being available; every +test suite must implement a method `enabled: Boolean` to determine this. The tests are then +registered as "conditional tests" via the `ctest()` functino, which, takes a key, +a detailed description (this is included in logs), and the actual function to execute. + +For example, here is the test `NewHadoopAPI`. + +```scala + + ctest("NewHadoopAPI", + "Use SparkContext.saveAsNewAPIHadoopFile() to save data to a file") { + sc = new SparkContext("local", "test", newSparkConf()) + val numbers = sc.parallelize(1 to testEntryCount) + val example1 = new Path(TestDir, "example1") + saveAsTextFile(numbers, example1, sc.hadoopConfiguration) + } +``` + +This test can be executed as part of the suite `S3aIOSuite`, by setting the `suites` maven property to the classname +of the test suite: + +``` +mvn test --pl cloud -Phadoop-2.7,cloud -Dcloud.test.configuration.file=/home/developer/aws/cloud.xml -Dsuites=org.apache.spark.cloud.s3.S3aIOSuite +``` + +If the test configuration in `/home/developer/aws/cloud.xml` does not have the property +`s3a.tests.enabled` set to `true`, the S3a suites are not enabled. +The named test suite will be skipped and a message logged to highlight this. + +A single test can be explicitly run by including the key in the `suites` property +after the suite name + +``` +mvn test --pl cloud -Phadoop-2.7,cloud -Dcloud.test.configuration.file=/home/developer/aws/cloud.xml `-Dsuites=org.apache.spark.cloud.s3.S3aIOSuite NewHadoopAPI` +``` + +This will run all tests in the `S3aIOSuite` suite whose name contains the string `NewHadoopAPI`; +here just one test. Again, the test will be skipped if the `cloud.xml` configuration file does +not enable s3a tests. + +To run all tests of a specific infrastructure, use the `wildcardSuites` property to list the package +under which all test suites should be executed. + +``` +mvn test --pl cloud -Phadoop-2.7 -Dcloud.test.configuration.file=/home/developer/aws/cloud.xml `-DwildcardSuites=org.apache.spark.cloud.s3` +``` + +Note that an absolute path is used to refer to the test configuration file in these examples. +If a relative path is supplied, it must be relative to the project base, *not the cloud module*. + +# Integration tests + +The module includes a set of tests which work as integration tests, as well as unit tests. These +can be executed against live spark clusters, and can be configured to scale up, so testing +scalability. + +| job | arguments | test | +|------|----------|------| +| `org.apache.spark.cloud.examples.CloudFileGenerator` | ` ` | Parallel generation of files | +| `org.apache.spark.cloud.examples.CloudStreaming` | ` []` | Verifies that file streaming works with object store | +| `org.apache.spark.cloud.examples.CloudDataFrames` | ` []` | Dataframe IO across multiple formats +| `org.apache.spark.cloud.s3.examples.S3LineCount` | `[] []` | S3A specific: count lines on a file, optionally write back. + +## Best Practices for Adding a New Test + +1. Use `ctest()` to define a test case conditional on the suite being enabled. +1. Keep the test time down through small values such as: numbers of files, dataset sizes, operations. +Avoid slow operations such as: globbing & listing files +1. Support a command line entry point for integration tests —and allow such tests to scale up +though command line arguments. +1. Give the test a unique name which can be used to explicitly execute it from the build via the `suite` property. +1. Give the test a meaningful description for logs and test reports. +1. Test against multiple infrastructure instances. +1. Allow for eventual consistency of deletion and list operations by using `eventually()` to +wait for object store changes to become visible. +1. Have a long enough timeout that remote tests over slower connections will not timeout. + +## Best Practices for Adding a New Test Suite + +1. Extend `CloudSuite` +1. Have an `after {}` clause which cleans up all object stores —this keeps costs down. +1. Do not assume that any test has exclusive access to any part of an object store other +than the specific test directory. This is critical to support parallel test execution. +1. Share setup costs across test cases, especially for slow directory/file setup operations. +1. If extra conditions are needed before a test suite can be executed, override the `enabled` method +to probe for the extra conditions being met. + +## Keeping Test Costs Down + +Object stores incur charges for storage and for GET operations out of the datacenter where +the data is stored. + +The tests try to keep costs down by not working with large amounts of data, and by deleting +all data on teardown. If a test run is aborted, data may be retained on the test filesystem. +While the charges should only be a small amount, period purges of the bucket will keep costs down. + +Rerunning the tests to completion again should achieve this. + +The large dataset tests read in public data, so storage and bandwidth costs +are incurred by Amazon and other cloud storage providers themselves. + +### Keeping Credentials Safe in Testing + +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. Keep the XML Configuration file with any secrets in a secure part of your filesystem. +1. When using Hadoop 2.8+, consider using Hadoop credential files to store secrets, referencing +these files in the relevant id/secret properties of the XML configuration file. +1. Do not execute object store tests as part of automated CI/Jenkins builds, unless the secrets +are not senstitive -for example, they refer to in-house (test) object stores, authentication is +done via IAM EC2 VM credentials, or the credentials are short-lived AWS STS-issued credentials +with a lifespan of minutes and access only to transient test buckets. diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index e2bf2d7ca77ca..52e59df9990e9 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -323,7 +323,7 @@ One important parameter for parallel collections is the number of *partitions* t Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3n://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight scala %} scala> val distFile = sc.textFile("data.txt") @@ -356,7 +356,7 @@ Apart from text files, Spark's Scala API also supports several other data format Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3n://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight java %} JavaRDD distFile = sc.textFile("data.txt"); @@ -388,7 +388,7 @@ Apart from text files, Spark's Java API also supports several other data formats PySpark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3n://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight python %} >>> distFile = sc.textFile("data.txt") diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index c39ef1ce59e1c..33e98f963dc6c 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -19,41 +19,32 @@ Although not mandatory, it is recommended to configure the proxy server of Swift # Dependencies -The Spark application should include hadoop-openstack dependency. +The Spark application should include hadoop-openstack 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 pom.xml file: {% highlight xml %} ... - org.apache.hadoop - hadoop-openstack - 2.3.0 + org.apache.spark + spark-cloud_2.11 + ${spark.version} ... {% endhighlight %} +If using the Scala 2.10-compatible version of Spark, the artifact is of course `spark-cloud_2.10`. # Configuration Parameters Create core-site.xml and place it inside Spark's conf directory. -There are two main categories of parameters that should to be configured: declaration of the -Swift driver and the parameters that are required by Keystone. +Ther main category of parameters that should to be configured are the authentication parameters +required by Keystone. -Configuration of Hadoop to use Swift File system achieved via - - - - - - - -
Property NameValue
fs.swift.implorg.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem
- -Additional parameters required by Keystone (v2.0) and should be provided to the Swift driver. Those -parameters will be used to perform authentication in Keystone to access Swift. The following table -contains a list of Keystone mandatory parameters. PROVIDER can be any name. +The following table contains a list of Keystone mandatory parameters. PROVIDER can be +any (alphanumeric) name. @@ -94,7 +85,7 @@ contains a list of Keystone mandatory parameters. PROVIDER can be a - +
Property NameMeaningRequired
fs.swift.service.PROVIDER.publicIndicates if all URLs are publicIndicates whether to use the public (off cloud) or private (in cloud; no transfer fees) endpoints Mandatory
@@ -104,10 +95,6 @@ defined for tenant test. Then core-site.xml should inc {% highlight xml %} - - fs.swift.impl - org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem - fs.swift.service.SparkTest.auth.url http://127.0.0.1:5000/v2.0/tokens diff --git a/pom.xml b/pom.xml index a65692e0d1318..87399f59f875c 100644 --- a/pom.xml +++ b/pom.xml @@ -620,6 +620,11 @@ jackson-module-jaxb-annotations ${fasterxml.jackson.version} + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + ${fasterxml.jackson.version} + org.glassfish.jersey.core jersey-server @@ -2550,6 +2555,26 @@ + + + cloud + + + + cloud + + + scala-2.10 From 028d9ed428638520239da7d2b619d20817df56fd Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 18 Nov 2016 17:02:53 +0000 Subject: [PATCH 02/28] [SPARK-7481] basic instantiation tests verify that dependency hadoop-azure, hadoop-aws, hadoop-openstack and implicitly their transitive dependencies are resolved. They don't verify all dependency setup, specifically that Jackson versions are consistent; that needs integration testing. --- cloud/src/test/resources/log4j.properties | 28 +++++++++ .../spark/cloud/AzureInstantiationSuite.scala | 29 +++++++++ .../spark/cloud/S3InstantiationSuite.scala | 35 +++++++++++ .../spark/cloud/SwiftInstantiationSuite.scala | 62 +++++++++++++++++++ 4 files changed, 154 insertions(+) create mode 100644 cloud/src/test/resources/log4j.properties create mode 100644 cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala create mode 100644 cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala create mode 100644 cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala diff --git a/cloud/src/test/resources/log4j.properties b/cloud/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..cb3e41ae05421 --- /dev/null +++ b/cloud/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# 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. +# log4j configuration used during build and unit tests + +log4j.rootLogger=info,stdout +log4j.threshold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n + + +log4.logger.org.apache.spark.cloud.s3=DEBUG +log4.logger.org.apache.hadoop.fs.s3a=DEBUG +log4.logger.org.mortbay.jetty=ERROR +# disable deprecation noise +log4.logger.org.apache.hadoop.conf.Configuration.deprecation=ERROR + +# turn off other logs which +log4.logger.org.eclipse.jetty=ERROR diff --git a/cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala new file mode 100644 index 0000000000000..019596bc98a7a --- /dev/null +++ b/cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.spark.cloud + +import org.apache.hadoop.fs.azure.NativeAzureFileSystem + +import org.apache.spark.SparkFunSuite + +class AzureInstantiationSuite extends SparkFunSuite { + + test("Create WASB FS Instance") { + new NativeAzureFileSystem() + } +} diff --git a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala new file mode 100644 index 0000000000000..3759d77a77b99 --- /dev/null +++ b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.spark.cloud + +import org.apache.hadoop.fs.s3a.S3AFileSystem +import org.apache.hadoop.fs.s3native.NativeS3FileSystem + +import org.apache.spark.SparkFunSuite + +class S3InstantiationSuite extends SparkFunSuite { + + test("Create S3A FS Instance") { + new S3AFileSystem() + } + + test("Create S3N FS Instance") { + new NativeS3FileSystem() + } + +} diff --git a/cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala new file mode 100644 index 0000000000000..7ccb9717763a1 --- /dev/null +++ b/cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.spark.cloud + +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.swift.http.SwiftProtocolConstants._ +import org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem + +import org.apache.spark.SparkFunSuite + +/** + * Force load in Hadoop Swift classes and some dependencies. + * Dependency problems should be picked up at compile time; runtime may + * identify problems with transitive libraries and FileSystem service registration + * (i.e. the mapping from `"swift://"` to the `SwiftNativeFileSystem` instance"). + **/ +private[cloud] class SwiftInstantiationSuite extends SparkFunSuite { + + test("Create Swift native FS class") { + new SwiftNativeFileSystem() + } + + test("Instantiate Swift FS") { + // create a spoof swift endpoint configuration + val conf = new Configuration() + + def opt(key: String, value: String): Unit = { + conf.set(s"fs.swift.service.example$key", value) + } + def opts(options: Seq[(String, String)]): Unit = { + options.foreach(e => opt(e._1, e._2)) + } + + opts(Seq( + (DOT_USERNAME, "user"), + (DOT_PASSWORD, "passwd"), + (DOT_AUTH_URL, "http://example.org/v2.0/") + )) + val fs = FileSystem.newInstance(new URI("swift://test.example"), conf) + logInfo(s"Loaded FS $fs") + assert(fs.isInstanceOf[SwiftNativeFileSystem], s"Not a swift FS: $fs") + } + +} From ace46e98e913ee68c0aca88d17eeb0f055da074b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 18 Nov 2016 19:04:53 +0000 Subject: [PATCH 03/28] [SPARK-7481] tests restricted to instantiation; logging modified appropriately --- cloud/src/test/resources/log4j.properties | 29 +++++++++++-------- .../spark/cloud/S3InstantiationSuite.scala | 14 +++++++++ 2 files changed, 31 insertions(+), 12 deletions(-) diff --git a/cloud/src/test/resources/log4j.properties b/cloud/src/test/resources/log4j.properties index cb3e41ae05421..05ff464479a27 100644 --- a/cloud/src/test/resources/log4j.properties +++ b/cloud/src/test/resources/log4j.properties @@ -11,18 +11,23 @@ # limitations under the License. # log4j configuration used during build and unit tests -log4j.rootLogger=info,stdout -log4j.threshold=ALL -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n +# Set everything to be logged to the file target/unit-tests.log +test.appender=file +log4j.rootCategory=INFO, ${test.appender} +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n -log4.logger.org.apache.spark.cloud.s3=DEBUG -log4.logger.org.apache.hadoop.fs.s3a=DEBUG -log4.logger.org.mortbay.jetty=ERROR -# disable deprecation noise -log4.logger.org.apache.hadoop.conf.Configuration.deprecation=ERROR +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%t: %m%n -# turn off other logs which -log4.logger.org.eclipse.jetty=ERROR +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN diff --git a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala index 3759d77a77b99..d42997705bdb2 100644 --- a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala +++ b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark.cloud +import com.amazonaws.services.s3.S3ClientOptions import org.apache.hadoop.fs.s3a.S3AFileSystem import org.apache.hadoop.fs.s3native.NativeS3FileSystem +import org.jets3t.service.S3ServiceException +import org.joda.time.LocalTime import org.apache.spark.SparkFunSuite @@ -32,4 +35,15 @@ class S3InstantiationSuite extends SparkFunSuite { new NativeS3FileSystem() } + test("Create Jets3t class") { + new S3ServiceException("jets3t") + } + + test("Create class in Amazon com.amazonaws.services.s3 JAR") { + new S3ClientOptions() + } + + test("Create Joda Time class") { + new LocalTime() + } } From 3f6dfdad893d083e4653c547fcd6406a91dd9544 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 21 Nov 2016 12:07:25 +0000 Subject: [PATCH 04/28] [SPARK-7481] declare httpcomponents:httpclient explicitly, as downstream tests which pulled in spark-cloud but not spark-hive were ending up with inconsistent versions. Add a test for the missing class being there too. --- cloud/pom.xml | 36 ++++++------------- .../spark/cloud/S3InstantiationSuite.scala | 5 +++ 2 files changed, 15 insertions(+), 26 deletions(-) diff --git a/cloud/pom.xml b/cloud/pom.xml index 0a9602afbb4b5..25e7eb792269e 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -65,32 +65,6 @@ test - - - org.apache.spark - spark-sql_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-hive_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-mllib_${scala.binary.version} - ${project.version} - provided - @@ -166,6 +140,16 @@ ${hadoop.deps.scope} --> + + + org.apache.httpcomponents + httpclient + + + + org.apache.httpcomponents + httpcore + diff --git a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala index d42997705bdb2..d8b53694f7882 100644 --- a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala +++ b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.cloud import com.amazonaws.services.s3.S3ClientOptions import org.apache.hadoop.fs.s3a.S3AFileSystem import org.apache.hadoop.fs.s3native.NativeS3FileSystem +import org.apache.http.message.TokenParser import org.jets3t.service.S3ServiceException import org.joda.time.LocalTime @@ -46,4 +47,8 @@ class S3InstantiationSuite extends SparkFunSuite { test("Create Joda Time class") { new LocalTime() } + + test("http core") { + new TokenParser() + } } From 5f8f996cea76a16391073b46023a981cba3b3cce Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 21 Nov 2016 17:56:05 +0000 Subject: [PATCH 05/28] [SPARK-7481] update docs by culling section on cloud integration tests; link to remaning docs from top level. --- docs/cloud-integration.md | 338 +------------------------------------- docs/index.md | 1 + 2 files changed, 4 insertions(+), 335 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index ec51ecb78cf14..1a85ae60595bf 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -245,7 +245,7 @@ waiting for the scheduler to find a node close to the data. This has to be set in the YARN cluster configuration, not in the Spark configuration. -### Parquet IO Settings +### Parquet I/O 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, @@ -257,7 +257,7 @@ enables the Parquet library to skip un-needed columns, so saving bandwidth. spark.sql.parquet.filterPushdown true spark.sql.hive.metastorePartitionPruning true -### ORC IO Settings +### ORC I/O 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. @@ -433,7 +433,7 @@ forwards in a file —at the expense of making full file reads slower. This opti on older S3A versions. When working with text formats (text, CSV), or any sequential read through an entire file, -this "random" IO policy should be disabled. This is actually the default, but can be done +this "random" I/O policy should be disabled. This is actually the default, but can be done explicitly: ``` @@ -619,335 +619,3 @@ Retain the object store as the final destination of persistent output, not as a HDFS. -## Testing Spark's Cloud Integration - -The `spark-cloud` module contains tests which can run against the object stores. These verify -functionality integration and performance. - -### Example Configuration for Testing Cloud Data - - -The test runs need a configuration file to declare the (secret) bindings to the cloud infrastructure. -The configuration used is the Hadoop XML format, because it allows XInclude importing of -secrets kept out of any source tree. - -The secret properties are defined using the Hadoop configuration option names, such as -`fs.s3a.access.key` and `fs.s3a.secret.key` - -The file must be declared to the maven test run in the property `cloud.test.configuration.file`, -which can be done in the command line - -``` -mvn test --pl cloud -Dcloud.test.configuration.file=../cloud.xml -``` - -*Important*: keep all credentials out of SCM-managed repositories. Even if `.gitignore` -or equivalent is used to exclude the file, they may unintenally get bundled and released -with an application. It is safest to keep the `cloud.xml` files out of the tree, -and keep the authentication secrets themselves in a single location for all applications -tested. - -Here is an example XML file `/home/developer/aws/cloud.xml` for running the S3A and Azure tests, -referencing the secret credentials kept in the file `/home/hadoop/aws/auth-keys.xml`. - -```xml - - - - - s3a.tests.enabled - true - Flag to enable S3A tests - - - - s3a.test.uri - s3a://testplan1 - S3A path to a bucket which the test runs are free to write, read and delete - data. - - - - azure.tests.enabled - true - - - - azure.test.uri - wasb://MYCONTAINER@TESTACCOUNT.blob.core.windows.net - - - -``` - -The configuration uses XInclude to pull in the secret credentials for the account -from the user's `/home/developer/.ssh/auth-keys.xml` file: - -```xml - - - fs.s3a.access.key - USERKEY - - - fs.s3a.secret.key - SECRET_AWS_KEY - - - fs.azure.account.key.TESTACCOUNT.blob.core.windows.net - SECRET_AZURE_KEY - - -``` - -Splitting the secret values out of the other XML files allows for the other files to -be managed via SCM and/or shared, with reduced risk. - -Note that the configuration file is used to define the entire Hadoop configuration used -within the Spark Context created; all options for the specific test filesystems may be -defined, such as endpoints and timeouts. - -### S3A Options - - - - - - - - - - - - - - - - - - - - - - - -
OptionMeaningDefault
s3a.tests.enabled - Execute tests using the S3A filesystem. - false
s3a.test.uri - URI for S3A tests. Required if S3A tests are enabled. -
s3a.test.csvfile.path - Path to a (possibly encrypted) CSV file used in linecount tests. - s3a://landsat-pds/scene_list.gz
s3a.test.csvfile.endpoint - Endpoint URI for CSV test file. This allows a different S3 instance - to be set for tests reading or writing data than against public CSV - source files. - Example: s3.amazonaws.com - s3.amazonaws.com
- -When testing against Amazon S3, their [public datasets](https://aws.amazon.com/public-data-sets/) -are used. - -The gzipped CSV file `s3a://landsat-pds/scene_list.gz` is used for testing line input and file IO; -the default is a 20+ MB file hosted by Amazon. This file is public and free for anyone to -access, making it convenient and cost effective. - -The size and number of lines in this file increases over time; -the current size of the file can be measured through `curl`: - -```bash -curl -I -X HEAD http://landsat-pds.s3.amazonaws.com/scene_list.gz -``` - -When testing against non-AWS infrastructure, an alternate file may be specified -in the option `s3a.test.csvfile.path`; with its endpoint set to that of the -S3 endpoint - - -```xml - - s3a.test.csvfile.path - s3a://testdata/landsat.gz - - - - fs.s3a.endpoint - s3server.example.org - - - - s3a.test.csvfile.endpoint - ${fs.s3a.endpoint} - - -``` - -When testing against an S3 instance which only supports the AWS V4 Authentication -API, such as Frankfurt and Seoul, the `fs.s3a.endpoint` property must be set to that of -the specific location. Because the public landsat dataset is hosted in AWS US-East, it must retain -the original S3 endpoint. This is done by default, though it can also be set explicitly: - - -```xml - - fs.s3a.endpoint - s3.eu-central-1.amazonaws.com - - - - s3a.test.csvfile.endpoint - s3.amazonaws.com - -``` - -Finally, the CSV file tests can be skipped entirely by declaring the URL to be "" - - -```xml - - s3a.test.csvfile.path - - -``` -## Azure Test Options - - - - - - - - - - - - - - -
OptionMeaningDefault
azure.tests.enabled - Execute tests using the Azure WASB filesystem - false
azure.test.uri - URI for Azure WASB tests. Required if Azure tests are enabled. -
- - -## Running a Single Test Case - -Each cloud test takes time, especially if the tests are being run outside of the -infrastructure of the specific cloud infrastructure provider. -Accordingly, it is important to be able to work on a single test case at a time -when implementing or debugging a test. - -Tests in a cloud suite must be conditional on the specific filesystem being available; every -test suite must implement a method `enabled: Boolean` to determine this. The tests are then -registered as "conditional tests" via the `ctest()` functino, which, takes a key, -a detailed description (this is included in logs), and the actual function to execute. - -For example, here is the test `NewHadoopAPI`. - -```scala - - ctest("NewHadoopAPI", - "Use SparkContext.saveAsNewAPIHadoopFile() to save data to a file") { - sc = new SparkContext("local", "test", newSparkConf()) - val numbers = sc.parallelize(1 to testEntryCount) - val example1 = new Path(TestDir, "example1") - saveAsTextFile(numbers, example1, sc.hadoopConfiguration) - } -``` - -This test can be executed as part of the suite `S3aIOSuite`, by setting the `suites` maven property to the classname -of the test suite: - -``` -mvn test --pl cloud -Phadoop-2.7,cloud -Dcloud.test.configuration.file=/home/developer/aws/cloud.xml -Dsuites=org.apache.spark.cloud.s3.S3aIOSuite -``` - -If the test configuration in `/home/developer/aws/cloud.xml` does not have the property -`s3a.tests.enabled` set to `true`, the S3a suites are not enabled. -The named test suite will be skipped and a message logged to highlight this. - -A single test can be explicitly run by including the key in the `suites` property -after the suite name - -``` -mvn test --pl cloud -Phadoop-2.7,cloud -Dcloud.test.configuration.file=/home/developer/aws/cloud.xml `-Dsuites=org.apache.spark.cloud.s3.S3aIOSuite NewHadoopAPI` -``` - -This will run all tests in the `S3aIOSuite` suite whose name contains the string `NewHadoopAPI`; -here just one test. Again, the test will be skipped if the `cloud.xml` configuration file does -not enable s3a tests. - -To run all tests of a specific infrastructure, use the `wildcardSuites` property to list the package -under which all test suites should be executed. - -``` -mvn test --pl cloud -Phadoop-2.7 -Dcloud.test.configuration.file=/home/developer/aws/cloud.xml `-DwildcardSuites=org.apache.spark.cloud.s3` -``` - -Note that an absolute path is used to refer to the test configuration file in these examples. -If a relative path is supplied, it must be relative to the project base, *not the cloud module*. - -# Integration tests - -The module includes a set of tests which work as integration tests, as well as unit tests. These -can be executed against live spark clusters, and can be configured to scale up, so testing -scalability. - -| job | arguments | test | -|------|----------|------| -| `org.apache.spark.cloud.examples.CloudFileGenerator` | ` ` | Parallel generation of files | -| `org.apache.spark.cloud.examples.CloudStreaming` | ` []` | Verifies that file streaming works with object store | -| `org.apache.spark.cloud.examples.CloudDataFrames` | ` []` | Dataframe IO across multiple formats -| `org.apache.spark.cloud.s3.examples.S3LineCount` | `[] []` | S3A specific: count lines on a file, optionally write back. - -## Best Practices for Adding a New Test - -1. Use `ctest()` to define a test case conditional on the suite being enabled. -1. Keep the test time down through small values such as: numbers of files, dataset sizes, operations. -Avoid slow operations such as: globbing & listing files -1. Support a command line entry point for integration tests —and allow such tests to scale up -though command line arguments. -1. Give the test a unique name which can be used to explicitly execute it from the build via the `suite` property. -1. Give the test a meaningful description for logs and test reports. -1. Test against multiple infrastructure instances. -1. Allow for eventual consistency of deletion and list operations by using `eventually()` to -wait for object store changes to become visible. -1. Have a long enough timeout that remote tests over slower connections will not timeout. - -## Best Practices for Adding a New Test Suite - -1. Extend `CloudSuite` -1. Have an `after {}` clause which cleans up all object stores —this keeps costs down. -1. Do not assume that any test has exclusive access to any part of an object store other -than the specific test directory. This is critical to support parallel test execution. -1. Share setup costs across test cases, especially for slow directory/file setup operations. -1. If extra conditions are needed before a test suite can be executed, override the `enabled` method -to probe for the extra conditions being met. - -## Keeping Test Costs Down - -Object stores incur charges for storage and for GET operations out of the datacenter where -the data is stored. - -The tests try to keep costs down by not working with large amounts of data, and by deleting -all data on teardown. If a test run is aborted, data may be retained on the test filesystem. -While the charges should only be a small amount, period purges of the bucket will keep costs down. - -Rerunning the tests to completion again should achieve this. - -The large dataset tests read in public data, so storage and bandwidth costs -are incurred by Amazon and other cloud storage providers themselves. - -### Keeping Credentials Safe in Testing - -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. Keep the XML Configuration file with any secrets in a secure part of your filesystem. -1. When using Hadoop 2.8+, consider using Hadoop credential files to store secrets, referencing -these files in the relevant id/secret properties of the XML configuration file. -1. Do not execute object store tests as part of automated CI/Jenkins builds, unless the secrets -are not senstitive -for example, they refer to in-house (test) object stores, authentication is -done via IAM EC2 VM credentials, or the credentials are short-lived AWS STS-issued credentials -with a lifespan of minutes and access only to transient test buckets. diff --git a/docs/index.md b/docs/index.md index ad4f24ff1a5d1..960b968454d0e 100644 --- a/docs/index.md +++ b/docs/index.md @@ -126,6 +126,7 @@ options for deployment: * [Security](security.html): Spark security support * [Hardware Provisioning](hardware-provisioning.html): recommendations for cluster hardware * Integration with other storage systems: + * [Cloud Infrastructures](cloud-integration.html) * [OpenStack Swift](storage-openstack-swift.html) * [Building Spark](building-spark.html): build Spark using the Maven system * [Contributing to Spark](http://spark.apache.org/contributing.html) From e92a49322dfdb777e996e9b07b298bb8ae8967d6 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 28 Nov 2016 15:44:10 +0000 Subject: [PATCH 06/28] [SPARK-7481] updated documentation as per review --- docs/cloud-integration.md | 209 ++++++++------------------------ docs/storage-openstack-swift.md | 5 +- 2 files changed, 53 insertions(+), 161 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 1a85ae60595bf..6384fa7651226 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -72,33 +72,11 @@ If using the Scala 2.10-compatible version of Spark, the artifact is of course ` ### Basic Use +You can refer to data in an object store just as you would data in a filesystem, by +using a URL to the data in methods like `SparkContext.textFile()` to read data, +`saveAsTextFile()` to write it back. -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://testbucket@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. @@ -109,79 +87,6 @@ The steps to do so are as follows 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. - -### 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://yourcontainer@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 %} - -### 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. - -#### 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. ## Object Stores as a substitute for HDFS @@ -195,19 +100,37 @@ The brief summary is: | Object Store Connector | Replace HDFS? | |-----------------------------|--------------------| -| Apache `s3a://` `s3n://` | No | +| `s3a://` `s3n://` from the ASF | 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 +`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`. +#### Spark Streaming and object stores + +Spark Streaming can monitor files added to object stores, by +creating a `FileInputDStream` DStream monitoring a path under a bucket through +`StreamingContext.textFileStream()`. + + +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. + +1. Streams should only be checkpointed to an object store considered compatible with +HDFS. Otherwise the checkpointing will be slow and potentially unreliable. + ### Recommended settings for writing to object stores Here are the settings to use when writing to object stores. This uses the "version 2" algorithm @@ -249,13 +172,15 @@ This has to be set in the YARN cluster configuration, not in the Spark configura 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. +and coalescing metadata from multiple files. The `filterPushdown` option +enables the Parquet library to optimize data reads itself, potentially saving bandwidth. - spark.hadoop.parquet.enable.summary-metadata false - spark.sql.parquet.mergeSchema false - spark.sql.parquet.filterPushdown true - spark.sql.hive.metastorePartitionPruning true +``` +spark.hadoop.parquet.enable.summary-metadata false +spark.sql.parquet.mergeSchema false +spark.sql.parquet.filterPushdown true +spark.sql.hive.metastorePartitionPruning true +``` ### ORC I/O Settings @@ -263,18 +188,19 @@ 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 +``` +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 `filterPushdown` option enables the ORC library to optimize data reads itself, +potentially saving bandwidth. 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. - ## Authenticating with Object Stores Apart from the special case of public read-only data, all object stores @@ -400,25 +326,9 @@ to buckets where the root paths are read only, or not readable at all. #### 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** - -1. The implementation is in `hadoop-aws`, which is included in `$SPARK_HOME/jars` when Spark -is built with cloud support. +is the sole S3 connector undergoing active maintenance at the Apache, and should be used wherever +possible. -1. Dependencies: `amazon-aws-sdk` JAR (Hadoop 2.7); -`amazon-s3-sdk` and `amazon-core-sdk` in Hadoop 2.8+. - -1. The Amazon JARs have proven very brittle —the version of the Amazon -libraries *must* match that which the Hadoop binaries were built against. - -1. S3A has authentication problems on Java 8u60+ if there is an old version -of Joda Time on the classpath. -If authentication is failing, see if`joda-time.jar` needs upgrading to 2.8.1 or later. **Tuning for performance:** @@ -429,11 +339,11 @@ spark.hadoop.fs.s3a.experimental.input.fadvise random ``` This reads from the object in blocks, which is efficient when seeking backwards as well as -forwards in a file —at the expense of making full file reads slower. This option is ignored -on older S3A versions. +forwards in a file —at the expense of making full file reads slower. -When working with text formats (text, CSV), or any sequential read through an entire file, -this "random" I/O policy should be disabled. This is actually the default, but can be done +When working with text formats (text, CSV), or any sequential read through an entire file +(including .gzip compressed data), +this "random" I/O policy should be disabled. This is the default, but can be done explicitly: ``` @@ -455,12 +365,6 @@ While stable, S3N is essentially unmaintained, and deprecated in favor of S3A. As well as being slower and limited in authentication mechanisms, the only maintenance it receives are critical security issues. -**Classpath** - -Hadoop 2.5 and earlier: add `jets3t.jar` to the classpath - -Hadoop 2.6+: bBoth `hadoop-aws.jar` and `jets3t.jar` (version 0.9.0 or later) -must be on the classpath. #### Amazon EMR's S3 Client: `s3://` @@ -494,11 +398,6 @@ The Apache implementation is that used by Microsoft in Azure itself: it can be u to access data in Azure as well as remotely. The object store itself is *consistent*, and can be reliably used as the destination of queries. -**Classpath** - -1. The `wasb` filesystem client is implemented in the`hadoop-azure` JAR available in Hadoop 2.7. -1. It also needs a matching `azure-storage` JAR. - ### Working with OpenStack Swift @@ -507,20 +406,13 @@ The OpenStack [`swift://` filesystem client](https://hadoop.apache.org/docs/stab works with Swift object stores in private OpenStack installations, public installations including Rackspace Cloud and IBM Softlayer. -**Classpath** - -1. `swift://` support comes from `hadoop-openstack`. -1. All other dependencies, including `httpclient`, `jackson`, and `commons-logging` are always -included in Spark distributions. - ### Working with Google Cloud Storage [Google Cloud Storage](https://cloud.google.com/storage) is supported via Google's own [GCS filesystem client](https://cloud.google.com/hadoop/google-cloud-storage-connector). -**Classpath** -1. For use outside of Google cloud, `gcs-connector.jar` must be be manually downloaded then added +For use outside of Google cloud, `gcs-connector.jar` must be be manually downloaded then added to `$SPARK_HOME/jars`. @@ -598,10 +490,8 @@ Places this can be visible include: - After deleting an obect: opening it may succeed, returning the data. - While reading an object, if it is updated or deleted during the process. -For many years, Amazon US East S3 lacked create consistency: attempting to open a newly created object -could return a 404 response, which Hadoop maps to a `FileNotFoundException`. This was fixed in August 2015 -—see [S3 Consistency Model](http://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel) -for the full details. +Microsoft Azure is consistent; Amazon S3 is "Create consistent" —but directory listing +operations may visibly lag behind changes to the underlying object. ### Read Operations May be Significantly Slower Than Normal Filesystem Operations. @@ -613,7 +503,8 @@ and block for responses. Each of these calls can be expensive. For maximum perfo the call. 1. Similarly, avoid wrapper methods such as `FileSystem.exists()`, `isDirectory()` or `isFile()`. 1. Try to forward `seek()` through a file, rather than backwards. -1. Avoid renaming files: This is slow and, if it fails, may fail leave the destination in a mess. +1. Avoid renaming files: This is slow and, if it fails, may fail leave the destination in +"an undefined state". 1. Use the local filesystem as the destination of output which you intend to reload in follow-on work. Retain the object store as the final destination of persistent output, not as a replacement for HDFS. diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index 33e98f963dc6c..5aea08af152d1 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -8,7 +8,8 @@ same URI formats as in Hadoop. You can specify a path in Swift as input through URI of the form swift://container.PROVIDER/path. You will also need to set your Swift security credentials, through core-site.xml or via SparkContext.hadoopConfiguration. -Current Swift driver requires Swift to use Keystone authentication method. +The current Swift driver requires Swift to use the Keystone authentication method, or +its Rackspace-specific predecessor. # Configuring Swift for Better Data Locality @@ -40,7 +41,7 @@ If using the Scala 2.10-compatible version of Spark, the artifact is of course ` # Configuration Parameters Create core-site.xml and place it inside Spark's conf directory. -Ther main category of parameters that should to be configured are the authentication parameters +The main category of parameters that should to be configured are the authentication parameters required by Keystone. The following table contains a list of Keystone mandatory parameters. PROVIDER can be From 97e80e1963b8f64905165c08974197cf4cd68356 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 28 Nov 2016 15:44:30 +0000 Subject: [PATCH 07/28] [SPARK-7481] SBT will build this now, optionally --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e52baf51aed1a..49b49c85d6018 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -57,7 +57,7 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(mesos, yarn, sparkGangliaLgpl, - streamingKinesisAsl, dockerIntegrationTests) = + streamingKinesisAsl, dockerIntegrationTests, cloud) = Seq("mesos", "yarn", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests").map(ProjectRef(buildLocation, _)) From ef3cebfd1baf928c3f30380f662eaee13ee6ca08 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 28 Nov 2016 15:45:44 +0000 Subject: [PATCH 08/28] [SPARK-7481] cloud POM includes jackson-dataformat-cbor, so that the CP is set up consistently for the later versions of the AWS SDK --- cloud/pom.xml | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/cloud/pom.xml b/cloud/pom.xml index 25e7eb792269e..0eff1b5ecbe49 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -36,9 +36,9 @@ Hadoop 2.7: hadoop-aws - aws-java-sdk-s3 + aws-java-sdk-s3 hadoop-azure - azure-storage + azure-storage hadoop-openstack WARNING: the signatures of methods in aws-java-sdk/aws-java-sdk-s3 can change between versions: @@ -131,15 +131,11 @@ jackson-annotations ${hadoop.deps.scope} - - org.apache.httpcomponents From 66650c7c7d4d9e2cb640175428bf16a343d6319b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 1 Dec 2016 13:30:48 +0000 Subject: [PATCH 09/28] [SPARK-7481] rebase with master; Pom had got out of sync --- pom.xml | 94 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 94 insertions(+) diff --git a/pom.xml b/pom.xml index 87399f59f875c..6df07873a77a1 100644 --- a/pom.xml +++ b/pom.xml @@ -2531,6 +2531,100 @@ 2.7.3 + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + + + + org.apache.hadoop + hadoop-openstack + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + junit + junit + + + org.mockito + mockito-all + + + + + + + org.apache.hadoop + hadoop-azure + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.jackson.core + jackson-core + + + + + From 31cc37e90f2dcb0ebbe696bc08d951e0526293f9 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 2 Dec 2016 17:39:52 +0000 Subject: [PATCH 10/28] [SPARK-7481] rename spark-cloud module to spark-hadoo-cloud, in POMs and docs --- assembly/pom.xml | 4 ++-- cloud/pom.xml | 4 ++-- docs/cloud-integration.md | 8 ++++---- docs/storage-openstack-swift.md | 6 +++--- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index ba34967037a0f..9728af2c53c1b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -228,14 +228,14 @@ cloud org.apache.spark - spark-cloud_${scala.binary.version} + spark-hadoop-cloud_${scala.binary.version} ${project.version} diff --git a/cloud/pom.xml b/cloud/pom.xml index 0eff1b5ecbe49..aa94fdc5f63f7 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -24,14 +24,14 @@ ../pom.xml - spark-cloud_2.11 + spark-hadoop-cloud_2.11 jar Spark Project Cloud Integration Contains support for cloud infrastructures, specifically the Hadoop JARs and transitive dependencies needed to interact with the infrastructures. - Any project which explicitly depends upon the spark-cloud artifact will get the dependencies; + Any project which explicitly depends upon the spark-hadoop-cloud artifact will get the dependencies; the exact versions of which will depend upon the hadoop version Spark was compiled against. Hadoop 2.7: diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 6384fa7651226..d5c3d75a50f94 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -50,10 +50,10 @@ 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. +be done by including the `spark-hadoop-cloud` module for the specific version of spark used. The Spark application should include hadoop-openstack dependency, which can -be done by including the `spark-cloud` module for the specific version of spark used. +be done by including the `spark-hadoop-cloud` module for the specific version of spark used. For example, for Maven support, add the following to the pom.xml file: {% highlight xml %} @@ -61,14 +61,14 @@ For example, for Maven support, add the following to the pom.xml fi ... org.apache.spark - spark-cloud_2.11 + spark-hadoop-cloud_2.11 ${spark.version} ... {% endhighlight %} -If using the Scala 2.10-compatible version of Spark, the artifact is of course `spark-cloud_2.10`. +If using the Scala 2.10-compatible version of Spark, the artifact is of course `spark-hadoop-cloud_2.10`. ### Basic Use diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index 5aea08af152d1..1234c5f8e2bf9 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -21,7 +21,7 @@ Although not mandatory, it is recommended to configure the proxy server of Swift # Dependencies The Spark application should include hadoop-openstack dependency, which can -be done by including the `spark-cloud` module for the specific version of spark used. +be done by including the `spark-hadoop-cloud` module for the specific version of spark used. For example, for Maven support, add the following to the pom.xml file: {% highlight xml %} @@ -29,14 +29,14 @@ For example, for Maven support, add the following to the pom.xml fi ... org.apache.spark - spark-cloud_2.11 + spark-hadoop-cloud_2.11 ${spark.version} ... {% endhighlight %} -If using the Scala 2.10-compatible version of Spark, the artifact is of course `spark-cloud_2.10`. +If using the Scala 2.10-compatible version of Spark, the artifact is of course `spark-hadoop-cloud_2.10`. # Configuration Parameters From 2fc6f23b5397f344583c0e192f88fb40bb88f6ad Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 14 Dec 2016 15:47:10 +0000 Subject: [PATCH 11/28] [SPARK-7841] bump up cloud pom to 2.2.0-SNAPSHOT; other minor pom cleanup --- cloud/pom.xml | 18 +++++++----------- pom.xml | 2 -- 2 files changed, 7 insertions(+), 13 deletions(-) diff --git a/cloud/pom.xml b/cloud/pom.xml index aa94fdc5f63f7..110dae17b5414 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0-SNAPSHOT + 2.2.0-SNAPSHOT ../pom.xml @@ -31,22 +31,18 @@ Contains support for cloud infrastructures, specifically the Hadoop JARs and transitive dependencies needed to interact with the infrastructures. - Any project which explicitly depends upon the spark-hadoop-cloud artifact will get the dependencies; - the exact versions of which will depend upon the hadoop version Spark was compiled against. + Any project which explicitly depends upon the spark-hadoop-cloud artifact will get the + dependencies; the exact versions of which will depend upon the hadoop version Spark was compiled + against. - Hadoop 2.7: - hadoop-aws - aws-java-sdk-s3 - hadoop-azure - azure-storage - hadoop-openstack + The imports of transitive dependencies (especially Jackson) are managed to make them consistent + with those of the Spark build. WARNING: the signatures of methods in aws-java-sdk/aws-java-sdk-s3 can change between versions: use the same version against which Hadoop was compiled. - - cloud + hadoop-cloud diff --git a/pom.xml b/pom.xml index 6df07873a77a1..81ad05957256e 100644 --- a/pom.xml +++ b/pom.xml @@ -2662,8 +2662,6 @@ --> cloud - - cloud From 65f6814ccba464dbba1c8a5390638291c7c3cf1a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 10 Jan 2017 14:07:18 +0000 Subject: [PATCH 12/28] [SPARK-7481] builds against Hadoop shaded 3.x clients failing as direct references to AWS classes failing. Cut them and rely on transitive load through FS class instantation to force the load. All that happens is that failures to link will be slightly less easy to debug. --- .../spark/cloud/S3InstantiationSuite.scala | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala index d8b53694f7882..2d495a6d38398 100644 --- a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala +++ b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala @@ -32,23 +32,4 @@ class S3InstantiationSuite extends SparkFunSuite { new S3AFileSystem() } - test("Create S3N FS Instance") { - new NativeS3FileSystem() - } - - test("Create Jets3t class") { - new S3ServiceException("jets3t") - } - - test("Create class in Amazon com.amazonaws.services.s3 JAR") { - new S3ClientOptions() - } - - test("Create Joda Time class") { - new LocalTime() - } - - test("http core") { - new TokenParser() - } } From 73820a341cbbdecdd386a1448300439577273671 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 20 Jan 2017 13:52:45 +0000 Subject: [PATCH 13/28] [SPARK-7481] update 2.7 dependencies to include azure, aws and openstack JARs, transitive dependencies on aws and azure SDKs --- dev/deps/spark-deps-hadoop-2.7 | 5 ----- 1 file changed, 5 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 16edf80d40712..6bf0923a1d751 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -16,8 +16,6 @@ arpack_combined_all-0.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar -aws-java-sdk-1.7.4.jar -azure-storage-2.0.0.jar base64-2.3.8.jar bcprov-jdk15on-1.51.jar bonecp-0.8.0.RELEASE.jar @@ -63,8 +61,6 @@ guice-3.0.jar guice-servlet-3.0.jar hadoop-annotations-2.7.3.jar hadoop-auth-2.7.3.jar -hadoop-aws-2.7.3.jar -hadoop-azure-2.7.3.jar hadoop-client-2.7.3.jar hadoop-common-2.7.3.jar hadoop-hdfs-2.7.3.jar @@ -73,7 +69,6 @@ hadoop-mapreduce-client-common-2.7.3.jar hadoop-mapreduce-client-core-2.7.3.jar hadoop-mapreduce-client-jobclient-2.7.3.jar hadoop-mapreduce-client-shuffle-2.7.3.jar -hadoop-hadoop-openstack-2.7.3.jar hadoop-yarn-api-2.7.3.jar hadoop-yarn-client-2.7.3.jar hadoop-yarn-common-2.7.3.jar From 824d801d43000161533dd50c9e2c7d2f1a1f7a0b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 30 Jan 2017 14:27:39 +0000 Subject: [PATCH 14/28] [SPARK-7481] add joda time as the dependency. Tested against hadoop branch-2, s3 ireland --- cloud/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/cloud/pom.xml b/cloud/pom.xml index 110dae17b5414..73bf3db954d50 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -116,6 +116,14 @@ hadoop-openstack ${hadoop.deps.scope} + + + joda-time + joda-time + com.fasterxml.jackson.core From 12a1b8488968917e4d99a39c7dd3ac2d39f87727 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 24 Feb 2017 14:30:29 +0000 Subject: [PATCH 15/28] SPARK-7481 purge all tests from the cloud module --- cloud/pom.xml | 8 +-- cloud/src/test/resources/log4j.properties | 33 ---------- .../spark/cloud/AzureInstantiationSuite.scala | 29 --------- .../spark/cloud/S3InstantiationSuite.scala | 35 ----------- .../spark/cloud/SwiftInstantiationSuite.scala | 62 ------------------- 5 files changed, 4 insertions(+), 163 deletions(-) delete mode 100644 cloud/src/test/resources/log4j.properties delete mode 100644 cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala delete mode 100644 cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala delete mode 100644 cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala diff --git a/cloud/pom.xml b/cloud/pom.xml index 73bf3db954d50..db4b98a2b8da8 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -35,11 +35,12 @@ dependencies; the exact versions of which will depend upon the hadoop version Spark was compiled against. - The imports of transitive dependencies (especially Jackson) are managed to make them consistent + The imports of transitive dependencies are managed to make them consistent with those of the Spark build. - WARNING: the signatures of methods in aws-java-sdk/aws-java-sdk-s3 can change between versions: - use the same version against which Hadoop was compiled. + WARNING: the signatures of methods in the AWS and Azure SDKs do change between + versions: use exactly the same version with which the Hadoop JARs were + built. hadoop-cloud @@ -61,7 +62,6 @@ test - net.java.dev.jets3t diff --git a/cloud/src/test/resources/log4j.properties b/cloud/src/test/resources/log4j.properties deleted file mode 100644 index 05ff464479a27..0000000000000 --- a/cloud/src/test/resources/log4j.properties +++ /dev/null @@ -1,33 +0,0 @@ -# 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. -# log4j configuration used during build and unit tests - - -# Set everything to be logged to the file target/unit-tests.log -test.appender=file -log4j.rootCategory=INFO, ${test.appender} -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n - -# Tests that launch java subprocesses can set the "test.appender" system property to -# "console" to avoid having the child process's logs overwrite the unit test's -# log file. -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%t: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN diff --git a/cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala deleted file mode 100644 index 019596bc98a7a..0000000000000 --- a/cloud/src/test/scala/org/apache/spark/cloud/AzureInstantiationSuite.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -package org.apache.spark.cloud - -import org.apache.hadoop.fs.azure.NativeAzureFileSystem - -import org.apache.spark.SparkFunSuite - -class AzureInstantiationSuite extends SparkFunSuite { - - test("Create WASB FS Instance") { - new NativeAzureFileSystem() - } -} diff --git a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala deleted file mode 100644 index 2d495a6d38398..0000000000000 --- a/cloud/src/test/scala/org/apache/spark/cloud/S3InstantiationSuite.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -package org.apache.spark.cloud - -import com.amazonaws.services.s3.S3ClientOptions -import org.apache.hadoop.fs.s3a.S3AFileSystem -import org.apache.hadoop.fs.s3native.NativeS3FileSystem -import org.apache.http.message.TokenParser -import org.jets3t.service.S3ServiceException -import org.joda.time.LocalTime - -import org.apache.spark.SparkFunSuite - -class S3InstantiationSuite extends SparkFunSuite { - - test("Create S3A FS Instance") { - new S3AFileSystem() - } - -} diff --git a/cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala b/cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala deleted file mode 100644 index 7ccb9717763a1..0000000000000 --- a/cloud/src/test/scala/org/apache/spark/cloud/SwiftInstantiationSuite.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -package org.apache.spark.cloud - -import java.net.URI - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.swift.http.SwiftProtocolConstants._ -import org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem - -import org.apache.spark.SparkFunSuite - -/** - * Force load in Hadoop Swift classes and some dependencies. - * Dependency problems should be picked up at compile time; runtime may - * identify problems with transitive libraries and FileSystem service registration - * (i.e. the mapping from `"swift://"` to the `SwiftNativeFileSystem` instance"). - **/ -private[cloud] class SwiftInstantiationSuite extends SparkFunSuite { - - test("Create Swift native FS class") { - new SwiftNativeFileSystem() - } - - test("Instantiate Swift FS") { - // create a spoof swift endpoint configuration - val conf = new Configuration() - - def opt(key: String, value: String): Unit = { - conf.set(s"fs.swift.service.example$key", value) - } - def opts(options: Seq[(String, String)]): Unit = { - options.foreach(e => opt(e._1, e._2)) - } - - opts(Seq( - (DOT_USERNAME, "user"), - (DOT_PASSWORD, "passwd"), - (DOT_AUTH_URL, "http://example.org/v2.0/") - )) - val fs = FileSystem.newInstance(new URI("swift://test.example"), conf) - logInfo(s"Loaded FS $fs") - assert(fs.isInstanceOf[SwiftNativeFileSystem], s"Not a swift FS: $fs") - } - -} From a7a2deca3cf00488682e355b41c716ecce57a62f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 20 Mar 2017 14:10:12 +0000 Subject: [PATCH 16/28] SPARK-7481 add cloud module to sbt sequence Change-Id: I3dea2544f089615493163f0fae482992873f9c35 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 49b49c85d6018..5073f34bc9498 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -59,7 +59,7 @@ object BuildCommons { val optionallyEnabledProjects@Seq(mesos, yarn, sparkGangliaLgpl, streamingKinesisAsl, dockerIntegrationTests, cloud) = Seq("mesos", "yarn", "ganglia-lgpl", "streaming-kinesis-asl", - "docker-integration-tests").map(ProjectRef(buildLocation, _)) + "docker-integration-tests", "cloud").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") From 02f6e19bef8d7e1e0622d04bf47bb2c785996877 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 20 Mar 2017 14:14:37 +0000 Subject: [PATCH 17/28] SPARK-7481 break line of mvn XML declaration Change-Id: Ibd6d40df2bc8a2edf19a058c458bea233ba414fd --- cloud/pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cloud/pom.xml b/cloud/pom.xml index db4b98a2b8da8..b8194b6550cb1 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -15,7 +15,9 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + 4.0.0 org.apache.spark From ce042d2405706bc7cd6b0d2a410c36346be0c86e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 20 Mar 2017 19:19:49 +0000 Subject: [PATCH 18/28] SPARK-7481 cloud pom is still JAR (not pom). works against Hadoop 2.6 as well as 2.7, keeping azure the 2.7.x dependency. All dependencies are scoped @ hadoop.scope Change-Id: I80bd95fd48e21cf2eb4d94907ac99081cd3bd375 --- cloud/pom.xml | 129 ++++++++++++++++------------------------------- pom.xml | 135 ++++++++++++++++++++++++-------------------------- 2 files changed, 109 insertions(+), 155 deletions(-) diff --git a/cloud/pom.xml b/cloud/pom.xml index b8194b6550cb1..7be69b0aabcd5 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -50,111 +50,68 @@ - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} + org.apache.hadoop + hadoop-aws + ${hadoop.deps.scope} - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test + org.apache.hadoop + hadoop-openstack + ${hadoop.deps.scope} - - + - net.java.dev.jets3t - jets3t + joda-time + joda-time + ${hadoop.deps.scope} - - + + + com.fasterxml.jackson.core + jackson-databind + ${hadoop.deps.scope} + + + com.fasterxml.jackson.core + jackson-annotations + ${hadoop.deps.scope} + + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + ${hadoop.deps.scope} + + - com.google.guava - guava + org.apache.httpcomponents + httpclient + ${hadoop.deps.scope} + + + + org.apache.httpcomponents + httpcore + ${hadoop.deps.scope} - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - - sbt + hadoop-2.7 - com.google.guava - guava - compile + org.apache.hadoop + hadoop-azure + ${hadoop.deps.scope} - - hadoop-2.7 - - - org.apache.hadoop - hadoop-aws - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-azure - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-openstack - ${hadoop.deps.scope} - - - - joda-time - joda-time - - - - com.fasterxml.jackson.core - jackson-databind - ${hadoop.deps.scope} - - - com.fasterxml.jackson.core - jackson-annotations - ${hadoop.deps.scope} - - - com.fasterxml.jackson.dataformat - jackson-dataformat-cbor - ${hadoop.deps.scope} - - - - org.apache.httpcomponents - httpclient - - - - org.apache.httpcomponents - httpcore - - - - diff --git a/pom.xml b/pom.xml index 81ad05957256e..8c7d2c06ce40f 100644 --- a/pom.xml +++ b/pom.xml @@ -1150,6 +1150,70 @@ + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + + + org.apache.hadoop + hadoop-openstack + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + junit + junit + + + org.mockito + mockito-all + + + org.apache.zookeeper zookeeper @@ -2533,75 +2597,10 @@ - - - org.apache.hadoop - hadoop-aws - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-common - - - commons-logging - commons-logging - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-core-asl - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.core - jackson-annotations - - - - - - org.apache.hadoop - hadoop-openstack - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-common - - - commons-logging - commons-logging - - - junit - junit - - - org.mockito - mockito-all - - - org.apache.hadoop @@ -2657,8 +2656,6 @@ hadoop versions to declare different include/exclude rules (especially transient dependencies). - To use this profile, the hadoop-2.7 profile must also - be declared --> cloud From a98575370d9af1cda2c8b05672beea101ec6e83e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 27 Apr 2017 16:07:10 +0100 Subject: [PATCH 19/28] SPARK-7481 move to Spark 2.3.0-SNAPSHOT Change-Id: I91f764aeed7d832df1538453d869a7fd83964d65 --- cloud/pom.xml | 2 +- pom.xml | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/cloud/pom.xml b/cloud/pom.xml index 7be69b0aabcd5..c25f1879f3053 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.2.0-SNAPSHOT + 2.3.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 8c7d2c06ce40f..37524e2641a84 100644 --- a/pom.xml +++ b/pom.xml @@ -2599,8 +2599,7 @@ org.apache.hadoop From 0e0527d62295b1d18a53ab12ac12fddaddf7be94 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 27 Apr 2017 21:18:06 +0100 Subject: [PATCH 20/28] tweaked pom; updated docs Change-Id: I12ea6ed72ffa9edee964c90c862ff4c45bc4f47f --- cloud/pom.xml | 11 - docs/cloud-integration.md | 489 +++++--------------------------- docs/storage-openstack-swift.md | 2 - pom.xml | 4 + 4 files changed, 75 insertions(+), 431 deletions(-) diff --git a/cloud/pom.xml b/cloud/pom.xml index c25f1879f3053..3b0251d955b8f 100644 --- a/cloud/pom.xml +++ b/cloud/pom.xml @@ -32,17 +32,6 @@ Contains support for cloud infrastructures, specifically the Hadoop JARs and transitive dependencies needed to interact with the infrastructures. - - Any project which explicitly depends upon the spark-hadoop-cloud artifact will get the - dependencies; the exact versions of which will depend upon the hadoop version Spark was compiled - against. - - The imports of transitive dependencies are managed to make them consistent - with those of the Spark build. - - WARNING: the signatures of methods in the AWS and Azure SDKs do change between - versions: use exactly the same version with which the Hadoop JARs were - built. hadoop-cloud diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index d5c3d75a50f94..eb69421d2f8de 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -24,37 +24,46 @@ description: Introduction to cloud storage support in Apache Spark SPARK_VERSION ## Introduction -All the public cloud infrastructures, Amazon AWS, Microsoft Azure, Google GCS and others offer +Amazon AWS, Microsoft Azure, Google GCS and other cloud infrastructures 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`. +or size limits, object stores, "blobstores", they replace the classic directory tree of +have a simpler model of `object-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. +Apache Spark can read and write data in object stores through filesystem connectors implemented +in Apache Hadoop or provided by the infrastructure suppliers themselves. +These connectors make the object stores look *almost* like filesystems, with directories and files +and the classic operations on them such as list, delete and rename. -With these connectors, Apache Spark supports object stores as the source -of data for analysis, including Spark Streaming and DataFrames. +## Important: Cloud Object Stores are Not Real Filesystems + +While the stores appear to be filesystems, underneath +they are still object stores, [and the difference is significant](http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) -## Quick Start +They cannot be used as a direct replacement for a cluster-wide filesystem such as HDFS +*except when this is explicitly stated*. -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. +Key differences are +* Directory renames may be very slow and leave the store in an unknown state on failure. +* Output written may only be visible when the writing process completes the write. +* Changes to stored objects may not be immediately visible, both in directory listings and actual data access. -### Dependencies +For these reasons, it is not always safe to use an object store as a direct destination of queries, or as +an intermediate store destination in chained queries. Consult the provider of the object store and the object store +connector's documentation, to determine which actions are considered safe. -The Spark application neeeds the relevant Hadoop libraries, which can -be done by including the `spark-hadoop-cloud` module for the specific version of spark used. +## Installation -The Spark application should include hadoop-openstack dependency, which can -be done by including the `spark-hadoop-cloud` module for the specific version of spark used. -For example, for Maven support, add the following to the pom.xml file: +Provided the relevant libraries are on the classpath, and Spark is configured with the credentials, +objects can be can be read or written through URLs referencing the data, +such as `s3a://landsat-pds/scene_list.gz`. + +The libraries can be added to an application's classpath by including the `spark-hadoop-cloud` +module and its dependencies. + +In Maven, add the following to the pom.xml file: {% highlight xml %} @@ -68,112 +77,47 @@ For example, for Maven support, add the following to the pom.xml fi {% endhighlight %} -If using the Scala 2.10-compatible version of Spark, the artifact is of course `spark-hadoop-cloud_2.10`. - -### Basic Use - -You can refer to data in an object store just as you would data in a filesystem, by -using a URL to the data in methods like `SparkContext.textFile()` to read data, -`saveAsTextFile()` to write it back. - - -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. +Commercial products based on Spark generally set up the classpath for talking to cloud infrastructures, +in which case this module is not needed. -## Object Stores as a substitute for HDFS +## Authenticating -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. +Spark jobs must authenticate with the services to access their data. -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? | -|-----------------------------|--------------------| -| `s3a://` `s3n://` from the ASF | 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`. - -#### Spark Streaming and object stores - -Spark Streaming can monitor files added to object stores, by -creating a `FileInputDStream` DStream monitoring a path under a bucket through -`StreamingContext.textFileStream()`. - - -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. When Spark is running in cloud infrastructure (for example, on Amazon EC2, Google Cloud or +Microsoft Azure), the credentials are usually automatically set up. +1. `spark-submit` picks up the `AWS_ACCESS_KEY`, `AWS_SECRET_KEY` +and `AWS_SESSION_TOKEN` environment variables and sets the associated configuration parameters +for`s3n` and `s3a` to these values +1. In a Hadoop cluster, settings may be set in the `core-site.xml` file. +1. Authentication details may be manually added to the Spark configuration in `spark-default.conf` +1. Alternatively, they can be programmatically set in the `SparkConf` instances used to configure +the application's `SparkContext`. -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. +*Important: never check in authentication secrets into source code repositories, +especially public ones* -1. Streams should only be checkpointed to an object store considered compatible with -HDFS. Otherwise the checkpointing will be slow and potentially unreliable. +Consult [the Hadoop documentation](http://hadoop.apache.org/docs/current/) for the relevant +configuration and security options. ### 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. +Here are some settings to use when writing to object stores. ``` -spark.speculation false spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2 spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true +spark.speculation false ``` -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 - - yarn.scheduler.capacity.node-locality-delay - 0 - -``` - -This has to be set in the YARN cluster configuration, not in the Spark configuration. +This uses the "version 2" algorithm +for committing files —which does less renaming than the v1 algorithm. Speculative execution is +disabled to reduce the risk of invalid output —but it may not eliminate it. ### Parquet I/O 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 `filterPushdown` option -enables the Parquet library to optimize data reads itself, potentially saving bandwidth. +For optimal performance when working with Parquet data use the following settings: ``` spark.hadoop.parquet.enable.summary-metadata false @@ -184,9 +128,7 @@ spark.sql.hive.metastorePartitionPruning true ### ORC I/O 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. - +For best performance when working with ORC data, use these settings: ``` spark.sql.orc.filterPushdown true @@ -195,318 +137,29 @@ spark.sql.orc.cache.stripe.details.size 10000 spark.sql.hive.metastorePartitionPruning true ``` -The `filterPushdown` option enables the ORC library to optimize data reads itself, -potentially saving bandwidth. - -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. - -## 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. - - -## 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. - - - - - - - - - - - - - - - - - - - - - - - - - - - -
SchemaStoreDetails
s3a://Amazon S3 - - Recommended S3 client for Spark releases built on Apache Hadoop 2.7 or later. -
s3n://Amazon S3 - - Deprected S3 client; only use for Spark releases built on Apache Hadoop 2.6 or earlier. -
s3://Amazon S3 on Amazon EMR - - Amazon's own S3 client; use only and exclusivley in Amazon EMR. -
wasb://Azure Storage - - Client for Microsoft Azure Storage; since Hadoop 2.7. -
swift://OpenStack Swift - - Client for OpenStack Swift object stores. -
gs://Google Cloud Storage - - Google's client for their cloud object store. -
- - -### 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. - - - - - - - - - - - - -
Hadoop Library VersionClient
Hadoop 2.7+ and commercial products based on it - s3a://
Hadoop 2.6 or earlier - s3n://
Amazon EMR - s3://
- -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. - -#### S3A Filesystem Client: `s3a://` - -The ["S3A" filesystem client](https://hadoop.apache.org/docs/stable2/hadoop-aws/tools/hadoop-aws/index.html) -is the sole S3 connector undergoing active maintenance at the Apache, and should be used wherever -possible. - - -**Tuning for performance:** - -For recent Hadoop versions, *when working with binary formats* (Parquet, ORC) use - -``` -spark.hadoop.fs.s3a.experimental.input.fadvise random -``` - -This reads from the object in blocks, which is efficient when seeking backwards as well as -forwards in a file —at the expense of making full file reads slower. - -When working with text formats (text, CSV), or any sequential read through an entire file -(including .gzip compressed data), -this "random" I/O policy should be disabled. This is the default, but can be done -explicitly: - -``` -spark.hadoop.fs.s3a.experimental.input.fadvise normal -spark.hadoop.fs.s3a.readahead.range 157810688 -``` - -This optimizes the object read for sequential input, and when there is a forward `seek()` call -up to that readahead range, will simply read the data in the current HTTPS request, rather than -abort it and start a new one. - - -#### S3 Native Client `s3n://` - -The ["S3N" filesystem client](https://hadoop.apache.org/docs/stable2/hadoop-aws/tools/hadoop-aws/index.html) -was implemented in 2008 and has been widely used. - -While stable, S3N is essentially unmaintained, and deprecated in favor of S3A. -As well as being slower and limited in authentication mechanisms, the -only maintenance it receives are critical security issues. - - -#### Amazon EMR's S3 Client: `s3://` - - -In Amazon EMR, `s3://` is the URL schema used to refer to -[Amazon's own filesystem client](https://aws.amazon.com/premiumsupport/knowledge-center/emr-file-system-s3/), -one that is closed-source. - -As EMR also maps `s3n://` to the same filesystem, using URLs with the `s3n://` schema avoids -some confusion. Bear in mind, however, that Amazon's S3 client library is not the Apache one: -only Amazon can field bug reports related to it. - -To work with this data outside of EMR itself, use `s3a://` or `s3n://` instead. - - -#### Obsolete: Apache Hadoop's S3 client, `s3://` - -Apache's own Hadoop releases (i.e not EMR), uses URL `s3://` to refer to a -deprecated inode-based filesystem implemented on top of S3. -This filesystem is obsolete, deprecated and has been dropped from Hadoop 3.x. - -*Important: * Do not use `s3://` URLs with Apache Spark except on Amazon EMR* -It is not the same as the Amazon EMR one and incompatible with all other applications. - - -### Working with Microsoft Azure Storage - -Azure support comes with the [`wasb` filesystem client](https://hadoop.apache.org/docs/stable2/hadoop-azure/index.html). - -The Apache implementation is that used by Microsoft in Azure itself: it can be used -to access data in Azure as well as remotely. The object store itself is *consistent*, and -can be reliably used as the destination of queries. - - -### Working with OpenStack Swift - - -The OpenStack [`swift://` filesystem client](https://hadoop.apache.org/docs/stable2/hadoop-openstack/index.html) -works with Swift object stores in private OpenStack installations, public installations -including Rackspace Cloud and IBM Softlayer. - -### Working with Google Cloud Storage - -[Google Cloud Storage](https://cloud.google.com/storage) is supported via Google's own -[GCS filesystem client](https://cloud.google.com/hadoop/google-cloud-storage-connector). - +### YARN Scheduler settings -For use outside of Google cloud, `gcs-connector.jar` must be be manually downloaded then added -to `$SPARK_HOME/jars`. +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. +{% highlight xml %} + + yarn.scheduler.capacity.node-locality-delay + 0 + +{% endhighlight %} -## Important: Cloud Object Stores are Not Real Filesystems +This must to be set in the cluster's `yarn-site.xml` file. -Object stores are not filesystems: they are not a hierarchical tree of directories and files. +### Further Reading -The Hadoop filesystem APIs offer a filesystem API to the object stores, but underneath -they are still object stores, [and the difference is significant](http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) +Here is the documentation on the standard connectors both from Apache and the cloud providers. -While object stores can be used as the source and store -for persistent data, they cannot be used as a direct replacement for a cluster-wide filesystem such as HDFS. -This is important to know, as the fact they are accessed with the same APIs can be misleading. - -### Directory Operations May be Slow and Non-atomic - -Directory rename and delete may be performed as a series of operations. Specifically, recursive -directory deletion may be implemented as "list the objects, delete them singly or in batches". -File and directory renames may be implemented as "copy all the objects" followed by the delete operation. - -1. The time to delete a directory depends on the number of files in the directory. -1. Directory deletion may fail partway through, leaving a partially deleted directory. -1. Directory renaming may fail part way through, leaving the destination directory containing some of the files -being renamed, the source directory untouched. -1. The time to rename files and directories increases with the amount of data to rename. -1. If the rename is done on the client, the time to rename -each file will depend upon the bandwidth between client and the filesystem. The further away the client -is, the longer the rename will take. -1. Recursive directory listing can be very slow. This can slow down some parts of job submission -and execution. - -Because of these behaviours, committing of work by renaming directories is neither efficient nor -reliable. In Spark 1.6 and predecessors, there was a special output committer for Parquet, -the `org.apache.spark.sql.execution.datasources.parquet.DirectParquetOutputCommitter` -which bypasses the rename phase. However, as well as having major problems when used - with speculative execution enabled, it handled failures badly. For this reason, it -[was removed from Spark 2.0](https://issues.apache.org/jira/browse/SPARK-10063). - -*Critical* speculative execution does not work against object -stores which do not support atomic directory renames. Your output may get -corrupted. - -*Warning* even non-speculative execution is at risk of leaving the output of a job in an inconsistent -state if a "Direct" output committer is used and executors fail. - -### Data is Not Written Until the OutputStream's `close()` Operation. - -Data written to the object store is often buffered to a local file or stored in memory, -until one of the following conditions of met: - -1. When the output stream's `close()` operation is invoked. -1. Where supported and enabled, there is enough data to create a partition in a - multi-partitioned upload. - -Calls to `OutputStream.flush()` are usually a no-op, or limited to flushing to any local buffer -file. - -- Data is not visible in the object store until the entire output stream has been written. -- If the operation of writing the data does not complete, no data is saved to the object store. -(this includes transient network failures as well as failures of the process itself) -- There may not be an entry in the object store for the file (even a zero-byte one) until -the write is complete. Hence: no indication that a file is being written. -- The time to close a file is usually proportional to `filesize/bandwidth`. - -### An Object Store May Display Eventual Consistency - -Object stores are often *Eventually Consistent*. Objects are replicated across servers -for availability —changes to a replica takes time to propagate to the other replicas; -the store is `inconsistent` during this process. - -Places this can be visible include: - -- When listing "a directory"; newly created files may not yet be visible, deleted ones still present. -- After updating an object: opening and reading the object may still return the previous data. -- After deleting an obect: opening it may succeed, returning the data. -- While reading an object, if it is updated or deleted during the process. - -Microsoft Azure is consistent; Amazon S3 is "Create consistent" —but directory listing -operations may visibly lag behind changes to the underlying object. - -### Read Operations May be Significantly Slower Than Normal Filesystem Operations. - -Object stores usually implement their APIs as HTTP operations; clients make HTTP(S) requests -and block for responses. Each of these calls can be expensive. For maximum performance - -1. Try to list filesystem paths in bulk. -1. Know that `FileSystem.getFileStatus()` is expensive: cache the results rather than repeat -the call. -1. Similarly, avoid wrapper methods such as `FileSystem.exists()`, `isDirectory()` or `isFile()`. -1. Try to forward `seek()` through a file, rather than backwards. -1. Avoid renaming files: This is slow and, if it fails, may fail leave the destination in -"an undefined state". -1. Use the local filesystem as the destination of output which you intend to reload in follow-on work. -Retain the object store as the final destination of persistent output, not as a replacement for -HDFS. +* [OpenStack Swift](http://hadoop.apache.org/docs/current/hadoop-openstack/index.html). Hadoop 2.6+ +* [Azure Blob Storage](http://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Since Hadoop 2.7 +* [Azure Data Lake](http://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). Since Hadoop 2.8 +* [Amazon S3 via S3A and S3N](http://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Hadoop 2.6+ +* [Amazon EMR File System (EMRFS)](http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon +* [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index 1234c5f8e2bf9..db8aae4a284f9 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -36,8 +36,6 @@ For example, for Maven support, add the following to the pom.xml fi
{% endhighlight %} -If using the Scala 2.10-compatible version of Spark, the artifact is of course `spark-hadoop-cloud_2.10`. - # Configuration Parameters Create core-site.xml and place it inside Spark's conf directory. diff --git a/pom.xml b/pom.xml index 37524e2641a84..3cceb3c094661 100644 --- a/pom.xml +++ b/pom.xml @@ -2619,6 +2619,10 @@ com.fasterxml.jackson.core jackson-core + + com.google.guava + guava +
From b78158f7aaeaebda206c30ea3e620b3775b3481b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 28 Apr 2017 15:50:58 +0100 Subject: [PATCH 21/28] SPARK-7481 strip down the docs to a bare minimum: FS differences, security, spark-specific options + links elsewhere Change-Id: I7e9efe20d116802a403af875b241b91178078d78 --- docs/cloud-integration.md | 69 +++++++++++++++++++++------------------ 1 file changed, 38 insertions(+), 31 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index eb69421d2f8de..5a5faeddb8b19 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -21,49 +21,51 @@ description: Introduction to cloud storage support in Apache Spark SPARK_VERSION * This will become a table of contents (this text will be scraped). {:toc} -## Introduction +## Introduction Amazon AWS, Microsoft Azure, Google GCS and other cloud infrastructures 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", they replace the classic directory tree of -have a simpler model of `object-name => data`. - -Apache Spark can read and write data in object stores through filesystem connectors implemented -in Apache Hadoop or provided by the infrastructure suppliers themselves. +persistent data storage in *object stores*. These are not classic "POSIX" file +systems. In order to store hundreds of petabytes of data without any single points of failure, +object stores replace the classic filesystem directory tree +with a simpler model of `object-name => data`. To enable remote access, operations +on objects are usually performed through (slow) HTTP REST operations. + +Spark can read and write data in object stores through filesystem connectors implemented +in Hadoop or provided by the infrastructure suppliers themselves. These connectors make the object stores look *almost* like filesystems, with directories and files and the classic operations on them such as list, delete and rename. -## Important: Cloud Object Stores are Not Real Filesystems +### Important: Cloud Object Stores are Not Real Filesystems While the stores appear to be filesystems, underneath they are still object stores, [and the difference is significant](http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) -They cannot be used as a direct replacement for a cluster-wide filesystem such as HDFS +They cannot be used as a direct replacement for a cluster filesystem such as HDFS *except when this is explicitly stated*. Key differences are -* Directory renames may be very slow and leave the store in an unknown state on failure. -* Output written may only be visible when the writing process completes the write. +* Directory renames may be very slow and, on failure, leave the store in an unknown state. +* Output may only be uploaded when the writing process closes the output stream. * Changes to stored objects may not be immediately visible, both in directory listings and actual data access. For these reasons, it is not always safe to use an object store as a direct destination of queries, or as -an intermediate store destination in chained queries. Consult the provider of the object store and the object store -connector's documentation, to determine which actions are considered safe. +an intermediate store in a chain of queries. Consult the documentation of the object store and its +connector to determine which uses are considered safe. -## Installation +### Installation Provided the relevant libraries are on the classpath, and Spark is configured with the credentials, objects can be can be read or written through URLs referencing the data, -such as `s3a://landsat-pds/scene_list.gz`. +such as `"s3a://landsat-pds/scene_list.gz"`. The libraries can be added to an application's classpath by including the `spark-hadoop-cloud` module and its dependencies. -In Maven, add the following to the pom.xml file: +In Maven, add the following to the `pom.xml` file, assuming `spark.version` +is set to the chosen version of Spark: {% highlight xml %} @@ -78,29 +80,33 @@ In Maven, add the following to the pom.xml file: {% endhighlight %} Commercial products based on Spark generally set up the classpath for talking to cloud infrastructures, -in which case this module is not needed. +in which case this module may not be needed. -## Authenticating +### Authenticating -Spark jobs must authenticate with the services to access their data. +Spark jobs must authenticate with the object stores to access data within them. -1. When Spark is running in cloud infrastructure (for example, on Amazon EC2, Google Cloud or -Microsoft Azure), the credentials are usually automatically set up. +1. When Spark is running in a cloud infrastructure, the credentials are usually automatically set up. 1. `spark-submit` picks up the `AWS_ACCESS_KEY`, `AWS_SECRET_KEY` -and `AWS_SESSION_TOKEN` environment variables and sets the associated configuration parameters -for`s3n` and `s3a` to these values +and `AWS_SESSION_TOKEN` environment variables and sets the associated authentication options +for the `s3n` and `s3a` filesystem clients. 1. In a Hadoop cluster, settings may be set in the `core-site.xml` file. 1. Authentication details may be manually added to the Spark configuration in `spark-default.conf` -1. Alternatively, they can be programmatically set in the `SparkConf` instances used to configure +1. Alternatively, they can be programmatically set in the `SparkConf` instance used to configure the application's `SparkContext`. -*Important: never check in authentication secrets into source code repositories, +*Important: never check authentication secrets into source code repositories, especially public ones* Consult [the Hadoop documentation](http://hadoop.apache.org/docs/current/) for the relevant configuration and security options. +## Configuring + +Each cloud connector has its own set of configuration parameters, again, +consult the relevant documentation. + ### Recommended settings for writing to object stores Here are some settings to use when writing to object stores. @@ -111,9 +117,10 @@ spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true spark.speculation false ``` -This uses the "version 2" algorithm -for committing files —which does less renaming than the v1 algorithm. Speculative execution is -disabled to reduce the risk of invalid output —but it may not eliminate it. +This uses the "version 2" algorithm for committing files, which does less +renaming than the "version 1" algorithm. +Speculative execution is disabled to reduce the risk of invalid output +—but it may not eliminate it. ### Parquet I/O Settings @@ -149,9 +156,9 @@ waiting for the scheduler to find a node close to the data.
{% endhighlight %} -This must to be set in the cluster's `yarn-site.xml` file. +This must be set in the cluster's `yarn-site.xml` file. -### Further Reading +## Further Reading Here is the documentation on the standard connectors both from Apache and the cloud providers. From de3e95bfaa012fe8003d030fe84b00259d7610aa Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 28 Apr 2017 16:52:06 +0100 Subject: [PATCH 22/28] SPARK-7481 doc review Change-Id: I1923a4b6a959d86aa2c5b3d71faaaf2541d3ba85 --- docs/cloud-integration.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 5a5faeddb8b19..6406048ba14ef 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -24,9 +24,9 @@ description: Introduction to cloud storage support in Apache Spark SPARK_VERSION ## Introduction -Amazon AWS, Microsoft Azure, Google GCS and other cloud infrastructures offer -persistent data storage in *object stores*. These are not classic "POSIX" file -systems. In order to store hundreds of petabytes of data without any single points of failure, +All major cloud providers offer persistent data storage in *object stores*. +These are not classic "POSIX" file systems. +In order to store hundreds of petabytes of data without any single points of failure, object stores replace the classic filesystem directory tree with a simpler model of `object-name => data`. To enable remote access, operations on objects are usually performed through (slow) HTTP REST operations. From 9b1579b04646e8581482d2b37e8b3d984be7dd75 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 28 Apr 2017 18:26:10 +0100 Subject: [PATCH 23/28] review comments Change-Id: I6a0b0b9f06a4adcdf55ef75161dc1039961bc7a1 --- docs/cloud-integration.md | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 6406048ba14ef..9e2e3702b7a99 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -50,6 +50,16 @@ Key differences are * Directory renames may be very slow and, on failure, leave the store in an unknown state. * Output may only be uploaded when the writing process closes the output stream. * Changes to stored objects may not be immediately visible, both in directory listings and actual data access. +* The means by which directories are emulated may make working with them slow. +* Seeking within a file may require new REST calls, so be slow. + +How does affect spark? + +1. Reading and writing data can be slower than expected. +1. Some directory structures may be very inefficient to scan during query split calculation. +1. The output of saved RDD may not be immediately visible to a follow-on query. +1. The internal mechanism by which Spark commits work when saving an RDD is potentially +both slow and unreliable. For these reasons, it is not always safe to use an object store as a direct destination of queries, or as an intermediate store in a chain of queries. Consult the documentation of the object store and its @@ -122,6 +132,9 @@ renaming than the "version 1" algorithm. Speculative execution is disabled to reduce the risk of invalid output —but it may not eliminate it. +Bear in mind that storing temporary files can run up charges; Delete +directories called `"_temporary"` on a regular basis to avoid this. + ### Parquet I/O Settings For optimal performance when working with Parquet data use the following settings: @@ -158,6 +171,23 @@ waiting for the scheduler to find a node close to the data. This must be set in the cluster's `yarn-site.xml` file. +#### Spark Streaming and object stores + +Spark Streaming can monitor files added to object stores, by +creating a `FileInputDStream` DStream monitoring a path under a bucket through +`StreamingContext.textFileStream()`. + +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. + +1. Streams should only be checkpointed to an object store considered compatible with +HDFS. Otherwise the checkpointing will be slow and potentially unreliable. + ## Further Reading Here is the documentation on the standard connectors both from Apache and the cloud providers. From 844e2551daad0ecfd1f870c4d3e130e361c454c1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 2 May 2017 14:44:10 +0100 Subject: [PATCH 24/28] SPARK-7481 more proofreading Change-Id: Ic4804667af8e52b7be11fb00621ad8b69a1d2569 --- docs/cloud-integration.md | 70 +++++++++++++-------------------- docs/storage-openstack-swift.md | 2 +- 2 files changed, 28 insertions(+), 44 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 9e2e3702b7a99..f7ccc006e9b00 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -29,7 +29,7 @@ These are not classic "POSIX" file systems. In order to store hundreds of petabytes of data without any single points of failure, object stores replace the classic filesystem directory tree with a simpler model of `object-name => data`. To enable remote access, operations -on objects are usually performed through (slow) HTTP REST operations. +on objects are usually offered as (slow) HTTP REST operations. Spark can read and write data in object stores through filesystem connectors implemented in Hadoop or provided by the infrastructure suppliers themselves. @@ -43,23 +43,22 @@ While the stores appear to be filesystems, underneath they are still object stores, [and the difference is significant](http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) They cannot be used as a direct replacement for a cluster filesystem such as HDFS -*except when this is explicitly stated*. +*except where this is explicitly stated*. Key differences are -* Directory renames may be very slow and, on failure, leave the store in an unknown state. -* Output may only be uploaded when the writing process closes the output stream. * Changes to stored objects may not be immediately visible, both in directory listings and actual data access. * The means by which directories are emulated may make working with them slow. -* Seeking within a file may require new REST calls, so be slow. +* Rename operations may be very slow and, on failure, leave the store in an unknown state. +* Seeking within a file may require new REST calls, hurting performance. -How does affect spark? +How does affect Spark? -1. Reading and writing data can be slower than expected. +1. Reading and writing data can be significantly slower than working with a normal filesystem. 1. Some directory structures may be very inefficient to scan during query split calculation. -1. The output of saved RDD may not be immediately visible to a follow-on query. -1. The internal mechanism by which Spark commits work when saving an RDD is potentially -both slow and unreliable. +1. The output of work may not be immediately visible to a follow-on query. +1. The rename-based algorithm by which Spark normally commits work when saving an RDD, DataFrame or Dataset + is potentially both slow and unreliable. For these reasons, it is not always safe to use an object store as a direct destination of queries, or as an intermediate store in a chain of queries. Consult the documentation of the object store and its @@ -67,11 +66,12 @@ connector to determine which uses are considered safe. ### Installation -Provided the relevant libraries are on the classpath, and Spark is configured with the credentials, -objects can be can be read or written through URLs referencing the data, -such as `"s3a://landsat-pds/scene_list.gz"`. +With the relevant libraries on the classpath and Spark configured with the credentials, +objects can be can be read or written by using their URLs as the path to data. +For example `sparkContext.textFile("s3a://landsat-pds/scene_list.gz")` will create +an RDD of the file `scene_list.gz` stored in S3, using the s3a connector. -The libraries can be added to an application's classpath by including the `spark-hadoop-cloud` +To add the relevant libraries to an application's classpath, include the `spark-hadoop-cloud` module and its dependencies. In Maven, add the following to the `pom.xml` file, assuming `spark.version` @@ -89,18 +89,17 @@ is set to the chosen version of Spark: {% endhighlight %} -Commercial products based on Spark generally set up the classpath for talking to cloud infrastructures, -in which case this module may not be needed. - +Commercial products based on Apache Spark generally directly set up the classpath +for talking to cloud infrastructures, in which case this module may not be needed. ### Authenticating Spark jobs must authenticate with the object stores to access data within them. 1. When Spark is running in a cloud infrastructure, the credentials are usually automatically set up. -1. `spark-submit` picks up the `AWS_ACCESS_KEY`, `AWS_SECRET_KEY` +1. `spark-submit` reads the `AWS_ACCESS_KEY`, `AWS_SECRET_KEY` and `AWS_SESSION_TOKEN` environment variables and sets the associated authentication options -for the `s3n` and `s3a` filesystem clients. +for the `s3n` and `s3a` connectors to Amazon S3. 1. In a Hadoop cluster, settings may be set in the `core-site.xml` file. 1. Authentication details may be manually added to the Spark configuration in `spark-default.conf` 1. Alternatively, they can be programmatically set in the `SparkConf` instance used to configure @@ -124,17 +123,16 @@ Here are some settings to use when writing to object stores. ``` spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2 spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true -spark.speculation false ``` This uses the "version 2" algorithm for committing files, which does less -renaming than the "version 1" algorithm. -Speculative execution is disabled to reduce the risk of invalid output -—but it may not eliminate it. +renaming than the "version 1" algorithm, though as it still uses `rename()` +to commit files, it is still unsafe to use in some environments. -Bear in mind that storing temporary files can run up charges; Delete +Bear in mind that storing temporary files can run up charges; delete directories called `"_temporary"` on a regular basis to avoid this. + ### Parquet I/O Settings For optimal performance when working with Parquet data use the following settings: @@ -157,36 +155,22 @@ spark.sql.orc.cache.stripe.details.size 10000 spark.sql.hive.metastorePartitionPruning true ``` -### 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. - -{% highlight xml %} - - yarn.scheduler.capacity.node-locality-delay - 0 - -{% endhighlight %} - -This must be set in the cluster's `yarn-site.xml` file. - -#### Spark Streaming and object stores +#### Spark Streaming and Object Storage Spark Streaming can monitor files added to object stores, by -creating a `FileInputDStream` DStream monitoring a path under a bucket through +creating a `FileInputDStream` to monitor a path in the store through a call to `StreamingContext.textFileStream()`. 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. +under the path, not the number of *new* files, so 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. -1. Streams should only be checkpointed to an object store considered compatible with -HDFS. Otherwise the checkpointing will be slow and potentially unreliable. +1. Streams should only be checkpointed to an store implementing a fast and +atomic `rename()` operation Otherwise the checkpointing may be slow and potentially unreliable. ## Further Reading diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index db8aae4a284f9..9df9b7b7e952f 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -39,7 +39,7 @@ For example, for Maven support, add the following to the pom.xml fi # Configuration Parameters Create core-site.xml and place it inside Spark's conf directory. -The main category of parameters that should to be configured are the authentication parameters +The main category of parameters that should be configured are the authentication parameters required by Keystone. The following table contains a list of Keystone mandatory parameters. PROVIDER can be From 72a03ed58331813b0ad4bc9517fcc1f23a5eda6f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 2 May 2017 19:21:46 +0100 Subject: [PATCH 25/28] SPARK-7481 proofreading docs Change-Id: I2b75a2722f0082b916b9be20bd23a0bdc2d36615 --- docs/cloud-integration.md | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index f7ccc006e9b00..76162865eb025 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -50,7 +50,7 @@ Key differences are * Changes to stored objects may not be immediately visible, both in directory listings and actual data access. * The means by which directories are emulated may make working with them slow. * Rename operations may be very slow and, on failure, leave the store in an unknown state. -* Seeking within a file may require new REST calls, hurting performance. +* Seeking within a file may require new HTTP calls, hurting performance. How does affect Spark? @@ -66,7 +66,7 @@ connector to determine which uses are considered safe. ### Installation -With the relevant libraries on the classpath and Spark configured with the credentials, +With the relevant libraries on the classpath and Spark configured with valid credentials, objects can be can be read or written by using their URLs as the path to data. For example `sparkContext.textFile("s3a://landsat-pds/scene_list.gz")` will create an RDD of the file `scene_list.gz` stored in S3, using the s3a connector. @@ -127,9 +127,9 @@ spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true This uses the "version 2" algorithm for committing files, which does less renaming than the "version 1" algorithm, though as it still uses `rename()` -to commit files, it is still unsafe to use in some environments. +to commit files, it may be unsafe to use. -Bear in mind that storing temporary files can run up charges; delete +As storing temporary files can run up charges; delete directories called `"_temporary"` on a regular basis to avoid this. @@ -144,6 +144,8 @@ spark.sql.parquet.filterPushdown true spark.sql.hive.metastorePartitionPruning true ``` +These minimise the amount of data read during queries. + ### ORC I/O Settings For best performance when working with ORC data, use these settings: @@ -155,7 +157,9 @@ spark.sql.orc.cache.stripe.details.size 10000 spark.sql.hive.metastorePartitionPruning true ``` -#### Spark Streaming and Object Storage +Again, these minimise the amount of data read during queries. + +## Spark Streaming and Object Storage Spark Streaming can monitor files added to object stores, by creating a `FileInputDStream` to monitor a path in the store through a call to From b788494cb63c91814309cdf22b55d3301292ac66 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 3 May 2017 22:04:55 +0100 Subject: [PATCH 26/28] SPARK-7481 module rename, POM movement, docs * module is renamed hadoop-cloud in POMs, sbt, docs * hadoop-aws/azure/openstack declarations pushed down to hadoop-cloud pom, along with jackson-cbor * docs around the commit algorithm option make clear that you should only worry about v1 vs v2 if the blobstore is consistent Change-Id: Ia114bc8cd2ef731d54a83774d9dc2cf9e4c6e7d4 --- assembly/pom.xml | 4 +- docs/cloud-integration.md | 22 ++++-- docs/storage-openstack-swift.md | 4 +- {cloud => hadoop-cloud}/pom.xml | 87 ++++++++++++++++++++++-- pom.xml | 114 +------------------------------- project/SparkBuild.scala | 4 +- 6 files changed, 107 insertions(+), 128 deletions(-) rename {cloud => hadoop-cloud}/pom.xml (56%) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9728af2c53c1b..30939727e3e82 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -231,11 +231,11 @@ Pull in spark-hadoop-cloud and its associated JARs, --> - cloud + hadoop-cloud org.apache.spark - spark-hadoop-cloud_${scala.binary.version} + hadoop-cloud_${scala.binary.version} ${project.version} diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 76162865eb025..98e29f6c19cf1 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -71,7 +71,7 @@ objects can be can be read or written by using their URLs as the path to data. For example `sparkContext.textFile("s3a://landsat-pds/scene_list.gz")` will create an RDD of the file `scene_list.gz` stored in S3, using the s3a connector. -To add the relevant libraries to an application's classpath, include the `spark-hadoop-cloud` +To add the relevant libraries to an application's classpath, include the `hadoop-cloud` module and its dependencies. In Maven, add the following to the `pom.xml` file, assuming `spark.version` @@ -82,7 +82,7 @@ is set to the chosen version of Spark: ... org.apache.spark - spark-hadoop-cloud_2.11 + hadoop-cloud_2.11 ${spark.version} ... @@ -118,16 +118,26 @@ consult the relevant documentation. ### Recommended settings for writing to object stores -Here are some settings to use when writing to object stores. +For object stores whose consistency model means that rename-based commits are safe +use the `FileOutputCommitter` v2 algorithm for performance: ``` spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2 +``` + + +This does less renaming at the end of a job than the "version 1" algorithm. +As it still uses `rename()` to commit files, it is unsafe to use +when the object store does not have consistent metadata/listings. + +The committer can also be set to ignore failures when cleaning up temporary +files; this reduces the risk that a transient network problem is escalated into a +job failure: + +``` spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true ``` -This uses the "version 2" algorithm for committing files, which does less -renaming than the "version 1" algorithm, though as it still uses `rename()` -to commit files, it may be unsafe to use. As storing temporary files can run up charges; delete directories called `"_temporary"` on a regular basis to avoid this. diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index 9df9b7b7e952f..f4bb2353e3c49 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -21,7 +21,7 @@ Although not mandatory, it is recommended to configure the proxy server of Swift # Dependencies The Spark application should include hadoop-openstack dependency, which can -be done by including the `spark-hadoop-cloud` module for the specific version of spark used. +be done by including the `hadoop-cloud` module for the specific version of spark used. For example, for Maven support, add the following to the pom.xml file: {% highlight xml %} @@ -29,7 +29,7 @@ For example, for Maven support, add the following to the pom.xml fi ... org.apache.spark - spark-hadoop-cloud_2.11 + hadoop-cloud_2.11 ${spark.version} ... diff --git a/cloud/pom.xml b/hadoop-cloud/pom.xml similarity index 56% rename from cloud/pom.xml rename to hadoop-cloud/pom.xml index 3b0251d955b8f..5a469c353a751 100644 --- a/cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -26,29 +26,84 @@ ../pom.xml - spark-hadoop-cloud_2.11 + hadoop-cloud_2.11 jar Spark Project Cloud Integration Contains support for cloud infrastructures, specifically the Hadoop JARs and - transitive dependencies needed to interact with the infrastructures. + transitive dependencies needed to interact with the infrastructures, + making everything consistent with Spark's other dependencies. hadoop-cloud + org.apache.hadoop hadoop-aws + ${hadoop.version} ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + - org.apache.hadoop hadoop-openstack + ${hadoop.version} ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + junit + junit + + + org.mockito + mockito-all + + + @@ -92,11 +147,35 @@ hadoop-2.7 + + + org.apache.hadoop hadoop-azure + ${hadoop.version} ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.google.guava + guava + + diff --git a/pom.xml b/pom.xml index 3cceb3c094661..3b26ae07ccea6 100644 --- a/pom.xml +++ b/pom.xml @@ -620,11 +620,6 @@ jackson-module-jaxb-annotations ${fasterxml.jackson.version} - - com.fasterxml.jackson.dataformat - jackson-dataformat-cbor - ${fasterxml.jackson.version} - org.glassfish.jersey.core jersey-server @@ -1150,70 +1145,6 @@ - - - org.apache.hadoop - hadoop-aws - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-common - - - commons-logging - commons-logging - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-core-asl - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.core - jackson-annotations - - - - - org.apache.hadoop - hadoop-openstack - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-common - - - commons-logging - commons-logging - - - junit - junit - - - org.mockito - mockito-all - - - org.apache.zookeeper zookeeper @@ -2595,38 +2526,6 @@ 2.7.3 - - - - - - org.apache.hadoop - hadoop-azure - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-common - - - org.codehaus.jackson - jackson-mapper-asl - - - com.fasterxml.jackson.core - jackson-core - - - com.google.guava - guava - - - - - @@ -2651,19 +2550,10 @@ - - cloud + hadoop-cloud - cloud + hadoop-cloud diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5073f34bc9498..b5362ec1ae452 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -57,9 +57,9 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(mesos, yarn, sparkGangliaLgpl, - streamingKinesisAsl, dockerIntegrationTests, cloud) = + streamingKinesisAsl, dockerIntegrationTests, hadoopCloud) = Seq("mesos", "yarn", "ganglia-lgpl", "streaming-kinesis-asl", - "docker-integration-tests", "cloud").map(ProjectRef(buildLocation, _)) + "docker-integration-tests", "hadoop-cloud").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") From e173e3f2a60a8ecc9875dbda24beba793d86d019 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 4 May 2017 16:05:13 +0100 Subject: [PATCH 27/28] SPARK-7481: artifact is spark-hadoop-cloud while project is hadoop-cloud. Added an extra callout to the docs "without some form of consistency layer, Amazon S3 cannot be safely used as the direct destination of work with the normal rename-based committer" That holds for all Spark-in-EC deployments; EMR has a consistency option, as do/will others; different committers can work reliably. Change-Id: Ibbf8b1b9de10b5bb83b647cc663ceb970f70ff2d Testing: SBT and maven, with & without the hadoop-2.7 option. --- assembly/pom.xml | 2 +- docs/cloud-integration.md | 3 +++ hadoop-cloud/pom.xml | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 30939727e3e82..464af16e46f6e 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -235,7 +235,7 @@ org.apache.spark - hadoop-cloud_${scala.binary.version} + spark-hadoop-cloud_${scala.binary.version} ${project.version} diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 98e29f6c19cf1..dbe104f529842 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -64,6 +64,9 @@ For these reasons, it is not always safe to use an object store as a direct dest an intermediate store in a chain of queries. Consult the documentation of the object store and its connector to determine which uses are considered safe. +In particular: *without some form of consistency layer, Amazon S3 cannot +be safely used as the direct destination of work with the normal rename-based committer.* + ### Installation With the relevant libraries on the classpath and Spark configured with valid credentials, diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 5a469c353a751..92ac500f7beee 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -26,7 +26,7 @@ ../pom.xml - hadoop-cloud_2.11 + spark-hadoop-cloud_2.11 jar Spark Project Cloud Integration From 32ebc8cd15cd3705279fbeee1b9527abd903023d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 5 May 2017 13:17:28 +0100 Subject: [PATCH 28/28] SPARK-7481: applied proofreading, moved links to https; also cut a couple of superflous blank lines Change-Id: Iee9f0e0527de7bb875d1c2a805a0847702bb4e11 --- docs/cloud-integration.md | 21 +++++++++------------ hadoop-cloud/pom.xml | 2 +- 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index dbe104f529842..751a192da4ffd 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -40,19 +40,19 @@ and the classic operations on them such as list, delete and rename. ### Important: Cloud Object Stores are Not Real Filesystems While the stores appear to be filesystems, underneath -they are still object stores, [and the difference is significant](http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) +they are still object stores, [and the difference is significant](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) They cannot be used as a direct replacement for a cluster filesystem such as HDFS *except where this is explicitly stated*. -Key differences are +Key differences are: * Changes to stored objects may not be immediately visible, both in directory listings and actual data access. * The means by which directories are emulated may make working with them slow. * Rename operations may be very slow and, on failure, leave the store in an unknown state. * Seeking within a file may require new HTTP calls, hurting performance. -How does affect Spark? +How does this affect Spark? 1. Reading and writing data can be significantly slower than working with a normal filesystem. 1. Some directory structures may be very inefficient to scan during query split calculation. @@ -111,7 +111,7 @@ the application's `SparkContext`. *Important: never check authentication secrets into source code repositories, especially public ones* -Consult [the Hadoop documentation](http://hadoop.apache.org/docs/current/) for the relevant +Consult [the Hadoop documentation](https://hadoop.apache.org/docs/current/) for the relevant configuration and security options. ## Configuring @@ -128,7 +128,6 @@ use the `FileOutputCommitter` v2 algorithm for performance: spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2 ``` - This does less renaming at the end of a job than the "version 1" algorithm. As it still uses `rename()` to commit files, it is unsafe to use when the object store does not have consistent metadata/listings. @@ -141,11 +140,9 @@ job failure: spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true ``` - As storing temporary files can run up charges; delete directories called `"_temporary"` on a regular basis to avoid this. - ### Parquet I/O Settings For optimal performance when working with Parquet data use the following settings: @@ -193,11 +190,11 @@ atomic `rename()` operation Otherwise the checkpointing may be slow and potentia Here is the documentation on the standard connectors both from Apache and the cloud providers. -* [OpenStack Swift](http://hadoop.apache.org/docs/current/hadoop-openstack/index.html). Hadoop 2.6+ -* [Azure Blob Storage](http://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Since Hadoop 2.7 -* [Azure Data Lake](http://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). Since Hadoop 2.8 -* [Amazon S3 via S3A and S3N](http://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Hadoop 2.6+ -* [Amazon EMR File System (EMRFS)](http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon +* [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html). Hadoop 2.6+ +* [Azure Blob Storage](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Since Hadoop 2.7 +* [Azure Data Lake](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). Since Hadoop 2.8 +* [Amazon S3 via S3A and S3N](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Hadoop 2.6+ +* [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon * [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 92ac500f7beee..aa36dd4774d86 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -28,7 +28,7 @@ spark-hadoop-cloud_2.11 jar - Spark Project Cloud Integration + Spark Project Cloud Integration through Hadoop Libraries Contains support for cloud infrastructures, specifically the Hadoop JARs and transitive dependencies needed to interact with the infrastructures,