28
28
import org .apache .iceberg .relocated .com .google .common .base .Preconditions ;
29
29
import org .apache .iceberg .relocated .com .google .common .util .concurrent .MoreExecutors ;
30
30
import org .apache .iceberg .relocated .com .google .common .util .concurrent .ThreadFactoryBuilder ;
31
+ import org .apache .iceberg .spark .BaseCatalog ;
31
32
import org .apache .iceberg .spark .Spark3Util ;
32
33
import org .apache .iceberg .spark .Spark3Util .CatalogAndIdentifier ;
33
34
import org .apache .iceberg .spark .actions .SparkActions ;
@@ -58,14 +59,14 @@ abstract class BaseProcedure implements Procedure {
58
59
protected static final DataType STRING_ARRAY = DataTypes .createArrayType (DataTypes .StringType );
59
60
60
61
private final SparkSession spark ;
61
- private final TableCatalog tableCatalog ;
62
+ private final BaseCatalog catalog ;
62
63
63
64
private SparkActions actions ;
64
65
private ExecutorService executorService = null ;
65
66
66
- protected BaseProcedure (TableCatalog tableCatalog ) {
67
+ protected BaseProcedure (BaseCatalog catalog ) {
67
68
this .spark = SparkSession .active ();
68
- this .tableCatalog = tableCatalog ;
69
+ this .catalog = catalog ;
69
70
}
70
71
71
72
protected SparkSession spark () {
@@ -80,7 +81,7 @@ protected SparkActions actions() {
80
81
}
81
82
82
83
protected TableCatalog tableCatalog () {
83
- return this .tableCatalog ;
84
+ return this .catalog ;
84
85
}
85
86
86
87
protected <T > T modifyIcebergTable (Identifier ident , Function <org .apache .iceberg .Table , T > func ) {
@@ -115,38 +116,38 @@ private <T> T execute(
115
116
116
117
protected Identifier toIdentifier (String identifierAsString , String argName ) {
117
118
CatalogAndIdentifier catalogAndIdentifier =
118
- toCatalogAndIdentifier (identifierAsString , argName , tableCatalog );
119
+ toCatalogAndIdentifier (identifierAsString , argName , catalog );
119
120
120
121
Preconditions .checkArgument (
121
- catalogAndIdentifier .catalog ().equals (tableCatalog ),
122
+ catalogAndIdentifier .catalog ().equals (catalog ),
122
123
"Cannot run procedure in catalog '%s': '%s' is a table in catalog '%s'" ,
123
- tableCatalog .name (),
124
+ catalog .name (),
124
125
identifierAsString ,
125
126
catalogAndIdentifier .catalog ().name ());
126
127
127
128
return catalogAndIdentifier .identifier ();
128
129
}
129
130
130
131
protected CatalogAndIdentifier toCatalogAndIdentifier (
131
- String identifierAsString , String argName , CatalogPlugin catalog ) {
132
+ String identifierAsString , String argName , CatalogPlugin catalogPlugin ) {
132
133
Preconditions .checkArgument (
133
134
identifierAsString != null && !identifierAsString .isEmpty (),
134
135
"Cannot handle an empty identifier for argument %s" ,
135
136
argName );
136
137
137
138
return Spark3Util .catalogAndIdentifier (
138
- "identifier for arg " + argName , spark , identifierAsString , catalog );
139
+ "identifier for arg " + argName , spark , identifierAsString , catalogPlugin );
139
140
}
140
141
141
142
protected SparkTable loadSparkTable (Identifier ident ) {
142
143
try {
143
- Table table = tableCatalog .loadTable (ident );
144
+ Table table = catalog .loadTable (ident );
144
145
ValidationException .check (
145
146
table instanceof SparkTable , "%s is not %s" , ident , SparkTable .class .getName ());
146
147
return (SparkTable ) table ;
147
148
} catch (NoSuchTableException e ) {
148
149
String errMsg =
149
- String .format ("Couldn't load table '%s' in catalog '%s'" , ident , tableCatalog .name ());
150
+ String .format ("Couldn't load table '%s' in catalog '%s'" , ident , catalog .name ());
150
151
throw new RuntimeException (errMsg , e );
151
152
}
152
153
}
@@ -159,13 +160,13 @@ protected Dataset<Row> loadRows(Identifier tableIdent, Map<String, String> optio
159
160
protected void refreshSparkCache (Identifier ident , Table table ) {
160
161
CacheManager cacheManager = spark .sharedState ().cacheManager ();
161
162
DataSourceV2Relation relation =
162
- DataSourceV2Relation .create (table , Option .apply (tableCatalog ), Option .apply (ident ));
163
+ DataSourceV2Relation .create (table , Option .apply (catalog ), Option .apply (ident ));
163
164
cacheManager .recacheByPlan (spark , relation );
164
165
}
165
166
166
167
protected Expression filterExpression (Identifier ident , String where ) {
167
168
try {
168
- String name = Spark3Util .quotedFullIdentifier (tableCatalog .name (), ident );
169
+ String name = Spark3Util .quotedFullIdentifier (catalog .name (), ident );
169
170
org .apache .spark .sql .catalyst .expressions .Expression expression =
170
171
SparkExpressionConverter .collectResolvedSparkExpression (spark , name , where );
171
172
return SparkExpressionConverter .convertToIcebergExpression (expression );
@@ -179,11 +180,11 @@ protected InternalRow newInternalRow(Object... values) {
179
180
}
180
181
181
182
protected abstract static class Builder <T extends BaseProcedure > implements ProcedureBuilder {
182
- private TableCatalog tableCatalog ;
183
+ private BaseCatalog catalog ;
183
184
184
185
@ Override
185
- public Builder <T > withTableCatalog ( TableCatalog newTableCatalog ) {
186
- this .tableCatalog = newTableCatalog ;
186
+ public Builder <T > withCatalog ( BaseCatalog newCatalog ) {
187
+ this .catalog = newCatalog ;
187
188
return this ;
188
189
}
189
190
@@ -194,8 +195,8 @@ public T build() {
194
195
195
196
protected abstract T doBuild ();
196
197
197
- TableCatalog tableCatalog () {
198
- return tableCatalog ;
198
+ BaseCatalog catalog () {
199
+ return catalog ;
199
200
}
200
201
}
201
202
0 commit comments