Hey,

Just wrapping this up --

I ended up following the instructions
<https://spark.apache.org/docs/1.6.2/building-spark.html> to build a custom
Spark release with Hadoop 2.7.2, stealing from Steve's SPARK-7481 PR a bit,
in order to get Spark 1.6.2 + Hadoop 2.7.2 + the hadoop-aws library (which
pulls in the proper AWS Java SDK dependency).

Now that there's an official Spark 2.0 + Hadoop 2.7.x release, this is
probably no longer necessary, but I haven't tried it, yet.

With the custom release, s3a paths work fine with EC2 role credentials
without doing anything special. The only thing I had to do was to add this
extra --conf flag to spark-submit in order to write to encrypted S3 buckets
--

    --conf spark.hadoop.fs.s3a.server-side-encryption-algorithm=AES256

Full instructions for building on Mac are here:

1) Download the Spark 1.6.2 source from
https://spark.apache.org/downloads.html

2) Install R

brew tap homebrew/science
brew install r

3) Set JAVA_HOME and the MAVEN_OPTS as in the instructions

4) Modify the root pom.xml to add a hadoop-2.7 profile (mostly stolen from
Spark 2.0)

    <profile>
      <id>hadoop-2.7</id>
      <properties>
        <hadoop.version>2.7.2</hadoop.version>
        <jets3t.version>0.9.3</jets3t.version>
        <zookeeper.version>3.4.6</zookeeper.version>
        <curator.version>2.6.0</curator.version>
      </properties>
      <dependencyManagement>
        <dependencies>
          <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>
            </exclusions>
          </dependency>
        </dependencies>
      </dependencyManagement>
    </profile>

5) Modify core/pom.xml to include the corresponding hadoop-aws and AWS SDK
libs

    <dependency>
      <groupId>org.apache.hadoop</groupId>
      <artifactId>hadoop-client</artifactId>
    </dependency>
    <dependency>
      <groupId>org.apache.hadoop</groupId>
      <artifactId>hadoop-aws</artifactId>
      <exclusions>
        <exclusion>
          <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-common</artifactId>
        </exclusion>
        <exclusion>
          <groupId>commons-logging</groupId>
          <artifactId>commons-logging</artifactId>
        </exclusion>
      </exclusions>
    </dependency>

6) Build with

./make-distribution.sh --name custom-hadoop-2.7-2-aws-s3a --tgz -Psparkr
-Phadoop-2.7 -Phive -Phive-thriftserver -Pyarn







On Sat, Jul 23, 2016 at 4:11 AM, Steve Loughran <ste...@hortonworks.com>
wrote:

