diff --git a/assembly/pom.xml b/assembly/pom.xml
index 742a4a1531e715a503f476f6aede792e1fd38b6e..464af16e46f6e723437da198e44d6afc707d6290 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -226,5 +226,19 @@
         <parquet.deps.scope>provided</parquet.deps.scope>
       </properties>
     </profile>
+
+    <!--
+     Pull in spark-hadoop-cloud and its associated JARs,
+    -->
+    <profile>
+      <id>hadoop-cloud</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-hadoop-cloud_${scala.binary.version}</artifactId>
+          <version>${project.version}</version>
+        </dependency>
+      </dependencies>
+    </profile>
   </profiles>
 </project>
diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md
new file mode 100644
index 0000000000000000000000000000000000000000..751a192da4ffde5a47c1207baa8910f9bb9a17ac
--- /dev/null
+++ b/docs/cloud-integration.md
@@ -0,0 +1,200 @@
+---
+layout: global
+displayTitle: Integration with Cloud Infrastructures
+title: Integration with Cloud Infrastructures
+description: Introduction to cloud storage support in Apache Spark SPARK_VERSION_SHORT
+---
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+## Introduction
+
+
+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 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.
+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
+
+While the stores appear to be filesystems, underneath
+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:
+
+* 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 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.
+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
+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,
+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 `hadoop-cloud` 
+module and its dependencies.
+
+In Maven, add the following to the `pom.xml` file, assuming `spark.version`
+is set to the chosen version of Spark:
+
+{% highlight xml %}
+<dependencyManagement>
+  ...
+  <dependency>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>hadoop-cloud_2.11</artifactId>
+    <version>${spark.version}</version>
+  </dependency>
+  ...
+</dependencyManagement>
+{% endhighlight %}
+
+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` 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` 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 
+the application's `SparkContext`.
+
+*Important: never check authentication secrets into source code repositories,
+especially public ones*
+
+Consult [the Hadoop documentation](https://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
+
+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
+```
+
+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:
+
+```
+spark.hadoop.parquet.enable.summary-metadata false
+spark.sql.parquet.mergeSchema false
+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:
+
+```
+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
+```
+
+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
+`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, 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 store implementing a fast and
+atomic `rename()` operation Otherwise the checkpointing may be slow and potentially unreliable.
+
+## Further Reading
+
+Here is the documentation on the standard connectors both from Apache and the cloud providers.
+
+* [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/docs/index.md b/docs/index.md
index ad4f24ff1a5d150066b3cf06704750deb6173404..960b968454d0e96438f0d33537f1a6ee75320b9e 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)
diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md
index e2bf2d7ca77ca14667d1d76c086eb4740aaa53c1..52e59df9990e9e7e4d3a61c389a06c8cad2c0f18 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<String> 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 c39ef1ce59e1c1f7b3e4fba7e1f1447f5bd7315b..f4bb2353e3c49bb45d1d212caebbb2d3bf0195cf 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 <code>swift://container.PROVIDER/path</code>. You will also need to set your 
 Swift security credentials, through <code>core-site.xml</code> or via
 <code>SparkContext.hadoopConfiguration</code>.
-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
 
@@ -19,41 +20,30 @@ Although not mandatory, it is recommended to configure the proxy server of Swift
 
 # Dependencies
 
-The Spark application should include <code>hadoop-openstack</code> dependency.
+The Spark application should include <code>hadoop-openstack</code> dependency, which can
+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 <code>pom.xml</code> file:
 
 {% highlight xml %}
 <dependencyManagement>
   ...
   <dependency>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-openstack</artifactId>
-    <version>2.3.0</version>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>hadoop-cloud_2.11</artifactId>
+    <version>${spark.version}</version>
   </dependency>
   ...
 </dependencyManagement>
 {% endhighlight %}
 
-
 # Configuration Parameters
 
 Create <code>core-site.xml</code> and place it inside Spark's <code>conf</code> 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. 
+The main category of parameters that should be configured are the authentication parameters
+required by Keystone.
 
-Configuration of Hadoop to use Swift File system achieved via 
-
-<table class="table">
-<tr><th>Property Name</th><th>Value</th></tr>
-<tr>
-  <td>fs.swift.impl</td>
-  <td>org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem</td>
-</tr>
-</table>
-
-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. <code>PROVIDER</code> can be any name.
+The following table  contains a list of Keystone mandatory parameters. <code>PROVIDER</code> can be
+any (alphanumeric) name.
 
 <table class="table">
 <tr><th>Property Name</th><th>Meaning</th><th>Required</th></tr>
@@ -94,7 +84,7 @@ contains a list of Keystone mandatory parameters. <code>PROVIDER</code> can be a
 </tr>
 <tr>
   <td><code>fs.swift.service.PROVIDER.public</code></td>
-  <td>Indicates if all URLs are public</td>
+  <td>Indicates whether to use the public (off cloud) or private (in cloud; no transfer fees) endpoints</td>
   <td>Mandatory</td>
 </tr>
 </table>
@@ -104,10 +94,6 @@ defined for tenant <code>test</code>. Then <code>core-site.xml</code> should inc
 
 {% highlight xml %}
 <configuration>
-  <property>
-    <name>fs.swift.impl</name>
-    <value>org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem</value>
-  </property>
   <property>
     <name>fs.swift.service.SparkTest.auth.url</name>
     <value>http://127.0.0.1:5000/v2.0/tokens</value>
diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..aa36dd4774d861993b5a8b780f5d2f0795756a98
--- /dev/null
+++ b/hadoop-cloud/pom.xml
@@ -0,0 +1,185 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent_2.11</artifactId>
+    <version>2.3.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>spark-hadoop-cloud_2.11</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project Cloud Integration through Hadoop Libraries</name>
+  <description>
+    Contains support for cloud infrastructures, specifically the Hadoop JARs and
+    transitive dependencies needed to interact with the infrastructures,
+    making everything consistent with Spark's other dependencies.
+  </description>
+  <properties>
+    <sbt.project.name>hadoop-cloud</sbt.project.name>
+  </properties>
+
+  <dependencies>
+    <!--
+      the AWS module pulls in jackson; its transitive dependencies can create
+      intra-jackson-module version problems.
+      -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-aws</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>${hadoop.deps.scope}</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-mapper-asl</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-core-asl</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-databind</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-annotations</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-openstack</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>${hadoop.deps.scope}</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mockito</groupId>
+          <artifactId>mockito-all</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!--
+    Add joda time to ensure that anything downstream which doesn't pull in spark-hive
+    gets the correct joda time artifact, so doesn't have auth failures on later Java 8 JVMs
+    -->
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <scope>${hadoop.deps.scope}</scope>
+    </dependency>
+    <!-- explicitly declare the jackson artifacts desired -->
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <scope>${hadoop.deps.scope}</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+      <scope>${hadoop.deps.scope}</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.dataformat</groupId>
+      <artifactId>jackson-dataformat-cbor</artifactId>
+      <version>${fasterxml.jackson.version}</version>
+    </dependency>
+    <!--Explicit declaration to force in Spark version into transitive dependencies -->
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <scope>${hadoop.deps.scope}</scope>
+    </dependency>
+    <!--Explicit declaration to force in Spark version into transitive dependencies -->
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+      <scope>${hadoop.deps.scope}</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+
+    <profile>
+      <id>hadoop-2.7</id>
+      <!-- Hadoop Azure is a new Jar with -->
+      <dependencies>
+
+        <!--
+        Hadoop WASB client only arrived in Hadoop 2.7
+        -->
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-azure</artifactId>
+          <version>${hadoop.version}</version>
+          <scope>${hadoop.deps.scope}</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-common</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.codehaus.jackson</groupId>
+              <artifactId>jackson-mapper-asl</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.fasterxml.jackson.core</groupId>
+              <artifactId>jackson-core</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.google.guava</groupId>
+              <artifactId>guava</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
+
+  </profiles>
+
+</project>
diff --git a/pom.xml b/pom.xml
index a1a1817e2f7d387f734f6f6ec0ccaf0c16ddf6f7..0533a8dcf2e0a4eea787b04be812883fdc9926a9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2546,6 +2546,13 @@
       </modules>
     </profile>
 
+    <profile>
+      <id>hadoop-cloud</id>
+      <modules>
+        <module>hadoop-cloud</module>
+      </modules>
+    </profile>
+
     <profile>
       <id>scala-2.10</id>
       <activation>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index e52baf51aed1af3a52b8c4bc54f384bf8d486fb8..b5362ec1ae4524736726506f4475a9ab1fde13c7 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) =
+    streamingKinesisAsl, dockerIntegrationTests, hadoopCloud) =
     Seq("mesos", "yarn", "ganglia-lgpl", "streaming-kinesis-asl",
-      "docker-integration-tests").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")