Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[1094] Fix config for S3 multi-cluster mode #1095

Conversation

scottsand-db
Copy link
Collaborator

@scottsand-db scottsand-db commented Apr 22, 2022

Problem Description

  • problem 1: there was a mismatch between the config params in the documentation (io.delta.storage.S3DynamoDBLogStore.ddb.tableName) and the config params actually used in the code + in the integration test (io.delta.storage.ddb.tableName).
  • solution 1: include the S3DynamoDBLogStore string in the config in the code.
  • problem 2: the io.delta.storage prefix didn't work when specified using --conf. they DID work using spark.conf.set() or SparkSession.builder.config() but not with --conf.
  • solution 2: we now allow 2 prefixes.
    • spark.io.delta.storage.S3DynamoDBLogStore.$key..... this will work all contexts (--conf, spark.conf.set(), etc).
    • io.delta.storage.S3DynamoDBLogStore.$key. this is the original prefix. this will be able to be used by delta-standalone and flink and hive. they just use hadoopConfs and don't need to have prefix starting with spark

PR Description

  • resolves [BUG] issues configuring AWS S3 multi-cluster mode #1094
  • update config for S3 multi-cluster mode (i.e. S3DynamoDBLogStore) to match the public documentation. the configs were missing the string literal S3DynamoDBLogStore from the conf prefix
  • now supports 2 confs. spark.io.delta.storage.S3DynamoDBLogStore.$key and io.delta.storage.S3DynamoDBLogStore.$key.

How was this patch tested?

  • added unit test for the 2 new confs
  • ran integration test using existing + new s3 table, and using existing + new ddb table
  • ran manual tests using locally published jars on pyspark + spark-shell + spark-submit (spark-submit via integration test)
  • tested using --conf as well as spark.conf.set as well as SparkSession.builder.config()

Does this PR introduce any user-facing changes?

Not really, just fixes a just-released, experimental LogStore config key

@scottsand-db scottsand-db self-assigned this Apr 22, 2022
@scottsand-db scottsand-db changed the title [WIP - NEED TO QA] [1094] Fix config for S3 multi-cluster mode [1094] Fix config for S3 multi-cluster mode Apr 22, 2022
@scottsand-db scottsand-db requested review from tdas and vkorukanti April 22, 2022 22:40
@@ -45,7 +45,7 @@
export DELTA_NUM_ROWS=16

./run-integration-tests.py --run-storage-s3-dynamodb-integration-tests \
--dbb-packages org.apache.hadoop:hadoop-aws:3.3.1,com.amazonaws:aws-java-sdk-bundle:1.12.142 \
--dbb-packages org.apache.hadoop:hadoop-aws:3.3.1 \
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

com.amazonaws:aws-java-sdk-bundle:1.12.142 is a compile dependency for org.apache.hadoop:hadoop-aws:3.3.1

Comment on lines +89 to +91
.config("spark.delta.logStore.s3.impl", delta_storage) \
.config("spark.delta.logStore.s3a.impl", delta_storage) \
.config("spark.delta.logStore.s3n.impl", delta_storage) \
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let's idiot-proof this to work with s3, s3a, s3n

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

*/
private static final String CONF_PREFIX = "io.delta.storage.";
private static final String DBB_CLIENT_TABLE = "ddb.tableName";
private static final String DBB_CLIENT_REGION = "ddb.region";
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

renamed from DBB to DDB :)

@scottsand-db
Copy link
Collaborator Author

Test Failed

[info] org.apache.spark.sql.delta.SnapshotManagementSuite *** ABORTED *** (19 seconds, 956 milliseconds)
[info]   The code passed to eventually never returned normally. Attempted 15 times over 10.012527203 seconds. Last failure message: There are 1 possibly leaked file streams.. (SharedSparkSession.scala:164)
[info]   org.scalatest.exceptions.TestFailedDueToTimeoutException:
[info]   at org.scalatest.enablers.Retrying$$anon$4.tryTryAgain$2(Retrying.scala:189)
[info]   at org.scalatest.enablers.Retrying$$anon$4.retry(Retrying.scala:196)
[info]   at org.scalatest.concurrent.Eventually.eventually(Eventually.scala:313)
[info]   at org.scalatest.concurrent.Eventually.eventually$(Eventually.scala:312)
[info]   at org.apache.spark.sql.delta.SnapshotManagementSuite.eventually(SnapshotManagementSuite.scala:31)
[info]   at org.apache.spark.sql.test.SharedSparkSessionBase.afterEach(SharedSparkSession.scala:164)
[info]   at org.apache.spark.sql.test.SharedSparkSessionBase.afterEach$(SharedSparkSession.scala:158)
[info]   at org.apache.spark.sql.delta.SnapshotManagementSuite.afterEach(SnapshotManagementSuite.scala:31)
[info]   at org.scalatest.BeforeAndAfterEach.$anonfun$runTest$1(BeforeAndAfterEach.scala:247)

