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

Steve Loughran commented on HADOOP-17241:
-----------------------------------------


bq. going forward s3a support for buckets with dots will be flaky/unreliable at 
best.

You can change back to path style access to get at buckets which have "." in 
their name or aren't valid hostnames

{code}
spark.hadoop.fs.s3a.path.style.access true
{code}


{code}
For those of us that have long established s3 naming conventions using dots 
this is a pervasive and serious development
{code}

As discussed, Amazon S3 have announced plans to remove path-style support for 
new buckets, though they've pushed out the date

See 
https://aws.amazon.com/blogs/aws/amazon-s3-path-deprecation-plan-the-rest-of-the-story/

This doesn't affect any of your current buckets, but consider following AWS 
best practises and only use "." in bucket names for anything other than virtual 
hosting. Certainly for any new buckets.





> s3a: bucket names which aren't parseable hostnames unsupported
> --------------------------------------------------------------
>
>                 Key: HADOOP-17241
>                 URL: https://issues.apache.org/jira/browse/HADOOP-17241
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 2.7.4, 3.2.0
>            Reporter: Ondrej Kokes
>            Priority: Minor
>
> Hi there,
>  I'm using Spark to read some data from S3 and I encountered an error when 
> reading from a bucket that contains a period (e.g. 
> `s3a://okokes-test-v1.1/foo.csv`). I have close to zero Java experience, but 
> I've tried to trace this as well as I can. Apologies for any misunderstanding 
> on my part.
> _Edit: the title is a little misleading - buckets can contain dots and s3a 
> will work, but only if these bucket names conform to hostname restrictions - 
> e.g. `s3a://foo.bar/bak.csv` would work, but my case - `okokes-test-v1.1` 
> does not, because `1` is not conform to a top level domain pattern._
> Using hadoop-aws:3.2.0, I get the following:
> {code:java}
> java.lang.NullPointerException: null uri host.
>  at java.base/java.util.Objects.requireNonNull(Objects.java:246)
>  at 
> org.apache.hadoop.fs.s3native.S3xLoginHelper.buildFSURI(S3xLoginHelper.java:71)
>  at org.apache.hadoop.fs.s3a.S3AFileSystem.setUri(S3AFileSystem.java:470)
>  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:235)
>  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3303)
>  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:124)
>  at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3352)
>  at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3320)
>  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:479)
>  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
>  at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:46)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:361)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:279)
>  at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:268)
>  at scala.Option.getOrElse(Option.scala:189)
>  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:268)
>  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:705)
>  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:535)
>  ... 47 elided{code}
> hadoop-aws:2.7.4 did lead to a similar outcome
> {code:java}
> java.lang.IllegalArgumentException: The bucketName parameter must be 
> specified.
>  at 
> com.amazonaws.services.s3.AmazonS3Client.assertParameterNotNull(AmazonS3Client.java:2816)
>  at 
> com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1026)
>  at 
> com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:994)
>  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:297)
>  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669)
>  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94)
>  at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703)
>  at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685)
>  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373)
>  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295)
>  at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:46)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:361)
>  at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:279)
>  at 
> org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:268)
>  at scala.Option.getOrElse(Option.scala:189)
>  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:268)
>  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:705)
>  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:535)
>  ... 47 elided{code}
> I investigated the issue a little bit and found buildFSURI to require the 
> host to be not null - [see 
> S3xLoginHelper.java|https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java#L70]
>  - but in my case the host is null and the authority part of the URL should 
> be used. When I checked AWS' handling of this case, they seem to be using 
> authority for all s3:// paths - 
> [https://github.com/aws/aws-sdk-java/blob/master/aws-java-sdk-s3/src/main/java/com/amazonaws/services/s3/AmazonS3URI.java#L85].
> I verified this URI in a Scala shell (openjdk 1.8.0_252)
>  
> {code:java}
> scala> (new URI("s3a://okokes-test-v1.1/foo.csv")).getHost()
> val res1: String = null
> scala> (new URI("s3a://okokes-test-v1.1/foo.csv")).getAuthority()
> val res2: String = okokes-test-v1.1
> {code}
>  
> Oh and this is indeed a bucket name. Not only did I create it in the console, 
> but there's also enough documentation on the topic - 
> [https://docs.aws.amazon.com/AmazonS3/latest/dev/BucketRestrictions.html#bucketnamingrules]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to