/
HiveClientImpl.scala
1308 lines (1197 loc) · 53.8 KB
/
HiveClientImpl.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hive.client
import java.io.PrintStream
import java.lang.{Iterable => JIterable}
import java.lang.reflect.InvocationTargetException
import java.nio.charset.StandardCharsets.UTF_8
import java.util.{Locale, Map => JMap}
import java.util.concurrent.TimeUnit._
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.common.StatsSetupConst
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hadoop.hive.metastore.{IMetaStoreClient, TableType => HiveTableType}
import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, Table => MetaStoreApiTable, _}
import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable}
import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC
import org.apache.hadoop.hive.ql.processors._
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.serde.serdeConstants
import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe
import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
import org.apache.hadoop.security.UserGroupInformation
import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.internal.Logging
import org.apache.spark.metrics.source.HiveCatalogMetrics
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException, NoSuchPartitionsException, PartitionAlreadyExistsException, PartitionsAlreadyExistException}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException}
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.connector.catalog.SupportsNamespaces._
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.hive.HiveExternalCatalog
import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_SCHEMA
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.util.{CircularBuffer, ShutdownHookManager, Utils}
/**
* A class that wraps the HiveClient and converts its responses to externally visible classes.
* Note that this class is typically loaded with an internal classloader for each instantiation,
* allowing it to interact directly with a specific isolated version of Hive. Loading this class
* with the isolated classloader however will result in it only being visible as a [[HiveClient]],
* not a [[HiveClientImpl]].
*
* This class needs to interact with multiple versions of Hive, but will always be compiled with
* the 'native', execution version of Hive. Therefore, any places where hive breaks compatibility
* must use reflection after matching on `version`.
*
* Every HiveClientImpl creates an internal HiveConf object. This object is using the given
* `hadoopConf` as the base. All options set in the `sparkConf` will be applied to the HiveConf
* object and overrides any exiting options. Then, options in extraConfig will be applied
* to the HiveConf object and overrides any existing options.
*
* @param version the version of hive used when pick function calls that are not compatible.
* @param sparkConf all configuration options set in SparkConf.
* @param hadoopConf the base Configuration object used by the HiveConf created inside
* this HiveClientImpl.
* @param extraConfig a collection of configuration options that will be added to the
* hive conf before opening the hive client.
* @param initClassLoader the classloader used when creating the `state` field of
* this [[HiveClientImpl]].
*/
private[hive] class HiveClientImpl(
override val version: HiveVersion,
warehouseDir: Option[String],
sparkConf: SparkConf,
hadoopConf: JIterable[JMap.Entry[String, String]],
extraConfig: Map[String, String],
initClassLoader: ClassLoader,
val clientLoader: IsolatedClientLoader)
extends HiveClient
with Logging {
import HiveClientImpl._
// Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur.
private val outputBuffer = new CircularBuffer()
private val shim = version match {
case hive.v12 => new Shim_v0_12()
case hive.v13 => new Shim_v0_13()
case hive.v14 => new Shim_v0_14()
case hive.v1_0 => new Shim_v1_0()
case hive.v1_1 => new Shim_v1_1()
case hive.v1_2 => new Shim_v1_2()
case hive.v2_0 => new Shim_v2_0()
case hive.v2_1 => new Shim_v2_1()
case hive.v2_2 => new Shim_v2_2()
case hive.v2_3 => new Shim_v2_3()
case hive.v3_0 => new Shim_v3_0()
case hive.v3_1 => new Shim_v3_1()
}
// Create an internal session state for this HiveClientImpl.
val state: SessionState = {
val original = Thread.currentThread().getContextClassLoader
if (clientLoader.isolationOn) {
// Switch to the initClassLoader.
Thread.currentThread().setContextClassLoader(initClassLoader)
try {
newState()
} finally {
Thread.currentThread().setContextClassLoader(original)
}
} else {
// Isolation off means we detect a CliSessionState instance in current thread.
// 1: Inside the spark project, we have already started a CliSessionState in
// `SparkSQLCLIDriver`, which contains configurations from command lines. Later, we call
// `SparkSQLEnv.init()` there, which would new a hive client again. so we should keep those
// configurations and reuse the existing instance of `CliSessionState`. In this case,
// SessionState.get will always return a CliSessionState.
// 2: In another case, a user app may start a CliSessionState outside spark project with built
// in hive jars, which will turn off isolation, if SessionSate.detachSession is
// called to remove the current state after that, hive client created later will initialize
// its own state by newState()
val ret = SessionState.get
if (ret != null) {
// hive.metastore.warehouse.dir is determined in SharedState after the CliSessionState
// instance constructed, we need to follow that change here.
warehouseDir.foreach { dir =>
ret.getConf.setVar(ConfVars.METASTOREWAREHOUSE, dir)
}
ret
} else {
newState()
}
}
}
private def closeState(): Unit = withHiveState {
// These temp files are registered in o.a.h.u.ShutdownHookManager too during state start.
// The state.close() will delete them if they are not null and try remove them from the
// o.a.h.u.ShutdownHookManager which causes undesirable IllegalStateException.
// We delete them ahead with a high priority hook here and set them to null to bypass the
// deletion in state.close().
if (state.getTmpOutputFile != null) {
state.getTmpOutputFile.delete()
state.setTmpOutputFile(null)
}
if (state.getTmpErrOutputFile != null) {
state.getTmpErrOutputFile.delete()
state.setTmpErrOutputFile(null)
}
state.close()
}
ShutdownHookManager.addShutdownHook(() => closeState())
// Log the default warehouse location.
logInfo(
s"Warehouse location for Hive client " +
s"(version ${version.fullVersion}) is ${conf.getVar(ConfVars.METASTOREWAREHOUSE)}")
private def newState(): SessionState = {
val hiveConf = newHiveConf(sparkConf, hadoopConf, extraConfig, Some(initClassLoader))
val state = new SessionState(hiveConf)
if (clientLoader.cachedHive != null) {
Hive.set(clientLoader.cachedHive.asInstanceOf[Hive])
}
// Hive 2.3 will set UDFClassLoader to hiveConf when initializing SessionState
// since HIVE-11878, and ADDJarsCommand will add jars to clientLoader.classLoader.
// For this reason we cannot load the jars added by ADDJarsCommand because of class loader
// got changed. We reset it to clientLoader.ClassLoader here.
state.getConf.setClassLoader(clientLoader.classLoader)
shim.setCurrentSessionState(state)
state.out = new PrintStream(outputBuffer, true, UTF_8.name())
state.err = new PrintStream(outputBuffer, true, UTF_8.name())
state
}
/** Returns the configuration for the current session. */
def conf: HiveConf = {
val hiveConf = state.getConf
// Hive changed the default of datanucleus.schema.autoCreateAll from true to false
// and hive.metastore.schema.verification from false to true since Hive 2.0.
// For details, see the JIRA HIVE-6113, HIVE-12463 and HIVE-1841.
// isEmbeddedMetaStore should not be true in the production environment.
// We hard-code hive.metastore.schema.verification and datanucleus.schema.autoCreateAll to allow
// bin/spark-shell, bin/spark-sql and sbin/start-thriftserver.sh to automatically create the
// Derby Metastore when running Spark in the non-production environment.
val isEmbeddedMetaStore = {
val msUri = hiveConf.getVar(ConfVars.METASTOREURIS)
val msConnUrl = hiveConf.getVar(ConfVars.METASTORECONNECTURLKEY)
(msUri == null || msUri.trim().isEmpty) &&
(msConnUrl != null && msConnUrl.startsWith("jdbc:derby"))
}
if (isEmbeddedMetaStore) {
hiveConf.setBoolean("hive.metastore.schema.verification", false)
hiveConf.setBoolean("datanucleus.schema.autoCreateAll", true)
}
hiveConf
}
override val userName = UserGroupInformation.getCurrentUser.getShortUserName
override def getConf(key: String, defaultValue: String): String = {
conf.get(key, defaultValue)
}
// We use hive's conf for compatibility.
private val retryLimit = conf.getIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES)
private val retryDelayMillis = shim.getMetastoreClientConnectRetryDelayMillis(conf)
/**
* Runs `f` with multiple retries in case the hive metastore is temporarily unreachable.
*/
private def retryLocked[A](f: => A): A = clientLoader.synchronized {
// Hive sometimes retries internally, so set a deadline to avoid compounding delays.
val deadline = System.nanoTime + (retryLimit * retryDelayMillis * 1e6).toLong
var numTries = 0
var caughtException: Exception = null
do {
numTries += 1
try {
return f
} catch {
case e: Exception if causedByThrift(e) =>
caughtException = e
logWarning(
"HiveClient got thrift exception, destroying client and retrying " +
s"(${retryLimit - numTries} tries remaining)", e)
clientLoader.cachedHive = null
Thread.sleep(retryDelayMillis)
}
} while (numTries <= retryLimit && System.nanoTime < deadline)
if (System.nanoTime > deadline) {
logWarning("Deadline exceeded")
}
throw caughtException
}
private def causedByThrift(e: Throwable): Boolean = {
var target = e
while (target != null) {
val msg = target.getMessage()
if (msg != null && msg.matches("(?s).*(TApplication|TProtocol|TTransport)Exception.*")) {
return true
}
target = target.getCause()
}
false
}
private def client: Hive = {
if (clientLoader.cachedHive != null) {
clientLoader.cachedHive.asInstanceOf[Hive]
} else {
val c = Hive.get(conf)
clientLoader.cachedHive = c
c
}
}
private def msClient: IMetaStoreClient = {
shim.getMSC(client)
}
/** Return the associated Hive [[SessionState]] of this [[HiveClientImpl]] */
override def getState: SessionState = withHiveState(state)
/**
* Runs `f` with ThreadLocal session state and classloaders configured for this version of hive.
*/
def withHiveState[A](f: => A): A = retryLocked {
val original = Thread.currentThread().getContextClassLoader
val originalConfLoader = state.getConf.getClassLoader
// We explicitly set the context class loader since "conf.setClassLoader" does
// not do that, and the Hive client libraries may need to load classes defined by the client's
// class loader. See SPARK-19804 for more details.
Thread.currentThread().setContextClassLoader(clientLoader.classLoader)
state.getConf.setClassLoader(clientLoader.classLoader)
// Set the thread local metastore client to the client associated with this HiveClientImpl.
Hive.set(client)
// Replace conf in the thread local Hive with current conf
// with the side-effect of Hive.get(conf) to avoid using out-of-date HiveConf.
// See discussion in https://github.com/apache/spark/pull/16826/files#r104606859
// for more details.
Hive.get(conf)
// setCurrentSessionState will use the classLoader associated
// with the HiveConf in `state` to override the context class loader of the current
// thread.
shim.setCurrentSessionState(state)
val ret = try {
f
} catch {
case e: NoClassDefFoundError if e.getMessage.contains("apache/hadoop/hive/serde2/SerDe") =>
throw new ClassNotFoundException("The SerDe interface removed since Hive 2.3(HIVE-15167)." +
" Please migrate your custom SerDes to Hive 2.3. See HIVE-15167 for more details.", e)
} finally {
state.getConf.setClassLoader(originalConfLoader)
Thread.currentThread().setContextClassLoader(original)
HiveCatalogMetrics.incrementHiveClientCalls(1)
}
ret
}
def setOut(stream: PrintStream): Unit = withHiveState {
state.out = stream
}
def setInfo(stream: PrintStream): Unit = withHiveState {
state.info = stream
}
def setError(stream: PrintStream): Unit = withHiveState {
state.err = stream
}
private def setCurrentDatabaseRaw(db: String): Unit = {
if (state.getCurrentDatabase != db) {
if (databaseExists(db)) {
state.setCurrentDatabase(db)
} else {
throw new NoSuchDatabaseException(db)
}
}
}
override def setCurrentDatabase(databaseName: String): Unit = withHiveState {
setCurrentDatabaseRaw(databaseName)
}
override def createDatabase(
database: CatalogDatabase,
ignoreIfExists: Boolean): Unit = withHiveState {
val hiveDb = toHiveDatabase(database, Some(userName))
client.createDatabase(hiveDb, ignoreIfExists)
}
override def dropDatabase(
name: String,
ignoreIfNotExists: Boolean,
cascade: Boolean): Unit = withHiveState {
client.dropDatabase(name, true, ignoreIfNotExists, cascade)
}
override def alterDatabase(database: CatalogDatabase): Unit = withHiveState {
if (!getDatabase(database.name).locationUri.equals(database.locationUri)) {
// SPARK-29260: Enable supported versions once it support altering database location.
if (!(version.equals(hive.v3_0) || version.equals(hive.v3_1))) {
throw new AnalysisException(
s"Hive ${version.fullVersion} does not support altering database location")
}
}
val hiveDb = toHiveDatabase(database)
client.alterDatabase(database.name, hiveDb)
}
private def toHiveDatabase(
database: CatalogDatabase, userName: Option[String] = None): HiveDatabase = {
val props = database.properties
val hiveDb = new HiveDatabase(
database.name,
database.description,
CatalogUtils.URIToString(database.locationUri),
(props -- Seq(PROP_OWNER)).asJava)
props.get(PROP_OWNER).orElse(userName).foreach { ownerName =>
shim.setDatabaseOwnerName(hiveDb, ownerName)
}
hiveDb
}
override def getDatabase(dbName: String): CatalogDatabase = withHiveState {
Option(client.getDatabase(dbName)).map { d =>
val params = Option(d.getParameters).map(_.asScala.toMap).getOrElse(Map()) ++
Map(PROP_OWNER -> shim.getDatabaseOwnerName(d))
CatalogDatabase(
name = d.getName,
description = Option(d.getDescription).getOrElse(""),
locationUri = CatalogUtils.stringToURI(d.getLocationUri),
properties = params)
}.getOrElse(throw new NoSuchDatabaseException(dbName))
}
override def databaseExists(dbName: String): Boolean = withHiveState {
client.databaseExists(dbName)
}
override def listDatabases(pattern: String): Seq[String] = withHiveState {
client.getDatabasesByPattern(pattern).asScala.toSeq
}
private def getRawTableOption(dbName: String, tableName: String): Option[HiveTable] = {
Option(client.getTable(dbName, tableName, false /* do not throw exception */))
}
private def getRawTablesByName(dbName: String, tableNames: Seq[String]): Seq[HiveTable] = {
try {
msClient.getTableObjectsByName(dbName, tableNames.asJava).asScala
.map(extraFixesForNonView).map(new HiveTable(_)).toSeq
} catch {
case ex: Exception =>
throw new HiveException(s"Unable to fetch tables of db $dbName", ex);
}
}
override def tableExists(dbName: String, tableName: String): Boolean = withHiveState {
getRawTableOption(dbName, tableName).nonEmpty
}
override def getTablesByName(
dbName: String,
tableNames: Seq[String]): Seq[CatalogTable] = withHiveState {
getRawTablesByName(dbName, tableNames).map(convertHiveTableToCatalogTable)
}
override def getTableOption(
dbName: String,
tableName: String): Option[CatalogTable] = withHiveState {
logDebug(s"Looking up $dbName.$tableName")
getRawTableOption(dbName, tableName).map(convertHiveTableToCatalogTable)
}
private def convertHiveTableToCatalogTable(h: HiveTable): CatalogTable = {
// Note: Hive separates partition columns and the schema, but for us the
// partition columns are part of the schema
val (cols, partCols) = try {
(h.getCols.asScala.map(fromHiveColumn), h.getPartCols.asScala.map(fromHiveColumn))
} catch {
case ex: SparkException =>
throw new SparkException(
s"${ex.getMessage}, db: ${h.getDbName}, table: ${h.getTableName}", ex)
}
val schema = StructType((cols ++ partCols).toSeq)
val bucketSpec = if (h.getNumBuckets > 0) {
val sortColumnOrders = h.getSortCols.asScala
// Currently Spark only supports columns to be sorted in ascending order
// but Hive can support both ascending and descending order. If all the columns
// are sorted in ascending order, only then propagate the sortedness information
// to downstream processing / optimizations in Spark
// TODO: In future we can have Spark support columns sorted in descending order
val allAscendingSorted = sortColumnOrders.forall(_.getOrder == HIVE_COLUMN_ORDER_ASC)
val sortColumnNames = if (allAscendingSorted) {
sortColumnOrders.map(_.getCol)
} else {
Seq.empty
}
Option(BucketSpec(h.getNumBuckets, h.getBucketCols.asScala.toSeq, sortColumnNames.toSeq))
} else {
None
}
// Skew spec and storage handler can't be mapped to CatalogTable (yet)
val unsupportedFeatures = ArrayBuffer.empty[String]
if (!h.getSkewedColNames.isEmpty) {
unsupportedFeatures += "skewed columns"
}
if (h.getStorageHandler != null) {
unsupportedFeatures += "storage handler"
}
if (h.getTableType == HiveTableType.VIRTUAL_VIEW && partCols.nonEmpty) {
unsupportedFeatures += "partitioned view"
}
val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
// Hive-generated Statistics are also recorded in ignoredProperties
val ignoredProperties = scala.collection.mutable.Map.empty[String, String]
for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
ignoredProperties += key -> value
}
val excludedTableProperties = HiveStatisticsProperties ++ Set(
// The property value of "comment" is moved to the dedicated field "comment"
"comment",
// For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added
// in the function toHiveTable.
"EXTERNAL"
)
val filteredProperties = properties.filterNot {
case (key, _) => excludedTableProperties.contains(key)
}
val comment = properties.get("comment")
CatalogTable(
identifier = TableIdentifier(h.getTableName, Option(h.getDbName)),
tableType = h.getTableType match {
case HiveTableType.EXTERNAL_TABLE => CatalogTableType.EXTERNAL
case HiveTableType.MANAGED_TABLE => CatalogTableType.MANAGED
case HiveTableType.VIRTUAL_VIEW => CatalogTableType.VIEW
case unsupportedType =>
val tableTypeStr = unsupportedType.toString.toLowerCase(Locale.ROOT).replace("_", " ")
throw new AnalysisException(s"Hive $tableTypeStr is not supported.")
},
schema = schema,
partitionColumnNames = partCols.map(_.name).toSeq,
// If the table is written by Spark, we will put bucketing information in table properties,
// and will always overwrite the bucket spec in hive metastore by the bucketing information
// in table properties. This means, if we have bucket spec in both hive metastore and
// table properties, we will trust the one in table properties.
bucketSpec = bucketSpec,
owner = Option(h.getOwner).getOrElse(""),
createTime = h.getTTable.getCreateTime.toLong * 1000,
lastAccessTime = h.getLastAccessTime.toLong * 1000,
storage = CatalogStorageFormat(
locationUri = shim.getDataLocation(h).map(CatalogUtils.stringToURI),
// To avoid ClassNotFound exception, we try our best to not get the format class, but get
// the class name directly. However, for non-native tables, there is no interface to get
// the format class name, so we may still throw ClassNotFound in this case.
inputFormat = Option(h.getTTable.getSd.getInputFormat).orElse {
Option(h.getStorageHandler).map(_.getInputFormatClass.getName)
},
outputFormat = Option(h.getTTable.getSd.getOutputFormat).orElse {
Option(h.getStorageHandler).map(_.getOutputFormatClass.getName)
},
serde = Option(h.getSerializationLib),
compressed = h.getTTable.getSd.isCompressed,
properties = Option(h.getTTable.getSd.getSerdeInfo.getParameters)
.map(_.asScala.toMap).orNull
),
// For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added
// in the function toHiveTable.
properties = filteredProperties,
stats = readHiveStats(properties),
comment = comment,
// In older versions of Spark(before 2.2.0), we expand the view original text and
// store that into `viewExpandedText`, that should be used in view resolution.
// We get `viewExpandedText` as viewText, and also get `viewOriginalText` in order to
// display the original view text in `DESC [EXTENDED|FORMATTED] table` command for views
// that created by older versions of Spark.
viewOriginalText = Option(h.getViewOriginalText),
viewText = Option(h.getViewExpandedText),
unsupportedFeatures = unsupportedFeatures.toSeq,
ignoredProperties = ignoredProperties.toMap)
}
override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState {
verifyColumnDataType(table.dataSchema)
client.createTable(toHiveTable(table, Some(userName)), ignoreIfExists)
}
override def dropTable(
dbName: String,
tableName: String,
ignoreIfNotExists: Boolean,
purge: Boolean): Unit = withHiveState {
shim.dropTable(client, dbName, tableName, true, ignoreIfNotExists, purge)
}
override def alterTable(
dbName: String,
tableName: String,
table: CatalogTable): Unit = withHiveState {
// getTableOption removes all the Hive-specific properties. Here, we fill them back to ensure
// these properties are still available to the others that share the same Hive metastore.
// If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect
// these user-specified values.
verifyColumnDataType(table.dataSchema)
val hiveTable = toHiveTable(
table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName))
// Do not use `table.qualifiedName` here because this may be a rename
val qualifiedTableName = s"$dbName.$tableName"
shim.alterTable(client, qualifiedTableName, hiveTable)
}
override def alterTableDataSchema(
dbName: String,
tableName: String,
newDataSchema: StructType,
schemaProps: Map[String, String]): Unit = withHiveState {
val oldTable = client.getTable(dbName, tableName)
verifyColumnDataType(newDataSchema)
val hiveCols = newDataSchema.map(toHiveColumn)
oldTable.setFields(hiveCols.asJava)
// remove old schema table properties
val it = oldTable.getParameters.entrySet.iterator
while (it.hasNext) {
val entry = it.next()
if (CatalogTable.isLargeTableProp(DATASOURCE_SCHEMA, entry.getKey)) {
it.remove()
}
}
// set new schema table properties
schemaProps.foreach { case (k, v) => oldTable.setProperty(k, v) }
val qualifiedTableName = s"$dbName.$tableName"
shim.alterTable(client, qualifiedTableName, oldTable)
}
override def createPartitions(
db: String,
table: String,
parts: Seq[CatalogTablePartition],
ignoreIfExists: Boolean): Unit = withHiveState {
def replaceExistException(e: Throwable): Unit = e match {
case _: HiveException if e.getCause.isInstanceOf[AlreadyExistsException] =>
throw new PartitionsAlreadyExistException(db, table, parts.map(_.spec))
case _ => throw e
}
try {
shim.createPartitions(client, db, table, parts, ignoreIfExists)
} catch {
case e: InvocationTargetException => replaceExistException(e.getCause)
case e: Throwable => replaceExistException(e)
}
}
override def dropPartitions(
db: String,
table: String,
specs: Seq[TablePartitionSpec],
ignoreIfNotExists: Boolean,
purge: Boolean,
retainData: Boolean): Unit = withHiveState {
// TODO: figure out how to drop multiple partitions in one call
val hiveTable = client.getTable(db, table, true /* throw exception */)
// do the check at first and collect all the matching partitions
val matchingParts =
specs.flatMap { s =>
assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
// The provided spec here can be a partial spec, i.e. it will match all partitions
// whose specs are supersets of this partial spec. E.g. If a table has partitions
// (b='1', c='1') and (b='1', c='2'), a partial spec of (b='1') will match both.
val parts = client.getPartitions(hiveTable, s.asJava).asScala
if (parts.isEmpty && !ignoreIfNotExists) {
throw new NoSuchPartitionsException(db, table, Seq(s))
}
parts.map(_.getValues)
}.distinct
var droppedParts = ArrayBuffer.empty[java.util.List[String]]
matchingParts.foreach { partition =>
try {
shim.dropPartition(client, db, table, partition, !retainData, purge)
} catch {
case e: Exception =>
val remainingParts = matchingParts.toBuffer --= droppedParts
logError(
s"""
|======================
|Attempt to drop the partition specs in table '$table' database '$db':
|${specs.mkString("\n")}
|In this attempt, the following partitions have been dropped successfully:
|${droppedParts.mkString("\n")}
|The remaining partitions have not been dropped:
|${remainingParts.mkString("\n")}
|======================
""".stripMargin)
throw e
}
droppedParts += partition
}
}
override def renamePartitions(
db: String,
table: String,
specs: Seq[TablePartitionSpec],
newSpecs: Seq[TablePartitionSpec]): Unit = withHiveState {
require(specs.size == newSpecs.size, "number of old and new partition specs differ")
val catalogTable = getTable(db, table)
val hiveTable = toHiveTable(catalogTable, Some(userName))
specs.zip(newSpecs).foreach { case (oldSpec, newSpec) =>
if (client.getPartition(hiveTable, newSpec.asJava, false) != null) {
throw new PartitionAlreadyExistsException(db, table, newSpec)
}
val hivePart = getPartitionOption(catalogTable, oldSpec)
.map { p => toHivePartition(p.copy(spec = newSpec), hiveTable) }
.getOrElse { throw new NoSuchPartitionException(db, table, oldSpec) }
client.renamePartition(hiveTable, oldSpec.asJava, hivePart)
}
}
override def alterPartitions(
db: String,
table: String,
newParts: Seq[CatalogTablePartition]): Unit = withHiveState {
// Note: Before altering table partitions in Hive, you *must* set the current database
// to the one that contains the table of interest. Otherwise you will end up with the
// most helpful error message ever: "Unable to alter partition. alter is not possible."
// See HIVE-2742 for more detail.
val original = state.getCurrentDatabase
try {
setCurrentDatabaseRaw(db)
val hiveTable = toHiveTable(getTable(db, table), Some(userName))
shim.alterPartitions(client, table, newParts.map { toHivePartition(_, hiveTable) }.asJava)
} finally {
state.setCurrentDatabase(original)
}
}
/**
* Returns the partition names for the given table that match the supplied partition spec.
* If no partition spec is specified, all partitions are returned.
*
* The returned sequence is sorted as strings.
*/
override def getPartitionNames(
table: CatalogTable,
partialSpec: Option[TablePartitionSpec] = None): Seq[String] = withHiveState {
val hivePartitionNames =
partialSpec match {
case None =>
// -1 for result limit means "no limit/return all"
client.getPartitionNames(table.database, table.identifier.table, -1)
case Some(s) =>
assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
client.getPartitionNames(table.database, table.identifier.table, s.asJava, -1)
}
hivePartitionNames.asScala.sorted.toSeq
}
override def getPartitionOption(
table: CatalogTable,
spec: TablePartitionSpec): Option[CatalogTablePartition] = withHiveState {
val hiveTable = toHiveTable(table, Some(userName))
val hivePartition = client.getPartition(hiveTable, spec.asJava, false)
Option(hivePartition).map(fromHivePartition)
}
/**
* Returns the partitions for the given table that match the supplied partition spec.
* If no partition spec is specified, all partitions are returned.
*/
override def getPartitions(
table: CatalogTable,
spec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = withHiveState {
val hiveTable = toHiveTable(table, Some(userName))
val partSpec = spec match {
case None => CatalogTypes.emptyTablePartitionSpec
case Some(s) =>
assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
s
}
val parts = client.getPartitions(hiveTable, partSpec.asJava).asScala.map(fromHivePartition)
HiveCatalogMetrics.incrementFetchedPartitions(parts.length)
parts.toSeq
}
override def getPartitionsByFilter(
table: CatalogTable,
predicates: Seq[Expression],
timeZoneId: String): Seq[CatalogTablePartition] = withHiveState {
val hiveTable = toHiveTable(table, Some(userName))
val parts = shim.getPartitionsByFilter(client, hiveTable, predicates, timeZoneId)
.map(fromHivePartition)
HiveCatalogMetrics.incrementFetchedPartitions(parts.length)
parts
}
override def listTables(dbName: String): Seq[String] = withHiveState {
client.getAllTables(dbName).asScala.toSeq
}
override def listTables(dbName: String, pattern: String): Seq[String] = withHiveState {
client.getTablesByPattern(dbName, pattern).asScala.toSeq
}
override def listTablesByType(
dbName: String,
pattern: String,
tableType: CatalogTableType): Seq[String] = withHiveState {
val hiveTableType = toHiveTableType(tableType)
try {
// Try with Hive API getTablesByType first, it's supported from Hive 2.3+.
shim.getTablesByType(client, dbName, pattern, hiveTableType)
} catch {
case _: UnsupportedOperationException =>
// Fallback to filter logic if getTablesByType not supported.
val tableNames = client.getTablesByPattern(dbName, pattern).asScala
getRawTablesByName(dbName, tableNames.toSeq)
.filter(_.getTableType == hiveTableType)
.map(_.getTableName)
}
}
/**
* Runs the specified SQL query using Hive.
*/
override def runSqlHive(sql: String): Seq[String] = {
val maxResults = 100000
val results = runHive(sql, maxResults)
// It is very confusing when you only get back some of the results...
if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED")
results
}
/**
* Execute the command using Hive and return the results as a sequence. Each element
* in the sequence is one row.
* Since upgrading the built-in Hive to 2.3, hive-llap-client is needed when
* running MapReduce jobs with `runHive`.
* Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false.
*/
protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = withHiveState {
def closeDriver(driver: Driver): Unit = {
// Since HIVE-18238(Hive 3.0.0), the Driver.close function's return type changed
// and the CommandProcessorFactory.clean function removed.
driver.getClass.getMethod("close").invoke(driver)
if (version != hive.v3_0 && version != hive.v3_1) {
CommandProcessorFactory.clean(conf)
}
}
// Hive query needs to start SessionState.
SessionState.start(state)
logDebug(s"Running hiveql '$cmd'")
if (cmd.toLowerCase(Locale.ROOT).startsWith("set")) { logDebug(s"Changing config: $cmd") }
try {
val cmd_trimmed: String = cmd.trim()
val tokens: Array[String] = cmd_trimmed.split("\\s+")
// The remainder of the command.
val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim()
val proc = shim.getCommandProcessor(tokens(0), conf)
proc match {
case driver: Driver =>
val response: CommandProcessorResponse = driver.run(cmd)
// Throw an exception if there is an error in query processing.
if (response.getResponseCode != 0) {
closeDriver(driver)
throw new QueryExecutionException(response.getErrorMessage)
}
driver.setMaxRows(maxRows)
val results = shim.getDriverResults(driver)
closeDriver(driver)
results
case _ =>
if (state.out != null) {
// scalastyle:off println
state.out.println(tokens(0) + " " + cmd_1)
// scalastyle:on println
}
val response: CommandProcessorResponse = proc.run(cmd_1)
// Throw an exception if there is an error in query processing.
if (response.getResponseCode != 0) {
throw new QueryExecutionException(response.getErrorMessage)
}
Seq(response.getResponseCode.toString)
}
} catch {
case e: Exception =>
logError(
s"""
|======================
|HIVE FAILURE OUTPUT
|======================
|${outputBuffer.toString}
|======================
|END HIVE FAILURE OUTPUT
|======================
""".stripMargin)
throw e
}
}
def loadPartition(
loadPath: String,
dbName: String,
tableName: String,
partSpec: java.util.LinkedHashMap[String, String],
replace: Boolean,
inheritTableSpecs: Boolean,
isSrcLocal: Boolean): Unit = withHiveState {
val hiveTable = client.getTable(dbName, tableName, true /* throw exception */)
shim.loadPartition(
client,
new Path(loadPath), // TODO: Use URI
s"$dbName.$tableName",
partSpec,
replace,
inheritTableSpecs,
isSkewedStoreAsSubdir = hiveTable.isStoredAsSubDirectories,
isSrcLocal = isSrcLocal)
}
def loadTable(
loadPath: String, // TODO URI
tableName: String,
replace: Boolean,
isSrcLocal: Boolean): Unit = withHiveState {
shim.loadTable(
client,
new Path(loadPath),
tableName,
replace,
isSrcLocal)
}
def loadDynamicPartitions(
loadPath: String,
dbName: String,
tableName: String,
partSpec: java.util.LinkedHashMap[String, String],
replace: Boolean,
numDP: Int): Unit = withHiveState {
val hiveTable = client.getTable(dbName, tableName, true /* throw exception */)
shim.loadDynamicPartitions(
client,
new Path(loadPath),
s"$dbName.$tableName",
partSpec,
replace,
numDP,
listBucketingEnabled = hiveTable.isStoredAsSubDirectories)
}
override def createFunction(db: String, func: CatalogFunction): Unit = withHiveState {
shim.createFunction(client, db, func)
}
override def dropFunction(db: String, name: String): Unit = withHiveState {
shim.dropFunction(client, db, name)
}
override def renameFunction(db: String, oldName: String, newName: String): Unit = withHiveState {
shim.renameFunction(client, db, oldName, newName)
}
override def alterFunction(db: String, func: CatalogFunction): Unit = withHiveState {
shim.alterFunction(client, db, func)
}
override def getFunctionOption(
db: String, name: String): Option[CatalogFunction] = withHiveState {
shim.getFunctionOption(client, db, name)
}
override def listFunctions(db: String, pattern: String): Seq[String] = withHiveState {
shim.listFunctions(client, db, pattern)
}
def addJar(path: String): Unit = {
val jarURI = Utils.resolveURI(path)
clientLoader.addJar(jarURI.toURL)
}
def newSession(): HiveClientImpl = {
clientLoader.createClient().asInstanceOf[HiveClientImpl]
}
def reset(): Unit = withHiveState {
val allTables = client.getAllTables("default")
val (mvs, others) = allTables.asScala.map(t => client.getTable("default", t))
.partition(_.getTableType.toString.equals("MATERIALIZED_VIEW"))
// Remove materialized view first, otherwise caused a violation of foreign key constraint.
mvs.foreach { table =>
val t = table.getTableName
logDebug(s"Deleting materialized view $t")
client.dropTable("default", t)
}
others.foreach { table =>
val t = table.getTableName
logDebug(s"Deleting table $t")
try {
client.getIndexes("default", t, 255).asScala.foreach { index =>
shim.dropIndex(client, "default", t, index.getIndexName)
}
if (!table.isIndexTable) {
client.dropTable("default", t)
}
} catch {
case _: NoSuchMethodError =>
// HIVE-18448 Hive 3.0 remove index APIs
client.dropTable("default", t)
}
}
client.getAllDatabases.asScala.filterNot(_ == "default").foreach { db =>
logDebug(s"Dropping Database: $db")
client.dropDatabase(db, true, false, true)
}