Skip to content

Conversation

@kbendick
Copy link
Contributor

@kbendick kbendick commented Jul 7, 2021

…er catalog

Allows users to set hadoop configuration overrides on any Iceberg tables that come from an Iceberg enabled Spark catalog.

Users specify the configurations similar to specifying global hadoop configuration overrides on the spark session via the spark config.

E.g. for a catalog foo, to override a hadoop config property fs.s3a.max.connections for iceberg table's in that catalog, a config would be added to the spark session config via --conf spark.sql.catalog.foo.hadoop.fs.s3a.max.connections=4.

For now this only works for Spark catalogs, in the future we should consider making this possible for other engines.

This closes #2607

@github-actions github-actions bot added the spark label Jul 7, 2021
@kbendick
Copy link
Contributor Author

kbendick commented Jul 7, 2021

@kbendick kbendick force-pushed the pull-spark-sql-hadoop-confs-per-catalog branch 5 times, most recently from 5192b6d to f174ae7 Compare July 7, 2021 22:54
@kbendick kbendick force-pushed the pull-spark-sql-hadoop-confs-per-catalog branch from d743768 to d18bae1 Compare July 8, 2021 00:51
@kbendick
Copy link
Contributor Author

kbendick commented Jul 8, 2021

@rdblue Could we possibly add this to the 0.12 release track? It's a relatively small change but it would be beneficial to us to have this released with 0.12 for users who might maintain their own environments.

If not possible, no worries.

@flyrain
Copy link
Contributor

flyrain commented Jul 8, 2021

I believe this is per catalog, not per table, right? Does it also support table level setting?

E.g. for a table foo, to override a hadoop config property fs.s3a.max.connections, a config would be added to the spark session config via --conf spark.sql.catalog.foo.hadoop.fs.s3a.max.connections=4.

@kbendick
Copy link
Contributor Author

kbendick commented Jul 9, 2021

I believe this is per catalog, not per table, right? Does it also support table level setting?

E.g. for a table foo, to override a hadoop config property fs.s3a.max.connections, a config would be added to the spark session config via --conf spark.sql.catalog.foo.hadoop.fs.s3a.max.connections=4.

That's correct. I have updated the description to mention catalog instead (as there's no way to change the hadoop config for multiple Iceberg tables in the same catalog).

Copy link
Contributor

@flyrain flyrain left a comment

Choose a reason for hiding this comment

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

LGTM

@kbendick kbendick force-pushed the pull-spark-sql-hadoop-confs-per-catalog branch from de67e05 to 16634c3 Compare July 9, 2021 23:53
Copy link
Contributor

@nastra nastra left a comment

Choose a reason for hiding this comment

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

LGTM with some nits

Copy link
Member

@szehon-ho szehon-ho left a comment

Choose a reason for hiding this comment

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

Hi, I added just a few comments

@kbendick kbendick force-pushed the pull-spark-sql-hadoop-confs-per-catalog branch from b0b981b to 605870f Compare July 15, 2021 22:07
@kbendick
Copy link
Contributor Author

I just rebased and fixed the merge conflicts. I've also added a synchronized block @szehon-ho.

@kbendick kbendick force-pushed the pull-spark-sql-hadoop-confs-per-catalog branch from bc48137 to 746434e Compare July 15, 2021 23:06
@kbendick kbendick force-pushed the pull-spark-sql-hadoop-confs-per-catalog branch from 746434e to 0adbea5 Compare July 15, 2021 23:11
@kbendick
Copy link
Contributor Author

@pvary @marton-bod would you mind taking a look at this as you were in the original discussion on the issue (and are much more informed on the subject of hadoop configs than myself 🙂 )

@RussellSpitzer
Copy link
Member

Solves #2607 - Thanks @kbendick! Thanks @pvary, @nastra , @flyrain and @szehon-ho for reviewing!

@RussellSpitzer RussellSpitzer merged commit 1b3dbb6 into apache:master Jul 19, 2021
minchowang pushed a commit to minchowang/iceberg that referenced this pull request Aug 2, 2021
…apache#2792)

Previously Iceberg Catalogs loaded into Spark would always use the Hadoop Configuration owned by the underlying Spark Session. This made it impossible to use a different set of configuration values which may be required to connect to a remote Catalog. This patch allows Spark catalogs to have hadoop configuration overrides per catalog permitting different configuration for different underlying Iceberg catalogs.
chenjunjiedada pushed a commit to chenjunjiedada/incubator-iceberg that referenced this pull request Oct 20, 2021
Merge remote-tracking branch 'upstream/merge-master-20210816' into master
## 该MR主要解决什么?

merge upstream/master,引入最近的一些bugFix和优化

## 该MR的修改是什么?

核心关注PR:
> Predicate PushDown 支持,https://github.com/apache/iceberg/pull/2358, https://github.com/apache/iceberg/pull/2926, https://github.com/apache/iceberg/pull/2777/files
> Spark场景写入空dataset 报错问题,直接skip掉即可, apache#2960
> Flink UI补充uidPrefix到operator方便跟踪多个iceberg sink任务, apache#288
> Spark 修复nested Struct Pruning问题, apache#2877
> 可以使用Table Properties指定创建v2 format表,apache#2887
> 补充SortRewriteStrategy框架,逐步支持不同rewrite策略, apache#2609 (WIP:apache#2829)
> Spark 为catalog配置hadoop属性支持, apache#2792
> Spark 针对timestamps without timezone读写支持, apache#2757
> Spark MicroBatch支持配置属性skip delete snapshots, apache#2752
> Spark V2 RewriteDatafilesAction 支持
> Core: Add validation for row-level deletes with rewrites, apache#2865 > schema time travel 功能相关,补充schema-id, Core: add schema id to snapshot 
> Spark Extension支持identifier fields操作, apache#2560
> Parquet: Update to 1.12.0, apache#2441
> Hive: Vectorized ORC reads for Hive, apache#2613
> Spark: Add an action to remove all referenced files, apache#2415

## 该MR是如何测试的?

UT
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Allow pass through of catalog options to the Hadoop configuration

6 participants