17
17
18
18
package org .apache .kyuubi .spark .connector .hive
19
19
20
+ import java .lang .{Boolean => JBoolean , Long => JLong }
20
21
import java .net .URI
21
22
import java .util
22
23
23
24
import scala .collection .JavaConverters ._
24
25
import scala .collection .mutable
26
+ import scala .util .Try
25
27
26
28
import org .apache .hadoop .conf .Configuration
27
29
import org .apache .spark .SparkConf
28
30
import org .apache .spark .internal .Logging
29
31
import org .apache .spark .sql .SparkSession
30
- import org .apache .spark .sql .catalyst .{SQLConfHelper , TableIdentifier }
32
+ import org .apache .spark .sql .catalyst .{CurrentUserContext , SQLConfHelper , TableIdentifier }
31
33
import org .apache .spark .sql .catalyst .analysis .{NamespaceAlreadyExistsException , NoSuchDatabaseException , NoSuchNamespaceException , NoSuchTableException , TableAlreadyExistsException }
32
34
import org .apache .spark .sql .catalyst .catalog ._
33
35
import org .apache .spark .sql .catalyst .catalog .CatalogTypes .TablePartitionSpec
@@ -47,6 +49,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
47
49
import org .apache .kyuubi .spark .connector .hive .HiveConnectorUtils .withSparkSQLConf
48
50
import org .apache .kyuubi .spark .connector .hive .HiveTableCatalog .{getStorageFormatAndProvider , toCatalogDatabase , CatalogDatabaseHelper , IdentifierHelper , NamespaceHelper }
49
51
import org .apache .kyuubi .spark .connector .hive .KyuubiHiveConnectorDelegationTokenProvider .metastoreTokenSignature
52
+ import org .apache .kyuubi .util .reflect .{DynClasses , DynConstructors }
50
53
51
54
/**
52
55
* A [[TableCatalog ]] that wrap HiveExternalCatalog to as V2 CatalogPlugin instance to access Hive.
@@ -100,6 +103,20 @@ class HiveTableCatalog(sparkSession: SparkSession)
100
103
catalogName
101
104
}
102
105
106
+ private def newHiveMetastoreCatalog (sparkSession : SparkSession ): HiveMetastoreCatalog = {
107
+ val sparkSessionClz = DynClasses .builder()
108
+ .impl(" org.apache.spark.sql.classic.SparkSession" ) // SPARK-49700 (4.0.0)
109
+ .impl(" org.apache.spark.sql.SparkSession" )
110
+ .buildChecked()
111
+
112
+ val hiveMetastoreCatalogCtor =
113
+ DynConstructors .builder()
114
+ .impl(" org.apache.spark.sql.hive.HiveMetastoreCatalog" , sparkSessionClz)
115
+ .buildChecked[HiveMetastoreCatalog ]()
116
+
117
+ hiveMetastoreCatalogCtor.newInstanceChecked(sparkSession)
118
+ }
119
+
103
120
override def initialize (name : String , options : CaseInsensitiveStringMap ): Unit = {
104
121
assert(catalogName == null , " The Hive table catalog is already initialed." )
105
122
assert(
@@ -110,7 +127,7 @@ class HiveTableCatalog(sparkSession: SparkSession)
110
127
catalog = new HiveSessionCatalog (
111
128
externalCatalogBuilder = () => externalCatalog,
112
129
globalTempViewManagerBuilder = () => globalTempViewManager,
113
- metastoreCatalog = new HiveMetastoreCatalog (sparkSession),
130
+ metastoreCatalog = newHiveMetastoreCatalog (sparkSession),
114
131
functionRegistry = sessionState.functionRegistry,
115
132
tableFunctionRegistry = sessionState.tableFunctionRegistry,
116
133
hadoopConf = hadoopConf,
@@ -166,6 +183,129 @@ class HiveTableCatalog(sparkSession: SparkSession)
166
183
HiveTable (sparkSession, catalog.getTableMetadata(ident.asTableIdentifier), this )
167
184
}
168
185
186
+ // scalastyle:off
187
+ private def newCatalogTable (
188
+ identifier : TableIdentifier ,
189
+ tableType : CatalogTableType ,
190
+ storage : CatalogStorageFormat ,
191
+ schema : StructType ,
192
+ provider : Option [String ] = None ,
193
+ partitionColumnNames : Seq [String ] = Seq .empty,
194
+ bucketSpec : Option [BucketSpec ] = None ,
195
+ owner : String = Option (CurrentUserContext .CURRENT_USER .get()).getOrElse(" " ),
196
+ createTime : JLong = System .currentTimeMillis,
197
+ lastAccessTime : JLong = - 1 ,
198
+ createVersion : String = " " ,
199
+ properties : Map [String , String ] = Map .empty,
200
+ stats : Option [CatalogStatistics ] = None ,
201
+ viewText : Option [String ] = None ,
202
+ comment : Option [String ] = None ,
203
+ collation : Option [String ] = None ,
204
+ unsupportedFeatures : Seq [String ] = Seq .empty,
205
+ tracksPartitionsInCatalog : JBoolean = false ,
206
+ schemaPreservesCase : JBoolean = true ,
207
+ ignoredProperties : Map [String , String ] = Map .empty,
208
+ viewOriginalText : Option [String ] = None ): CatalogTable = {
209
+ // scalastyle:on
210
+ Try { // SPARK-50675 (4.0.0)
211
+ DynConstructors .builder()
212
+ .impl(
213
+ classOf [CatalogTable ],
214
+ classOf [TableIdentifier ],
215
+ classOf [CatalogTableType ],
216
+ classOf [CatalogStorageFormat ],
217
+ classOf [StructType ],
218
+ classOf [Option [String ]],
219
+ classOf [Seq [String ]],
220
+ classOf [Option [BucketSpec ]],
221
+ classOf [String ],
222
+ classOf [Long ],
223
+ classOf [Long ],
224
+ classOf [String ],
225
+ classOf [Map [String , String ]],
226
+ classOf [Option [CatalogStatistics ]],
227
+ classOf [Option [String ]],
228
+ classOf [Option [String ]],
229
+ classOf [Option [String ]],
230
+ classOf [Seq [String ]],
231
+ classOf [Boolean ],
232
+ classOf [Boolean ],
233
+ classOf [Map [String , String ]],
234
+ classOf [Option [String ]])
235
+ .buildChecked()
236
+ .invokeChecked[CatalogTable ](
237
+ null ,
238
+ identifier,
239
+ tableType,
240
+ storage,
241
+ schema,
242
+ provider,
243
+ partitionColumnNames,
244
+ bucketSpec,
245
+ owner,
246
+ createTime,
247
+ lastAccessTime,
248
+ createVersion,
249
+ properties,
250
+ stats,
251
+ viewText,
252
+ comment,
253
+ collation,
254
+ unsupportedFeatures,
255
+ tracksPartitionsInCatalog,
256
+ schemaPreservesCase,
257
+ ignoredProperties,
258
+ viewOriginalText)
259
+ }.recover { case _ : Exception => // Spark 3.5 and previous
260
+ DynConstructors .builder()
261
+ .impl(
262
+ classOf [CatalogTable ],
263
+ classOf [TableIdentifier ],
264
+ classOf [CatalogTableType ],
265
+ classOf [CatalogStorageFormat ],
266
+ classOf [StructType ],
267
+ classOf [Option [String ]],
268
+ classOf [Seq [String ]],
269
+ classOf [Option [BucketSpec ]],
270
+ classOf [String ],
271
+ classOf [Long ],
272
+ classOf [Long ],
273
+ classOf [String ],
274
+ classOf [Map [String , String ]],
275
+ classOf [Option [CatalogStatistics ]],
276
+ classOf [Option [String ]],
277
+ classOf [Option [String ]],
278
+ classOf [Seq [String ]],
279
+ classOf [Boolean ],
280
+ classOf [Boolean ],
281
+ classOf [Map [String , String ]],
282
+ classOf [Option [String ]])
283
+ .buildChecked()
284
+ .invokeChecked[CatalogTable ](
285
+ null ,
286
+ identifier,
287
+ tableType,
288
+ storage,
289
+ schema,
290
+ provider,
291
+ partitionColumnNames,
292
+ bucketSpec,
293
+ owner,
294
+ createTime,
295
+ lastAccessTime,
296
+ createVersion,
297
+ properties,
298
+ stats,
299
+ viewText,
300
+ comment,
301
+ unsupportedFeatures,
302
+ tracksPartitionsInCatalog,
303
+ schemaPreservesCase,
304
+ ignoredProperties,
305
+ viewOriginalText)
306
+ }.get
307
+ }
308
+
169
309
override def createTable (
170
310
ident : Identifier ,
171
311
schema : StructType ,
@@ -190,7 +330,7 @@ class HiveTableCatalog(sparkSession: SparkSession)
190
330
CatalogTableType .MANAGED
191
331
}
192
332
193
- val tableDesc = CatalogTable (
333
+ val tableDesc = newCatalogTable (
194
334
identifier = ident.asTableIdentifier,
195
335
tableType = tableType,
196
336
storage = storage,
0 commit comments