This is fork version from Databricks's spark-redshift repository. Our custom changes only tested with Spark 2.4.0 version. These custom changes may not be worked with older version of Spark
To ensure the best experience for our customers, we have decided to inline this connector directly in Databricks Runtime. The latest version of Databricks Runtime (3.0+) includes an advanced version of the RedShift connector for Spark that features both performance improvements (full query pushdown) as well as security improvements (automatic encryption). For more information, refer to the Databricks documentation. As a result, we will no longer be making releases separately from Databricks Runtime.
A library to load data into Spark SQL DataFrames from Amazon Redshift, and write them back to
Redshift tables. Amazon S3 is used to efficiently transfer data in and out of Redshift, and
JDBC is used to automatically trigger the appropriate COPY
and UNLOAD
commands on Redshift.
This library is more suited to ETL than interactive queries, since large amounts of data could be extracted to S3 for each query execution. If you plan to perform many queries against the same Redshift tables then we recommend saving the extracted data in a format such as Parquet.
- Installation
- Usage:
- Data sources API: Scala, Python, SQL, R
- Hadoop InputFormat
- Configuration
- Additional configuration options
- Transactional Guarantees
- Common problems and solutions
- S3 bucket and Redshift cluster are in different AWS regions
- Migration Guide
This library requires Apache Spark 2.0+ and Amazon Redshift 1.0.963+.
For version that works with Spark 1.x, please check for the 1.x branch.
You may use this library in your applications with the following dependency information:
Scala 2.10
groupId: com.databricks
artifactId: spark-redshift_2.10
version: 3.0.0-preview1
Scala 2.11
groupId: com.databricks
artifactId: spark-redshift_2.11
version: 3.0.0-preview1
You will also need to provide a JDBC driver that is compatible with Redshift. Amazon recommend that you use their driver, which is distributed as a JAR that is hosted on Amazon's website. This library has also been successfully tested using the Postgres JDBC driver.
Note on Hadoop versions: This library depends on spark-avro
, which should automatically be downloaded because it is declared as a dependency. However, you may need to provide the corresponding avro-mapred
dependency which matches your Hadoop distribution. In most deployments, however, this dependency will be automatically provided by your cluster's Spark assemblies and no additional action will be required.
Note on Amazon SDK dependency: This library declares a provided
dependency on components of the AWS Java SDK. In most cases, these libraries will be provided by your deployment environment. However, if you get ClassNotFoundExceptions for Amazon SDK classes then you will need to add explicit dependencies on com.amazonaws.aws-java-sdk-core
and com.amazonaws.aws-java-sdk-s3
as part of your build / runtime configuration. See the comments in project/SparkRedshiftBuild.scala
for more details.
Master snapshot builds of this library are built using jitpack.io. In order to use these snapshots in your build, you'll need to add the JitPack repository to your build file.
-
In Maven:
<repositories> <repository> <id>jitpack.io</id> <url>https://jitpack.io</url> </repository> </repositories>
then
<dependency> <groupId>com.github.databricks</groupId> <artifactId>spark-redshift_2.10</artifactId> <!-- For Scala 2.11, use spark-redshift_2.11 instead --> <version>master-SNAPSHOT</version> </dependency>
-
In SBT:
resolvers += "jitpack" at "https://jitpack.io"
then
libraryDependencies += "com.github.databricks" %% "spark-redshift" % "master-SNAPSHOT"
-
In Databricks: use the "Advanced Options" toggle in the "Create Library" screen to specify a custom Maven repository:
Use
https://jitpack.io
as the repository.- For Scala 2.10: use the coordinate
com.github.databricks:spark-redshift_2.10:master-SNAPSHOT
- For Scala 2.11: use the coordinate
com.github.databricks:spark-redshift_2.11:master-SNAPSHOT
- For Scala 2.10: use the coordinate
Once you have configured your AWS credentials, you can use this library via the Data Sources API in Scala, Python or SQL, as follows:
import org.apache.spark.sql._
val sc = // existing SparkContext
val sqlContext = new SQLContext(sc)
// Get some data from a Redshift table
val df: DataFrame = sqlContext.read
.format("com.databricks.spark.redshift")
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass")
.option("dbtable", "my_table")
.option("tempdir", "s3n://path/for/temp/data")
.load()
// Can also load data from a Redshift query
val df: DataFrame = sqlContext.read
.format("com.databricks.spark.redshift")
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass")
.option("query", "select x, count(*) my_table group by x")
.option("tempdir", "s3n://path/for/temp/data")
.load()
// Apply some transformations to the data as per normal, then you can use the
// Data Source API to write the data back to another table
df.write
.format("com.databricks.spark.redshift")
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass")
.option("dbtable", "my_table_copy")
.option("tempdir", "s3n://path/for/temp/data")
.mode("error")
.save()
// Using IAM Role based authentication
df.write
.format("com.databricks.spark.redshift")
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass")
.option("dbtable", "my_table_copy")
.option("aws_iam_role", "arn:aws:iam::123456789000:role/redshift_iam_role")
.option("tempdir", "s3n://path/for/temp/data")
.mode("error")
.save()
from pyspark.sql import SQLContext
sc = # existing SparkContext
sql_context = SQLContext(sc)
# Read data from a table
df = sql_context.read \
.format("com.databricks.spark.redshift") \
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass") \
.option("dbtable", "my_table") \
.option("tempdir", "s3n://path/for/temp/data") \
.load()
# Read data from a query
df = sql_context.read \
.format("com.databricks.spark.redshift") \
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass") \
.option("query", "select x, count(*) my_table group by x") \
.option("tempdir", "s3n://path/for/temp/data") \
.load()
# Write back to a table
df.write \
.format("com.databricks.spark.redshift") \
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass") \
.option("dbtable", "my_table_copy") \
.option("tempdir", "s3n://path/for/temp/data") \
.mode("error") \
.save()
# Using IAM Role based authentication
df.write \
.format("com.databricks.spark.redshift") \
.option("url", "jdbc:redshift://redshifthost:5439/database?user=username&password=pass") \
.option("dbtable", "my_table_copy") \
.option("tempdir", "s3n://path/for/temp/data") \
.option("aws_iam_role", "arn:aws:iam::123456789000:role/redshift_iam_role") \
.mode("error") \
.save()
Reading data using SQL:
CREATE TABLE my_table
USING com.databricks.spark.redshift
OPTIONS (
dbtable 'my_table',
tempdir 's3n://path/for/temp/data',
url 'jdbc:redshift://redshifthost:5439/database?user=username&password=pass'
);
Writing data using SQL:
-- Create a new table, throwing an error if a table with the same name already exists:
CREATE TABLE my_table
USING com.databricks.spark.redshift
OPTIONS (
dbtable 'my_table',
tempdir 's3n://path/for/temp/data'
url 'jdbc:redshift://redshifthost:5439/database?user=username&password=pass'
)
AS SELECT * FROM tabletosave;
Note that the SQL API only supports the creation of new tables and not overwriting or appending; this corresponds to the default save mode of the other language APIs.
Reading data using R:
df <- read.df(
NULL,
"com.databricks.spark.redshift",
tempdir = "s3n://path/for/temp/data",
dbtable = "my_table",
url = "jdbc:redshift://redshifthost:5439/database?user=username&password=pass")
The library contains a Hadoop input format for Redshift tables unloaded with the ESCAPE option, which you may make direct use of as follows:
import com.databricks.spark.redshift.RedshiftInputFormat
val records = sc.newAPIHadoopFile(
path,
classOf[RedshiftInputFormat],
classOf[java.lang.Long],
classOf[Array[String]])
The use of this library involves several connections which must be authenticated / secured, all of which are illustrated in the following diagram:
┌───────┐
┌───────────────────▶│ S3 │◀─────────────────┐
│ IAM or keys └───────┘ IAM or keys │
│ ▲ │
│ │ IAM or keys │
▼ ▼ ┌──────▼────┐
┌────────────┐ ┌───────────┐ │┌──────────┴┐
│ Redshift │ │ Spark │ ││ Spark │
│ │◀──────────▶│ Driver │◀────────▶┤ Executors │
└────────────┘ └───────────┘ └───────────┘
JDBC with Configured
username / in
password Spark
(can enable SSL)
This library reads and writes data to S3 when transferring data to/from Redshift. As a result, it
requires AWS credentials with read and write access to a S3 bucket (specified using the tempdir
configuration parameter).
⚠️ Note: This library does not clean up the temporary files that it creates in S3. As a result, we recommend that you use a dedicated temporary S3 bucket with an object lifecycle configuration to ensure that temporary files are automatically deleted after a specified expiration period. See the Encryption section of this document for a discussion of how these files may be encrypted.
The following describes how each connection can be authenticated:
-
Spark driver to Redshift: The Spark driver connects to Redshift via JDBC using a username and password. Redshift does not support the use of IAM roles to authenticate this connection. This connection can be secured using SSL; for more details, see the Encryption section below.
-
Spark to S3: S3 acts as a middleman to store bulk data when reading from or writing to Redshift. Spark connects to S3 using both the Hadoop FileSystem interfaces and directly using the Amazon Java SDK's S3 client.
This connection can be authenticated using either AWS keys or IAM roles (DBFS mountpoints are not currently supported, so Databricks users who do not want to rely on AWS keys should use cluster IAM roles instead).
There are multiple ways of providing these credentials:
-
Default Credential Provider Chain (best option for most users): AWS credentials will automatically be retrieved through the DefaultAWSCredentialsProviderChain.
If you use IAM instance roles to authenticate to S3 (e.g. on Databricks, EMR, or EC2), then you should probably use this method.
If another method of providing credentials is used (methods 2 or 3), then that will take precedence over this default.
-
Set keys in Hadoop conf: You can specify AWS keys via Hadoop configuration properties. For example, if your
tempdir
configuration points to as3n://
filesystem then you can set thefs.s3n.awsAccessKeyId
andfs.s3n.awsSecretAccessKey
properties in a Hadoop XML configuration file or callsc.hadoopConfiguration.set()
to mutate Spark's global Hadoop configuration.For example, if you are using the
s3n
filesystem then addsc.hadoopConfiguration.set("fs.s3n.awsAccessKeyId", "YOUR_KEY_ID") sc.hadoopConfiguration.set("fs.s3n.awsSecretAccessKey", "YOUR_SECRET_ACCESS_KEY")
and for the
s3a
filesystem addsc.hadoopConfiguration.set("fs.s3a.access.key", "YOUR_KEY_ID") sc.hadoopConfiguration.set("fs.s3a.secret.key", "YOUR_SECRET_ACCESS_KEY")
Python users will have to use a slightly different method to modify the
hadoopConfiguration
, since this field is not exposed in all versions of PySpark. Although the following command relies on some Spark internals, it should work with all PySpark versions and is unlikely to break or change in the future:sc._jsc.hadoopConfiguration().set("fs.s3n.awsAccessKeyId", "YOUR_KEY_ID") sc._jsc.hadoopConfiguration().set("fs.s3n.awsSecretAccessKey", "YOUR_SECRET_ACCESS_KEY")
-
Encode keys in
tempdir
URI: For example, the URIs3n://ACCESSKEY:SECRETKEY@bucket/path/to/temp/dir
encodes the key pair (ACCESSKEY
,SECRETKEY
).
Due to [Hadoop limitations](https://issues.apache.org/jira/browse/HADOOP-3733), this approach will not work for secret keys which contain forward slash (`/`) characters, even if those characters are urlencoded.
-
-
Redshift to S3: Redshift also connects to S3 during
COPY
andUNLOAD
queries. There are three methods of authenticating this connection:-
Have Redshift assume an IAM role (most secure): You can grant Redshift permission to assume an IAM role during
COPY
orUNLOAD
operations and then configure this library to instruct Redshift to use that role:- Create an IAM role granting appropriate S3 permissions to your bucket.
- Follow the guide Authorizing Amazon Redshift to Access Other AWS Services On Your Behalf to configure this role's trust policy in order to allow Redshift to assume this role.
- Follow the steps in the Authorizing COPY and UNLOAD Operations Using IAM Roles guide to associate that IAM role with your Redshift cluster.
- Set this library's
aws_iam_role
option to the role's ARN.
-
Forward Spark's S3 credentials to Redshift: if the
forward_spark_s3_credentials
option is set totrue
then this library will automatically discover the credentials that Spark is using to connect to S3 and will forward those credentials to Redshift over JDBC. If Spark is authenticating to S3 using an IAM instance role then a set of temporary STS credentials will be passed to Redshift; otherwise, AWS keys will be passed. These credentials are sent as part of the JDBC query, so therefore it is strongly recommended to enable SSL encryption of the JDBC connection when using this authentication method. -
Use Security Token Service (STS) credentials: You may configure the
temporary_aws_access_key_id
,temporary_aws_secret_access_key
, andtemporary_aws_session_token
configuration properties to point to temporary keys created via the AWS Security Token Service. These credentials are sent as part of the JDBC query, so therefore it is strongly recommended to enable SSL encryption of the JDBC connection when using this authentication method. If you choose this option then please be aware of the risk that the credentials expire before the read / write operation succeeds.
These three options are mutually-exclusive and you must explicitly choose which one to use.
-
-
Securing JDBC: The Redshift and Postgres JDBC drivers both support SSL. To enable SSL support, first configure Java to add the required certificates by following the Using SSL and Server Certificates in Java instructions in the Redshift documentation. Then, follow the instructions in JDBC Driver Configuration Options to add the appropriate SSL options to the JDBC
url
used with this library. -
Encrypting
UNLOAD
data stored in S3 (data stored when reading from Redshift): According to the Redshift documentation on Unloading Data to S3, "UNLOAD automatically encrypts data files using Amazon S3 server-side encryption (SSE-S3)."Redshift also supports client-side encryption with a custom key (see: Unloading Encrypted Data Files) but this library currently lacks the capability to specify the required symmetric key.
-
Encrypting
COPY
data stored in S3 (data stored when writing to Redshift): According to the Redshift documentation on Loading Encrypted Data Files from Amazon S3:You can use the COPY command to load data files that were uploaded to Amazon S3 using server-side encryption with AWS-managed encryption keys (SSE-S3 or SSE-KMS), client-side encryption, or both. COPY does not support Amazon S3 server-side encryption with a customer-supplied key (SSE-C)
To use this capability, you should configure your Hadoop S3 FileSystem to use encryption by setting the appropriate configuration properties (which will vary depending on whether you are using
s3a
,s3n
, EMRFS, etc.). Note that theMANIFEST
file (a list of all files written) will not be encrypted.
The parameter map or OPTIONS provided in Spark SQL supports the following settings.
Parameter | Required | Default | Notes |
---|---|---|---|
dbtable | Yes, unless query is specified | No default | The table to create or read from in Redshift. This parameter is required when saving data back to Redshift. |
query | Yes, unless dbtable is specified | No default | The query to read from in Redshift |
user | No | No default | The Redshift username. Must be used in tandem with password option. May only be used if the user and password are not passed in the URL, passing both will result in an error. |
password | No | No default | The Redshift password. Must be used in tandem with user option. May only be used if the user and password are not passed in the URL; passing both will result in an error. |
url | Yes | No default |
A JDBC URL, of the format, jdbc:subprotocol://host:port/database?user=username&password=password
|
aws_iam_role | Only if using IAM roles to authorize Redshift COPY/UNLOAD operations | No default | Fully specified ARN of the IAM Role attached to the Redshift cluster, ex: arn:aws:iam::123456789000:role/redshift_iam_role |
forward_spark_s3_credentials | No | false | If true then this library will automatically discover the credentials that Spark is using to connect to S3 and will forward those credentials to Redshift over JDBC. These credentials are sent as part of the JDBC query, so therefore it is strongly recommended to enable SSL encryption of the JDBC connection when using this option. |
temporary_aws_access_key_id | No | No default | AWS access key, must have write permissions to the S3 bucket. |
temporary_aws_secret_access_key | No | No default | AWS secret access key corresponding to provided access key. |
temporary_aws_session_token | No | No default | AWS session token corresponding to provided access key. |
tempdir | Yes | No default | A writeable location in Amazon S3, to be used for unloaded data when reading and Avro data to be loaded into Redshift when writing. If you're using Redshift data source for Spark as part of a regular ETL pipeline, it can be useful to set a Lifecycle Policy on a bucket and use that as a temp location for this data. |
jdbcdriver | No | Determined by the JDBC URL's subprotocol | The class name of the JDBC driver to use. This class must be on the classpath. In most cases, it should not be necessary to specify this option, as the appropriate driver classname should automatically be determined by the JDBC URL's subprotocol. |
diststyle | No | EVEN | The Redshift Distribution Style to be used when creating a table. Can be one of EVEN, KEY or ALL (see Redshift docs). When using KEY, you must also set a distribution key with the distkey option. |
distkey | No, unless using DISTSTYLE KEY | No default | The name of a column in the table to use as the distribution key when creating a table. |
sortkeyspec | No | No default |
A full Redshift Sort Key definition. Examples include:
|
No | true |
Setting this deprecated option to false will cause an overwrite operation's destination table to be dropped immediately at the beginning of the write, making the overwrite operation non-atomic and reducing the availability of the destination table. This may reduce the temporary disk space requirements for overwrites.
| |
description | No | No default |
A description for the table. Will be set using the SQL COMMENT command, and should show up in most query tools. See also the description metadata to set descriptions on individual columns. |
preactions | No | No default |
This can be a ; separated list of SQL commands to be executed before loading COPY command. It may be useful to have some DELETE commands or similar run here before loading new data. If the command contains %s, the table name will be formatted in before execution (in case you're using a staging table). Be warned that if this commands fail, it is treated as an error and you'll get an exception. If using a staging table, the changes will be reverted and the backup table restored if pre actions fail. |
postactions | No | No default |
This can be a ; separated list of SQL commands to be executed after a successful COPY when loading data. It may be useful to have some GRANT commands or similar run here when loading new data. If the command contains %s, the table name will be formatted in before execution (in case you're using a staging table). Be warned that if this commands fail, it is treated as an error and you'll get an exception. If using a staging table, the changes will be reverted and the backup table restored if post actions fail. |
extracopyoptions | No | No default |
A list extra options to append to the Redshift COPY command when loading data, e.g. TRUNCATECOLUMNS or MAXERROR n (see the Redshift docs for other options). Note that since these options are appended to the end of the COPY command, only options that make sense at the end of the command can be used, but that should cover most possible use cases. |
tempformat (Experimental) | No | AVRO |
The format in which to save temporary files in S3 when writing to Redshift. Defaults to "AVRO"; the other allowed values are "CSV" and "CSV GZIP" for CSV and gzipped CSV, respectively. Redshift is significantly faster when loading CSV than when loading Avro files, so using that tempformat may provide a large performance boost when writing to Redshift. |
csvnullstring (Experimental) | No | @NULL@ |
The String value to write for nulls when using the CSV tempformat. This should be a value which does not appear in your actual data. |
When creating Redshift tables, this library's default behavior is to create TEXT
columns for string columns. Redshift stores TEXT
columns as VARCHAR(256)
, so these columns have a maximum size of 256 characters (source).
To support larger columns, you can use the maxlength
column metadata field to specify the maximum length of individual string columns. This can also be done as a space-savings performance optimization in order to declare columns with a smaller maximum length than the default.
⚠️ Note: Due to limitations in Spark, metadata modification is unsupported in the Python, SQL, and R language APIs.
Here is an example of updating multiple columns' metadata fields using Spark's Scala API:
import org.apache.spark.sql.types.MetadataBuilder
// Specify the custom width of each column
val columnLengthMap = Map(
"language_code" -> 2,
"country_code" -> 2,
"url" -> 2083
)
var df = ... // the dataframe you'll want to write to Redshift
// Apply each column metadata customization
columnLengthMap.foreach { case (colName, length) =>
val metadata = new MetadataBuilder().putLong("maxlength", length).build()
df = df.withColumn(colName, df(colName).as(colName, metadata))
}
df.write
.format("com.databricks.spark.redshift")
.option("url", jdbcURL)
.option("tempdir", s3TempDirectory)
.option("dbtable", sessionTable)
.save()
If you need to manually set a column type, you can use the redshift_type
column metadata. For example, if you desire to override
the Spark SQL Schema -> Redshift SQL
type matcher to assign a user-defined column type, you can do the following:
import org.apache.spark.sql.types.MetadataBuilder
// Specify the custom width of each column
val columnTypeMap = Map(
"language_code" -> "CHAR(2)",
"country_code" -> "CHAR(2)",
"url" -> "BPCHAR(111)"
)
var df = ... // the dataframe you'll want to write to Redshift
// Apply each column metadata customization
columnTypeMap.foreach { case (colName, colType) =>
val metadata = new MetadataBuilder().putString("redshift_type", colType).build()
df = df.withColumn(colName, df(colName).as(colName, metadata))
}
When creating a table, this library can be configured to use a specific compression encoding on individual columns. You can use the encoding
column metadata field to specify a compression encoding for each column (see Amazon docs for available encodings).
Redshift allows columns to have descriptions attached that should show up in most query tools (using the COMMENT
command). You can set the description
column metadata field to specify a description for individual columns.
This section describes the transactional guarantees of the Redshift data source for Spark
For general information on Redshift's transactional guarantees, see the Managing Concurrent Write Operations chapter in the Redshift documentation. In a nutshell, Redshift provides serializable isolation (according to the documentation for Redshift's BEGIN
command, "[although] you can use any of the four transaction isolation levels, Amazon Redshift processes all isolation levels as serializable"). According to its documentation, "Amazon Redshift supports a default automatic commit behavior in which each separately-executed SQL command commits individually." Thus, individual commands like COPY
and UNLOAD
are atomic and transactional, while explicit BEGIN
and END
should only be necessary to enforce the atomicity of multiple commands / queries.
When reading from / writing to Redshift, this library reads and writes data in S3. Both Spark and Redshift produce partitioned output which is stored in multiple files in S3. According to the Amazon S3 Data Consistency Model documentation, S3 bucket listing operations are eventually-consistent, so the files must to go to special lengths to avoid missing / incomplete data due to this source of eventual-consistency.
Appending to an existing table: In the COPY
command, this library uses manifests to guard against certain eventually-consistent S3 operations. As a result, it appends to existing tables have the same atomic and transactional properties as regular Redshift COPY
commands.
Appending to an existing table: When inserting rows into Redshift, this library uses the COPY
command and specifies manifests to guard against certain eventually-consistent S3 operations. As a result, spark-redshift
appends to existing tables have the same atomic and transactional properties as regular Redshift COPY
commands.
Creating a new table (SaveMode.CreateIfNotExists
): Creating a new table is a two-step process, consisting of a CREATE TABLE
command followed by a COPY
command to append the initial set of rows. Both of these operations are performed in a single transaction.
Overwriting an existing table: By default, this library uses transactions to perform overwrites, which are implemented by deleting the destination table, creating a new empty table, and appending rows to it.
If the deprecated usestagingtable
setting is set to false
then this library will commit the DELETE TABLE
command before appending rows to the new table, sacrificing the atomicity of the overwrite operation but reducing the amount of staging space that Redshift needs during the overwrite.
Querying Redshift tables: Queries use Redshift's UNLOAD
command to execute a query and save its results to S3 and use manifests to guard against certain eventually-consistent S3 operations. As a result, queries from Redshift data source for Spark should have the same consistency properties as regular Redshift queries.
By default, S3 <-> Redshift copies will not work if the S3 bucket and Redshift cluster are in different AWS regions.
If you attempt to perform a read of a Redshift table and the regions are mismatched then you may see a confusing error, such as
java.sql.SQLException: [Amazon](500310) Invalid operation: S3ServiceException:The bucket you are attempting to access must be addressed using the specified endpoint. Please send all future requests to this endpoint.
Similarly, attempting to write to Redshift using a S3 bucket in a different region may cause the following error:
error: Problem reading manifest file - S3ServiceException:The bucket you are attempting to access must be addressed using the specified endpoint. Please send all future requests to this endpoint.,Status 301,Error PermanentRedirect
For writes: Redshift's COPY
command allows the S3 bucket's region to be explicitly specified, so you can make writes to Redshift work properly in these cases by adding
region 'the-region-name'
to the extracopyoptions
setting. For example, with a bucket in the US East (Virginia) region and the Scala API, use
.option("extracopyoptions", "region 'us-east-1'")
For reads: According to its documentation, the Redshift UNLOAD
command does not support writing to a bucket in a different region:
Important
The Amazon S3 bucket where Amazon Redshift will write the output files must reside in the same region as your cluster.
As a result, this use-case is not supported by this library. The only workaround is to use a new bucket in the same region as your Redshift cluster.
- Version 3.0 now requires
forward_spark_s3_credentials
to be explicitly set before Spark S3 credentials will be forwarded to Redshift. Users who use theaws_iam_role
ortemporary_aws_*
authentication mechanisms will be unaffected by this change. Users who relied on the old default behavior will now need to explicitly setforward_spark_s3_credentials
totrue
to continue using their previous Redshift to S3 authentication mechanism. For a discussion of the three authentication mechanisms and their security trade-offs, see the Authenticating to S3 and Redshift section of this README.