[ 
https://issues.apache.org/jira/browse/CRUNCH-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13688071#comment-13688071
 ] 

Deepak Subhramanian commented on CRUNCH-220:
--------------------------------------------

[~joshwills] cc [~davebeech] 

Hi Josh , 

It build properly when I tried today building 0.7 snapshot. Not sure what 
different environment settings I had yesterday. 

I ran the code with the patch and removing fs.default.name and it is giving me 
the same error.   Also when I give fs.default.name it works in the local mode 
but not in the cluster mode since it is looking job.jar in s3 dir.   I will get 
help of Dave [~dbeech] tomorrow and double check if I am doing something wrong. 

[ec2-ip.compute.amazonaws.com] run: hadoop   jar 
/mnt1/home/dsubhramanian/deployed_code/hadoop_jars/hbasetocsvmrjobs-1.0.0.jar 
uk.co.newsint.cip.datascience.crunch.ConvertSeqToText -libjars 
/usr/lib/hbase/hbase-0.90.4-cdh3u3.jar,/usr/lib/hbase/lib/zookeeper.jar 
-Dhbase.zookeeper.quorum=ip-10-248-23-44.eu-west-1.compute.internal 
-Dfs.s3n.awsAccessKeyId=accesskeyedited 
-Dfs.s3n.awsSecretAccessKey=secretkeyedited SSH  
s3n://analyticsbkt/prod/hive/ds/customerfiltered/tmp_s3_contractsubs/hive 
s3n://analyticsbkt/prod/hive/ds/customerfiltered/tmp_s3_contractsubs/csv 
'cip_uid,subscription_date_created,service_activation_date_start,service_cancellation_date,cancellation_requested_date,subscription_date_created_month,service_activation_date_start_month,trial_date_end,trial_length_round,contract_date_end_plus_filter,contract_date_end,contract_date_end_minus_filter'
[ec2-ip.compute.amazonaws.com] out: 13/06/19 14:24:46 INFO impl.FileTargetImpl: 
Will write output files to new path: 
s3n://analyticsbkt/prod/hive/ds/customerfiltered/tmp_s3_contractsubs/csv
[ec2-ip.compute.amazonaws.com] out: 13/06/19 14:24:49 INFO 
input.FileInputFormat: Total input paths to process : 1
[ec2-ip.compute.amazonaws.com] out: 13/06/19 14:24:50 INFO 
jobcontrol.CrunchControlledJob: Running job 
"uk.co.newsint.cip.datascience.crunch.ConvertSeqToText: 
SeqFile(s3n://analyticsbkt/prod/hive/ds/customerfiltered/tmp_s3_contractsubs/hive)+S0+asText+Text(s3n://analyticsbkt/prod/hive/ds/customerfiltered/tmp_s3_contractsubs/csv)"
[ec2-ip.compute.amazonaws.com] out: 13/06/19 14:24:50 INFO 
jobcontrol.CrunchControlledJob: Job status available at: 
http://ip-edited.eu-west-1.compute.interna:50030/jobdetails.jsp?jobid=job_201306051137_3898
[ec2-ip.compute.amazonaws.com] out: Exception in thread "Thread-4" 
java.lang.IllegalArgumentException: Wrong FS: 
s3n://analyticsbkt/prod/hive/ds/customerfiltered/tmp_s3_contractsubs/csv, 
expected: hdfs://ip-10-edited.eu-west-1.compute.internal
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:410)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.hadoop.hdfs.DistributedFileSystem.checkPath(DistributedFileSystem.java:106)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:162)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:558)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:797)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.io.impl.FileTargetImpl.handleOutputs(FileTargetImpl.java:94)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.impl.mr.exec.CrunchJobHooks$CompletionHook.handleMultiPaths(CrunchJobHooks.java:87)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.impl.mr.exec.CrunchJobHooks$CompletionHook.run(CrunchJobHooks.java:78)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.hadoop.mapreduce.lib.jobcontrol.CrunchControlledJob.checkRunningState(CrunchControlledJob.java:251)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.hadoop.mapreduce.lib.jobcontrol.CrunchControlledJob.checkState(CrunchControlledJob.java:261)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.hadoop.mapreduce.lib.jobcontrol.CrunchJobControl.checkRunningJobs(CrunchJobControl.java:170)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.hadoop.mapreduce.lib.jobcontrol.CrunchJobControl.pollJobStatusAndStartNewOnes(CrunchJobControl.java:221)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.impl.mr.exec.MRExecutor.monitorLoop(MRExecutor.java:101)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.impl.mr.exec.MRExecutor.access$000(MRExecutor.java:52)
[ec2-ip.compute.amazonaws.com] out:     at 
org.apache.crunch.impl.mr.exec.MRExecutor$1.run(MRExecutor.java:76)
[ec2-ip.compute.amazonaws.com] out:     at java.lang.Thread.run(Thread.java:662)

                
> Crunch not working with S3
> --------------------------
>
>                 Key: CRUNCH-220
>                 URL: https://issues.apache.org/jira/browse/CRUNCH-220
>             Project: Crunch
>          Issue Type: Bug
>          Components: IO
>    Affects Versions: 0.6.0
>         Environment: Cloudera Hadoop with Amazon S3
>            Reporter: Deepak Subhramanian
>            Assignee: Josh Wills
>            Priority: Minor
>             Fix For: 0.7.0
>
>         Attachments: CRUNCH-220.patch
>
>
> I am trying to use crunch to read file from S3 and write to S3. I am able to 
> read the file .But giving an error while writing to s3.  Not sure if it is a 
> bug or I am missing a hadoop configuration.  I am able to read from s3 and 
> write to a local file or hdfs directly.  Here is the code and error. I am 
> passing s3 key and secret as parameters.  
> PCollection<String> lines =pipeline.read(From.sequenceFile(inputdir,   
> Writables.strings()));
>     
>     PCollection<String> textline = lines.parallelDo(new DoFn<String, 
> String>() {
>         public void process(String line, Emitter<String> emitter) {
>             if (headerNotWritten) {
>               
>                 //emitter.emit("Writing Header");
>                 emitter.emit(table_header.getTable_header());
>                 emitter.emit(line);
>                 headerNotWritten =false;
>                 
>             }else {
>             emitter.emit(line);
>             }
>         }
>       }, Writables.strings()); // Indicates the serialization format
>     
>     pipeline.writeTextFile(textline, outputdir);
>  Exception in thread "main" java.lang.IllegalArgumentException: Wrong FS: 
> s3n://bktname/testcsv, expected: hdfs://ip-address.compute.internal
> [ip-addresscompute.amazonaws.com] out:        at 
> org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:410)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.hadoop.hdfs.DistributedFileSystem.checkPath(DistributedFileSystem.java:106)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:162)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:558)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:797)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.crunch.io.impl.FileTargetImpl.handleExisting(FileTargetImpl.java:133)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.crunch.impl.mr.MRPipeline.write(MRPipeline.java:212)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.crunch.impl.mr.MRPipeline.write(MRPipeline.java:200)
> [ip-address-82.eu-west-1.compute.amazonaws.com] out:  at 
> org.apache.crunch.impl.mr.collect.PCollectionImpl.write(PCollectionImpl.java:132)
> [ec2-79-125-102-82.eu-west-1.compute.amazonaws.com] out:      at 
> org.apache.crunch.impl.mr.MRPipeline.writeTextFile(MRPipeline.java:356)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to