test("java LogStore is instantiated with hadoopConf with SQLConf values") {
val sparkConf = new SparkConf()
.setMaster("local")
// equivalent to --conf spark.delta.storage.custom.key=foo
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

turns out this just is not true

val basePrefixKey = "io.delta.storage.S3DynamoDBLogStore.ddb.tableName"

// Sanity check
assert(sparkPrefixKey == SPARK_CONF_PREFIX + "." + DDB_CLIENT_TABLE)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: typically these should be require as this failure does not mean some thing is wrong in the production code... this failure means something is wrong in the test code.

scottsand-db added a commit to scottsand-db/delta that referenced this pull request Apr 26, 2022
- problem 1: there was a mismatch between the config params in the documentation (`io.delta.storage.S3DynamoDBLogStore.ddb.tableName`) and the config params actually used in the code + in the integration test (`io.delta.storage.ddb.tableName`).
- solution 1: include the `S3DynamoDBLogStore` string in the config in the code.
- problem 2: the `io.delta.storage` prefix didn't work when specified using `--conf`. they DID work using `spark.conf.set()` or `SparkSession.builder.config()` but not with `--conf`.
- solution 2: we now allow 2 prefixes.
  - `spark.io.delta.storage.S3DynamoDBLogStore.$key.....` this will work all contexts (`--conf`, `spark.conf.set()`, etc).
  - `io.delta.storage.S3DynamoDBLogStore.$key`. this is the original prefix. this will be able to be used by delta-standalone and flink and hive. they just use hadoopConfs and don't need to have prefix starting with `spark`

- resolves delta-io#1094
- update config for S3 multi-cluster mode (i.e. S3DynamoDBLogStore) to match the public documentation. the configs were missing the string literal `S3DynamoDBLogStore` from the conf prefix
- now supports 2 confs. `spark.io.delta.storage.S3DynamoDBLogStore.$key` and `io.delta.storage.S3DynamoDBLogStore.$key`.

- added unit test for the 2 new confs
- ran integration test using existing + new s3 table, and using existing + new ddb table
- ran manual tests using locally published jars on pyspark + spark-shell + spark-submit (spark-submit via integration test)
- tested using `--conf` as well as `spark.conf.set` as well as `SparkSession.builder.config()`

Not really, just fixes a just-released, experimental LogStore config key

Closes delta-io#1095

Signed-off-by: Scott Sandre <[email protected]>
GitOrigin-RevId: e5db1e6b0dfe958e3234644462891f269313ca33
scottsand-db added a commit that referenced this pull request Apr 26, 2022
- problem 1: there was a mismatch between the config params in the documentation (`io.delta.storage.S3DynamoDBLogStore.ddb.tableName`) and the config params actually used in the code + in the integration test (`io.delta.storage.ddb.tableName`).
- solution 1: include the `S3DynamoDBLogStore` string in the config in the code.
- problem 2: the `io.delta.storage` prefix didn't work when specified using `--conf`. they DID work using `spark.conf.set()` or `SparkSession.builder.config()` but not with `--conf`.
- solution 2: we now allow 2 prefixes.
  - `spark.io.delta.storage.S3DynamoDBLogStore.$key.....` this will work all contexts (`--conf`, `spark.conf.set()`, etc).
  - `io.delta.storage.S3DynamoDBLogStore.$key`. this is the original prefix. this will be able to be used by delta-standalone and flink and hive. they just use hadoopConfs and don't need to have prefix starting with `spark`

- resolves #1094
- update config for S3 multi-cluster mode (i.e. S3DynamoDBLogStore) to match the public documentation. the configs were missing the string literal `S3DynamoDBLogStore` from the conf prefix
- now supports 2 confs. `spark.io.delta.storage.S3DynamoDBLogStore.$key` and `io.delta.storage.S3DynamoDBLogStore.$key`.

- added unit test for the 2 new confs
- ran integration test using existing + new s3 table, and using existing + new ddb table
- ran manual tests using locally published jars on pyspark + spark-shell + spark-submit (spark-submit via integration test)
- tested using `--conf` as well as `spark.conf.set` as well as `SparkSession.builder.config()`

Not really, just fixes a just-released, experimental LogStore config key

Closes #1095

Signed-off-by: Scott Sandre <[email protected]>
GitOrigin-RevId: e5db1e6b0dfe958e3234644462891f269313ca33
@vkorukanti vkorukanti added this to the 2.0 milestone Jun 28, 2022
jbguerraz pushed a commit to jbguerraz/delta that referenced this pull request Jul 6, 2022
## Problem Description
- problem 1: there was a mismatch between the config params in the documentation (`io.delta.storage.S3DynamoDBLogStore.ddb.tableName`) and the config params actually used in the code + in the integration test (`io.delta.storage.ddb.tableName`).
- solution 1: include the `S3DynamoDBLogStore` string in the config in the code.
- problem 2: the `io.delta.storage` prefix didn't work when specified using `--conf`. they DID work using `spark.conf.set()` or `SparkSession.builder.config()` but not with `--conf`.
- solution 2: we now allow 2 prefixes.
  - `spark.io.delta.storage.S3DynamoDBLogStore.$key.....` this will work all contexts (`--conf`, `spark.conf.set()`, etc).
  - `io.delta.storage.S3DynamoDBLogStore.$key`. this is the original prefix. this will be able to be used by delta-standalone and flink and hive. they just use hadoopConfs and don't need to have prefix starting with `spark`

## PR Description
- resolves delta-io#1094
- update config for S3 multi-cluster mode (i.e. S3DynamoDBLogStore) to match the public documentation. the configs were missing the string literal `S3DynamoDBLogStore` from the conf prefix
- now supports 2 confs. `spark.io.delta.storage.S3DynamoDBLogStore.$key` and `io.delta.storage.S3DynamoDBLogStore.$key`.

- added unit test for the 2 new confs
- ran integration test using existing + new s3 table, and using existing + new ddb table
- ran manual tests using locally published jars on pyspark + spark-shell + spark-submit (spark-submit via integration test)
- tested using `--conf` as well as `spark.conf.set` as well as `SparkSession.builder.config()`

## Does this PR introduce _any_ user-facing changes?

Not really, just fixes a just-released, experimental LogStore config key

Closes delta-io#1095

Signed-off-by: Scott Sandre <[email protected]>
GitOrigin-RevId: e5db1e6b0dfe958e3234644462891f269313ca33
jbguerraz pushed a commit to jbguerraz/delta that referenced this pull request Jul 6, 2022
## Problem Description
- problem 1: there was a mismatch between the config params in the documentation (`io.delta.storage.S3DynamoDBLogStore.ddb.tableName`) and the config params actually used in the code + in the integration test (`io.delta.storage.ddb.tableName`).
- solution 1: include the `S3DynamoDBLogStore` string in the config in the code.
- problem 2: the `io.delta.storage` prefix didn't work when specified using `--conf`. they DID work using `spark.conf.set()` or `SparkSession.builder.config()` but not with `--conf`.
- solution 2: we now allow 2 prefixes.
  - `spark.io.delta.storage.S3DynamoDBLogStore.$key.....` this will work all contexts (`--conf`, `spark.conf.set()`, etc).
  - `io.delta.storage.S3DynamoDBLogStore.$key`. this is the original prefix. this will be able to be used by delta-standalone and flink and hive. they just use hadoopConfs and don't need to have prefix starting with `spark`

## PR Description
- resolves delta-io#1094
- update config for S3 multi-cluster mode (i.e. S3DynamoDBLogStore) to match the public documentation. the configs were missing the string literal `S3DynamoDBLogStore` from the conf prefix
- now supports 2 confs. `spark.io.delta.storage.S3DynamoDBLogStore.$key` and `io.delta.storage.S3DynamoDBLogStore.$key`.

- added unit test for the 2 new confs
- ran integration test using existing + new s3 table, and using existing + new ddb table
- ran manual tests using locally published jars on pyspark + spark-shell + spark-submit (spark-submit via integration test)
- tested using `--conf` as well as `spark.conf.set` as well as `SparkSession.builder.config()`

## Does this PR introduce _any_ user-facing changes?

Not really, just fixes a just-released, experimental LogStore config key

Closes delta-io#1095

Signed-off-by: Scott Sandre <[email protected]>
GitOrigin-RevId: e5db1e6b0dfe958e3234644462891f269313ca33
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[BUG] issues configuring AWS S3 multi-cluster mode
3 participants