JonasJ-ap commented on PR #5644:
URL: https://github.com/apache/iceberg/pull/5644#issuecomment-1229349125
I tried to add the dual stack feature to all clients. However, when I
compiled a debug version of iceberg-spark-runtime-3.1 (with some logging
message) and tested it on Glue 3.0, the following error occurred:
The script I used is the following:
```
import org.apache.spark.SparkContext
import org.apache.spark.sql.SparkSession
import org.apache.iceberg.Table
import org.apache.iceberg.aws.glue.GlueCatalog
import org.apache.iceberg.catalog.Catalog
import org.apache.iceberg.catalog.TableIdentifier
import org.apache.iceberg.spark.actions.SparkActions
import scala.jdk.CollectionConverters._
object GlueApp {
def main(sysArgs: Array[String]) {
val sparkContext: SparkContext = new SparkContext()
val spark: SparkSession = SparkSession.builder.
config("spark.sql.catalog.demo",
"org.apache.iceberg.spark.SparkCatalog").
config("spark.sql.catalog.demo.warehouse",
"s3://myicebergtest/glueiceberg2").
config("spark.sql.catalog.demo.catalog-impl",
"org.apache.iceberg.aws.glue.GlueCatalog").
config("spark.sql.catalog.demo.client.dualstack-enabled", "true").
getOrCreate()
spark.sql("CREATE DATABASE IF NOT EXISTS demo.reviews")
val book_reviews_location =
"s3://amazon-reviews-pds/parquet/product_category=Books/*.parquet"
val book_reviews = spark.read.parquet(book_reviews_location)
book_reviews.writeTo("demo.reviews.book_reviews2").
tableProperty("format-version", "2").
createOrReplace()
// read using SQL
// spark.sql("SELECT * FROM demo.reviews.book_reviews2").show()
}
}
```
The job is designed to read from a DEMO S3 bucket into an S3 bucket that
lives in my account.
At the job initialization stage, I got the following message indicating that
this job enabled dual stack on a glue client:
```
2022-08-28 00:14:50,293 INFO [main]
aws.AwsClientFactories$DefaultAwsClientFactory
(AwsClientFactories.java:glue(133)): The glue dual stack is enabled
```
Then, an error occurs:
```
Exception in User Class:
software.amazon.awssdk.core.exception.SdkClientException : Received an
UnknownHostException when attempting to interact with a service. See cause for
the exact endpoint that is failing to resolve. If this is happening on an
endpoint that previously worked, there may be a network connectivity issue or
your DNS cache could be storing endpoints for too long.
```
implying that the dual-stack endpoint is not valid in this connection.
To further verify this issue, I changed
```
config("spark.sql.catalog.demo.client.dualstack-enabled", "true").
```
to
```
config("spark.sql.catalog.demo.client.dualstack-enabled", "false").
```
and then the error disappeared, the job runs successfully.
After that, I rollbacked to my initial commit, which only enables dual stack
on S3 client and did the test again. This time I got
```
2022-08-28 00:35:01,362 INFO [Executor task launch worker for task 15.0 in
stage 1.0 (TID 16)] aws.AwsClientFactories$DefaultAwsClientFactory
(AwsClientFactories.java:s3(111)): The s3 dual stack is enabled
```
indicating that the dual stack is enabled for this client.
The job runs successfully this time.
Hence, it seems that although `dualStackEnabled()` can be used by all AWS
clients, only the S3 client can have it manually set to true safely.
Thus, I think maybe we should set a unique dual stack flag for each AWS
client in the future and only add the flag for S3 clients in this PR
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]