forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 2
/
hive3-support.patch
282 lines (274 loc) · 11.9 KB
/
hive3-support.patch
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
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
index 496d16d..28c6081 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
@@ -26,6 +26,7 @@ import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
+import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.common.StatsSetupConst
import org.apache.hadoop.hive.conf.HiveConf
@@ -86,7 +87,7 @@ private[hive] class HiveClientImpl(
override val version: HiveVersion,
warehouseDir: Option[String],
sparkConf: SparkConf,
- hadoopConf: JIterable[JMap.Entry[String, String]],
+ hadoopConf: Configuration,
extraConfig: Map[String, String],
initClassLoader: ClassLoader,
val clientLoader: IsolatedClientLoader)
@@ -107,8 +108,19 @@ private[hive] class HiveClientImpl(
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()
}
+ if (version == hive.v3_0) {
+ hadoopConf.set("hive.execution.engine", "mr")
+ }
+
+ if (version == hive.v3_1) {
+ hadoopConf.set("hive.execution.engine", "mr")
+ }
+
+
// Create an internal session state for this HiveClientImpl.
val state: SessionState = {
val original = Thread.currentThread().getContextClassLoader
@@ -169,15 +181,14 @@ private[hive] class HiveClientImpl(
// has hive-site.xml. So, HiveConf will use that to override its default values.
// 2: we set all spark confs to this hiveConf.
// 3: we set all entries in config to this hiveConf.
- val confMap = (hadoopConf.iterator().asScala.map(kv => kv.getKey -> kv.getValue) ++
- sparkConf.getAll.toMap ++ extraConfig).toMap
- confMap.foreach { case (k, v) => hiveConf.set(k, v) }
- SQLConf.get.redactOptions(confMap).foreach { case (k, v) =>
+ (hadoopConf.iterator().asScala.map(kv => kv.getKey -> kv.getValue)
+ ++ sparkConf.getAll.toMap ++ extraConfig).foreach { case (k, v) =>
logDebug(
s"""
|Applying Hadoop/Hive/Spark and extra properties to Hive Conf:
- |$k=$v
+ |$k=${if (k.toLowerCase(Locale.ROOT).contains("password")) "xxx" else v}
""".stripMargin)
+ hiveConf.set(k, v)
}
val state = new SessionState(hiveConf)
if (clientLoader.cachedHive != null) {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
index cee795d..244fda4 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
@@ -20,7 +20,8 @@ package org.apache.spark.sql.hive.client
import java.lang.{Boolean => JBoolean, Integer => JInteger, Long => JLong}
import java.lang.reflect.{InvocationTargetException, Method, Modifier}
import java.net.URI
-import java.util.{ArrayList => JArrayList, List => JList, Locale, Map => JMap, Set => JSet}
+import java.util.Locale
+import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JSet}
import java.util.concurrent.TimeUnit
import scala.collection.JavaConverters._
@@ -39,7 +40,7 @@ import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.serde.serdeConstants
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.sql.catalyst.FunctionIdentifier
import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException
import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, CatalogUtils, FunctionResource, FunctionResourceType}
@@ -1181,3 +1182,163 @@ private[client] class Shim_v2_1 extends Shim_v2_0 {
private[client] class Shim_v2_2 extends Shim_v2_1
private[client] class Shim_v2_3 extends Shim_v2_1
+
+private[client] class Shim_v3_0 extends Shim_v2_3 {
+ // Naive Spark does not support transactional operations.
+ protected lazy val transactional = JBoolean.FALSE
+
+ // Spark supports only non-ACID operations
+ protected lazy val isAcidIUDoperation = JBoolean.FALSE
+
+ // Writer ID can be 0 for non-ACID operations
+ protected lazy val writeIdInLoadTableOrPartition: JLong = 0L
+
+ // Statement ID
+ protected lazy val stmtIdInLoadTableOrPartition: JInteger = 0
+
+ protected lazy val listBucketingLevel: JInteger = 0
+
+ private lazy val clazzLoadFileType = getClass.getClassLoader.loadClass(
+ "org.apache.hadoop.hive.ql.plan.LoadTableDesc$LoadFileType")
+
+ private lazy val loadPartitionMethod =
+ findMethod(
+ classOf[Hive],
+ "loadPartition",
+ classOf[Path],
+ classOf[Table],
+ classOf[JMap[String, String]],
+ clazzLoadFileType,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ classOf[JLong],
+ JInteger.TYPE,
+ JBoolean.TYPE)
+ private lazy val loadTableMethod =
+ findMethod(
+ classOf[Hive],
+ "loadTable",
+ classOf[Path],
+ classOf[String],
+ clazzLoadFileType,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ classOf[JLong],
+ JInteger.TYPE,
+ JBoolean.TYPE)
+ private lazy val loadDynamicPartitionsMethod =
+ findMethod(
+ classOf[Hive],
+ "loadDynamicPartitions",
+ classOf[Path],
+ classOf[String],
+ classOf[JMap[String, String]],
+ clazzLoadFileType,
+ JInteger.TYPE,
+ JInteger.TYPE,
+ JBoolean.TYPE,
+ JLong.TYPE,
+ JInteger.TYPE,
+ JBoolean.TYPE,
+ classOf[AcidUtils.Operation],
+ JBoolean.TYPE)
+ private lazy val alterTableMethod =
+ findMethod(
+ classOf[Hive],
+ "alterTable",
+ classOf[String],
+ classOf[Table],
+ classOf[EnvironmentContext],
+ JBoolean.TYPE)
+ private lazy val alterPartitionsMethod =
+ findMethod(
+ classOf[Hive],
+ "alterPartitions",
+ classOf[String],
+ classOf[JList[Partition]],
+ classOf[EnvironmentContext],
+ JBoolean.TYPE)
+
+ override def loadPartition(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ inheritTableSpecs: Boolean,
+ isSkewedStoreAsSubdir: Boolean,
+ isSrcLocal: Boolean): Unit = {
+ val session = SparkSession.getActiveSession
+ assert(session.nonEmpty)
+ val database = session.get.sessionState.catalog.getCurrentDatabase
+ val table = hive.getTable(database, tableName)
+ val loadFileType = if (replace) {
+ clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("REPLACE_ALL"))
+ } else {
+ clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("KEEP_EXISTING"))
+ }
+ assert(loadFileType.isDefined)
+ // Looks we should handle HIVE-19891 but looks no way to access the appropriate value here.
+ // Here, it just uses 'inheritTableSpecs' to keep the existing behaviour.
+ val inheritLocation = inheritTableSpecs
+ loadPartitionMethod.invoke(hive, loadPath, table, partSpec, loadFileType.get,
+ inheritTableSpecs: JBoolean, inheritLocation: JBoolean, isSkewedStoreAsSubdir: JBoolean,
+ isSrcLocal: JBoolean, isAcid, hasFollowingStatsTask,
+ writeIdInLoadTableOrPartition, stmtIdInLoadTableOrPartition, replace: JBoolean)
+ }
+
+
+ override def loadTable(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ replace: Boolean,
+ isSrcLocal: Boolean): Unit = {
+ val loadFileType = if (replace) {
+ clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("REPLACE_ALL"))
+ } else {
+ clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("KEEP_EXISTING"))
+ }
+ assert(loadFileType.isDefined)
+ loadTableMethod.invoke(hive, loadPath, tableName, loadFileType.get, isSrcLocal: JBoolean,
+ isSkewedStoreAsSubdir, isAcidIUDoperation, hasFollowingStatsTask,
+ writeIdInLoadTableOrPartition, stmtIdInLoadTableOrPartition: JInteger, replace: JBoolean)
+ }
+
+ override def loadDynamicPartitions(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ numDP: Int,
+ listBucketingEnabled: Boolean): Unit = {
+ val loadFileType = if (replace) {
+ clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("REPLACE_ALL"))
+ } else {
+ clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("KEEP_EXISTING"))
+ }
+ assert(loadFileType.isDefined)
+ loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, loadFileType.get,
+ numDP: JInteger, listBucketingLevel, isAcid, writeIdInLoadTableOrPartition,
+ stmtIdInLoadTableOrPartition, hasFollowingStatsTask, AcidUtils.Operation.NOT_ACID,
+ replace: JBoolean)
+ }
+
+ override def alterTable(hive: Hive, tableName: String, table: Table): Unit = {
+ alterTableMethod.invoke(hive, tableName, table, environmentContextInAlterTable, transactional)
+ }
+
+ override def alterPartitions(hive: Hive, tableName: String, newParts: JList[Partition]): Unit = {
+ alterPartitionsMethod.invoke(
+ hive, tableName, newParts, environmentContextInAlterTable, transactional)
+ }
+}
+
+private[client] class Shim_v3_1 extends Shim_v3_0
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
index 8a12673..bf1b324 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
@@ -99,6 +99,8 @@ private[hive] object IsolatedClientLoader extends Logging {
case "2.1" | "2.1.0" | "2.1.1" => hive.v2_1
case "2.2" | "2.2.0" => hive.v2_2
case "2.3" | "2.3.0" | "2.3.1" | "2.3.2" | "2.3.3" => hive.v2_3
+ case "3.0" | "3.0.0" => hive.v3_0
+ case "3.1" | "3.1.0" | "3.1.1" | "3.1.2" => hive.v3_1
}
private def downloadVersion(
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
index 25e9886..0b3d5bb 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
@@ -79,7 +79,15 @@ package object client {
exclusions = Seq("org.apache.curator:*",
"org.pentaho:pentaho-aggdesigner-algorithm"))
- val allSupportedHiveVersions = Set(v12, v13, v14, v1_0, v1_1, v1_2, v2_0, v2_1, v2_2, v2_3)
+ case object v3_0 extends HiveVersion("3.0.0",
+ exclusions = Seq("org.apache.curator:*",
+ "org.pentaho:pentaho-aggdesigner-algorithm"))
+
+ case object v3_1 extends HiveVersion("3.1.0",
+ exclusions = Seq("org.apache.curator:*",
+ "org.pentaho:pentaho-aggdesigner-algorithm"))
+
+ val allSupportedHiveVersions = Set(v12, v13, v14, v1_0, v1_1, v1_2, v2_0, v2_1, v2_2, v2_3, v3_0, v3_1)
}
// scalastyle:on