Ondrej Kokes created HADOOP-17241: ------------------------------------- Summary: s3a: dots not allowed in bucket names Key: HADOOP-17241 URL: https://issues.apache.org/jira/browse/HADOOP-17241 Project: Hadoop Common Issue Type: Bug Components: fs/s3 Affects Versions: 3.2.0, 2.7.4 Reporter: Ondrej Kokes
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. 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: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org