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

peter weissbrod commented on HADOOP-17241:
------------------------------------------

Further information: I've started to experience this issue in separate places. 
Hive 3.0.0 metastore related issues.

Existing buckets (created march 2020) containing dots in the names. Creating 
new/external table definitions using LOCATION 's3a://...; are successful (hive 
seems to automatically scan the LOCATION for file stats upon creation. It makes 
several low-level attempts but ultimately compiles a list of files and builds 
statistics)

After creation all calls to SELECT from the creates tables fails with 
{{InvalidInputException: Input path does not exist}}

In this scenario I attempt to load from the created table using pyspark v3 
(running example below). My takeaway from this is that going forward s3a 
support for buckets with dots will be flaky/unreliable at best.

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

 
{{Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0
      /_/

Using Scala version 2.12.10, Java HotSpot(TM) 64-Bit Server VM, 1.8.0_121
Branch HEAD
Compiled by user ubuntu on 2020-06-06T11:32:25Z
Revision 3fdfce3120f307147244e5eaf46d61419a723d50
Url https://gitbox.apache.org/repos/asf/spark.git

# load a text file via sparkContext and ensure you can read it. This works!
>>> s3File = spark.sparkContext.textFile("s3a://mybucket.with.dots/_mydir/*")
>>> s3File.count()
1985
# attempt to (re) build a sql table from that same directory
>>> spark.sql('drop table if exists exp.baz')
DataFrame[]
>>> spark.sql("""
... create external table exp.baz (...) stored as textfile location 
's3a://mybucket.with.dots/_mydir/'
""")
DataFrame[]
>>> spark.sql('select count(*) from exp.baz').show()
20/09/07 17:14:14 WARN DAGScheduler: Creating new stage failed due to exception 
- job: 1
org.apache.hadoop.mapred.InvalidInputException: Input path does not exist: 
s3a://mybucket.with.dots/_mydir
        at 
org.apache.hadoop.mapred.FileInputFormat.singleThreadedListStatus(FileInputFormat.java:287)
        at 
org.apache.hadoop.mapred.FileInputFormat.listStatus(FileInputFormat.java:229)
        at 
org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:315)
        at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:205)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:276)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:272)
        at 
org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:276)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:272)
        at 
org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:276)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:272)
        at 
org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:276)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:272)
        at 
org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:276)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:272)
        at 
org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
        at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:276)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:272)
        at org.apache.spark.rdd.RDD.getNumPartitions(RDD.scala:292)
        at 
org.apache.spark.scheduler.DAGScheduler.checkBarrierStageWithNumSlots(DAGScheduler.scala:435)
        at 
org.apache.spark.scheduler.DAGScheduler.createShuffleMapStage(DAGScheduler.scala:388)
        at 
org.apache.spark.scheduler.DAGScheduler.getOrCreateShuffleMapStage(DAGScheduler.scala:358)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$getOrCreateParentStages$1(DAGScheduler.scala:468)
        at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
        at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
        at scala.collection.TraversableLike.map(TraversableLike.scala:238)
        at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
        at 
scala.collection.mutable.AbstractSet.scala$collection$SetLike$$super$map(Set.scala:48)
        at scala.collection.SetLike.map(SetLike.scala:104)
        at scala.collection.SetLike.map$(SetLike.scala:104)
        at scala.collection.mutable.AbstractSet.map(Set.scala:48)
        at 
org.apache.spark.scheduler.DAGScheduler.getOrCreateParentStages(DAGScheduler.scala:467)
        at 
org.apache.spark.scheduler.DAGScheduler.createResultStage(DAGScheduler.scala:454)
        at 
org.apache.spark.scheduler.DAGScheduler.handleJobSubmitted(DAGScheduler.scala:986)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2160)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2152)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2141)
        at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
        at 
org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:752)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2093)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2114)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2133)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:467)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:420)
        at 
org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:47)
        at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3625)
        at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2695)
        at 
org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3616)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:100)
        at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
        at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3614)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:2695)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:2902)
        at org.apache.spark.sql.Dataset.getRows(Dataset.scala:300)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:337)}}

> 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: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to