3636import org .apache .spark .sql .types .StructType ;
3737import java .util .List ;
3838import java .util .Locale ;
39+ import java .util .Map ;
3940import java .util .Optional ;
4041
4142import static com .netflix .iceberg .TableProperties .DEFAULT_FILE_FORMAT ;
@@ -53,16 +54,18 @@ public String shortName() {
5354
5455 @ Override
5556 public DataSourceReader createReader (DataSourceOptions options ) {
56- Table table = findTable (options );
57- return new Reader (table , lazyConf ());
57+ Configuration conf = new Configuration (lazyBaseConf ());
58+ Table table = getTableAndResolveHadoopConfiguration (options , conf );
59+
60+ return new Reader (table , conf );
5861 }
5962
6063 @ Override
6164 public Optional <DataSourceWriter > createWriter (String jobId , StructType dfStruct , SaveMode mode ,
6265 DataSourceOptions options ) {
6366 Preconditions .checkArgument (mode == SaveMode .Append , "Save mode %s is not supported" , mode );
64-
65- Table table = findTable (options );
67+ Configuration conf = new Configuration ( lazyBaseConf ());
68+ Table table = getTableAndResolveHadoopConfiguration (options , conf );
6669
6770 Schema dfSchema = SparkSchemaUtil .convert (table .schema (), dfStruct );
6871 List <String > errors = CheckCompatibility .writeCompatibilityErrors (table .schema (), dfSchema );
@@ -86,30 +89,49 @@ public Optional<DataSourceWriter> createWriter(String jobId, StructType dfStruct
8689 .toUpperCase (Locale .ENGLISH ));
8790 }
8891
89- return Optional .of (new Writer (table , lazyConf () , format ));
92+ return Optional .of (new Writer (table , conf , format ));
9093 }
9194
92- protected Table findTable (DataSourceOptions options ) {
95+ protected Table findTable (DataSourceOptions options , Configuration conf ) {
9396 Optional <String > location = options .get ("path" );
9497 Preconditions .checkArgument (location .isPresent (),
9598 "Cannot open table without a location: path is not set" );
9699
97- HadoopTables tables = new HadoopTables (lazyConf () );
100+ HadoopTables tables = new HadoopTables (conf );
98101
99102 return tables .load (location .get ());
100103 }
101104
102- protected SparkSession lazySparkSession () {
105+ private SparkSession lazySparkSession () {
103106 if (lazySpark == null ) {
104107 this .lazySpark = SparkSession .builder ().getOrCreate ();
105108 }
106109 return lazySpark ;
107110 }
108111
109- protected Configuration lazyConf () {
112+ private Configuration lazyBaseConf () {
110113 if (lazyConf == null ) {
111114 this .lazyConf = lazySparkSession ().sparkContext ().hadoopConfiguration ();
112115 }
113116 return lazyConf ;
114117 }
118+
119+ private Table getTableAndResolveHadoopConfiguration (
120+ DataSourceOptions options , Configuration conf ) {
121+ // Overwrite configurations from the Spark Context with configurations from the options.
122+ mergeIcebergHadoopConfs (conf , options .asMap ());
123+ Table table = findTable (options , conf );
124+ // Set confs from table properties
125+ mergeIcebergHadoopConfs (conf , table .properties ());
126+ // Re-overwrite values set in options and table properties but were not in the environment.
127+ mergeIcebergHadoopConfs (conf , options .asMap ());
128+ return table ;
129+ }
130+
131+ private static void mergeIcebergHadoopConfs (
132+ Configuration baseConf , Map <String , String > options ) {
133+ options .keySet ().stream ()
134+ .filter (key -> key .startsWith ("iceberg.hadoop" ))
135+ .forEach (key -> baseConf .set (key .replaceFirst ("iceberg.hadoop" , "" ), options .get (key )));
136+ }
115137}
0 commit comments