>
>
> Amazon S3 has stronger consistency guarantees than the ASF s3 clients, it
> uses dynamo to do this.
>
> there is some work underway to do something similar atop S3a, S3guard, see
> https://issues.apache.org/jira/browse/HADOOP-13345  .
>
> Regarding IAM support in Spark, The latest version of S3A, which will ship
> in Hadoop 2.8, adds: IAM, temporary credential, direct env var pickup —and
> the ability to add your own.
>
> Regarding getting the relevant binaries into your app, you need a version
> of the hadoop-aws library consistent with the rest of hadoop, and the
> version of the amazon AWS SDKs that hadoop was built against. APIs in the
> SDK have changed and attempting to upgrade the amazon JAR will fail.
>
> There's a PR attached to SPARK-7481 which does the bundling and adds a
> suite of tests...it's designed to work with Hadoop 2.7+ builds. if you are
> building Spark locally, please try it and provide feedback on the PR
>
> finally, don't try an use s3a  on hadoop-2.6...that was really in preview
> state, and it let bugs surface which were fixed in 2.7.
>
> -Steve
>
> ps: More on S3a in Hadoop 2.8. Things will be way better!
> http://slideshare.net/HadoopSummit/hadoop-cloud-storage-object-store-integration-in-production
>
>
> On 21 Jul 2016, at 17:23, Ewan Leith <ewan.le...@realitymine.com> wrote:
>
> If you use S3A rather than S3N, it supports IAM roles.
>
> I think you can make s3a used for s3:// style URLs so it’s consistent with
> your EMR paths by adding this to your Hadoop config, probably in
> core-site.xml:
>
> fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem
> fs.s3.impl=org.apache.hadoop.fs.s3a.S3AFileSystem
> fs.AbstractFileSystem.s3.impl=org.apache.hadoop.fs.s3a.S3A
> fs.AbstractFileSystem.s3a.impl=org.apache.hadoop.fs.s3a.S3A
>
> And make sure the s3a jars are in your classpath
>
> Thanks,
> Ewan
>
> *From:* Everett Anderson [mailto:ever...@nuna.com.INVALID
> <ever...@nuna.com.INVALID>]
> *Sent:* 21 July 2016 17:01
> *To:* Gourav Sengupta <gourav.sengu...@gmail.com>
> *Cc:* Teng Qiu <teng...@gmail.com>; Andy Davidson <
> a...@santacruzintegration.com>; user <user@spark.apache.org>
> *Subject:* Re: Role-based S3 access outside of EMR
>
> Hey,
>
> FWIW, we are using EMR, actually, in production.
>
> The main case I have for wanting to access S3 with Spark outside of EMR is
> that during development, our developers tend to run EC2 sandbox instances
> that have all the rest of our code and access to some of the input data on
> S3. It'd be nice if S3 access "just worked" on these without storing the
> access keys in an exposed manner.
>
> Teng -- when you say you use EMRFS, does that mean you copied AWS's EMRFS
> JAR from an EMR cluster and are using it outside? My impression is that AWS
> hasn't released the EMRFS implementation as part of the aws-java-sdk, so
> I'm wary of using it. Do you know if it's supported?
>
>
> On Thu, Jul 21, 2016 at 2:32 AM, Gourav Sengupta <
> gourav.sengu...@gmail.com> wrote:
>
> Hi Teng,
>
> This is totally a flashing news for me, that people cannot use EMR in
> production because its not open sourced, I think that even Werner is not
> aware of such a problem. Is EMRFS opensourced? I am curious to know what
> does HA stand for?
> Regards,
> Gourav
>
> On Thu, Jul 21, 2016 at 8:37 AM, Teng Qiu <teng...@gmail.com> wrote:
>
> there are several reasons that AWS users do (can) not use EMR, one
> point for us is that security compliance problem, EMR is totally not
> open sourced, we can not use it in production system. second is that
> EMR do not support HA yet.
>
> but to the original question from @Everett :
>
> -> Credentials and Hadoop Configuration
>
> as you said, best practice should be "rely on machine roles", they
> called IAM roles.
>
> we are using EMRFS impl for accessing s3, it supports IAM role-based
> access control well. you can take a look here:
> https://github.com/zalando/spark/tree/branch-1.6-zalando
>
> or simply use our docker image (Dockerfile on github:
> https://github.com/zalando/spark-appliance/tree/master/Dockerfile)
>
> docker run -d --net=host \
>            -e START_MASTER="true" \
>            -e START_WORKER="true" \
>            -e START_WEBAPP="true" \
>            -e START_NOTEBOOK="true" \
>            registry.opensource.zalan.do/bi/spark:1.6.2-6
>
>
> -> SDK and File System Dependencies
>
> as mentioned above, using EMRFS libs solved this problem:
>
> http://docs.aws.amazon.com//ElasticMapReduce/latest/ReleaseGuide/emr-fs.html
> <http://docs.aws.amazon.com/ElasticMapReduce/latest/ReleaseGuide/emr-fs.html>
>
>
> 2016-07-21 8:37 GMT+02:00 Gourav Sengupta <gourav.sengu...@gmail.com>:
> > But that would mean you would be accessing data over internet increasing
> > data read latency, data transmission failures. Why are you not using EMR?
> >
> > Regards,
> > Gourav
> >
> > On Thu, Jul 21, 2016 at 1:06 AM, Everett Anderson <
> ever...@nuna.com.invalid>
> > wrote:
> >>
> >> Thanks, Andy.
> >>
> >> I am indeed often doing something similar, now -- copying data locally
> >> rather than dealing with the S3 impl selection and AWS credentials
> issues.
> >> It'd be nice if it worked a little easier out of the box, though!
> >>
> >>
> >> On Tue, Jul 19, 2016 at 2:47 PM, Andy Davidson
> >> <a...@santacruzintegration.com> wrote:
> >>>
> >>> Hi Everett
> >>>
> >>> I always do my initial data exploration and all our product development
> >>> in my local dev env. I typically select a small data set and copy it
> to my
> >>> local machine
> >>>
> >>> My main() has an optional command line argument ‘- - runLocal’
> Normally I
> >>> load data from either hdfs:/// or S3n:// . If the arg is set I read
> from
> >>> file:///
> >>>
> >>> Sometime I use a CLI arg ‘- -dataFileURL’
> >>>
> >>> So in your case I would log into my data cluster and use “AWS s3 cp" to
> >>> copy the data into my cluster and then use “SCP” to copy the data from
> the
> >>> data center back to my local env.
> >>>
> >>> Andy
> >>>
> >>> From: Everett Anderson <ever...@nuna.com.INVALID>
> >>> Date: Tuesday, July 19, 2016 at 2:30 PM
> >>> To: "user @spark" <user@spark.apache.org>
> >>> Subject: Role-based S3 access outside of EMR
> >>>
> >>> Hi,
> >>>
> >>> When running on EMR, AWS configures Hadoop to use their EMRFS Hadoop
> >>> FileSystem implementation for s3:// URLs and seems to install the
> necessary
> >>> S3 credentials properties, as well.
> >>>
> >>> Often, it's nice during development to run outside of a cluster even
> with
> >>> the "local" Spark master, though, which I've found to be more
> troublesome.
> >>> I'm curious if I'm doing this the right way.
> >>>
> >>> There are two issues -- AWS credentials and finding the right
> combination
> >>> of compatible AWS SDK and Hadoop S3 FileSystem dependencies.
> >>>
> >>> Credentials and Hadoop Configuration
> >>>
> >>> For credentials, some guides recommend setting AWS_SECRET_ACCESS_KEY
> and
> >>> AWS_ACCESS_KEY_ID environment variables or putting the corresponding
> >>> properties in Hadoop XML config files, but it seems better practice to
> rely
> >>> on machine roles and not expose these.
> >>>
> >>> What I end up doing is, in code, when not running on EMR, creating a
> >>> DefaultAWSCredentialsProviderChain and then installing the following
> >>> properties in the Hadoop Configuration using it:
> >>>
> >>> fs.s3.awsAccessKeyId
> >>> fs.s3n.awsAccessKeyId
> >>> fs.s3a.awsAccessKeyId
> >>> fs.s3.awsSecretAccessKey
> >>> fs.s3n.awsSecretAccessKey
> >>> fs.s3a.awsSecretAccessKey
> >>>
> >>> I also set the fs.s3.impl and fs.s3n.impl properties to
> >>> org.apache.hadoop.fs.s3a.S3AFileSystem to force them to use the S3A
> >>> implementation since people usually use "s3://" URIs.
> >>>
> >>> SDK and File System Dependencies
> >>>
> >>> Some special combination of the Hadoop version, AWS SDK version, and
> >>> hadoop-aws is necessary.
> >>>
> >>> One working S3A combination with Spark 1.6.1 + Hadoop 2.7.x for me
> seems
> >>> to be with
> >>>
> >>> --packages
> >>> com.amazonaws:aws-java-sdk:1.7.4,org.apache.hadoop:hadoop-aws:2.7.2
> >>>
> >>> Is this generally what people do? Is there a better way?
> >>>
> >>> I realize this isn't entirely a Spark-specific problem, but as so many
> >>> people seem to be using S3 with Spark, I imagine this community's
> faced the
> >>> problem a lot.
> >>>
> >>> Thanks!
> >>>
> >>> - Everett
> >>>
> >>
> >
>
>
>

Reply via email to