diff --git a/assembly/pom.xml b/assembly/pom.xml
index 3a6b73b4522d..34fa9f8077c4 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -255,16 +255,49 @@
spark-hive_${scala.binary.version}
${project.version}
+
+ org.eclipse.jetty
+ jetty-util
+ runtime
+
hive-thriftserver
+
+ provided
+
org.apache.spark
spark-hive-thriftserver_${scala.binary.version}
${project.version}
+
+ org.eclipse.jetty
+ jetty-server
+ runtime
+
+
+ org.eclipse.jetty
+ jetty-security
+ runtime
+
+
+ org.eclipse.jetty
+ jetty-servlet
+ runtime
+
+
+ org.apache.datasketches
+ datasketches-hive
+ runtime
+
+
+ ${hive.group}
+ hive-llap-common
+ runtime
+
@@ -306,6 +339,7 @@
provided
provided
+ provided
diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh
index 68c61232ea2a..03c4f7fad2eb 100755
--- a/dev/test-dependencies.sh
+++ b/dev/test-dependencies.sh
@@ -34,7 +34,7 @@ HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pkubernetes -Pyarn -Phive \
-Pspark-ganglia-lgpl -Pkinesis-asl -Phadoop-cloud -Pjvm-profiler"
MVN="build/mvn"
HADOOP_HIVE_PROFILES=(
- hadoop-3-hive-2.3
+ hadoop-3-hive-4.0
)
MVN_EXEC_PLUGIN_VERSION=$(build/mvn help:evaluate \
@@ -85,7 +85,7 @@ $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /de
# Generate manifests for each Hadoop profile:
for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do
- if [[ $HADOOP_HIVE_PROFILE == **hadoop-3-hive-2.3** ]]; then
+ if [[ $HADOOP_HIVE_PROFILE == **hadoop-3-hive-4.0** ]]; then
HADOOP_PROFILE=hadoop-3
fi
echo "Performing Maven install for $HADOOP_HIVE_PROFILE"
diff --git a/dev/test-jars.txt b/dev/test-jars.txt
index bd8fc93bc9f0..58e2bbef0b8d 100644
--- a/dev/test-jars.txt
+++ b/dev/test-jars.txt
@@ -10,7 +10,6 @@ sql/connect/common/src/test/resources/artifact-tests/smallJar.jar
sql/core/src/test/resources/SPARK-33084.jar
sql/core/src/test/resources/artifact-tests/udf_noA.jar
sql/hive-thriftserver/src/test/resources/TestUDTF.jar
-sql/hive/src/test/noclasspath/hive-test-udfs.jar
sql/hive/src/test/resources/SPARK-21101-1.0.jar
sql/hive/src/test/resources/TestUDTF.jar
sql/hive/src/test/resources/data/files/TestSerDe.jar
diff --git a/pom.xml b/pom.xml
index 96e67c7a2697..0d77dea0c52e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -134,7 +134,8 @@
org.apache.hive
core
- 2.3.10
+ 4.1.0
+ ${hive.version}
3.9.1
@@ -196,7 +197,7 @@
3.18.0
2.12.1
- 4.1.17
+ 6.0.11
33.4.0-jre
2.11.0
3.1.9
@@ -205,7 +206,7 @@
3.0.0
2.2.11
0.16.0
- 4.13.1
+ 4.9.3
1.1
4.21.0
4.21.0
@@ -276,9 +277,9 @@
needing Hadoop classes in the classpath to compile).
-->
compile
+ compile
compile
compile
- 2.8.1
compile
compile
test
@@ -335,6 +336,7 @@
-Djdk.reflect.useDirectMethodHandle=false
-Dio.netty.tryReflectionSetAccessible=true
--enable-native-access=ALL-UNNAMED
+ -Dmvn.executable=${maven.multiModuleProjectDirectory}/build/mvn
2.7.12
9.2.0
@@ -830,6 +832,11 @@
log4j-slf4j2-impl
${log4j.version}
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+ ${log4j.version}
+
org.apache.logging.log4j
log4j-api
@@ -1647,6 +1654,10 @@
${hive.group}
hive-metastore
+
+ ${hive.group}
+ hive-standalone-metastore-server
+
${hive.group}
hive-service
@@ -1679,6 +1690,10 @@
commons-logging
commons-logging
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+
@@ -1743,6 +1758,10 @@
${hive.version}
${hive.deps.scope}
+
+ ${hive.group}
+ hive-standalone-metastore-common
+
${hive.group}
hive-shims
@@ -1807,6 +1826,14 @@
${hive.group}
hive-storage-api
+
+ org.apache.hadoop
+ hadoop-yarn-common
+
+
+ org.apache.hadoop
+ hadoop-hdfs-client
+
@@ -1852,8 +1879,8 @@
ant
- com.esotericsoftware.kryo
- kryo
+ org.datanucleus
+ datanucleus-core
commons-codec
@@ -1863,11 +1890,6 @@
org.apache.avro
avro-mapred
-
-
- org.apache.calcite
- calcite-core
-
org.apache.calcite
calcite-avatica
@@ -1959,6 +1981,18 @@
net.hydromatic
aggdesigner-algorithm
+
+ org.apache.atlas
+ atlas-intg
+
+
+ org.apache.hadoop
+ hadoop-yarn-registry
+
+
+ com.jayway.jsonpath
+ json-path
+
@@ -2084,6 +2118,10 @@
com.jolbox
bonecp
+
+ org.apache.orc
+ orc-core
+
@@ -2168,7 +2206,7 @@
${hive.group}
hive-service-rpc
- 4.0.0
+ ${hive.version}
*
@@ -2262,6 +2300,10 @@
${hive.group}
hive-serde
+
+ org.apache.hadoop
+ hadoop-common
+
org.slf4j
slf4j-api
@@ -2305,6 +2347,57 @@
+
+ ${hive.group}
+ hive-udf
+ ${hive.version}
+
+
+ ${hive.group}
+ hive-standalone-metastore-server
+ ${hive.version}
+
+
+ org.antlr
+ antlr-runtime
+
+
+ org.apache.orc
+ orc-core
+
+
+ org.apache.datasketches
+ datasketches-hive
+
+
+ com.github.joshelser
+ dropwizard-metrics-hadoop-metrics2-reporter
+
+
+
+
+ org.apache.datasketches
+ datasketches-hive
+ 2.0.0
+
+
+ org.slf4j
+ slf4j-simple
+
+
+
+
+ ${hive.group}
+ hive-parser
+ ${hive.version}
+ runtime
+
+
+ org.apache.calcite
+ calcite-core
+ 1.38.0
+ runtime
+
org.apache.orc
@@ -2457,6 +2550,11 @@
datanucleus-core
${datanucleus-core.version}
+
+ org.datanucleus
+ datanucleus-api-jdo
+ 6.0.5
+
org.apache.thrift
libthrift
@@ -2606,6 +2704,12 @@
javax.servlet-api
${javaxservlet.version}
+
+ org.apache.maven.shared
+ maven-invoker
+ 3.3.0
+ test
+
@@ -2783,7 +2887,7 @@
-Xss128m
-Xms4g
- -Xmx4g
+ -Xmx8g
-XX:MaxMetaspaceSize=2g
-XX:ReservedCodeCacheSize=${CodeCacheSize}
--enable-native-access=ALL-UNNAMED
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 3ef0e174fe78..dbee5b0c9de6 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -290,6 +290,12 @@ object SparkBuild extends PomBuild {
sparkGenjavadocSettings ++
compilerWarningSettings ++
(if (noLintOnCompile) Nil else enableScalaStyle) ++ Seq(
+ (Compile / dependencyClasspath) := (Compile / dependencyClasspath).value
+ .filterNot(file => {
+ val name = file.toString
+ (name.contains("orc-core") || name.contains("orc-format")) &&
+ !name.contains("shaded-protobuf")
+ }),
(Compile / exportJars) := true,
(Test / exportJars) := false,
javaHome := sys.env.get("JAVA_HOME")
@@ -1090,10 +1096,15 @@ object KubernetesIntegrationTests {
*/
object DependencyOverrides {
lazy val guavaVersion = sys.props.get("guava.version").getOrElse("33.4.0-jre")
+ lazy val log4jVersion = sys.props.get("log4j.version").getOrElse("2.24.3")
+ lazy val derbyVersion = sys.props.get("derby.version").getOrElse("10.16.1.1")
lazy val settings = Seq(
dependencyOverrides += "com.google.guava" % "guava" % guavaVersion,
dependencyOverrides += "jline" % "jline" % "2.14.6",
- dependencyOverrides += "org.apache.avro" % "avro" % "1.12.0")
+ dependencyOverrides += "org.apache.avro" % "avro" % "1.12.0",
+ dependencyOverrides += "org.apache.logging.log4j" % "log4j-slf4j-impl" % log4jVersion,
+ dependencyOverrides += "org.apache.derby" % "derby" % derbyVersion
+ )
}
/**
@@ -1105,8 +1116,7 @@ object ExcludedDependencies {
libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") },
excludeDependencies ++= Seq(
ExclusionRule(organization = "ch.qos.logback"),
- ExclusionRule("org.slf4j", "slf4j-simple"),
- ExclusionRule("javax.servlet", "javax.servlet-api"))
+ ExclusionRule("org.slf4j", "slf4j-simple"))
)
}
@@ -1749,6 +1759,7 @@ object TestSettings {
}.getOrElse(Nil): _*),
// Show full stack trace and duration in test cases.
(Test / testOptions) += Tests.Argument("-oDF"),
+ (Test / testOptions) += Tests.Argument(TestFrameworks.ScalaTest, "-fG", "scalatest.txt"),
// Slowpoke notifications: receive notifications every 5 minute of tests that have been running
// longer than two minutes.
(Test / testOptions) += Tests.Argument(TestFrameworks.ScalaTest, "-W", "120", "300"),
diff --git a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala
index 067f08cb6752..9f3bc4a94d1e 100644
--- a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala
@@ -95,6 +95,25 @@ class SparkShellSuite extends SparkFunSuite {
}
}
+ def handleException(cause: Throwable): Unit = lock.synchronized {
+ val message =
+ s"""
+ |=======================
+ |SparkShellSuite failure output
+ |=======================
+ |Spark Shell command line: ${command.mkString(" ")}
+ |Exception: $cause
+ |Failed to capture next expected output "${expectedAnswers(next)}" within $timeout.
+ |
+ |${buffer.mkString("\n")}
+ |===========================
+ |End SparkShellSuite failure output
+ |===========================
+ """.stripMargin
+ logError(message, cause)
+ fail(message, cause)
+ }
+
val process = new ProcessBuilder(command: _*).start()
val stdinWriter = new OutputStreamWriter(process.getOutputStream, StandardCharsets.UTF_8)
@@ -119,23 +138,7 @@ class SparkShellSuite extends SparkFunSuite {
}
ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeoutForQuery)
log.info("Found all expected output.")
- } catch { case cause: Throwable =>
- val message =
- s"""
- |=======================
- |SparkShellSuite failure output
- |=======================
- |Spark Shell command line: ${command.mkString(" ")}
- |Exception: $cause
- |Failed to capture next expected output "${expectedAnswers(next)}" within $timeout.
- |
- |${buffer.mkString("\n")}
- |===========================
- |End SparkShellSuite failure output
- |===========================
- """.stripMargin
- logError(message, cause)
- fail(message, cause)
+ } catch { case cause: Throwable => handleException(cause)
} finally {
if (!process.waitFor(1, MINUTES)) {
try {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 0ed7bdb1f282..86100c103d6e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -4722,8 +4722,7 @@ class AstBuilder extends DataTypeAstBuilder
entry("field.delim", ctx.fieldsTerminatedBy) ++
entry("serialization.format", ctx.fieldsTerminatedBy) ++
entry("escape.delim", ctx.escapedBy) ++
- // The following typo is inherited from Hive...
- entry("colelction.delim", ctx.collectionItemsTerminatedBy) ++
+ entry("collection.delim", ctx.collectionItemsTerminatedBy) ++
entry("mapkey.delim", ctx.keysTerminatedBy) ++
Option(ctx.linesSeparatedBy).toSeq.map { token =>
val value = string(visitStringLit(token))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
index 5bb81873449c..60baddc452ea 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
@@ -901,8 +901,12 @@ abstract class ExternalCatalogSuite extends SparkFunSuite {
assert(exists(db.locationUri, "my_table"))
catalog.renameTable("db1", "my_table", "your_table")
- assert(!exists(db.locationUri, "my_table"))
- assert(exists(db.locationUri, "your_table"))
+ // Hive does not rename location for managed tables if the location was specified when table
+ // was created.
+ if (this.isInstanceOf[InMemoryCatalogSuite]) {
+ assert(!exists(db.locationUri, "my_table"))
+ assert(exists(db.locationUri, "your_table"))
+ }
catalog.dropTable("db1", "your_table", ignoreIfNotExists = false, purge = false)
assert(!exists(db.locationUri, "your_table"))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index 0cb6f53f273a..78bb75ce773b 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -525,7 +525,7 @@ class DDLParserSuite extends AnalysisTest {
None,
Some(SerdeInfo(storedAs = Some("textfile"), serdeProperties = Map(
"field.delim" -> ",", "serialization.format" -> ",", "escape.delim" -> "\\",
- "colelction.delim" -> "#", "mapkey.delim" -> "=", "line.delim" -> "\n"
+ "collection.delim" -> "#", "mapkey.delim" -> "=", "line.delim" -> "\n"
))))
Seq(createSql, replaceSql).foreach { sql =>
testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false)
diff --git a/sql/core/pom.xml b/sql/core/pom.xml
index 39d8c3995441..b0b94324f401 100644
--- a/sql/core/pom.xml
+++ b/sql/core/pom.xml
@@ -123,7 +123,11 @@
${hive.group}
- hive-storage-api
+ hive-common
+
+
+ ${hive.group}
+ hive-serde
org.apache.parquet
@@ -279,6 +283,10 @@
bcpkix-jdk18on
test
+
+ org.apache.maven.shared
+ maven-invoker
+
target/scala-${scala.binary.version}/classes
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala
new file mode 100644
index 000000000000..a5c23642e352
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala
@@ -0,0 +1,64 @@
+/*
+ * 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.catalyst.util
+
+import java.sql.{Timestamp => SqlTimestamp}
+import java.time.{Instant, LocalDate, LocalDateTime, ZoneId}
+import java.time.ZoneOffset.UTC
+
+import org.apache.hadoop.hive.common.`type`.{Date => HiveDate, Timestamp => HiveTimestamp}
+
+import org.apache.spark.sql.catalyst.util.RebaseDateTime.rebaseJulianToGregorianDays
+import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.instantToMicros
+
+object HiveDateTimeUtils {
+ private val zoneId = ZoneId.systemDefault()
+
+ private def toSqlTimestamp(t: HiveTimestamp): SqlTimestamp = {
+ val millis = t.toEpochMilli(zoneId)
+ val ts = new SqlTimestamp(millis)
+ ts.setNanos(t.getNanos)
+ ts
+ }
+
+ def fromHiveTimestamp(t: HiveTimestamp): Long = {
+ // get Hive localDateTime
+ var localDateTime = LocalDateTime.ofInstant(Instant.ofEpochSecond(t.toEpochSecond, t.getNanos),
+ UTC)
+ val julianDate = rebaseJulianToGregorianDays(localDateTime.toLocalDate.toEpochDay.toInt)
+ localDateTime = LocalDateTime.of(LocalDate.ofEpochDay(julianDate), localDateTime.toLocalTime)
+ instantToMicros(localDateTime.toInstant(zoneId.getRules.getOffset(localDateTime)))
+ }
+
+ def fromHiveDate(d: HiveDate): Int = {
+ d.toEpochDay
+ }
+
+ def toHiveTimestamp(t: Long): HiveTimestamp = {
+ val javaTimestamp = DateTimeUtils.toJavaTimestamp(t)
+ val hiveTimestamp = new HiveTimestamp(javaTimestamp.toLocalDateTime)
+ // hiveTimestamp.setNanos(javaTimestamp.getNanos)
+ hiveTimestamp
+ }
+
+ def toHiveDate(d: Int): HiveDate = {
+ // val julian = RebaseDateTime.rebaseGregorianToJulianDays(d)
+ HiveDate.ofEpochDay(d)
+ }
+
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala
new file mode 100644
index 000000000000..c7e3e8f0147e
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala
@@ -0,0 +1,89 @@
+/*
+ * 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.execution.datasources
+
+import java.io.{DataInput, DataOutput, IOException}
+
+import org.apache.hadoop.hive.common.`type`.Date
+import org.apache.hadoop.hive.serde2.io.DateWritableV2
+
+import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays}
+
+/**
+ * The class accepts/returns days in Gregorian calendar and rebase them
+ * via conversion to local date in Julian calendar for dates before 1582-10-15
+ * in read/write for backward compatibility with Spark 2.4 and earlier versions.
+ *
+ * @param gregorianDays The number of days since the epoch 1970-01-01 in
+ * Gregorian calendar.
+ * @param julianDays The number of days since the epoch 1970-01-01 in
+ * Julian calendar.
+ */
+class DaysWritableV2 extends DateWritableV2 {
+ private var gregorianDays: Date = null
+
+ def this(d: Date) = {
+ this()
+ gregorianDays = d
+ super.set(d)
+ }
+
+ def this(d: Int) = {
+ this(Date.ofEpochDay(d))
+ }
+
+ def this(dateWritableV2: DateWritableV2) = {
+ this()
+ gregorianDays = dateWritableV2 match {
+ case daysWritableV2: DaysWritableV2 => daysWritableV2.gregorianDays
+ case dateWritableV2: DateWritableV2 =>
+ Date.ofEpochDay(rebaseJulianToGregorianDays(dateWritableV2.getDays))
+ }
+ super.set(rebaseGregorianToJulianDays(gregorianDays.toEpochDay))
+ }
+
+ def getGregorianDays: Int = gregorianDays.toEpochDay
+
+ override def getDays: Int = super.getDays
+
+ override def get: Date = {
+ super.get()
+ }
+
+ override def set(d: Date): Unit = {
+ gregorianDays = d
+ super.set(d.toEpochDay)
+ }
+
+ override def set(d: Int): Unit = {
+ gregorianDays = Date.ofEpochDay(d)
+ super.set(d)
+ }
+
+ @throws[IOException]
+ override def write(out: DataOutput): Unit = {
+ super.write(out)
+ }
+
+ @throws[IOException]
+ override def readFields(in: DataInput): Unit = {
+ super.readFields(in)
+ gregorianDays = Date.ofEpochDay(rebaseJulianToGregorianDays(getDays))
+ }
+
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index 3d4da878869e..87c5b38a8184 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -2116,7 +2116,8 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase {
}
}
- test(s"Support alter table command with CASE_SENSITIVE is true") {
+ test("Support alter table command with CASE_SENSITIVE is true") {
+ assume(isInstanceOf[InMemoryCatalogedDDLSuite], "Hive is no longer case preserving")
withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"true") {
withLocale("tr") {
val dbName = "DaTaBaSe_I"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
index ccf502d79c00..d86cbd3b393c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
@@ -3067,7 +3067,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest {
"escape.delim" -> "y",
"serialization.format" -> "x",
"line.delim" -> "\n",
- "colelction.delim" -> "a", // yes, it's a typo from Hive :)
+ "collection.delim" -> "a",
"mapkey.delim" -> "b"))
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
index 040999476ece..e9e725956613 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY}
import org.apache.spark.sql.execution.datasources.{CommonFileDataSourceSuite, SchemaMergeUtils}
import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtilsBase}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtilsBase, TestSparkSession}
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -508,7 +508,14 @@ abstract class OrcSuite
withAllNativeOrcReaders {
checkAnswer(
spark.read.orc(path),
- Seq(Row(Date.valueOf("1001-01-01")), Row(Date.valueOf("1582-10-15"))))
+ Seq(Row(Date.valueOf("1001-01-01")),
+ if (spark.isInstanceOf[TestSparkSession]) {
+ // Spark rebases 1582-10-05 through 1582-10-15 to 1582-10-15
+ Row(Date.valueOf("1582-10-15"))
+ } else {
+ // Hive rebases 1582-10-05 through 1582-10-15 by adding 10 days
+ Row(Date.valueOf("1582-10-20"))
+ }))
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
index f0f3f94b811f..c08d4417e442 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.test
-import java.io.File
+import java.io.{File, InputStream}
import java.net.URI
import java.nio.file.Files
import java.util.{Locale, UUID}
@@ -28,9 +28,11 @@ import scala.language.implicitConversions
import scala.util.control.NonFatal
import org.apache.hadoop.fs.Path
+import org.apache.maven.shared.invoker.{DefaultInvocationRequest, DefaultInvoker}
import org.scalactic.source.Position
import org.scalatest.{BeforeAndAfterAll, Suite, Tag}
import org.scalatest.concurrent.Eventually
+import org.scalatest.matchers.should.Matchers._
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.{AnalysisException, Row}
@@ -535,4 +537,16 @@ private[sql] object SQLTestUtils {
None
}
}
+
+ private val invoker = new DefaultInvoker()
+ invoker.setMavenExecutable(new File(System.getProperty("mvn.executable", "mvn")))
+
+ final def invokeMaven(pomFile: File): Unit = {
+ val request = new DefaultInvocationRequest()
+ .setPomFile(pomFile)
+ .addArgs(Seq("clean", "package", "-DskipTests").toList.asJava)
+ .setInputStream(InputStream.nullInputStream())
+ invoker.execute(request).getExitCode should equal (0)
+ }
+
}
diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml
index 135b84cd01f8..7741e04547a8 100644
--- a/sql/hive-thriftserver/pom.xml
+++ b/sql/hive-thriftserver/pom.xml
@@ -81,6 +81,10 @@
${hive.group}
hive-beeline
+
+ ${hive.group}
+ hive-standalone-metastore-server
+
${hive.group}
hive-service-rpc
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java
index da4c170f93fd..6dd67be7de11 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java
@@ -30,10 +30,10 @@
import org.apache.hadoop.hive.ql.metadata.Hive;
import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim;
import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.hive.thrift.DBTokenStore;
-import org.apache.hadoop.hive.thrift.HiveDelegationTokenManager;
-import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server.ServerMode;
+import org.apache.hadoop.hive.metastore.security.DBTokenStore;
+import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge.Server.ServerMode;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.ProxyUsers;
@@ -80,7 +80,7 @@ public String getAuthName() {
private String authTypeStr;
private final String transportMode;
private final HiveConf conf;
- private HiveDelegationTokenManager delegationTokenManager = null;
+ private MetastoreDelegationTokenManager delegationTokenManager = null;
public static final String HS2_PROXY_USER = "hive.server2.proxy.user";
public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken";
@@ -114,18 +114,19 @@ public HiveAuthFactory(HiveConf conf) throws TTransportException, IOException {
authTypeStr = AuthTypes.NONE.getAuthName();
}
if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
- String principal = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL);
String keytab = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB);
+ String principal = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL);
+ String client = conf.getVar(ConfVars.HIVE_SERVER2_CLIENT_KERBEROS_PRINCIPAL);
if (needUgiLogin(UserGroupInformation.getCurrentUser(),
SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keytab)) {
- saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(keytab, principal);
+ saslServer = HadoopThriftAuthBridge.getBridge().createServer(keytab, principal, client);
} else {
// Using the default constructor to avoid unnecessary UGI login.
saslServer = new HadoopThriftAuthBridge.Server();
}
// start delegation token manager
- delegationTokenManager = new HiveDelegationTokenManager();
+ delegationTokenManager = new MetastoreDelegationTokenManager();
try {
// rawStore is only necessary for DBTokenStore
Object rawStore = null;
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java
index a76bcc9b873e..57931e600fcb 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java
@@ -31,7 +31,7 @@
import javax.security.auth.Subject;
import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.http.protocol.BasicHttpContext;
import org.apache.http.protocol.HttpContext;
@@ -68,8 +68,7 @@ public final class HttpAuthUtils {
*/
public static String getKerberosServiceTicket(String principal, String host,
String serverHttpUrl, boolean assumeSubject) throws Exception {
- String serverPrincipal =
- ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host);
+ String serverPrincipal = HadoopThriftAuthBridge.getBridge().getServerPrincipal(principal, host);
if (assumeSubject) {
// With this option, we're assuming that the external application,
// using the JDBC driver has done a JAAS kerberos login already
@@ -82,7 +81,7 @@ public static String getKerberosServiceTicket(String principal, String host,
} else {
// JAAS login from ticket cache to setup the client UserGroupInformation
UserGroupInformation clientUGI =
- ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos");
+ HadoopThriftAuthBridge.getBridge().getCurrentUGIWithConf("kerberos");
return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl));
}
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java
index ef91f94eeec2..e6d425753482 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java
@@ -20,9 +20,8 @@
import java.util.Map;
import javax.security.sasl.SaslException;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge.Server;
import org.apache.hive.service.cli.thrift.ThriftCLIService;
import org.apache.hive.service.rpc.thrift.TCLIService;
import org.apache.hive.service.rpc.thrift.TCLIService.Iface;
@@ -52,7 +51,7 @@ public static TTransport getKerberosTransport(String principal, String host,
return createSubjectAssumedTransport(principal, underlyingTransport, saslProps);
} else {
HadoopThriftAuthBridge.Client authBridge =
- ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
+ HadoopThriftAuthBridge.getBridge().createClientWithConf("kerberos");
return authBridge.createClientTransport(principal, host, "KERBEROS", null,
underlyingTransport, saslProps);
}
@@ -77,7 +76,7 @@ public static TTransport createSubjectAssumedTransport(String principal,
public static TTransport getTokenTransport(String tokenStr, String host,
TTransport underlyingTransport, Map saslProps) throws SaslException {
HadoopThriftAuthBridge.Client authBridge =
- ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
+ HadoopThriftAuthBridge.getBridge().createClientWithConf("kerberos");
try {
return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport,
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java
index 2452691a2ffe..3276d83594d8 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java
@@ -33,22 +33,20 @@ public class RowBasedSet implements RowSet {
private long startOffset;
private final TypeDescriptor[] descriptors; // non-null only for writing (server-side)
- private final RemovableList rows;
+ private final RemovableList rows;
public RowBasedSet(TableSchema schema) {
descriptors = schema.toTypeDescriptors();
- rows = new RemovableList();
+ rows = new RemovableList();
}
public RowBasedSet(TRowSet tRowSet) {
- descriptors = null;
- rows = new RemovableList(tRowSet.getRows());
- startOffset = tRowSet.getStartRowOffset();
+ this(null, tRowSet.getRows(), tRowSet.getStartRowOffset());
}
private RowBasedSet(TypeDescriptor[] descriptors, List rows, long startOffset) {
this.descriptors = descriptors;
- this.rows = new RemovableList(rows);
+ this.rows = rows != null ? new RemovableList(rows) : new RemovableList();
this.startOffset = startOffset;
}
@@ -128,9 +126,9 @@ public void remove() {
};
}
- private static class RemovableList extends ArrayList {
+ private static class RemovableList extends ArrayList {
RemovableList() { super(); }
- RemovableList(List rows) { super(rows); }
+ RemovableList(List rows) { super(rows); }
@Override
public void removeRange(int fromIndex, int toIndex) {
super.removeRange(fromIndex, toIndex);
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index 815a369b6b23..8de82d0a05e7 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -18,16 +18,15 @@
import java.util.Map;
-import org.apache.hadoop.hive.ql.session.OperationLog;
import org.apache.hive.service.cli.OperationType;
import org.apache.hive.service.cli.session.HiveSession;
public abstract class ExecuteStatementOperation extends Operation {
protected String statement = null;
- public ExecuteStatementOperation(HiveSession parentSession, String statement,
+ public ExecuteStatementOperation(HiveSession parentSession, OperationManager operationManager, String statement,
Map confOverlay, boolean runInBackground) {
- super(parentSession, confOverlay, OperationType.EXECUTE_STATEMENT, runInBackground);
+ super(parentSession, operationManager, confOverlay, OperationType.EXECUTE_STATEMENT, runInBackground);
this.statement = statement;
}
@@ -43,7 +42,7 @@ protected void registerCurrentOperationLog() {
isOperationLogEnabled = false;
return;
}
- OperationLog.setCurrentOperationLog(operationLog);
+ operationManager.setCurrentOperationLog(operationLog, operationLogFile);
}
}
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
index ef4bbb45e8f4..e597d328635a 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
@@ -33,8 +33,8 @@ public class GetCatalogsOperation extends MetadataOperation {
protected final RowSet rowSet;
- protected GetCatalogsOperation(HiveSession parentSession) {
- super(parentSession, OperationType.GET_CATALOGS);
+ protected GetCatalogsOperation(HiveSession parentSession, OperationManager operationManager) {
+ super(parentSession, operationManager, OperationType.GET_CATALOGS);
rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false);
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
index 250adc51f81e..1cfee892adac 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
@@ -33,7 +33,7 @@
import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.metadata.TableIterable;
+import org.apache.hadoop.hive.metastore.TableIterable;
import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
@@ -119,9 +119,9 @@ public class GetColumnsOperation extends MetadataOperation {
protected final RowSet rowSet;
- protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName,
- String tableName, String columnName) {
- super(parentSession, OperationType.GET_COLUMNS);
+ protected GetColumnsOperation(HiveSession parentSession, OperationManager operationManager,
+ String catalogName, String schemaName, String tableName, String columnName) {
+ super(parentSession, operationManager, OperationType.GET_COLUMNS);
this.catalogName = catalogName;
this.schemaName = schemaName;
this.tableName = tableName;
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java
index 3a29859a2074..b21ba0d09271 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java
@@ -99,10 +99,10 @@ public class GetCrossReferenceOperation extends MetadataOperation {
private final String foreignTableName;
private final RowSet rowSet;
- public GetCrossReferenceOperation(HiveSession parentSession,
+ public GetCrossReferenceOperation(HiveSession parentSession, OperationManager operationManager,
String parentCatalogName, String parentSchemaName, String parentTableName,
String foreignCatalog, String foreignSchema, String foreignTable) {
- super(parentSession, OperationType.GET_FUNCTIONS);
+ super(parentSession, operationManager, OperationType.GET_FUNCTIONS);
this.parentCatalogName = parentCatalogName;
this.parentSchemaName = parentSchemaName;
this.parentTableName = parentTableName;
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
index 3f02f753bf87..a78cd6fb08c4 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
@@ -66,9 +66,9 @@ public class GetFunctionsOperation extends MetadataOperation {
protected final RowSet rowSet;
- public GetFunctionsOperation(HiveSession parentSession,
+ public GetFunctionsOperation(HiveSession parentSession, OperationManager operationManager,
String catalogName, String schemaName, String functionName) {
- super(parentSession, OperationType.GET_FUNCTIONS);
+ super(parentSession, operationManager, OperationType.GET_FUNCTIONS);
this.catalogName = catalogName;
this.schemaName = schemaName;
this.functionName = functionName;
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java
index 927328342974..9339865f29f6 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java
@@ -61,9 +61,9 @@ public class GetPrimaryKeysOperation extends MetadataOperation {
private final RowSet rowSet;
- public GetPrimaryKeysOperation(HiveSession parentSession,
+ public GetPrimaryKeysOperation(HiveSession parentSession, OperationManager operationManager,
String catalogName, String schemaName, String tableName) {
- super(parentSession, OperationType.GET_FUNCTIONS);
+ super(parentSession, operationManager, OperationType.GET_FUNCTIONS);
this.catalogName = catalogName;
this.schemaName = schemaName;
this.tableName = tableName;
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
index 865e264bd5f4..a71908519322 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
@@ -44,9 +44,9 @@ public class GetSchemasOperation extends MetadataOperation {
protected RowSet rowSet;
- protected GetSchemasOperation(HiveSession parentSession,
+ protected GetSchemasOperation(HiveSession parentSession, OperationManager operationManager,
String catalogName, String schemaName) {
- super(parentSession, OperationType.GET_SCHEMAS);
+ super(parentSession, operationManager, OperationType.GET_SCHEMAS);
this.catalogName = catalogName;
this.schemaName = schemaName;
this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false);
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
index b75eaec5ff65..9009da795d1d 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
@@ -43,8 +43,8 @@ public class GetTableTypesOperation extends MetadataOperation {
protected final RowSet rowSet;
private final TableTypeMapping tableTypeMapping;
- protected GetTableTypesOperation(HiveSession parentSession) {
- super(parentSession, OperationType.GET_TABLE_TYPES);
+ protected GetTableTypesOperation(HiveSession parentSession, OperationManager operationManager) {
+ super(parentSession, operationManager, OperationType.GET_TABLE_TYPES);
String tableMappingStr = getParentSession().getHiveConf()
.getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING);
tableTypeMapping =
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
index bd9f0814814f..c3bf9ffa0629 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
@@ -66,10 +66,10 @@ public class GetTablesOperation extends MetadataOperation {
.addStringColumn("REF_GENERATION",
"Specifies how values in SELF_REFERENCING_COL_NAME are created.");
- protected GetTablesOperation(HiveSession parentSession,
+ protected GetTablesOperation(HiveSession parentSession, OperationManager operationManager,
String catalogName, String schemaName, String tableName,
List tableTypes) {
- super(parentSession, OperationType.GET_TABLES);
+ super(parentSession, operationManager, OperationType.GET_TABLES);
this.catalogName = catalogName;
this.schemaName = schemaName;
this.tableName = tableName;
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
index ad692d46edd2..9cdff0635334 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
@@ -76,8 +76,8 @@ public class GetTypeInfoOperation extends MetadataOperation {
protected final RowSet rowSet;
- protected GetTypeInfoOperation(HiveSession parentSession) {
- super(parentSession, OperationType.GET_TYPE_INFO);
+ protected GetTypeInfoOperation(HiveSession parentSession, OperationManager operationManager) {
+ super(parentSession, operationManager, OperationType.GET_TYPE_INFO);
rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false);
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java
index 32cc42f008bd..d84653b05cd8 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java
@@ -17,6 +17,9 @@
package org.apache.hive.service.cli.operation;
import java.io.CharArrayWriter;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
import java.io.Serializable;
import java.util.Map;
import java.util.regex.Pattern;
@@ -180,7 +183,7 @@ public Result filter(org.apache.logging.log4j.core.Logger logger, Level level, M
@Override
public Result filter(LogEvent logEvent) {
- OperationLog log = operationManager.getOperationLogByThread();
+ OperationLog log = operationManager.getCurrentOperationLog();
boolean excludeMatches = (loggingMode == OperationLog.LoggingLevel.VERBOSE);
if (log == null) {
@@ -296,7 +299,7 @@ private LogDivertAppender(OperationManager operationManager,
@Override
public void append(LogEvent event) {
- OperationLog log = operationManager.getOperationLogByThread();
+ OperationLog log = operationManager.getCurrentOperationLog();
// Set current layout depending on the verbose/non-verbose mode.
if (log != null) {
@@ -318,6 +321,12 @@ public void append(LogEvent event) {
LOG.debug(" ---+++=== Dropped log event from thread " + event.getThreadName());
return;
}
- log.writeOperationLog(logOutput);
+ try {
+ PrintStream out =
+ new PrintStream(new FileOutputStream(operationManager.getCurrentOperationLogFile()));
+ out.print(logOutput);
+ } catch (FileNotFoundException e) {
+ LOG.debug(" ---+++=== Dropped log event from thread " + event.getThreadName());
+ }
}
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java
index a818d1ecfd16..a111285df6f5 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java
@@ -42,8 +42,8 @@ public abstract class MetadataOperation extends Operation {
protected static TableSchema RESULT_SET_SCHEMA;
private static final char SEARCH_STRING_ESCAPE = '\\';
- protected MetadataOperation(HiveSession parentSession, OperationType opType) {
- super(parentSession, opType);
+ protected MetadataOperation(HiveSession parentSession, OperationManager operationManager, OperationType opType) {
+ super(parentSession, operationManager, opType);
setHasResultSet(true);
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java
index b5cf8991a4cb..4d3d9ff337e5 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java
@@ -17,16 +17,16 @@
package org.apache.hive.service.cli.operation;
import java.io.File;
-import java.io.FileNotFoundException;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Map;
+import java.util.Objects;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.QueryState;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorException;
import org.apache.hadoop.hive.ql.session.OperationLog;
import org.apache.hive.service.cli.FetchOrientation;
import org.apache.hive.service.cli.HiveSQLException;
@@ -46,6 +46,7 @@
public abstract class Operation {
protected final HiveSession parentSession;
+ protected final OperationManager operationManager;
private OperationState state = OperationState.INITIALIZED;
private final OperationHandle opHandle;
private HiveConf configuration;
@@ -56,6 +57,7 @@ public abstract class Operation {
protected volatile HiveSQLException operationException;
protected final boolean runAsync;
protected volatile Future> backgroundHandle;
+ protected File operationLogFile;
protected OperationLog operationLog;
protected boolean isOperationLogEnabled;
protected Map confOverlay = new HashMap();
@@ -71,25 +73,29 @@ public abstract class Operation {
FetchOrientation.FETCH_FIRST,
FetchOrientation.FETCH_PRIOR);
- protected Operation(HiveSession parentSession, OperationType opType) {
- this(parentSession, null, opType);
+ protected Operation(HiveSession parentSession, OperationManager operationManager, OperationType opType) {
+ this(parentSession, operationManager, null, opType);
}
- protected Operation(HiveSession parentSession, Map confOverlay,
+ protected Operation(HiveSession parentSession, OperationManager operationManager, Map confOverlay,
OperationType opType) {
- this(parentSession, confOverlay, opType, false);
+ this(parentSession, operationManager, confOverlay, opType, false);
}
protected Operation(HiveSession parentSession,
- Map confOverlay, OperationType opType, boolean runInBackground) {
+ OperationManager operationManager, Map confOverlay, OperationType opType, boolean runInBackground) {
this.parentSession = parentSession;
+ this.operationManager = Objects.requireNonNull(operationManager);
this.confOverlay = confOverlay;
this.runAsync = runInBackground;
this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion());
lastAccessTime = System.currentTimeMillis();
operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(),
HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS);
- queryState = new QueryState(parentSession.getHiveConf(), confOverlay, runInBackground);
+ queryState = new QueryState.Builder()
+ .withHiveConf(parentSession.getHiveConf())
+ .withConfOverlay(confOverlay)
+ .build();
}
public Future> getBackgroundHandle() {
@@ -204,7 +210,7 @@ public boolean isFailed() {
protected void createOperationLog() {
if (parentSession.isOperationLogEnabled()) {
- File operationLogFile = new File(parentSession.getOperationLogSessionDir(),
+ operationLogFile = new File(parentSession.getOperationLogSessionDir(),
opHandle.getHandleIdentifier().toString());
isOperationLogEnabled = true;
@@ -234,23 +240,16 @@ protected void createOperationLog() {
}
// create OperationLog object with above log file
- try {
- operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf());
- } catch (FileNotFoundException e) {
- LOG.warn("Unable to instantiate OperationLog object for operation: {}", e,
- MDC.of(LogKeys.OPERATION_HANDLE, opHandle));
- isOperationLogEnabled = false;
- return;
- }
+ operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf());
// register this operationLog to current thread
- OperationLog.setCurrentOperationLog(operationLog);
+ operationManager.setCurrentOperationLog(operationLog, operationLogFile);
}
}
protected void unregisterOperationLog() {
if (isOperationLogEnabled) {
- OperationLog.removeCurrentOperationLog();
+ operationManager.removeCurrentOperationLog();
}
}
@@ -336,11 +335,11 @@ protected void validateFetchOrientation(FetchOrientation orientation,
}
}
- protected HiveSQLException toSQLException(String prefix, CommandProcessorResponse response) {
- HiveSQLException ex = new HiveSQLException(prefix + ": " + response.getErrorMessage(),
- response.getSQLState(), response.getResponseCode());
- if (response.getException() != null) {
- ex.initCause(response.getException());
+ protected HiveSQLException toSQLException(String prefix, CommandProcessorException exception) {
+ HiveSQLException ex = new HiveSQLException(prefix + ": " + exception.getMessage(),
+ exception.getSqlState(), exception.getResponseCode());
+ if (exception.getCause() != null) {
+ ex.initCause(exception.getCause());
}
return ex;
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java
index 79c12d387a9d..5488a3db7fe9 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java
@@ -17,6 +17,7 @@
package org.apache.hive.service.cli.operation;
+import java.io.File;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.HashMap;
@@ -56,6 +57,11 @@ public class OperationManager extends AbstractService {
private final Map handleToOperation =
new HashMap();
+ private final ThreadLocal threadLocalOperationLog =
+ ThreadLocal.withInitial(() -> null);
+
+ private final ThreadLocal threadLocalFile = ThreadLocal.withInitial(() -> null);
+
public OperationManager() {
super(OperationManager.class.getSimpleName());
}
@@ -97,20 +103,20 @@ public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parent
}
public GetTypeInfoOperation newGetTypeInfoOperation(HiveSession parentSession) {
- GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession);
+ GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession, this);
addOperation(operation);
return operation;
}
public GetCatalogsOperation newGetCatalogsOperation(HiveSession parentSession) {
- GetCatalogsOperation operation = new GetCatalogsOperation(parentSession);
+ GetCatalogsOperation operation = new GetCatalogsOperation(parentSession, this);
addOperation(operation);
return operation;
}
public GetSchemasOperation newGetSchemasOperation(HiveSession parentSession,
String catalogName, String schemaName) {
- GetSchemasOperation operation = new GetSchemasOperation(parentSession, catalogName, schemaName);
+ GetSchemasOperation operation = new GetSchemasOperation(parentSession, this, catalogName, schemaName);
addOperation(operation);
return operation;
}
@@ -119,20 +125,20 @@ public MetadataOperation newGetTablesOperation(HiveSession parentSession,
String catalogName, String schemaName, String tableName,
List tableTypes) {
MetadataOperation operation =
- new GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes);
+ new GetTablesOperation(parentSession, this, catalogName, schemaName, tableName, tableTypes);
addOperation(operation);
return operation;
}
public GetTableTypesOperation newGetTableTypesOperation(HiveSession parentSession) {
- GetTableTypesOperation operation = new GetTableTypesOperation(parentSession);
+ GetTableTypesOperation operation = new GetTableTypesOperation(parentSession, this);
addOperation(operation);
return operation;
}
public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession,
String catalogName, String schemaName, String tableName, String columnName) {
- GetColumnsOperation operation = new GetColumnsOperation(parentSession,
+ GetColumnsOperation operation = new GetColumnsOperation(parentSession, this,
catalogName, schemaName, tableName, columnName);
addOperation(operation);
return operation;
@@ -140,7 +146,7 @@ public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession,
public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession,
String catalogName, String schemaName, String functionName) {
- GetFunctionsOperation operation = new GetFunctionsOperation(parentSession,
+ GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, this,
catalogName, schemaName, functionName);
addOperation(operation);
return operation;
@@ -148,7 +154,7 @@ public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession,
public GetPrimaryKeysOperation newGetPrimaryKeysOperation(HiveSession parentSession,
String catalogName, String schemaName, String tableName) {
- GetPrimaryKeysOperation operation = new GetPrimaryKeysOperation(parentSession,
+ GetPrimaryKeysOperation operation = new GetPrimaryKeysOperation(parentSession, this,
catalogName, schemaName, tableName);
addOperation(operation);
return operation;
@@ -158,7 +164,7 @@ public GetCrossReferenceOperation newGetCrossReferenceOperation(
HiveSession session, String primaryCatalog, String primarySchema,
String primaryTable, String foreignCatalog, String foreignSchema,
String foreignTable) {
- GetCrossReferenceOperation operation = new GetCrossReferenceOperation(session,
+ GetCrossReferenceOperation operation = new GetCrossReferenceOperation(session, this,
primaryCatalog, primarySchema, primaryTable, foreignCatalog, foreignSchema,
foreignTable);
addOperation(operation);
@@ -283,8 +289,22 @@ private Schema getLogSchema() {
return schema;
}
- public OperationLog getOperationLogByThread() {
- return OperationLog.getCurrentOperationLog();
+ public void setCurrentOperationLog(OperationLog log, File file) {
+ threadLocalOperationLog.set(log);
+ threadLocalFile.set(file);
+ }
+
+ public OperationLog getCurrentOperationLog() {
+ return threadLocalOperationLog.get();
+ }
+
+ public File getCurrentOperationLogFile() {
+ return threadLocalFile.get();
+ }
+
+ public void removeCurrentOperationLog() {
+ threadLocalOperationLog.remove();
+ threadLocalFile.remove();
}
public List removeExpiredOperations(OperationHandle[] handles) {
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
index 19aa7ab385ae..764483eee3f0 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
@@ -116,8 +116,7 @@ public HiveSessionImpl(TProtocolVersion protocol, String username, String passwo
try {
// In non-impersonation mode, map scheduler queue to current user
// if fair scheduler is configured.
- if (! hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) &&
- hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) {
+ if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
ShimLoader.getHadoopShims().refreshDefaultQueue(hiveConf, username);
}
} catch (IOException e) {
@@ -273,7 +272,7 @@ private static void setConf(String varname, String key, String varvalue, boolean
new VariableSubstitution(() -> SessionState.get().getHiveVariables());
HiveConf conf = SessionState.get().getConf();
String value = substitution.substitute(conf, varvalue);
- if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) {
+ if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CONF_VALIDATION)) {
HiveConf.ConfVars confVars = HiveConf.getConfVars(key);
if (confVars != null) {
if (!confVars.isType(value)) {
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
index 0ec13424fd0f..b24aefab02b0 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
@@ -23,8 +23,10 @@
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.metadata.Hive;
import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.shims.Utils;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hive.service.auth.HiveAuthFactory;
import org.apache.hive.service.cli.HiveSQLException;
import org.apache.hive.service.rpc.thrift.TProtocolVersion;
@@ -96,6 +98,14 @@ public void close() throws HiveSQLException {
}
}
+ private static Token createToken(String tokenStr, String tokenService)
+ throws IOException {
+ Token delegationToken = new Token<>();
+ delegationToken.decodeFromUrlString(tokenStr);
+ delegationToken.setService(new Text(tokenService));
+ return delegationToken;
+ }
+
/**
* Enable delegation token for the session
* save the token string and set the token.signature in hive conf. The metastore client uses
@@ -108,7 +118,8 @@ private void setDelegationToken(String delegationTokenStr) throws HiveSQLExcepti
if (delegationTokenStr != null) {
getHiveConf().set("hive.metastore.token.signature", HS2TOKEN);
try {
- Utils.setTokenStr(sessionUgi, delegationTokenStr, HS2TOKEN);
+ Token delegationToken = createToken(delegationTokenStr, HS2TOKEN);
+ sessionUgi.addToken(delegationToken);
} catch (IOException e) {
throw new HiveSQLException("Couldn't setup delegation token in the ugi", e);
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
index c7fa7b5f3e0a..1cef6c8c4711 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
@@ -81,8 +81,13 @@ protected void initializeServer() {
}
String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf,
HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname);
+ String keyStoreType = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_TYPE).trim();
+ String keyStoreAlgorithm = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYMANAGERFACTORY_ALGORITHM).trim();
+ String includeCiphersuites = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_BINARY_INCLUDE_CIPHERSUITES).trim();
+
serverSocket = HiveAuthUtils.getServerSSLSocket(hiveHost, portNum, keyStorePath,
- keyStorePassword, sslVersionBlacklist);
+ keyStorePassword, keyStoreType, keyStoreAlgorithm, sslVersionBlacklist,
+ includeCiphersuites);
}
// In case HIVE_SERVER2_THRIFT_PORT or hive.server2.thrift.port is configured with 0 which
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java
index 16d8540b4056..66ef2aceea8b 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java
@@ -20,7 +20,7 @@
import java.util.Map;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.HMSHandler;
import org.apache.hadoop.hive.metastore.RawStore;
import org.apache.spark.internal.SparkLogger;
@@ -67,7 +67,7 @@ private void cleanRawStore() {
*/
public void cacheThreadLocalRawStore() {
Long threadId = this.getId();
- RawStore threadLocalRawStore = HiveMetaStore.HMSHandler.getRawStore();
+ RawStore threadLocalRawStore = HMSHandler.getRawStore();
if (threadLocalRawStore != null && !threadRawStoreMap.containsKey(threadId)) {
LOG.debug("Adding RawStore: " + threadLocalRawStore + ", for the thread: " +
this.getName() + " to threadRawStoreMap for future cleanup.");
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index 350aba1ab175..534c3ee72b37 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -26,7 +26,7 @@ import scala.util.control.NonFatal
import org.apache.hadoop.hive.shims.Utils
import org.apache.hive.service.cli._
-import org.apache.hive.service.cli.operation.ExecuteStatementOperation
+import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
import org.apache.hive.service.rpc.thrift.{TCLIServiceConstants, TColumnDesc, TPrimitiveTypeEntry, TRowSet, TTableSchema, TTypeDesc, TTypeEntry, TTypeId, TTypeQualifiers, TTypeQualifierValue}
@@ -42,11 +42,13 @@ import org.apache.spark.util.{Utils => SparkUtils}
private[hive] class SparkExecuteStatementOperation(
val session: SparkSession,
parentSession: HiveSession,
+ operationManager: OperationManager,
statement: String,
confOverlay: JMap[String, String],
runInBackground: Boolean = true,
queryTimeout: Long)
- extends ExecuteStatementOperation(parentSession, statement, confOverlay, runInBackground)
+ extends ExecuteStatementOperation(parentSession, operationManager, statement, confOverlay,
+ runInBackground)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala
index a02b2ca8966d..8a3c0cf8d998 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver
import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType
import org.apache.hive.service.cli.OperationState
-import org.apache.hive.service.cli.operation.GetCatalogsOperation
+import org.apache.hive.service.cli.operation.{GetCatalogsOperation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
import org.apache.spark.internal.Logging
@@ -34,8 +34,9 @@ import org.apache.spark.sql.SparkSession
*/
private[hive] class SparkGetCatalogsOperation(
val session: SparkSession,
- parentSession: HiveSession)
- extends GetCatalogsOperation(parentSession)
+ parentSession: HiveSession,
+ operationManager: OperationManager)
+ extends GetCatalogsOperation(parentSession, operationManager)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala
index 6c573ceb14ec..9aeba93ce084 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala
@@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._
import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObject}
import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType
import org.apache.hive.service.cli._
-import org.apache.hive.service.cli.operation.GetColumnsOperation
+import org.apache.hive.service.cli.operation.{GetColumnsOperation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
import org.apache.spark.internal.Logging
@@ -46,11 +46,13 @@ import org.apache.spark.sql.types._
private[hive] class SparkGetColumnsOperation(
val session: SparkSession,
parentSession: HiveSession,
+ operationManager: OperationManager,
catalogName: String,
schemaName: String,
tableName: String,
columnName: String)
- extends GetColumnsOperation(parentSession, catalogName, schemaName, tableName, columnName)
+ extends GetColumnsOperation(parentSession, operationManager, catalogName, schemaName, tableName,
+ columnName)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala
index ec0da495d874..c4a64fea0f22 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala
@@ -23,7 +23,7 @@ import scala.jdk.CollectionConverters._
import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils}
import org.apache.hive.service.cli._
-import org.apache.hive.service.cli.operation.GetFunctionsOperation
+import org.apache.hive.service.cli.operation.{GetFunctionsOperation, OperationManager}
import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG
import org.apache.hive.service.cli.session.HiveSession
@@ -42,10 +42,12 @@ import org.apache.spark.sql.SparkSession
private[hive] class SparkGetFunctionsOperation(
val session: SparkSession,
parentSession: HiveSession,
+ operationManager: OperationManager,
catalogName: String,
schemaName: String,
functionName: String)
- extends GetFunctionsOperation(parentSession, catalogName, schemaName, functionName)
+ extends GetFunctionsOperation(parentSession, operationManager, catalogName, schemaName,
+ functionName)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala
index 4b8b603eede5..458b9110c209 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala
@@ -21,7 +21,7 @@ import java.util.regex.Pattern
import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType
import org.apache.hive.service.cli._
-import org.apache.hive.service.cli.operation.GetSchemasOperation
+import org.apache.hive.service.cli.operation.{GetSchemasOperation, OperationManager}
import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG
import org.apache.hive.service.cli.session.HiveSession
@@ -40,9 +40,10 @@ import org.apache.spark.sql.SparkSession
private[hive] class SparkGetSchemasOperation(
val session: SparkSession,
parentSession: HiveSession,
+ operationManager: OperationManager,
catalogName: String,
schemaName: String)
- extends GetSchemasOperation(parentSession, catalogName, schemaName)
+ extends GetSchemasOperation(parentSession, operationManager, catalogName, schemaName)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala
index e3e1cdbd3681..baa09a233ba2 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala
@@ -21,7 +21,7 @@ import java.util.UUID
import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType
import org.apache.hive.service.cli._
-import org.apache.hive.service.cli.operation.GetTableTypesOperation
+import org.apache.hive.service.cli.operation.{GetTableTypesOperation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
import org.apache.spark.internal.Logging
@@ -37,8 +37,9 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType
*/
private[hive] class SparkGetTableTypesOperation(
val session: SparkSession,
- parentSession: HiveSession)
- extends GetTableTypesOperation(parentSession)
+ parentSession: HiveSession,
+ operationManager: OperationManager)
+ extends GetTableTypesOperation(parentSession, operationManager)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala
index 0579d567d022..0393dd2b1824 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala
@@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._
import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils}
import org.apache.hive.service.cli._
-import org.apache.hive.service.cli.operation.GetTablesOperation
+import org.apache.hive.service.cli.operation.{GetTablesOperation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
import org.apache.spark.internal.Logging
@@ -45,11 +45,13 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._
private[hive] class SparkGetTablesOperation(
val session: SparkSession,
parentSession: HiveSession,
+ operationManager: OperationManager,
catalogName: String,
schemaName: String,
tableName: String,
tableTypes: JList[String])
- extends GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes)
+ extends GetTablesOperation(parentSession, operationManager, catalogName, schemaName, tableName,
+ tableTypes)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala
index 4af2265841b5..6b141bb595fa 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala
@@ -23,7 +23,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType
import org.apache.hadoop.hive.serde2.thrift.Type
import org.apache.hadoop.hive.serde2.thrift.Type._
import org.apache.hive.service.cli.OperationState
-import org.apache.hive.service.cli.operation.GetTypeInfoOperation
+import org.apache.hive.service.cli.operation.{GetTypeInfoOperation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
import org.apache.spark.internal.Logging
@@ -38,8 +38,9 @@ import org.apache.spark.sql.SparkSession
*/
private[hive] class SparkGetTypeInfoOperation(
val session: SparkSession,
- parentSession: HiveSession)
- extends GetTypeInfoOperation(parentSession)
+ parentSession: HiveSession,
+ operationManager: OperationManager)
+ extends GetTypeInfoOperation(parentSession, operationManager)
with SparkOperation
with Logging {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index 02826abef857..96e504222479 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -30,8 +30,11 @@ import jline.console.history.FileHistory
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor}
import org.apache.hadoop.hive.common.HiveInterruptUtils
+import org.apache.hadoop.hive.common.io.SessionStream
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.ql.Driver
+import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper
+import org.apache.hadoop.hive.ql.exec.tez.TezJobExecHelper
import org.apache.hadoop.hive.ql.processors._
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
@@ -46,7 +49,6 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
import org.apache.spark.sql.catalyst.util.SQLKeywordUtils
import org.apache.spark.sql.hive.client.HiveClientImpl
import org.apache.spark.sql.hive.security.HiveDelegationTokenProvider
-import org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.closeHiveSessionStateIfStarted
import org.apache.spark.sql.internal.{SharedState, SQLConf}
import org.apache.spark.sql.internal.SQLConf.LEGACY_EMPTY_CURRENT_DB_IN_CLI
import org.apache.spark.util.{SparkStringUtils, Utils}
@@ -99,17 +101,17 @@ private[hive] object SparkSQLCLIDriver extends Logging {
sessionState.in = System.in
try {
- sessionState.out = new PrintStream(System.out, true, UTF_8.name())
- sessionState.info = new PrintStream(System.err, true, UTF_8.name())
- sessionState.err = new PrintStream(System.err, true, UTF_8.name())
+ sessionState.out = new SessionStream(System.out, true, UTF_8.name())
+ sessionState.info = new SessionStream(System.err, true, UTF_8.name())
+ sessionState.err = new SessionStream(System.err, true, UTF_8.name())
} catch {
case e: UnsupportedEncodingException =>
- closeHiveSessionStateIfStarted(sessionState)
+ sessionState.close()
exit(ERROR_PATH_NOT_FOUND)
}
if (!oproc.process_stage2(sessionState)) {
- closeHiveSessionStateIfStarted(sessionState)
+ sessionState.close()
exit(ERROR_MISUSE_SHELL_BUILTIN)
}
@@ -144,7 +146,7 @@ private[hive] object SparkSQLCLIDriver extends Logging {
// Clean up after we exit
ShutdownHookManager.addShutdownHook { () =>
- closeHiveSessionStateIfStarted(sessionState)
+ sessionState.close()
SparkSQLEnv.stop(exitCode)
}
@@ -182,9 +184,9 @@ private[hive] object SparkSQLCLIDriver extends Logging {
// will set the output into an invalid buffer.
sessionState.in = System.in
try {
- sessionState.out = new PrintStream(System.out, true, UTF_8.name())
- sessionState.info = new PrintStream(System.err, true, UTF_8.name())
- sessionState.err = new PrintStream(System.err, true, UTF_8.name())
+ sessionState.out = new SessionStream(System.out, true, UTF_8.name())
+ sessionState.info = new SessionStream(System.err, true, UTF_8.name())
+ sessionState.err = new SessionStream(System.err, true, UTF_8.name())
} catch {
case e: UnsupportedEncodingException => exit(ERROR_PATH_NOT_FOUND)
}
@@ -206,17 +208,28 @@ private[hive] object SparkSQLCLIDriver extends Logging {
cli.printMasterAndAppId()
if (sessionState.execString != null) {
- exit(cli.processLine(sessionState.execString))
+ try {
+ cli.processLine(sessionState.execString)
+ exit(0)
+ } catch {
+ case e: CommandProcessorException =>
+ logError(log"Could not execute. (${MDC(ERROR, e.getMessage)})")
+ exit(e.getErrorCode)
+ }
}
try {
if (sessionState.fileName != null) {
- exit(cli.processFile(sessionState.fileName))
+ cli.processFile(sessionState.fileName)
+ exit(0)
}
} catch {
case e: FileNotFoundException =>
logError(log"Could not open input file for reading. (${MDC(ERROR, e.getMessage)})")
exit(ERROR_PATH_NOT_FOUND)
+ case e: CommandProcessorException =>
+ logError(log"Could not process input file. (${MDC(ERROR, e.getMessage)})")
+ exit(e.getErrorCode)
}
val reader = new ConsoleReader()
@@ -257,7 +270,6 @@ private[hive] object SparkSQLCLIDriver extends Logging {
}
}
- var ret = 0
var prefix = ""
def currentDB = {
@@ -285,7 +297,11 @@ private[hive] object SparkSQLCLIDriver extends Logging {
if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) {
line = prefix + line
- ret = cli.processLine(line, true)
+ try {
+ cli.processLine(line, true)
+ } catch {
+ case _: CommandProcessorException | _: RuntimeException => None
+ }
prefix = ""
currentPrompt = promptWithCurrentDB
} else {
@@ -296,9 +312,9 @@ private[hive] object SparkSQLCLIDriver extends Logging {
line = reader.readLine(currentPrompt + "> ")
}
- closeHiveSessionStateIfStarted(sessionState)
+ sessionState.close()
- exit(ret)
+ exit(0)
}
def printUsage(): Unit = {
@@ -421,14 +437,15 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
console.printInfo(s"Spark master: $master, Application Id: $appId")
}
- override def processCmd(cmd: String): Int = {
+ override def processCmd(cmd: String): CommandProcessorResponse = {
+ val ret = new CommandProcessorResponse()
val cmd_trimmed: String = cmd.trim()
val cmd_lower = cmd_trimmed.toLowerCase(Locale.ROOT)
val tokens: Array[String] = cmd_trimmed.split("\\s+")
val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim()
if (cmd_lower.equals("quit") ||
cmd_lower.equals("exit")) {
- closeHiveSessionStateIfStarted(sessionState)
+ sessionState.close()
SparkSQLCLIDriver.exit(EXIT_SUCCESS)
}
if (tokens(0).toLowerCase(Locale.ROOT).equals("source") || cmd_trimmed.startsWith("!")) {
@@ -437,9 +454,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
val endTimeNs = System.nanoTime()
val timeTaken: Double = TimeUnit.NANOSECONDS.toMillis(endTimeNs - startTimeNs) / 1000.0
console.printInfo(s"Time taken: $timeTaken seconds")
- 0
+ ret
} else {
- var ret = 0
val hconf = conf.asInstanceOf[HiveConf]
val proc: CommandProcessor = CommandProcessorFactory.get(tokens, hconf)
@@ -451,7 +467,6 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
proc.isInstanceOf[ResetProcessor] ) {
val driver = new SparkSQLDriver
- driver.init()
val out = sessionState.out
val err = sessionState.err
val startTimeNs: Long = System.nanoTime()
@@ -462,7 +477,6 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
driver.run(cmd)
} catch {
case t: Throwable =>
- ret = 1
val format = SparkSQLEnv.sparkSession.sessionState.conf.errorMessageFormat
val msg = t match {
case st: SparkThrowable with Throwable =>
@@ -476,7 +490,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
t.printStackTrace(err)
}
driver.close()
- return ret
+ throw new CommandProcessorException(t)
}
val endTimeNs = System.nanoTime()
val timeTaken: Double = TimeUnit.NANOSECONDS.toMillis(endTimeNs - startTimeNs) / 1000.0
@@ -506,12 +520,9 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
s"""Failed with exception ${e.getClass.getName}: ${e.getMessage}
|${Utils.stringifyException(e)}
""".stripMargin)
- ret = 1
- }
-
- val cret = driver.close()
- if (ret == 0) {
- ret = cret
+ throw new CommandProcessorException(e)
+ } finally {
+ driver.close()
}
var responseMsg = s"Time taken: $timeTaken seconds"
@@ -525,7 +536,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
if (sessionState.getIsVerbose) {
sessionState.out.println(tokens(0) + " " + cmd_1)
}
- ret = proc.run(cmd_1).getResponseCode
+ proc.run(cmd_1)
}
// scalastyle:on println
}
@@ -534,7 +545,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
}
// Adapted processLine from Hive 2.3's CliDriver.processLine.
- override def processLine(line: String, allowInterrupting: Boolean): Int = {
+ // Updated for Hive 4.1
+ override def processLine(line: String, allowInterrupting: Boolean): CommandProcessorResponse = {
var oldSignal: SignalHandler = null
var interruptSignal: Signal = null
@@ -559,38 +571,41 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
// to prompt
console.printInfo("Interrupting... Be patient, this might take some time.")
console.printInfo("Press Ctrl+C again to kill JVM")
-
+ // First, kill any running MR jobs
+ HadoopJobExecHelper.killRunningJobs()
+ TezJobExecHelper.killRunningJobs()
HiveInterruptUtils.interrupt()
}
})
}
try {
- var lastRet: Int = 0
-
+ var ret: CommandProcessorResponse = new CommandProcessorResponse
// we can not use "split" function directly as ";" may be quoted
val commands = splitSemiColon(line).asScala
- var command: String = ""
+ val commandBuilder = new StringBuilder
for (oneCmd <- commands) {
if (oneCmd.endsWith("\\")) {
- command += oneCmd.dropRight(1) + ";"
+ commandBuilder.append(oneCmd.dropRight(1) + ";")
} else {
- command += oneCmd
+ commandBuilder.append(oneCmd)
+ val command = commandBuilder.toString
if (!SparkStringUtils.isBlank(command)) {
- val ret = processCmd(command)
- command = ""
- lastRet = ret
- val ignoreErrors =
- HiveConf.getBoolVar(conf, HiveConf.getConfVars("hive.cli.errors.ignore"))
- if (ret != 0 && !ignoreErrors) {
- CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf])
- return ret
+ try {
+ ret = processCmd(command)
+ } catch {
+ case e: CommandProcessorException =>
+ val ignoreErrors = HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLI_IGNORE_ERRORS)
+ if (!ignoreErrors) {
+ throw e
+ }
+ } finally {
+ commandBuilder.clear()
}
}
}
}
- CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf])
- lastRet
+ ret
} finally {
// Once we are done processing the line, restore the old handler
if (oldSignal != null && interruptSignal != null) {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
index 8b9b7352fdca..a51f61213c1d 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
@@ -23,6 +23,7 @@ import scala.jdk.CollectionConverters._
import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema}
import org.apache.hadoop.hive.ql.Driver
+import org.apache.hadoop.hive.ql.QueryState
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse
import org.apache.spark.SparkThrowable
@@ -38,15 +39,12 @@ import org.apache.spark.util.Utils
private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv.sparkSession)
- extends Driver
+ extends Driver(new QueryState.Builder().build())
with Logging {
private[hive] var tableSchema: Schema = _
private[hive] var hiveResponse: Seq[String] = _
- override def init(): Unit = {
- }
-
private def getResultSetSchema(query: QueryExecution): Schema = {
val analyzed = query.analyzed
logDebug(s"Result Schema: ${analyzed.output}")
@@ -79,7 +77,7 @@ private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv.
}
}
tableSchema = getResultSetSchema(execution)
- new CommandProcessorResponse(0)
+ new CommandProcessorResponse()
} catch {
case st: SparkThrowable =>
logDebug(s"Failed in [$command]", st)
@@ -90,10 +88,9 @@ private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv.
}
}
- override def close(): Int = {
+ override def close(): Unit = {
hiveResponse = null
tableSchema = null
- 0
}
override def getResults(res: JList[_]): Boolean = {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
index 8e12165dd6f1..da4a9299a0b8 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
@@ -52,7 +52,7 @@ private[thriftserver] class SparkSQLOperationManager()
val conf = sparkSession.sessionState.conf
val runInBackground = async && conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
val operation = new SparkExecuteStatementOperation(
- sparkSession, parentSession, statement, confOverlay, runInBackground, queryTimeout)
+ sparkSession, parentSession, this, statement, confOverlay, runInBackground, queryTimeout)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created Operation for $statement with session=$parentSession, " +
s"runInBackground=$runInBackground")
@@ -64,7 +64,7 @@ private[thriftserver] class SparkSQLOperationManager()
val session = sessionToContexts.get(parentSession.getSessionHandle)
require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" +
" initialized or had already closed.")
- val operation = new SparkGetCatalogsOperation(session, parentSession)
+ val operation = new SparkGetCatalogsOperation(session, parentSession, this)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created GetCatalogsOperation with session=$parentSession.")
operation
@@ -77,7 +77,8 @@ private[thriftserver] class SparkSQLOperationManager()
val session = sessionToContexts.get(parentSession.getSessionHandle)
require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" +
" initialized or had already closed.")
- val operation = new SparkGetSchemasOperation(session, parentSession, catalogName, schemaName)
+ val operation = new SparkGetSchemasOperation(session, parentSession, this, catalogName,
+ schemaName)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created GetSchemasOperation with session=$parentSession.")
operation
@@ -92,7 +93,7 @@ private[thriftserver] class SparkSQLOperationManager()
val session = sessionToContexts.get(parentSession.getSessionHandle)
require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" +
" initialized or had already closed.")
- val operation = new SparkGetTablesOperation(session, parentSession,
+ val operation = new SparkGetTablesOperation(session, parentSession, this,
catalogName, schemaName, tableName, tableTypes)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created GetTablesOperation with session=$parentSession.")
@@ -108,7 +109,7 @@ private[thriftserver] class SparkSQLOperationManager()
val session = sessionToContexts.get(parentSession.getSessionHandle)
require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" +
" initialized or had already closed.")
- val operation = new SparkGetColumnsOperation(session, parentSession,
+ val operation = new SparkGetColumnsOperation(session, parentSession, this,
catalogName, schemaName, tableName, columnName)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created GetColumnsOperation with session=$parentSession.")
@@ -120,7 +121,7 @@ private[thriftserver] class SparkSQLOperationManager()
val session = sessionToContexts.get(parentSession.getSessionHandle)
require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" +
" initialized or had already closed.")
- val operation = new SparkGetTableTypesOperation(session, parentSession)
+ val operation = new SparkGetTableTypesOperation(session, parentSession, this)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created GetTableTypesOperation with session=$parentSession.")
operation
@@ -134,7 +135,7 @@ private[thriftserver] class SparkSQLOperationManager()
val session = sessionToContexts.get(parentSession.getSessionHandle)
require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" +
" initialized or had already closed.")
- val operation = new SparkGetFunctionsOperation(session, parentSession,
+ val operation = new SparkGetFunctionsOperation(session, parentSession, this,
catalogName, schemaName, functionName)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created GetFunctionsOperation with session=$parentSession.")
@@ -146,7 +147,7 @@ private[thriftserver] class SparkSQLOperationManager()
val session = sessionToContexts.get(parentSession.getSessionHandle)
require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" +
" initialized or had already closed.")
- val operation = new SparkGetTypeInfoOperation(session, parentSession)
+ val operation = new SparkGetTypeInfoOperation(session, parentSession, this)
handleToOperation.put(operation.getHandle, operation)
logDebug(s"Created GetTypeInfoOperation with session=$parentSession.")
operation
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
index 655c3526f5cb..b2abb28a7e77 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver
import java.io._
import java.nio.charset.StandardCharsets
-import java.util.concurrent.CountDownLatch
+import java.util.concurrent.{CountDownLatch, TimeUnit}
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.Promise
@@ -238,6 +238,7 @@ class CliSuite extends SparkFunSuite {
// --hiveconf will overrides hive-site.xml
runCliWithin(2.minute, useExternalHiveFile = true)(
"desc database default;" -> warehousePath.getAbsolutePath,
+ "drop database if exists cliTestDb;" -> "",
"create database cliTestDb;" -> "",
"desc database cliTestDb;" -> warehousePath.getAbsolutePath,
"set spark.sql.warehouse.dir;" -> warehousePath.getAbsolutePath)
@@ -255,6 +256,7 @@ class CliSuite extends SparkFunSuite {
useExternalHiveFile = true,
metastore = metastore)(
"desc database default;" -> sparkWareHouseDir.getAbsolutePath,
+ "drop database if exists cliTestDb;" -> "",
"create database cliTestDb;" -> "",
"desc database cliTestDb;" -> sparkWareHouseDir.getAbsolutePath,
"set spark.sql.warehouse.dir;" -> sparkWareHouseDir.getAbsolutePath)
@@ -697,7 +699,7 @@ class CliSuite extends SparkFunSuite {
}
t.start()
t.start()
- cd.await()
+ assert(cd.await(5, TimeUnit.MINUTES))
}
// scalastyle:off line.size.limit
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala
index 1bc9aaf672c3..3f09e6045490 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala
@@ -20,12 +20,12 @@ import java.nio.ByteBuffer
import java.util.UUID
import org.apache.hive.service.cli.OperationHandle
-import org.apache.hive.service.cli.operation.GetCatalogsOperation
+import org.apache.hive.service.cli.operation.{GetCatalogsOperation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType}
-class GetCatalogsOperationMock(parentSession: HiveSession)
- extends GetCatalogsOperation(parentSession) {
+class GetCatalogsOperationMock(parentSession: HiveSession, operationManager: OperationManager)
+ extends GetCatalogsOperation(parentSession, operationManager) {
override def runInternal(): Unit = {}
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala
index 035b71a37a69..423e9ed04510 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala
@@ -37,6 +37,8 @@ class HiveCliSessionStateSuite extends SparkFunSuite {
HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false).foreach {
case (key, value) => hiveConf.set(key, value)
}
+ // Set Hive execution engine to map-reduce for backward compatibility with Hive 2.3.x
+ HiveConf.setVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "mr")
val sessionState: SessionState = new CliSessionState(hiveConf)
SessionState.start(sessionState)
val s1 = SessionState.get
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala
index 6f89fbfb788b..eb0e91728fda 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala
@@ -67,7 +67,7 @@ class OperationManagerMock extends OperationManager {
private val calledHandles: mutable.Set[OperationHandle] = new mutable.HashSet[OperationHandle]()
override def newGetCatalogsOperation(parentSession: HiveSession): GetCatalogsOperation = {
- val operation = new GetCatalogsOperationMock(parentSession)
+ val operation = new GetCatalogsOperationMock(parentSession, this)
try {
val m = classOf[OperationManager].getDeclaredMethod("addOperation", classOf[Operation])
m.setAccessible(true)
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
index 6f0fedcb8536..590df2d28850 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
@@ -113,7 +113,7 @@ trait SharedThriftServer extends SharedSparkSession {
val interceptor = new HttpBasicAuthInterceptor(
username,
"anonymous",
- null, null, true, new util.HashMap[String, String]())
+ null, null, true, new util.HashMap[String, String](), new util.HashMap[String, String]())
new THttpClient(
s"http://localhost:$serverPort/cliservice",
HttpClientBuilder.create.addInterceptorFirst(interceptor).build())
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala
index 5abf034c1dea..902d604c08d6 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala
@@ -24,6 +24,7 @@ import scala.concurrent.duration._
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hive.service.cli.OperationState
+import org.apache.hive.service.cli.operation.OperationManager
import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl}
import org.apache.hive.service.rpc.thrift.{TProtocolVersion, TTypeId}
import org.mockito.Mockito.{doReturn, mock, spy, when, RETURNS_DEEP_STUBS}
@@ -93,7 +94,7 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark
doReturn(dataFrame, Nil: _*).when(spySparkSession).sql(statement)
val executeStatementOperation = new MySparkExecuteStatementOperation(spySparkSession,
- hiveSession, statement, signal, finalState)
+ hiveSession, new OperationManagerMock(), statement, signal, finalState)
val run = new Thread() {
override def run(): Unit = executeStatementOperation.runInternal()
@@ -112,10 +113,11 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark
private class MySparkExecuteStatementOperation(
session: SparkSession,
hiveSession: HiveSession,
+ operationManager: OperationManager,
statement: String,
signal: Semaphore,
finalState: OperationState)
- extends SparkExecuteStatementOperation(session, hiveSession, statement,
+ extends SparkExecuteStatementOperation(session, hiveSession, operationManager, statement,
new util.HashMap, false, 0) {
override def cleanup(): Unit = {
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
index abd2b1983b34..41cfe928fbb7 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
@@ -471,24 +471,9 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase {
val metaData = statement.getConnection.getMetaData
Seq(
() => metaData.allProceduresAreCallable,
- () => metaData.getURL,
- () => metaData.getUserName,
() => metaData.isReadOnly,
- () => metaData.nullsAreSortedHigh,
- () => metaData.nullsAreSortedLow,
- () => metaData.nullsAreSortedAtStart,
- () => metaData.nullsAreSortedAtEnd,
() => metaData.usesLocalFiles,
() => metaData.usesLocalFilePerTable,
- () => metaData.supportsMixedCaseIdentifiers,
- () => metaData.supportsMixedCaseQuotedIdentifiers,
- () => metaData.storesUpperCaseIdentifiers,
- () => metaData.storesUpperCaseQuotedIdentifiers,
- () => metaData.storesLowerCaseIdentifiers,
- () => metaData.storesLowerCaseQuotedIdentifiers,
- () => metaData.storesMixedCaseIdentifiers,
- () => metaData.storesMixedCaseQuotedIdentifiers,
- () => metaData.getSQLKeywords,
() => metaData.nullPlusNonNullIsNull,
() => metaData.supportsConvert,
() => metaData.supportsTableCorrelationNames,
@@ -569,7 +554,6 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase {
() => metaData.getRowIdLifetime,
() => metaData.supportsStoredFunctionsUsingCallSyntax,
() => metaData.autoCommitFailureClosesAllResultSets,
- () => metaData.getClientInfoProperties,
() => metaData.getFunctionColumns("", "%", "%", "%"),
() => metaData.getPseudoColumns("", "%", "%", "%"),
() => metaData.generatedKeyAlwaysReturned).foreach { func =>
@@ -594,8 +578,7 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase {
assert(metaData.getDriverVersion === HiveVersionInfo.getVersion)
assert(metaData.getDatabaseMajorVersion === VersionUtils.majorVersion(SPARK_VERSION))
assert(metaData.getDatabaseMinorVersion === VersionUtils.minorVersion(SPARK_VERSION))
- assert(metaData.getIdentifierQuoteString === " ",
- "This method returns a space \" \" if identifier quoting is not supported")
+ assert(metaData.getIdentifierQuoteString === "`")
assert(metaData.getNumericFunctions === "")
assert(metaData.getStringFunctions === "")
assert(metaData.getSystemFunctions === "")
@@ -658,6 +641,22 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase {
assert(metaData.getSQLStateType === DatabaseMetaData.sqlStateSQL)
assert(metaData.getMaxLogicalLobSize === 0)
assert(!metaData.supportsRefCursors)
+ assert(metaData.getURL.startsWith("jdbc:hive2://"))
+ assert(metaData.getUserName === System.getProperty("user.name"))
+ assert(metaData.nullsAreSortedHigh)
+ assert(!metaData.nullsAreSortedLow)
+ assert(!metaData.nullsAreSortedAtStart)
+ assert(!metaData.nullsAreSortedAtEnd)
+ assert(!metaData.supportsMixedCaseIdentifiers)
+ assert(!metaData.supportsMixedCaseQuotedIdentifiers)
+ assert(!metaData.storesUpperCaseIdentifiers)
+ assert(!metaData.storesUpperCaseQuotedIdentifiers)
+ assert(metaData.storesLowerCaseIdentifiers)
+ assert(metaData.storesLowerCaseQuotedIdentifiers)
+ assert(!metaData.storesMixedCaseIdentifiers)
+ assert(!metaData.storesMixedCaseQuotedIdentifiers)
+ assert(metaData.getSQLKeywords != "")
+ assert(metaData.getClientInfoProperties != null)
}
}
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala
index 60afcf815361..3581bfc478b1 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala
@@ -64,12 +64,15 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftServer2TestBase {
rs = new HiveQueryResultSet.Builder(connection)
.setClient(client)
- .setSessionHandle(sessHandle)
- .setStmtHandle(stmtHandle).setMaxRows(Int.MaxValue).setFetchSize(Int.MaxValue)
+ .setStmtHandle(stmtHandle)
+ .setMaxRows(Int.MaxValue)
+ .setFetchSize(Int.MaxValue)
.build()
f(rs)
} finally {
- rs.close()
+ if (rs != null) {
+ rs.close()
+ }
connection.close()
transport.close()
rawTransport.close()
@@ -133,7 +136,6 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftServer2TestBase {
rs = new HiveQueryResultSet.Builder(connection)
.setClient(client)
- .setSessionHandle(sessHandle)
.setStmtHandle(getTableResp.getOperationHandle)
.build()
f(rs)
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 4f5f50433ea1..08becc04301c 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -218,6 +218,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"alter1",
"input16",
+ // HIVE-23483: Hive 4 does not support DynamicSerDe
+ "implicit_cast1",
+ "input19",
+
// No support for unpublished test udfs.
"autogen_colalias",
@@ -734,7 +738,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"groupby_ppr",
"groupby_sort_6",
"having",
- "implicit_cast1",
"index_serde",
"infer_bucket_sort_dyn_part",
"innerjoin",
@@ -748,7 +751,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"input12_hadoop20",
"input14",
"input15",
- "input19",
"input1_limit",
"input2",
"input21",
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index 1d4c7b784c0a..0ec0891a8fc7 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -138,6 +138,37 @@
hive-llap-client
${hive.llap.scope}
+
+ ${hive.group}
+ hive-udf
+
+
+ ${hive.group}
+ hive-standalone-metastore-server
+ test
+
+
+ org.apache.datasketches
+ datasketches-hive
+ test
+
+
+ ${hive.group}
+ hive-parser
+ ${hive.deps.scope}
+
+
+ org.apache.calcite
+ calcite-core
+ ${calcite.deps.scope}
+
+
+ org.apache.hadoop
+ hadoop-hdfs
+ ${hadoop.version}
+ ${hadoop.deps.scope}
+
+
org.apache.avro
@@ -172,6 +203,10 @@
org.datanucleus
datanucleus-core
+
+ org.datanucleus
+ datanucleus-api-jdo
+
org.apache.hadoop
hadoop-client-runtime
@@ -185,13 +220,25 @@
org.apache.thrift
libfb303
+
+ com.esotericsoftware.kryo
+ kryo5
+ 5.5.0
+
+
+ org.jodd
+ jodd-util
+ 6.0.0
+
org.apache.derby
derby
+ runtime
org.apache.derby
derbytools
+ runtime
org.scala-lang
@@ -203,6 +250,10 @@
scalacheck_${scala.binary.version}
test
+
+ org.apache.maven.shared
+ maven-invoker
+
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
index 6389e130b6a2..bacc98dc6f15 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
@@ -345,19 +345,16 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
// converts the table metadata to Hive compatible format, i.e. set the serde information.
def newHiveCompatibleMetastoreTable(serde: HiveSerDe): CatalogTable = {
- val location = if (table.tableType == EXTERNAL) {
+ if (table.tableType == EXTERNAL) {
// When we hit this branch, we are saving an external data source table with hive
// compatible format, which means the data source is file-based and must have a `path`.
require(table.storage.locationUri.isDefined,
"External file-based data source table must have a `path` entry in storage properties.")
- Some(table.location)
- } else {
- None
}
table.copy(
storage = table.storage.copy(
- locationUri = location,
+ locationUri = Some(table.location),
inputFormat = serde.inputFormat,
outputFormat = serde.outputFormat,
serde = serde.serde,
@@ -552,8 +549,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
// properties, to avoid adding a unnecessary path option for Hive serde tables.
val hasPathOption = CaseInsensitiveMap(rawTable.storage.properties).contains("path")
val storageWithNewPath = if (rawTable.tableType == MANAGED && hasPathOption) {
- // If it's a managed table with path option and we are renaming it, then the path option
- // becomes inaccurate and we need to update it according to the new table name.
+ // If it's a managed table with path option, and we are renaming it, then the path option
+ // becomes inaccurate, and we need to update it according to the new table name.
val newTablePath = defaultTablePath(TableIdentifier(newName, Some(db)))
updateLocationInStorageProps(rawTable, Some(newTablePath))
} else {
@@ -854,7 +851,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
// Get the original table properties as defined by the user.
table.copy(
createVersion = version,
- properties = table.properties.filterNot { case (key, _) => key.startsWith(SPARK_SQL_PREFIX) })
+ properties = table.properties.filterNot { case (key, value) =>
+ key.startsWith(SPARK_SQL_PREFIX) ||
+ key == "bucketing_version" && value == "2" ||
+ key == "numFilesErasureCoded" && value == "0"})
}
// Reorder table schema to put partition columns at the end. Before Spark 2.2, the partition
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 087f68ce3c79..4f4e2df65eb6 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -34,8 +34,8 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.catalyst.util.RebaseDateTime.rebaseJulianToGregorianDays
import org.apache.spark.sql.errors.DataTypeErrors.toSQLType
-import org.apache.spark.sql.execution.datasources.DaysWritable
import org.apache.spark.sql.types
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -93,8 +93,8 @@ import org.apache.spark.unsafe.types.UTF8String
* org.apache.hadoop.hive.serde2.io.ShortWritable
* org.apache.hadoop.hive.serde2.io.ByteWritable
* org.apache.hadoop.io.BytesWritable
- * org.apache.hadoop.hive.serde2.io.DateWritable
- * org.apache.hadoop.hive.serde2.io.TimestampWritable
+ * org.apache.hadoop.hive.serde2.io.DateWritableV2
+ * org.apache.hadoop.hive.serde2.io.TimestampWritableV2
* org.apache.hadoop.hive.serde2.io.HiveDecimalWritable
* Complex Type
* List: Object[] / java.util.List
@@ -192,7 +192,9 @@ private[hive] trait HiveInspectors {
case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType
case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType
case c: Class[_] if c == classOf[hiveIo.DateWritable] => DateType
+ case c: Class[_] if c == classOf[hiveIo.DateWritableV2] => DateType
case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType
+ case c: Class[_] if c == classOf[hiveIo.TimestampWritableV2] => TimestampType
case c: Class[_] if c == classOf[hadoopIo.Text] => StringType
case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType
case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType
@@ -328,10 +330,10 @@ private[hive] trait HiveInspectors {
HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal))
case _: JavaDateObjectInspector =>
withNullSafe(o =>
- DateTimeUtils.toJavaDate(o.asInstanceOf[Int]))
+ HiveDateTimeUtils.toHiveDate(o.asInstanceOf[Int]))
case _: JavaTimestampObjectInspector =>
withNullSafe(o =>
- DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long]))
+ HiveDateTimeUtils.toHiveTimestamp(o.asInstanceOf[Long]))
case _: HiveDecimalObjectInspector if x.preferWritable() =>
withNullSafe(o => getDecimalWritable(o.asInstanceOf[Decimal]))
case _: HiveDecimalObjectInspector =>
@@ -342,11 +344,11 @@ private[hive] trait HiveInspectors {
case _: BinaryObjectInspector =>
withNullSafe(o => o.asInstanceOf[Array[Byte]])
case _: DateObjectInspector if x.preferWritable() =>
- withNullSafe(o => getDateWritable(o))
+ withNullSafe(o => getDateWritableV2(o))
case _: DateObjectInspector =>
withNullSafe(o => DateTimeUtils.toJavaDate(o.asInstanceOf[Int]))
case _: TimestampObjectInspector if x.preferWritable() =>
- withNullSafe(o => getTimestampWritable(o))
+ withNullSafe(o => getTimestampWritableV2(o))
case _: TimestampObjectInspector =>
withNullSafe(o => DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long]))
case _: HiveIntervalDayTimeObjectInspector if x.preferWritable() =>
@@ -480,7 +482,7 @@ private[hive] trait HiveInspectors {
_ => constant
case poi: WritableConstantTimestampObjectInspector =>
val t = poi.getWritableConstantValue
- val constant = DateTimeUtils.fromJavaTimestamp(t.getTimestamp)
+ val constant = HiveDateTimeUtils.fromHiveTimestamp(t.getTimestamp)
_ => constant
case poi: WritableConstantIntObjectInspector =>
val constant = poi.getWritableConstantValue.get()
@@ -509,7 +511,7 @@ private[hive] trait HiveInspectors {
System.arraycopy(writable.getBytes, 0, constant, 0, constant.length)
_ => constant
case poi: WritableConstantDateObjectInspector =>
- val constant = DateTimeUtils.fromJavaDate(poi.getWritableConstantValue.get())
+ val constant = HiveDateTimeUtils.fromHiveDate(poi.getWritableConstantValue.get())
_ => constant
case mi: StandardConstantMapObjectInspector =>
val keyUnwrapper = unwrapperFor(mi.getMapKeyObjectInspector)
@@ -633,7 +635,8 @@ private[hive] trait HiveInspectors {
case x: DateObjectInspector if x.preferWritable() =>
data: Any => {
if (data != null) {
- new DaysWritable(x.getPrimitiveWritableObject(data)).gregorianDays
+ val d = rebaseJulianToGregorianDays(x.getPrimitiveWritableObject(data).getDays)
+ d
} else {
null
}
@@ -641,7 +644,7 @@ private[hive] trait HiveInspectors {
case x: DateObjectInspector =>
data: Any => {
if (data != null) {
- DateTimeUtils.fromJavaDate(x.getPrimitiveJavaObject(data))
+ HiveDateTimeUtils.fromHiveDate(x.getPrimitiveJavaObject(data))
} else {
null
}
@@ -649,15 +652,15 @@ private[hive] trait HiveInspectors {
case x: TimestampObjectInspector if x.preferWritable() =>
data: Any => {
if (data != null) {
- DateTimeUtils.fromJavaTimestamp(x.getPrimitiveWritableObject(data).getTimestamp)
+ HiveDateTimeUtils.fromHiveTimestamp(x.getPrimitiveWritableObject(data).getTimestamp)
} else {
null
}
}
- case ti: TimestampObjectInspector =>
+ case x: TimestampObjectInspector =>
data: Any => {
if (data != null) {
- DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data))
+ HiveDateTimeUtils.fromHiveTimestamp(x.getPrimitiveJavaObject(data))
} else {
null
}
@@ -1029,11 +1032,11 @@ private[hive] trait HiveInspectors {
private def getDateWritableConstantObjectInspector(value: Any): ObjectInspector =
PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.dateTypeInfo, getDateWritable(value))
+ TypeInfoFactory.dateTypeInfo, getDateWritableV2(value))
private def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector =
PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.timestampTypeInfo, getTimestampWritable(value))
+ TypeInfoFactory.timestampTypeInfo, getTimestampWritableV2(value))
private def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector =
PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
@@ -1094,18 +1097,20 @@ private[hive] trait HiveInspectors {
new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])
}
- private def getDateWritable(value: Any): DaysWritable =
+ private def getDateWritableV2(value: Any): hiveIo.DateWritableV2 =
if (value == null) {
null
} else {
- new DaysWritable(value.asInstanceOf[Int])
+ val d = new hiveIo.DateWritableV2(value.asInstanceOf[Int])
+ d
}
- private def getTimestampWritable(value: Any): hiveIo.TimestampWritable =
+ private def getTimestampWritableV2(value: Any): hiveIo.TimestampWritableV2 =
if (value == null) {
null
} else {
- new hiveIo.TimestampWritable(DateTimeUtils.toJavaTimestamp(value.asInstanceOf[Long]))
+ new hiveIo.TimestampWritableV2(
+ HiveDateTimeUtils.toHiveTimestamp(value.asInstanceOf[Long]))
}
private def getHiveIntervalDayTimeWritable(value: Any): hiveIo.HiveIntervalDayTimeWritable =
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
index a36501d1eaf4..869afb636d11 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
@@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.common.FileUtils
import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.util.VersionInfo
import org.apache.hive.common.util.HiveVersionInfo
@@ -62,7 +62,7 @@ private[spark] object HiveUtils extends Logging {
" Note that, this a read-only conf and only used to report the built-in hive version." +
" If you want a different metastore client for Spark to call, please refer to" +
" spark.sql.hive.metastore.version.")
- .version("1.1.1")
+ .version("4.1.0")
.stringConf
.checkValue(_ == builtinHiveVersion,
"The builtin Hive version is read-only, please use spark.sql.hive.metastore.version")
@@ -77,7 +77,7 @@ private[spark] object HiveUtils extends Logging {
"2.0.0 through 2.3.10, " +
"3.0.0 through 3.1.3 and " +
"4.0.0 through 4.1.0.")
- .version("1.4.0")
+ .version("4.1.0")
.stringConf
.checkValue(isCompatibleHiveVersion, "Unsupported Hive Metastore version")
.createWithDefault(builtinHiveVersion)
@@ -474,14 +474,14 @@ private[spark] object HiveUtils extends Logging {
// You can search hive.metastore.uris in the code of HiveConf (in Hive's repo).
// Then, you will find that the local metastore mode is only set to true when
// hive.metastore.uris is not set.
- propMap.put("hive.metastore.uris", "")
+ propMap.put(ConfVars.THRIFT_URIS.getVarname, "")
// The execution client will generate garbage events, therefore the listeners that are generated
// for the execution clients are useless. In order to not output garbage, we don't generate
// these listeners.
- propMap.put(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname, "")
- propMap.put(ConfVars.METASTORE_EVENT_LISTENERS.varname, "")
- propMap.put(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname, "")
+ propMap.put(ConfVars.PRE_EVENT_LISTENERS.getVarname, "")
+ propMap.put(ConfVars.EVENT_LISTENERS.getVarname, "")
+ propMap.put(ConfVars.END_FUNCTION_LISTENERS.getVarname, "")
// SPARK-21451: Spark will gather all `spark.hadoop.*` properties from a `SparkConf` to a
// Hadoop Configuration internally, as long as it happens after SparkContext initialized.
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index d6f62284d2e7..3d4cfb9f3dd6 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._
import org.apache.hadoop.hive.ql.exec.Utilities
import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable}
import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc}
-import org.apache.hadoop.hive.serde2.Deserializer
+import org.apache.hadoop.hive.serde2.{AbstractSerDe, Deserializer}
import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils.AvroTableProperties
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector}
import org.apache.hadoop.hive.serde2.objectinspector.primitive._
@@ -43,11 +43,11 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper}
import org.apache.spark.sql.catalyst.analysis.CastSupport
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util.HiveDateTimeUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.unsafe.types.UTF8String
-import org.apache.spark.util.{SerializableConfiguration, Utils}
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.SerializableConfiguration
/**
* A trait for subclasses that handle table scans.
@@ -96,7 +96,7 @@ class HadoopTableReader(
override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] =
makeRDDForTable(
hiveTable,
- Utils.classForName[Deserializer](tableDesc.getSerdeClassName),
+ tableDesc.getDeserializer(_broadcastedHadoopConf.value.value).getClass,
filterOpt = None)
/**
@@ -110,7 +110,7 @@ class HadoopTableReader(
*/
def makeRDDForTable(
hiveTable: HiveTable,
- deserializerClass: Class[_ <: Deserializer],
+ abstractSerDeClass: Class[_ <: AbstractSerDe],
filterOpt: Option[PathFilter]): RDD[InternalRow] = {
assert(!hiveTable.isPartitioned,
@@ -133,11 +133,11 @@ class HadoopTableReader(
val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter =>
val hconf = broadcastedHadoopConf.value.value
- val deserializer = deserializerClass.getConstructor().newInstance()
+ val abstractSerDe = abstractSerDeClass.getConstructor().newInstance()
DeserializerLock.synchronized {
- deserializer.initialize(hconf, localTableDesc.getProperties)
+ abstractSerDe.initialize(hconf, localTableDesc.getProperties, null)
}
- HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer)
+ HadoopTableReader.fillObject(iter, abstractSerDe, attrsWithIndex, mutableRow, abstractSerDe)
}
deserializedHadoopRDD
@@ -145,7 +145,7 @@ class HadoopTableReader(
override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[InternalRow] = {
val partitionToDeserializer = partitions.map(part =>
- (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap
+ (part, part.getDeserializer.getClass)).toMap
makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None)
}
@@ -162,7 +162,7 @@ class HadoopTableReader(
def makeRDDForPartitionedTable(
partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]],
filterOpt: Option[PathFilter]): RDD[InternalRow] = {
- val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) =>
+ val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializerClass) =>
val partDesc = Utilities.getPartitionDescFromTableDesc(tableDesc, partition, true)
val partPath = partition.getDataLocation
val inputPathStr = applyFilterIfNeeded(partPath, filterOpt)
@@ -181,7 +181,7 @@ class HadoopTableReader(
}
val broadcastedHiveConf = _broadcastedHadoopConf
- val localDeserializer = partDeserializer
+ val localAbstractSerDeClass = partDeserializerClass
val mutableRow = new SpecificInternalRow(attributes.map(_.dataType))
// Splits all attributes into two groups, partition key attributes and those that are not.
@@ -210,7 +210,8 @@ class HadoopTableReader(
createHadoopRDD(partDesc, inputPathStr).mapPartitions { iter =>
val hconf = broadcastedHiveConf.value.value
- val deserializer = localDeserializer.getConstructor().newInstance()
+ val deserializer = localAbstractSerDeClass.getConstructor().newInstance()
+ .asInstanceOf[AbstractSerDe]
// SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema
// information) may be defined in table properties. Here we should merge table properties
// and partition properties before initializing the deserializer. Note that partition
@@ -226,13 +227,10 @@ class HadoopTableReader(
case (key, value) => props.setProperty(key, value)
}
DeserializerLock.synchronized {
- deserializer.initialize(hconf, props)
+ deserializer.initialize(hconf, props, partProps)
}
// get the table deserializer
- val tableSerDe = localTableDesc.getDeserializerClass.getConstructor().newInstance()
- DeserializerLock.synchronized {
- tableSerDe.initialize(hconf, tableProperties)
- }
+ val tableSerDe = localTableDesc.getDeserializer(hconf)
// fill the non partition key attributes
HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs,
@@ -488,10 +486,11 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging {
row.update(ordinal, HiveShim.toCatalystDecimal(oi, value))
case oi: TimestampObjectInspector =>
(value: Any, row: InternalRow, ordinal: Int) =>
- row.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(oi.getPrimitiveJavaObject(value)))
+ row.setLong(ordinal,
+ HiveDateTimeUtils.fromHiveTimestamp(oi.getPrimitiveJavaObject(value)))
case oi: DateObjectInspector =>
(value: Any, row: InternalRow, ordinal: Int) =>
- row.setInt(ordinal, DateTimeUtils.fromJavaDate(oi.getPrimitiveJavaObject(value)))
+ row.setInt(ordinal, HiveDateTimeUtils.fromHiveDate(oi.getPrimitiveJavaObject(value)))
case oi: BinaryObjectInspector =>
(value: Any, row: InternalRow, ordinal: Int) =>
row.update(ordinal, oi.getPrimitiveJavaObject(value))
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 f1fe95b8a318..f2df6a12651d 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
@@ -28,6 +28,7 @@ import scala.annotation.tailrec
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._
+import scala.util.{Failure, Success, Try, Using}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@@ -35,11 +36,13 @@ import org.apache.hadoop.hive.common.StatsSetupConst
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.metastore.{HiveMetaStoreClient, IMetaStoreClient, RetryingMetaStoreClient, 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.metastore.conf.MetastoreConf
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.{AUTO_CREATE_ALL, METASTORE_METADATA_TRANSFORMER_CLASS, SCHEMA_VERIFICATION, STATS_AUTO_GATHER}
+import org.apache.hadoop.hive.ql.IDriver
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.processors.CommandProcessorException
import org.apache.hadoop.hive.ql.session.SessionState
+import org.apache.hadoop.hive.ql.util.DirectionUtils
import org.apache.hadoop.hive.serde.serdeConstants
import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe
import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
@@ -67,6 +70,7 @@ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils}
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.sql.util.CaseInsensitiveStringMap
import org.apache.spark.util.{CircularBuffer, Utils}
/**
@@ -193,6 +197,9 @@ private[hive] class HiveClientImpl(
/** Returns the configuration for the current session. */
def conf: HiveConf = {
val hiveConf = state.getConf
+ // Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false.
+ HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, false)
+
// 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.
@@ -207,8 +214,12 @@ private[hive] class HiveClientImpl(
(msConnUrl != null && msConnUrl.startsWith("jdbc:derby"))
}
if (isEmbeddedMetaStore) {
- hiveConf.setBoolean("hive.metastore.schema.verification", false)
- hiveConf.setBoolean("datanucleus.schema.autoCreateAll", true)
+ MetastoreConf.setBoolVar(hiveConf, SCHEMA_VERIFICATION, false)
+ MetastoreConf.setBoolVar(hiveConf, AUTO_CREATE_ALL, true)
+ MetastoreConf.setVar(hiveConf, METASTORE_METADATA_TRANSFORMER_CLASS, "")
+ MetastoreConf.setBoolVar(hiveConf, STATS_AUTO_GATHER, true)
+ // TODO: check if this can be enabled back
+ // MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.TRY_DIRECT_SQL, false)
}
hiveConf
}
@@ -473,7 +484,7 @@ private[hive] class HiveClientImpl(
// 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 allAscendingSorted = sortColumnOrders.forall(_.getOrder == DirectionUtils.ASCENDING_CODE)
val sortColumnNames = if (allAscendingSorted) {
sortColumnOrders.map(_.getCol)
@@ -868,28 +879,8 @@ private[hive] class HiveClientImpl(
* 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 && version != hive.v4_0) {
- CommandProcessorFactory.clean(conf)
- }
- }
-
- def getResponseCode(response: CommandProcessorResponse): Int = {
- if (version < hive.v4_0) {
- response.getResponseCode
- } else {
- // Since Hive 4.0, response code is removed from CommandProcessorResponse.
- // Here we simply return 0 for the positive cases as for error cases it will
- // throw exceptions early.
- 0
- }
- }
// Hive query needs to start SessionState.
SessionState.start(state)
@@ -900,47 +891,34 @@ private[hive] class HiveClientImpl(
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 =>
- try {
- val response: CommandProcessorResponse = driver.run(cmd)
- if (getResponseCode(response) != 0) {
- // Throw an exception if there is an error in query processing.
- // This works for hive 3.x and earlier versions.
- throw new QueryExecutionException(response.getErrorMessage)
- }
- driver.setMaxRows(maxRows)
- val results = shim.getDriverResults(driver)
- results
- } catch {
- case e @ (_: QueryExecutionException | _: SparkThrowable) =>
- throw e
- case e: Exception =>
- // Wrap the original hive error with QueryExecutionException and throw it
- // if there is an error in query processing.
- // This works for hive 4.x and later versions.
- throw new QueryExecutionException(Utils.stackTraceToString(e))
- } finally {
- closeDriver(driver)
- }
-
- case _ =>
+ val results: Try[Seq[String]] = Using(shim.getCommandProcessor(tokens(0), conf)) {
+ case driver: IDriver =>
+ driver.run(cmd)
+ driver.setMaxRows(maxRows)
+ shim.getDriverResults(driver)
+ case proc =>
val out = state.getClass.getField("out").get(state)
if (out != null) {
// scalastyle:off println
out.asInstanceOf[PrintStream].println(tokens(0) + " " + cmd_1)
// scalastyle:on println
}
- val response: CommandProcessorResponse = proc.run(cmd_1)
- val responseCode = getResponseCode(response)
- if (responseCode != 0) {
- // Throw an exception if there is an error in query processing.
- // This works for hive 3.x and earlier versions. For 4.x and later versions,
- // It will go to the catch block directly.
- throw new QueryExecutionException(response.getErrorMessage)
+ proc.run(cmd_1)
+ Seq.empty
+ }
+ results match {
+ case Success(s) =>
+ s
+ case Failure(e) =>
+ e match {
+ case e@(_: QueryExecutionException | _: SparkThrowable) =>
+ throw e
+ case e: CommandProcessorException =>
+ // Wrap the original hive error with QueryExecutionException and throw it
+ // if there is an error in query processing.
+ // This works for hive 4.x and later versions.
+ throw new QueryExecutionException(Utils.stackTraceToString(e))
}
- Seq(responseCode.toString)
}
} catch {
case e: Exception =>
@@ -1063,12 +1041,7 @@ private[hive] class HiveClientImpl(
val t = table.getTableName
logDebug(s"Deleting table $t")
try {
- shim.getIndexes(client, "default", t, 255).foreach { index =>
- shim.dropIndex(client, "default", t, index.getIndexName)
- }
- if (!table.isIndexTable) {
- shim.dropTable(client, "default", t)
- }
+ shim.dropTable(client, "default", t)
} catch {
case _: NoSuchMethodError =>
// HIVE-18448 Hive 3.0 remove index APIs
@@ -1198,18 +1171,25 @@ private[hive] object HiveClientImpl extends Logging {
}
table.bucketSpec match {
- case Some(bucketSpec) if !HiveExternalCatalog.isDatasourceTable(table) =>
- hiveTable.setNumBuckets(bucketSpec.numBuckets)
- hiveTable.setBucketCols(bucketSpec.bucketColumnNames.toList.asJava)
-
- if (bucketSpec.sortColumnNames.nonEmpty) {
- hiveTable.setSortCols(
- bucketSpec.sortColumnNames
- .map(col => new Order(col, HIVE_COLUMN_ORDER_ASC))
- .toList
- .asJava
- )
+ case Some(bucketSpec) =>
+ def setBuckets(): Unit = {
+ if (HiveExternalCatalog.isDatasourceTable(table)) {
+ hiveTable.setNumBuckets(0)
+ } else {
+ hiveTable.setNumBuckets(bucketSpec.numBuckets)
+ hiveTable.setBucketCols(bucketSpec.bucketColumnNames.toList.asJava)
+
+ if (bucketSpec.sortColumnNames.nonEmpty) {
+ hiveTable.setSortCols(
+ bucketSpec.sortColumnNames
+ .map(col => new Order(col, DirectionUtils.ASCENDING_CODE))
+ .toList
+ .asJava
+ )
+ }
+ }
}
+ setBuckets()
case _ =>
}
@@ -1224,8 +1204,9 @@ private[hive] object HiveClientImpl extends Logging {
p: CatalogTablePartition,
ht: HiveTable): HivePartition = {
val tpart = new org.apache.hadoop.hive.metastore.api.Partition
+ val spec = new CaseInsensitiveStringMap(p.spec.asJava).asScala.view
val partValues = ht.getPartCols.asScala.map { hc =>
- p.spec.getOrElse(hc.getName, throw new IllegalArgumentException(
+ spec.getOrElse(hc.getName, throw new IllegalArgumentException(
s"Partition spec is missing a value for column '${hc.getName}': ${p.spec}"))
}
val storageDesc = new StorageDescriptor
@@ -1373,6 +1354,8 @@ private[hive] object HiveClientImpl extends Logging {
SQLConf.get.redactOptions(confMap).foreach { case (k, v) =>
logDebug(s"Applying Hadoop/Hive/Spark and extra properties to Hive Conf:$k=$v")
}
+ // Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false.
+ HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, false)
// Disable CBO because we removed the Calcite dependency.
hiveConf.setBoolean("hive.cbo.enable", false)
// If this is true, SessionState.start will create a file to log hive job which will not be
@@ -1414,25 +1397,7 @@ private[hive] object HiveClientImpl extends Logging {
case _ =>
new HiveConf(conf, classOf[HiveConf])
}
- val hive = try {
- Hive.getWithoutRegisterFns(hiveConf)
- } catch {
- // SPARK-37069: not all Hive versions have the above method (e.g., Hive 2.3.9 has it but
- // 2.3.8 doesn't), therefore here we fallback when encountering the exception.
- case _: NoSuchMethodError =>
- Hive.get(hiveConf)
- }
-
- // Follow behavior of HIVE-26633 (4.0.0), only apply the max message size when
- // `hive.thrift.client.max.message.size` is set and the value is positive
- Option(hiveConf.get("hive.thrift.client.max.message.size"))
- .map(HiveConf.toSizeBytes(_).toInt).filter(_ > 0)
- .foreach { maxMessageSize =>
- logDebug(s"Trying to set metastore client thrift max message to $maxMessageSize")
- configureMaxThriftMessageSize(hiveConf, hive.getMSC, maxMessageSize)
- }
-
- hive
+ Hive.getWithoutRegisterFns(hiveConf)
}
private def getFieldValue[T](obj: Any, fieldName: String): T = {
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 ef27669f5ba0..4ff1e9855fa5 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
@@ -29,11 +29,11 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.metastore.{IMetaStoreClient, PartitionDropOptions, TableType}
-import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, Index, MetaException, PrincipalType, ResourceType, ResourceUri}
-import org.apache.hadoop.hive.ql.Driver
+import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, MetaException, PrincipalType, ResourceType, ResourceUri}
+import org.apache.hadoop.hive.ql.IDriver
import org.apache.hadoop.hive.ql.io.AcidUtils
import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table}
-import org.apache.hadoop.hive.ql.plan.{AddPartitionDesc, DynamicPartitionCtx}
+import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx
import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory}
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.serde.serdeConstants
@@ -109,7 +109,7 @@ private[client] sealed abstract class Shim {
def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor
- def getDriverResults(driver: Driver): Seq[String]
+ def getDriverResults(driver: IDriver): Seq[String]
def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long
@@ -211,8 +211,6 @@ private[client] sealed abstract class Shim {
def listFunctions(hive: Hive, db: String, pattern: String): Seq[String]
- def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit
-
def dropTable(
hive: Hive,
dbName: String,
@@ -235,8 +233,6 @@ private[client] sealed abstract class Shim {
def getMSC(hive: Hive): IMetaStoreClient
- def getIndexes(hive: Hive, dbName: String, tableName: String, max: Short): Seq[Index]
-
protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = {
klass.getMethod(name, args: _*)
}
@@ -324,16 +320,10 @@ private[client] class Shim_v2_0 extends Shim with Logging {
table: Table,
parts: Seq[CatalogTablePartition],
ignoreIfExists: Boolean): Unit = {
- val addPartitionDesc = new AddPartitionDesc(table.getDbName, table.getTableName, ignoreIfExists)
- parts.zipWithIndex.foreach { case (s, i) =>
- addPartitionDesc.addPartition(
- s.spec.asJava, s.storage.locationUri.map(CatalogUtils.URIToString).orNull)
- if (s.parameters.nonEmpty) {
- addPartitionDesc.getPartition(i).setPartParams(s.parameters.asJava)
- }
+ parts.zipWithIndex.foreach { case (s, _) =>
+ hive.createPartition(table, s.parameters.asJava)
+ recordHiveCall()
}
- recordHiveCall()
- hive.createPartitions(addPartitionDesc)
}
override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = {
@@ -456,7 +446,7 @@ private[client] class Shim_v2_0 extends Shim with Logging {
override def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor =
CommandProcessorFactory.get(Array(token), conf)
- override def getDriverResults(driver: Driver): Seq[String] = {
+ override def getDriverResults(driver: IDriver): Seq[String] = {
val res = new JArrayList[Object]()
driver.getResults(res)
res.asScala.map {
@@ -516,11 +506,6 @@ private[client] class Shim_v2_0 extends Shim with Logging {
numDP: JInteger, listBucketingEnabled: JBoolean, isAcid, txnIdInLoadDynamicPartitions)
}
- override def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit = {
- recordHiveCall()
- hive.dropIndex(dbName, tableName, indexName, throwExceptionInDropIndex, deleteDataInDropIndex)
- }
-
override def dropTable(
hive: Hive,
dbName: String,
@@ -535,6 +520,7 @@ private[client] class Shim_v2_0 extends Shim with Logging {
override def alterTable(hive: Hive, tableName: String, table: Table): Unit = {
recordHiveCall()
alterTableMethod.invoke(hive, tableName, table)
+ // hive.alterTable(tableName, table, )
}
override def alterPartitions(hive: Hive, tableName: String, newParts: JList[Partition]): Unit = {
@@ -995,17 +981,9 @@ private[client] class Shim_v2_0 extends Shim with Logging {
oldPartSpec: JMap[String, String],
newPart: Partition): Unit = {
recordHiveCall()
- hive.renamePartition(table, oldPartSpec, newPart)
+ hive.renamePartition(table, oldPartSpec, newPart, 0)
}
- override def getIndexes(
- hive: Hive,
- dbName: String,
- tableName: String,
- max: Short): Seq[Index] = {
- recordHiveCall()
- hive.getIndexes(dbName, tableName, max).asScala.toSeq
- }
}
private[client] class Shim_v2_1 extends Shim_v2_0 {
@@ -1398,7 +1376,7 @@ private[client] class Shim_v4_0 extends Shim_v3_1 {
}
assert(loadFileType.isDefined)
recordHiveCall()
- val resetStatistics = false
+ val resetStatistics = true
val isDirectInsert = false
loadTableMethod.invoke(
hive,
@@ -1437,7 +1415,7 @@ private[client] class Shim_v4_0 extends Shim_v3_1 {
val partitions = parts.map(HiveClientImpl.toHivePartition(_, table).getTPartition).asJava
recordHiveCall()
val needResults = false
- addPartitionsMethod.invoke(hive, partitions, ignoreIfExists, needResults: JBoolean)
+ hive.addPartitions(partitions, ignoreIfExists, needResults: JBoolean)
}
override def loadPartition(
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 0bfe588df72c..8c13e2b458e8 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
@@ -210,6 +210,7 @@ private[hive] class IsolatedClientLoader(
name.startsWith("org.apache.hadoop.") && !name.startsWith("org.apache.hadoop.hive.")
name.startsWith("org.slf4j") ||
+ name.startsWith("org.apache.derby.") || // derby JDBC for embedded HMS
name.startsWith("org.apache.log4j") || // log4j1.x
name.startsWith("org.apache.logging.log4j") || // log4j2
name.startsWith("org.apache.spark.") ||
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 d7a0c58b4016..e424f503862a 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
@@ -102,21 +102,19 @@ package object client {
"org.apache.hive:hive-vector-code-gen"))
case object v4_1 extends HiveVersion("4.1.0",
- extraDeps =
- "org.antlr:antlr4-runtime:4.9.3" ::
- "org.apache.derby:derby:10.14.1.0" ::
- "org.apache.hadoop:hadoop-hdfs:3.4.1" ::
- "org.datanucleus:datanucleus-api-jdo:6.0.5" ::
- "org.datanucleus:datanucleus-core:6.0.11" ::
- "org.datanucleus:datanucleus-rdbms:6.0.10" ::
- "org.datanucleus:javax.jdo:3.2.1" ::
- "org.springframework:spring-core:5.3.39" ::
- "org.springframework:spring-jdbc:5.3.39" :: Nil,
- exclusions =
- "org.apache.curator:*" ::
- "org.apache.hive:hive-service-rpc" ::
- "org.apache.tez:tez-api" ::
- "org.apache.zookeeper:zookeeper" :: Nil)
+ extraDeps = Seq("org.apache.hadoop:hadoop-hdfs:3.4.1",
+ "org.datanucleus:datanucleus-api-jdo:6.0.5",
+ "org.datanucleus:datanucleus-rdbms:6.0.10",
+ "org.datanucleus:datanucleus-core:6.0.11",
+ "org.datanucleus:javax.jdo:3.2.0-release",
+ "org.springframework:spring-core:5.3.21",
+ "org.springframework:spring-jdbc:5.3.21",
+ "org.antlr:antlr4-runtime:4.9.3",
+ "org.apache.derby:derby:10.16.1.1"),
+ exclusions = Seq("org.apache.calcite:calcite-druid",
+ "org.apache.curator:*",
+ "org.pentaho:pentaho-aggdesigner-algorithm",
+ "org.apache.hive:hive-vector-code-gen"))
val allSupportedHiveVersions: Set[HiveVersion] =
Set(v2_0, v2_1, v2_2, v2_3, v3_0, v3_1, v4_0, v4_1)
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
index 33bf6ae1554c..95e9aa9cf01f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
@@ -23,7 +23,6 @@ import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.hive.ql.exec.Utilities
import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat}
import org.apache.hadoop.hive.ql.plan.FileSinkDesc
-import org.apache.hadoop.hive.serde2.Serializer
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector}
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
@@ -89,7 +88,7 @@ case class HiveFileFormat(fileSinkConf: FileSinkDesc)
// Add table properties from storage handler to hadoopConf, so any custom storage
// handler settings can be set to hadoopConf
HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false)
- Utilities.copyTableJobPropertiesToConf(tableDesc, conf)
+ Utilities.copyTableJobPropertiesToConf(tableDesc, conf.asInstanceOf[JobConf])
// Avoid referencing the outer object.
val fileSinkConfSer = fileSinkConf
@@ -135,12 +134,7 @@ class HiveOutputWriter(
private def tableDesc = fileSinkConf.getTableInfo
- private val serializer = {
- val serializer = tableDesc.getDeserializerClass.getConstructor().
- newInstance().asInstanceOf[Serializer]
- serializer.initialize(jobConf, tableDesc.getProperties)
- serializer
- }
+ private val serializer = tableDesc.getDeserializer(jobConf)
private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter(
jobConf,
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala
index 0f2eee6798ea..15f10bbd7911 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala
@@ -107,8 +107,7 @@ object HiveOptions {
val delimiterOptions = Map(
"fieldDelim" -> "field.delim",
"escapeDelim" -> "escape.delim",
- // The following typo is inherited from Hive...
- "collectionDelim" -> "colelction.delim",
+ "collectionDelim" -> "collection.delim",
"mapkeyDelim" -> "mapkey.delim",
"lineDelim" -> "line.delim").map { case (k, v) => k.toLowerCase(Locale.ROOT) -> v }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala
index de2d15415837..0ad77bff8e35 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala
@@ -145,7 +145,8 @@ private[hive] case class HiveScriptTransformationExec(
val (outputStream, proc, inputStream, stderrBuffer) = initProc
- val (inputSerde, inputSoi) = initInputSerDe(ioschema, child.output).getOrElse((null, null))
+ val (inputSerde, inputSoi) =
+ initInputSerDe(hadoopConf, ioschema, child.output).getOrElse((null, null))
// For HiveScriptTransformationExec, if inputSerde == null, but outputSerde != null
// We will use StringBuffer to pass data, in this case, we should cast data as string too.
@@ -173,7 +174,7 @@ private[hive] case class HiveScriptTransformationExec(
)
val (outputSerde, outputSoi) = {
- initOutputSerDe(ioschema, output).getOrElse((null, null))
+ initOutputSerDe(hadoopConf, ioschema, output).getOrElse((null, null))
}
val outputIterator = if (outputSerde == null) {
@@ -239,11 +240,12 @@ private[hive] case class HiveScriptTransformationWriterThread(
object HiveScriptIOSchema extends HiveInspectors {
def initInputSerDe(
+ hadoopConf: Configuration,
ioschema: ScriptTransformationIOSchema,
input: Seq[Expression]): Option[(AbstractSerDe, StructObjectInspector)] = {
ioschema.inputSerdeClass.map { serdeClass =>
val (columns, columnTypes) = parseAttrs(input)
- val serde = initSerDe(serdeClass, columns, columnTypes, ioschema.inputSerdeProps)
+ val serde = initSerDe(hadoopConf, serdeClass, columns, columnTypes, ioschema.inputSerdeProps)
val fieldObjectInspectors = columnTypes.map(toInspector)
val objectInspector = ObjectInspectorFactory
.getStandardStructObjectInspector(columns.asJava, fieldObjectInspectors.asJava)
@@ -252,11 +254,12 @@ object HiveScriptIOSchema extends HiveInspectors {
}
def initOutputSerDe(
+ hadoopConf: Configuration,
ioschema: ScriptTransformationIOSchema,
output: Seq[Attribute]): Option[(AbstractSerDe, StructObjectInspector)] = {
ioschema.outputSerdeClass.map { serdeClass =>
val (columns, columnTypes) = parseAttrs(output)
- val serde = initSerDe(serdeClass, columns, columnTypes, ioschema.outputSerdeProps)
+ val serde = initSerDe(hadoopConf, serdeClass, columns, columnTypes, ioschema.outputSerdeProps)
val structObjectInspector = serde.getObjectInspector().asInstanceOf[StructObjectInspector]
(serde, structObjectInspector)
}
@@ -269,6 +272,7 @@ object HiveScriptIOSchema extends HiveInspectors {
}
def initSerDe(
+ hadoopConf: Configuration,
serdeClassName: String,
columns: Seq[String],
columnTypes: Seq[DataType],
@@ -284,7 +288,7 @@ object HiveScriptIOSchema extends HiveInspectors {
val properties = new Properties()
properties.putAll(propsMap.asJava)
- serde.initialize(null, properties)
+ serde.initialize(hadoopConf, properties, null)
serde
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
index 335d552fd50b..0a6776c7205e 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
@@ -123,8 +123,7 @@ case class HiveTableScanExec(
HiveShim.appendReadColumns(hiveConf, neededColumnIDs, neededColumnNames)
- val deserializer = tableDesc.getDeserializerClass.getConstructor().newInstance()
- deserializer.initialize(hiveConf, tableDesc.getProperties)
+ val deserializer = tableDesc.getDeserializer(hiveConf)
// Specifies types and object inspectors of columns to be scanned.
val structOI = ObjectInspectorUtils
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala
index 3864d1601504..b4e94be7f597 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala
@@ -149,7 +149,7 @@ class HiveTempPath(session: SparkSession, val hadoopConf: Configuration, path: P
try {
stagingDirForCreating.foreach { stagingDir =>
val fs: FileSystem = stagingDir.getFileSystem(hadoopConf)
- if (!FileUtils.mkdir(fs, stagingDir, true, hadoopConf)) {
+ if (!FileUtils.mkdir(fs, stagingDir, hadoopConf)) {
throw SparkException.internalError(
"Cannot create staging directory '" + stagingDir.toString + "'")
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index b5d3fb699d62..06b32f9810b4 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -146,9 +146,9 @@ case class InsertIntoHiveTable(
if (numDynamicPartitions > 0) {
if (overwrite && table.tableType == CatalogTableType.EXTERNAL) {
val numWrittenParts = writtenParts.size
- val maxDynamicPartitionsKey = HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname
+ val maxDynamicPartitionsKey = HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS.varname
val maxDynamicPartitions = hadoopConf.getInt(maxDynamicPartitionsKey,
- HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.defaultIntVal)
+ HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS.defaultIntVal)
if (numWrittenParts > maxDynamicPartitions) {
throw QueryExecutionErrors.writePartitionExceedConfigSizeWhenDynamicPartitionError(
numWrittenParts, maxDynamicPartitions, maxDynamicPartitionsKey)
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
index 7f58f69269dd..f5bce5acacbb 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
@@ -28,7 +28,7 @@ import org.apache.commons.codec.binary.Base64
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.hive.ql.io.orc._
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument
+import org.apache.hadoop.hive.ql.io.sarg.{ConvertAstToSearchArg, SearchArgument}
import org.apache.hadoop.hive.serde2.objectinspector
import org.apache.hadoop.hive.serde2.objectinspector.{SettableStructObjectInspector, StructObjectInspector}
import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils}
@@ -137,7 +137,7 @@ case class OrcFileFormat() extends FileFormat with DataSourceRegister with Seria
if (sparkSession.sessionState.conf.orcFilterPushDown) {
// Sets pushed predicates
OrcFilters.createFilter(requiredSchema, filters).foreach { f =>
- hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, toKryo(f))
+ hadoopConf.set(ConvertAstToSearchArg.SARG_PUSHDOWN, ConvertAstToSearchArg.sargToKryo(f))
hadoopConf.setBoolean("hive.optimize.index.filter", true)
}
}
@@ -170,7 +170,11 @@ case class OrcFileFormat() extends FileFormat with DataSourceRegister with Seria
// ObjectInspector during recordReader creation itself and can
// avoid NameNode call in unwrapOrcStructs per file.
// Specifically would be helpful for partitioned datasets.
- val orcReader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf))
+ val orcReader = OrcFile.createReader(filePath,
+ // Hive blindly assumes that Timestamps are in UTC time zone
+ OrcFile.readerOptions(conf)
+ .convertToProlepticGregorian(false)
+ .useUTCTimestamp(true))
new SparkOrcNewRecordReader(orcReader, conf, file.start, file.length)
}
@@ -233,7 +237,7 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration)
table.setProperty("columns.types", dataSchema.map(_.dataType.catalogString).mkString(":"))
val serde = new OrcSerde
- serde.initialize(conf, table)
+ serde.initialize(conf, table, null)
serde
}
@@ -288,6 +292,7 @@ private[orc] class OrcOutputWriter(
}
override def write(row: InternalRow): Unit = {
+ // getOrCreateInternalWriter()
recordWriter.write(NullWritable.get(), serializer.serialize(row))
}
@@ -308,7 +313,9 @@ private[orc] class OrcOutputWriter(
// Hive ORC initializes its private `writer` field at the first write.
// For empty write task, we need to create it manually to record our meta.
val options = OrcFile.writerOptions(context.getConfiguration)
- options.inspector(serializer.structOI)
+ .inspector(serializer.structOI)
+ // .setProlepticGregorian(true)
+ // .useUTCTimestamp(true)
writer = OrcFile.createWriter(new Path(path), options)
// set the writer to make it flush meta on close
writerField.set(recordWriter, writer)
@@ -318,8 +325,6 @@ private[orc] class OrcOutputWriter(
}
private[orc] object OrcFileFormat extends HiveInspectors with Logging {
- // This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public.
- private[orc] val SARG_PUSHDOWN = "sarg.pushdown"
def unwrapOrcStructs(
conf: Configuration,
diff --git a/sql/hive/src/test/noclasspath/hive-test-udfs.jar b/sql/hive/src/test/noclasspath/hive-test-udfs.jar
deleted file mode 100644
index a5bfa456f668..000000000000
Binary files a/sql/hive/src/test/noclasspath/hive-test-udfs.jar and /dev/null differ
diff --git a/sql/hive/src/test/noclasspath/pom.xml b/sql/hive/src/test/noclasspath/pom.xml
new file mode 100644
index 000000000000..838676e287ca
--- /dev/null
+++ b/sql/hive/src/test/noclasspath/pom.xml
@@ -0,0 +1,74 @@
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ hive-test-udfs
+ 4.1.0-SNAPSHOT
+ jar
+ Spark Hive Test UDFS
+
+
+ Apache-2.0
+ https://www.apache.org/licenses/LICENSE-2.0.html
+ repo
+
+
+
+ Apache Software Foundation
+ https://www.apache.org
+
+
+
+ 8
+ 8
+ 8
+ 3.4.1
+ 4.1.0
+
+
+
+ ${project.artifactId}
+
+
+ maven-jar-plugin
+ 3.4.2
+
+ ${project.basedir}
+
+
+
+
+
+
+
+ org.apache.hive
+ hive-exec
+ core
+ ${hive.version}
+ provided
+
+
+ org.apache.hive
+ hive-udf
+ ${hive.version}
+ provided
+
+
+
+
diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udaf/example/UDAFExampleMax2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udaf/example/UDAFExampleMax2.java
new file mode 100644
index 000000000000..00681405dd3c
--- /dev/null
+++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udaf/example/UDAFExampleMax2.java
@@ -0,0 +1,268 @@
+/**
+ * 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.hadoop.hive.contrib.udaf.example;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDAF;
+import org.apache.hadoop.hive.ql.exec.UDAFEvaluator;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+@Description(name = "udaf_max2", value = "_FUNC_(expr) - Returns the maximum value of expr")
+public class UDAFExampleMax2 extends UDAF {
+
+ static public class MaxShortEvaluator implements UDAFEvaluator {
+ private short mMax;
+ private boolean mEmpty;
+
+ public MaxShortEvaluator() {
+ super();
+ init();
+ }
+
+ public void init() {
+ mMax = 0;
+ mEmpty = true;
+ }
+
+ public boolean iterate(ShortWritable o) {
+ if (o != null) {
+ if (mEmpty) {
+ mMax = o.get();
+ mEmpty = false;
+ } else {
+ mMax = (short) Math.max(mMax, o.get());
+ }
+ }
+ return true;
+ }
+
+ public ShortWritable terminatePartial() {
+ return mEmpty ? null : new ShortWritable(mMax);
+ }
+
+ public boolean merge(ShortWritable o) {
+ return iterate(o);
+ }
+
+ public ShortWritable terminate() {
+ return mEmpty ? null : new ShortWritable(mMax);
+ }
+ }
+
+ static public class MaxIntEvaluator implements UDAFEvaluator {
+ private int mMax;
+ private boolean mEmpty;
+
+ public MaxIntEvaluator() {
+ super();
+ init();
+ }
+
+ public void init() {
+ mMax = 0;
+ mEmpty = true;
+ }
+
+ public boolean iterate(IntWritable o) {
+ if (o != null) {
+ if (mEmpty) {
+ mMax = o.get();
+ mEmpty = false;
+ } else {
+ mMax = Math.max(mMax, o.get());
+ }
+ }
+ return true;
+ }
+
+ public IntWritable terminatePartial() {
+ return mEmpty ? null : new IntWritable(mMax);
+ }
+
+ public boolean merge(IntWritable o) {
+ return iterate(o);
+ }
+
+ public IntWritable terminate() {
+ return mEmpty ? null : new IntWritable(mMax);
+ }
+ }
+
+ static public class MaxLongEvaluator implements UDAFEvaluator {
+ private long mMax;
+ private boolean mEmpty;
+
+ public MaxLongEvaluator() {
+ super();
+ init();
+ }
+
+ public void init() {
+ mMax = 0;
+ mEmpty = true;
+ }
+
+ public boolean iterate(LongWritable o) {
+ if (o != null) {
+ if (mEmpty) {
+ mMax = o.get();
+ mEmpty = false;
+ } else {
+ mMax = Math.max(mMax, o.get());
+ }
+ }
+ return true;
+ }
+
+ public LongWritable terminatePartial() {
+ return mEmpty ? null : new LongWritable(mMax);
+ }
+
+ public boolean merge(LongWritable o) {
+ return iterate(o);
+ }
+
+ public LongWritable terminate() {
+ return mEmpty ? null : new LongWritable(mMax);
+ }
+ }
+
+ static public class MaxFloatEvaluator implements UDAFEvaluator {
+ private float mMax;
+ private boolean mEmpty;
+
+ public MaxFloatEvaluator() {
+ super();
+ init();
+ }
+
+ public void init() {
+ mMax = 0;
+ mEmpty = true;
+ }
+
+ public boolean iterate(FloatWritable o) {
+ if (o != null) {
+ if (mEmpty) {
+ mMax = o.get();
+ mEmpty = false;
+ } else {
+ mMax = Math.max(mMax, o.get());
+ }
+ }
+ return true;
+ }
+
+ public FloatWritable terminatePartial() {
+ return mEmpty ? null : new FloatWritable(mMax);
+ }
+
+ public boolean merge(FloatWritable o) {
+ return iterate(o);
+ }
+
+ public FloatWritable terminate() {
+ return mEmpty ? null : new FloatWritable(mMax);
+ }
+ }
+
+ static public class MaxDoubleEvaluator implements UDAFEvaluator {
+ private double mMax;
+ private boolean mEmpty;
+
+ public MaxDoubleEvaluator() {
+ super();
+ init();
+ }
+
+ public void init() {
+ mMax = 0;
+ mEmpty = true;
+ }
+
+ public boolean iterate(DoubleWritable o) {
+ if (o != null) {
+ if (mEmpty) {
+ mMax = o.get();
+ mEmpty = false;
+ } else {
+ mMax = Math.max(mMax, o.get());
+ }
+ }
+ return true;
+ }
+
+ public DoubleWritable terminatePartial() {
+ return mEmpty ? null : new DoubleWritable(mMax);
+ }
+
+ public boolean merge(DoubleWritable o) {
+ return iterate(o);
+ }
+
+ public DoubleWritable terminate() {
+ return mEmpty ? null : new DoubleWritable(mMax);
+ }
+ }
+
+ static public class MaxStringEvaluator implements UDAFEvaluator {
+ private Text mMax;
+ private boolean mEmpty;
+
+ public MaxStringEvaluator() {
+ super();
+ init();
+ }
+
+ public void init() {
+ mMax = null;
+ mEmpty = true;
+ }
+
+ public boolean iterate(Text o) {
+ if (o != null) {
+ if (mEmpty) {
+ mMax = new Text(o);
+ mEmpty = false;
+ } else if (mMax.compareTo(o) < 0) {
+ mMax.set(o);
+ }
+ }
+ return true;
+ }
+
+ public Text terminatePartial() {
+ return mEmpty ? null : mMax;
+ }
+
+ public boolean merge(Text o) {
+ return iterate(o);
+ }
+
+ public Text terminate() {
+ return mEmpty ? null : mMax;
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java
new file mode 100644
index 000000000000..0b6432fac3a4
--- /dev/null
+++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java
@@ -0,0 +1,47 @@
+/**
+ * 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.hadoop.hive.contrib.udf.example;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringTrimCol;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringTrimColScalar;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseTrim;
+
+/**
+ * UDFTrim.
+ *
+ */
+@Description(name = "generic_udf_trim2",
+ value = "_FUNC_(str) - Removes the leading and trailing space characters from str ",
+ extended = "Example:\n"
+ + " > SELECT _FUNC_(' facebook ') FROM src LIMIT 1;\n" + " 'facebook'")
+@VectorizedExpressions({ StringTrimCol.class, StringTrimColScalar.class })
+public class GenericUDFTrim2 extends GenericUDFBaseTrim {
+ public GenericUDFTrim2() {
+ super("generic_udf_trim2");
+ }
+
+ @Override
+ protected String performOp(String val, String trimChars) {
+ return StringUtils.strip(val, trimChars);
+ }
+
+}
\ No newline at end of file
diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/UDFExampleAdd2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/UDFExampleAdd2.java
new file mode 100644
index 000000000000..33e6f0954282
--- /dev/null
+++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/UDFExampleAdd2.java
@@ -0,0 +1,38 @@
+/**
+ * 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.hadoop.hive.contrib.udf.example;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDF;
+
+/**
+ * UDFExampleAdd2.
+ *
+ */
+@Description(name = "udf_add2", value = "_FUNC_(expr) - Example UDF that returns the sum")
+public class UDFExampleAdd2 extends UDF {
+
+ public Integer evaluate(Integer a, Integer b) {
+ return a + b;
+ }
+
+ public Double evaluate(Double a, Double b) {
+ return a + b;
+ }
+
+}
\ No newline at end of file
diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udtf/example/GenericUDTFCount3.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udtf/example/GenericUDTFCount3.java
new file mode 100644
index 000000000000..46866d49046a
--- /dev/null
+++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udtf/example/GenericUDTFCount3.java
@@ -0,0 +1,68 @@
+/**
+ * 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.hadoop.hive.contrib.udtf.example;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+
+
+/**
+ * GenericUDTFCount3 outputs the number of rows seen, three times. It's output three times
+ * to test outputting of rows on close with lateral view.
+ *
+ */
+@Description(name = "udtf_count3",
+ value = "_FUNC_(col) - UDF outputs the number of rows seen, three times.")
+public class GenericUDTFCount3 extends GenericUDTF {
+
+ private transient Integer count = Integer.valueOf(0);
+ private transient Object forwardObj[] = new Object[1];
+
+ @Override
+ public void close() throws HiveException {
+ forwardObj[0] = count;
+ forward(forwardObj);
+ forward(forwardObj);
+ forward(forwardObj);
+ }
+
+ @Override
+ public StructObjectInspector initialize(ObjectInspector[] argOIs) throws UDFArgumentException {
+ ArrayList fieldNames = new ArrayList();
+ ArrayList fieldOIs = new ArrayList();
+ fieldNames.add("col1");
+ fieldOIs.add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
+ return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames,
+ fieldOIs);
+ }
+
+ @Override
+ public void process(Object[] args) throws HiveException {
+ count = Integer.valueOf(count.intValue() + 1);
+ }
+
+}
\ No newline at end of file
diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum2.java
new file mode 100644
index 000000000000..eee4029cbc17
--- /dev/null
+++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum2.java
@@ -0,0 +1,462 @@
+/**
+ * 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.hadoop.hive.ql.udf.generic;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.parse.WindowingSpec.BoundarySpec;
+import org.apache.hadoop.hive.ql.plan.ptf.BoundaryDef;
+import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * GenericUDAFSum.
+ *
+ */
+@Description(name = "generic_udaf_sum2", value = "_FUNC_(x) - Returns the sum of a set of numbers")
+public class GenericUDAFSum2 extends AbstractGenericUDAFResolver {
+
+ static final Log LOG = LogFactory.getLog(GenericUDAFSum2.class.getName());
+
+ @Override
+ public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+ throws SemanticException {
+ if (parameters.length != 1) {
+ throw new UDFArgumentTypeException(parameters.length - 1,
+ "Exactly one argument is expected.");
+ }
+
+ if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ throw new UDFArgumentTypeException(0,
+ "Only primitive type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ return new GenericUDAFSumLong();
+ case TIMESTAMP:
+ case FLOAT:
+ case DOUBLE:
+ case STRING:
+ case VARCHAR:
+ case CHAR:
+ return new GenericUDAFSumDouble();
+ case DECIMAL:
+ return new GenericUDAFSumHiveDecimal();
+ case BOOLEAN:
+ case DATE:
+ default:
+ throw new UDFArgumentTypeException(0,
+ "Only numeric or string type arguments are accepted but "
+ + parameters[0].getTypeName() + " is passed.");
+ }
+ }
+
+ public static PrimitiveObjectInspector.PrimitiveCategory getReturnType(TypeInfo type) {
+ if (type.getCategory() != ObjectInspector.Category.PRIMITIVE) {
+ return null;
+ }
+ switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) {
+ case BYTE:
+ case SHORT:
+ case INT:
+ case LONG:
+ return PrimitiveObjectInspector.PrimitiveCategory.LONG;
+ case TIMESTAMP:
+ case FLOAT:
+ case DOUBLE:
+ case STRING:
+ case VARCHAR:
+ case CHAR:
+ return PrimitiveObjectInspector.PrimitiveCategory.DOUBLE;
+ case DECIMAL:
+ return PrimitiveObjectInspector.PrimitiveCategory.DECIMAL;
+ }
+ return null;
+ }
+
+ /**
+ * GenericUDAFSumHiveDecimal.
+ *
+ */
+ public static class GenericUDAFSumHiveDecimal extends GenericUDAFEvaluator {
+ private PrimitiveObjectInspector inputOI;
+ private HiveDecimalWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+ result = new HiveDecimalWritable(HiveDecimal.ZERO);
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ // The output precision is 10 greater than the input which should cover at least
+ // 10b rows. The scale is the same as the input.
+ DecimalTypeInfo outputTypeInfo = null;
+ if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+ int precision = Math.min(HiveDecimal.MAX_PRECISION, inputOI.precision() + 10);
+ outputTypeInfo = TypeInfoFactory.getDecimalTypeInfo(precision, inputOI.scale());
+ } else {
+ outputTypeInfo = (DecimalTypeInfo) inputOI.getTypeInfo();
+ }
+ return PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(outputTypeInfo);
+ }
+
+ /** class for storing decimal sum value. */
+ @AggregationType(estimable = false) // hard to know exactly for decimals
+ static class SumHiveDecimalAgg extends AbstractAggregationBuffer {
+ boolean empty;
+ HiveDecimal sum;
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ SumHiveDecimalAgg agg = new SumHiveDecimalAgg();
+ reset(agg);
+ return agg;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ SumHiveDecimalAgg bdAgg = (SumHiveDecimalAgg) agg;
+ bdAgg.empty = true;
+ bdAgg.sum = HiveDecimal.ZERO;
+ }
+
+ boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ try {
+ merge(agg, parameters[0]);
+ } catch (NumberFormatException e) {
+ if (!warned) {
+ warned = true;
+ LOG.warn(getClass().getSimpleName() + " "
+ + StringUtils.stringifyException(e));
+ LOG
+ .warn(getClass().getSimpleName()
+ + " ignoring similar exceptions.");
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ return terminate(agg);
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) agg;
+ if (myagg.sum == null) {
+ return;
+ }
+
+ myagg.empty = false;
+ myagg.sum = myagg.sum.add(PrimitiveObjectInspectorUtils.getHiveDecimal(partial, inputOI));
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) agg;
+ if (myagg.empty || myagg.sum == null) {
+ return null;
+ }
+ result.set(myagg.sum);
+ return result;
+ }
+
+ @Override
+ public GenericUDAFEvaluator getWindowingEvaluator(WindowFrameDef wFrmDef) {
+ return new GenericUDAFStreamingEvaluator.SumAvgEnhancer(
+ this, wFrmDef) {
+
+ @Override
+ protected HiveDecimalWritable getNextResult(
+ org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss)
+ throws HiveException {
+ SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) ss.wrappedBuf;
+ HiveDecimal r = myagg.empty ? null : myagg.sum;
+ HiveDecimal d = ss.retrieveNextIntermediateValue();
+
+ d = d == null ? HiveDecimal.ZERO : d;
+ r = r == null ? null : r.subtract(d);
+
+ return r == null ? null : new HiveDecimalWritable(r);
+ }
+
+ @Override
+ protected HiveDecimal getCurrentIntermediateResult(
+ org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss)
+ throws HiveException {
+ SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) ss.wrappedBuf;
+ return myagg.empty ? null : myagg.sum;
+ }
+
+ };
+ }
+ }
+
+ /**
+ * GenericUDAFSumDouble.
+ *
+ */
+ public static class GenericUDAFSumDouble extends GenericUDAFEvaluator {
+ private PrimitiveObjectInspector inputOI;
+ private DoubleWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+ result = new DoubleWritable(0);
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+ }
+
+ /** class for storing double sum value. */
+ @AggregationType(estimable = true)
+ static class SumDoubleAgg extends AbstractAggregationBuffer {
+ boolean empty;
+ double sum;
+ @Override
+ public int estimate() { return JavaDataModel.PRIMITIVES1 + JavaDataModel.PRIMITIVES2; }
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ SumDoubleAgg result = new SumDoubleAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ SumDoubleAgg myagg = (SumDoubleAgg) agg;
+ myagg.empty = true;
+ myagg.sum = 0;
+ }
+
+ boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ try {
+ merge(agg, parameters[0]);
+ } catch (NumberFormatException e) {
+ if (!warned) {
+ warned = true;
+ LOG.warn(getClass().getSimpleName() + " "
+ + StringUtils.stringifyException(e));
+ LOG
+ .warn(getClass().getSimpleName()
+ + " ignoring similar exceptions.");
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ return terminate(agg);
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ SumDoubleAgg myagg = (SumDoubleAgg) agg;
+ myagg.empty = false;
+ myagg.sum += PrimitiveObjectInspectorUtils.getDouble(partial, inputOI);
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ SumDoubleAgg myagg = (SumDoubleAgg) agg;
+ if (myagg.empty) {
+ return null;
+ }
+ result.set(myagg.sum);
+ return result;
+ }
+
+ @Override
+ public GenericUDAFEvaluator getWindowingEvaluator(WindowFrameDef wFrmDef) {
+ return new GenericUDAFStreamingEvaluator.SumAvgEnhancer(this,
+ wFrmDef) {
+
+ @Override
+ protected DoubleWritable getNextResult(
+ org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss)
+ throws HiveException {
+ SumDoubleAgg myagg = (SumDoubleAgg) ss.wrappedBuf;
+ Double r = myagg.empty ? null : myagg.sum;
+ Double d = ss.retrieveNextIntermediateValue();
+ d = d == null ? (double)0.0F : d;
+ r = r == null ? null : r - d;
+
+ return r == null ? null : new DoubleWritable(r);
+ }
+
+ @Override
+ protected Double getCurrentIntermediateResult(
+ org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss)
+ throws HiveException {
+ SumDoubleAgg myagg = (SumDoubleAgg) ss.wrappedBuf;
+ return myagg.empty ? null : new Double(myagg.sum);
+ }
+
+ };
+ }
+
+ }
+
+ /**
+ * GenericUDAFSumLong.
+ *
+ */
+ public static class GenericUDAFSumLong extends GenericUDAFEvaluator {
+ private PrimitiveObjectInspector inputOI;
+ private LongWritable result;
+
+ @Override
+ public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ super.init(m, parameters);
+ result = new LongWritable(0);
+ inputOI = (PrimitiveObjectInspector) parameters[0];
+ return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
+ }
+
+ /** class for storing double sum value. */
+ @AggregationType(estimable = true)
+ static class SumLongAgg extends AbstractAggregationBuffer {
+ boolean empty;
+ long sum;
+ @Override
+ public int estimate() { return JavaDataModel.PRIMITIVES1 + JavaDataModel.PRIMITIVES2; }
+ }
+
+ @Override
+ public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+ SumLongAgg result = new SumLongAgg();
+ reset(result);
+ return result;
+ }
+
+ @Override
+ public void reset(AggregationBuffer agg) throws HiveException {
+ SumLongAgg myagg = (SumLongAgg) agg;
+ myagg.empty = true;
+ myagg.sum = 0;
+ }
+
+ private boolean warned = false;
+
+ @Override
+ public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+ assert (parameters.length == 1);
+ try {
+ merge(agg, parameters[0]);
+ } catch (NumberFormatException e) {
+ if (!warned) {
+ warned = true;
+ LOG.warn(getClass().getSimpleName() + " "
+ + StringUtils.stringifyException(e));
+ }
+ }
+ }
+
+ @Override
+ public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+ return terminate(agg);
+ }
+
+ @Override
+ public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+ if (partial != null) {
+ SumLongAgg myagg = (SumLongAgg) agg;
+ myagg.sum += PrimitiveObjectInspectorUtils.getLong(partial, inputOI);
+ myagg.empty = false;
+ }
+ }
+
+ @Override
+ public Object terminate(AggregationBuffer agg) throws HiveException {
+ SumLongAgg myagg = (SumLongAgg) agg;
+ if (myagg.empty) {
+ return null;
+ }
+ result.set(myagg.sum);
+ return result;
+ }
+
+ @Override
+ public GenericUDAFEvaluator getWindowingEvaluator(WindowFrameDef wFrmDef) {
+ return new GenericUDAFStreamingEvaluator.SumAvgEnhancer(this,
+ wFrmDef) {
+
+ @Override
+ protected LongWritable getNextResult(
+ org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss)
+ throws HiveException {
+ SumLongAgg myagg = (SumLongAgg) ss.wrappedBuf;
+ Long r = myagg.empty ? null : myagg.sum;
+ Long d = ss.retrieveNextIntermediateValue();
+ d = d == null ? 0L : d;
+ r = r == null ? null : r - d;
+
+ return r == null ? null : new LongWritable(r);
+ }
+
+ @Override
+ protected Long getCurrentIntermediateResult(
+ org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss)
+ throws HiveException {
+ SumLongAgg myagg = (SumLongAgg) ss.wrappedBuf;
+ return myagg.empty ? null : new Long(myagg.sum);
+ }
+
+ };
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala
index c136c4c9790f..4dc293ecd591 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala
@@ -70,7 +70,7 @@ class ClasspathDependenciesSuite extends SparkFunSuite {
}
test("hive-exec") {
- assertLoads("org.apache.hadoop.hive.ql.CommandNeedRetryException")
+ assertLoads("org.apache.hadoop.hive.ql.processors.CommandProcessorException")
}
private val STD_INSTANTIATOR = "org.objenesis.strategy.StdInstantiatorStrategy"
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala
index db522b72e4cc..c12089d0e5ba 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.command.DDLUtils
import org.apache.spark.sql.types.{StringType, StructField, StructType}
+import org.apache.spark.util.Utils
/**
* Test suite for the [[HiveExternalCatalog]].
@@ -34,7 +35,9 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType}
class HiveExternalCatalogSuite extends ExternalCatalogSuite {
private val externalCatalog: HiveExternalCatalog = {
- val catalog = new HiveExternalCatalog(new SparkConf, new Configuration)
+ val hadoopConf = new Configuration
+ hadoopConf.set("hive.metastore.warehouse.dir", Utils.createTempDir().toURI.toString)
+ val catalog = new HiveExternalCatalog(new SparkConf, hadoopConf)
catalog.client.reset()
catalog
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
index 6581d39c707e..cf7c74881f37 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
@@ -216,7 +216,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils {
tryDownloadSpark(version, sparkTestingDir.getCanonicalPath)
}
- val hiveMetastoreVersion = """^\d+\.\d+""".r.findFirstIn(hiveVersion).get
+ val hiveMetastoreVersion = "2.3"
val args = Seq(
"--name", "prepare testing tables",
"--master", "local[2]",
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
index fad374827418..f43c5d6ec696 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
@@ -298,7 +298,7 @@ class DataSourceWithHiveMetastoreCatalogSuite
// It's not a bucketed table at Hive side
val hiveSide = sparkSession.metadataHive.runSqlHive("DESC FORMATTED t")
- assert(hiveSide.contains("Num Buckets: \t-1 \t "))
+ assert(hiveSide.contains("Num Buckets: \t0 \t "))
assert(hiveSide.contains("Bucket Columns: \t[] \t "))
assert(hiveSide.contains("\tspark.sql.sources.schema.numBuckets\t2 "))
assert(hiveSide.contains("\tspark.sql.sources.schema.bucketCol.0\tc1 "))
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
index f44f8b80fea5..cf3c2c58dccf 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -149,7 +149,7 @@ class HiveSparkSubmitSuite
"--conf", s"${EXECUTOR_MEMORY.key}=512m",
"--conf", "spark.ui.enabled=false",
"--conf", "spark.master.rest.enabled=false",
- "--conf", "spark.sql.hive.metastore.version=2.3.10",
+ "--conf", "spark.sql.hive.metastore.version=4.1.0",
"--conf", "spark.sql.hive.metastore.jars=maven",
"--driver-java-options", "-Dderby.system.durability=test",
unusedJar.toString)
@@ -371,7 +371,7 @@ class HiveSparkSubmitSuite
"--master", "local-cluster[2,1,512]",
"--conf", s"${EXECUTOR_MEMORY.key}=512m",
"--conf", s"${LEGACY_TIME_PARSER_POLICY.key}=LEGACY",
- "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=2.3.10",
+ "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=4.1.0",
"--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven",
"--conf", s"spark.hadoop.javax.jdo.option.ConnectionURL=$metastore",
unusedJar.toString)
@@ -388,7 +388,7 @@ object SetMetastoreURLTest extends Logging {
val builder = SparkSession.builder()
.config(sparkConf)
.config(UI_ENABLED.key, "false")
- .config(HiveUtils.HIVE_METASTORE_VERSION.key, "2.3.10")
+ .config(HiveUtils.HIVE_METASTORE_VERSION.key, "4.1.0")
// The issue described in SPARK-16901 only appear when
// spark.sql.hive.metastore.jars is not set to builtin.
.config(HiveUtils.HIVE_METASTORE_JARS.key, "maven")
@@ -699,7 +699,7 @@ object SparkSQLConfTest extends Logging {
val filteredSettings = super.getAll.filterNot(e => isMetastoreSetting(e._1))
// Always add these two metastore settings at the beginning.
- (HiveUtils.HIVE_METASTORE_VERSION.key -> "2.3.10") +:
+ (HiveUtils.HIVE_METASTORE_VERSION.key -> "4.1.0") +:
(HiveUtils.HIVE_METASTORE_JARS.key -> "maven") +:
filteredSettings
}
@@ -727,7 +727,7 @@ object SPARK_9757 extends QueryTest {
val hiveWarehouseLocation = Utils.createTempDir()
val sparkContext = new SparkContext(
new SparkConf()
- .set(HiveUtils.HIVE_METASTORE_VERSION.key, "2.3.10")
+ .set(HiveUtils.HIVE_METASTORE_VERSION.key, "4.1.0")
.set(HiveUtils.HIVE_METASTORE_JARS.key, "maven")
.set(UI_ENABLED, false)
.set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString))
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala
index 4b27082e188f..68ccf049a915 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.hive
+import java.io.File
+
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
@@ -140,6 +142,8 @@ class HiveUDFDynamicLoadSuite extends QueryTest with SQLTestUtils with TestHiveS
)
).toImmutableArraySeq
+ SQLTestUtils.invokeMaven(new File("src/test/noclasspath/pom.xml"))
+
udfTestInfos.foreach { udfInfo =>
// The test jars are built from below commit:
// https://github.com/HeartSaVioR/hive/commit/12f3f036b6efd0299cd1d457c0c0a65e0fd7e5f2
@@ -147,7 +151,6 @@ class HiveUDFDynamicLoadSuite extends QueryTest with SQLTestUtils with TestHiveS
// This jar file should not be placed to the classpath.
val jarPath = "src/test/noclasspath/hive-test-udfs.jar"
- assume(new java.io.File(jarPath).exists)
val jarUrl = s"file://${System.getProperty("user.dir")}/$jarPath"
test("Spark should be able to run Hive UDF using jar regardless of " +
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala
index ac0618be73fc..1513ba557a35 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.hive
import java.io.File
import java.net.URI
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars
import org.apache.spark.{SparkConf, TestUtils}
import org.apache.spark.deploy.SparkHadoopUtil
@@ -35,9 +35,9 @@ class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton
test("newTemporaryConfiguration overwrites listener configurations") {
Seq(true, false).foreach { useInMemoryDerby =>
val conf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby)
- assert(conf(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname) === "")
- assert(conf(ConfVars.METASTORE_EVENT_LISTENERS.varname) === "")
- assert(conf(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname) === "")
+ assert(conf(ConfVars.EVENT_LISTENERS.getVarname) === "")
+ assert(conf(ConfVars.EVENT_LISTENERS.getVarname) === "")
+ assert(conf(ConfVars.END_FUNCTION_LISTENERS.getVarname) === "")
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala
index a90fe4493dee..412eb9ab278a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala
@@ -45,7 +45,7 @@ class HadoopVersionInfoSuite extends SparkFunSuite {
// Download jars for Hive 2.0
val client = IsolatedClientLoader.forVersion(
- hiveMetastoreVersion = "2.0",
+ hiveMetastoreVersion = "4.1.0",
// 3.0.x is chosen because that HADOOP-14067 got fixed in 3.1.0
hadoopVersion = "3.0.3",
sparkConf = new SparkConf(),
@@ -58,7 +58,7 @@ class HadoopVersionInfoSuite extends SparkFunSuite {
.filter(!_.getName.startsWith("org.apache.hadoop_hadoop-"))
val sparkConf = new SparkConf()
- sparkConf.set(HiveUtils.HIVE_METASTORE_VERSION, "2.0")
+ sparkConf.set(HiveUtils.HIVE_METASTORE_VERSION, "4.1.0")
sparkConf.set(
HiveUtils.HIVE_METASTORE_JARS,
jars.map(_.getCanonicalPath).mkString(File.pathSeparator))
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
index d0d848bcb562..388f7c39e1ee 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
@@ -338,7 +338,7 @@ class HiveClientSuite(version: String) extends HiveVersionSuite(version) {
}
test("listTables(database)") {
- assert((client.listTables("default") diff Seq("src", "temporary", "view1")) === Nil)
+ assert(client.listTables("default").toSet === Set("src", "temporary", "view1"))
}
test("listTables(database, pattern)") {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala
index e43308f62a49..e9446c32989a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala
@@ -31,7 +31,7 @@ import org.apache.spark.tags.{ExtendedHiveTest, SlowHiveTest}
/**
* A simple set of tests that call the methods of a [[HiveClient]], loading different version
* of hive from maven central. These tests are simple in that they are mostly just testing to make
- * sure that reflective calls are not throwing NoSuchMethod error, but the actually functionality
+ * sure that reflective calls are not throwing NoSuchMethod error, but the actual functionality
* is not fully tested.
*/
@SlowHiveTest
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala
index c06e2dea40f9..a9fc295911b1 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala
@@ -22,6 +22,6 @@ private[client] trait HiveClientVersions {
protected val versions = if (testVersions.nonEmpty) {
testVersions.get.split(",").map(_.trim).filter(_.nonEmpty).toIndexedSeq
} else {
- IndexedSeq("2.0", "2.1", "2.2", "2.3", "3.0", "3.1", "4.0", "4.1")
+ IndexedSeq("4.1")
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala
index 1a45f6b15096..ee3a677b886e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala
@@ -35,9 +35,8 @@ private[client] abstract class HiveVersionSuite(version: String) extends SparkFu
hadoopConf.set("datanucleus.autoStartMechanismMode", "ignored")
hadoopConf.set("hive.metastore.schema.verification", "false")
// Since Hive 3.0, HIVE-19310 skipped `ensureDbInit` if `hive.in.test=false`.
- if (version == "3.0" || version == "3.1" || version == "4.0") {
+ if (version == "3.0" || version == "3.1" || version == "4.1") {
hadoopConf.set("hive.in.test", "true")
- hadoopConf.set("hive.query.reexecution.enabled", "false")
}
HiveClientBuilder.buildClient(version, hadoopConf)
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index e43d1cdb19d4..dabb08f7ec90 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -25,6 +25,7 @@ import java.util.Locale
import scala.jdk.CollectionConverters._
import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.metastore.Warehouse.{DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME}
import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.{spy, times, verify}
@@ -1393,7 +1394,8 @@ class HiveDDLSuite
case "false" => Map("database" -> "`spark_catalog`.`default`")
case _ => Map(
"clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException",
- "msg" -> "MetaException(message:Can not drop default database)")
+ "msg" -> (s"MetaException(message:Can not drop $DEFAULT_DATABASE_NAME database " +
+ s"in catalog $DEFAULT_CATALOG_NAME)"))
}
)
}
@@ -1756,7 +1758,7 @@ class HiveDDLSuite
sql(s"SELECT * FROM ${targetTable.identifier}"))
}
- test("create table with the same name as an index table") {
+ ignore("[Hive 4.x: does not support index] create table with the same name as an index table") {
val tabName = "tab1"
val indexName = tabName + "_index"
withTable(tabName) {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index d58dec6f1126..0b2e03679b7a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -23,7 +23,7 @@ import java.sql.Timestamp
import scala.util.Try
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars
import org.scalatest.BeforeAndAfter
import org.apache.spark.{SparkFiles, TestUtils}
@@ -1212,7 +1212,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
.zip(parts)
.map { case (k, v) =>
if (v == "NULL") {
- s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultStrVal}"
+ s"$k=${ConfVars.DEFAULTPARTITIONNAME.getDefaultVal}"
} else {
s"$k=$v"
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
index b7c466f3ed6a..ae4fd1db6231 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
@@ -18,11 +18,10 @@
package org.apache.spark.sql.hive.execution
import java.io.{DataInput, DataOutput, File, PrintWriter}
-import java.util.{ArrayList, Arrays, Properties}
+import java.util.{ArrayList, Arrays}
import scala.jdk.CollectionConverters._
-import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hive.ql.exec.UDF
import org.apache.hadoop.hive.ql.metadata.HiveException
import org.apache.hadoop.hive.ql.udf.{UDAFPercentile, UDFType}
@@ -739,7 +738,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {
val df = sql("SELECT CodeGenHiveGenericUDF(v) from HiveGenericUDFTable")
val plan = df.queryExecution.executedPlan
assert(plan.isInstanceOf[WholeStageCodegenExec])
- checkAnswer(df, Seq(Row("14ab8df5135825bc9f5ff7c30609f02f")))
+ checkAnswer(df,
+ Seq(Row("95e053f9451c5f73215101a92682782327c69bca5f1450419c606b09b26534ab")))
}
}
}
@@ -856,8 +856,6 @@ class TestPair(x: Int, y: Int) extends Writable with Serializable {
}
class PairSerDe extends AbstractSerDe {
- override def initialize(p1: Configuration, p2: Properties): Unit = {}
-
override def getObjectInspector: ObjectInspector = {
ObjectInspectorFactory
.getStandardStructObjectInspector(
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala
index 232916b6e05b..126c9dd37fb0 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala
@@ -28,7 +28,8 @@ class AlterNamespaceSetLocationSuite extends v1.AlterNamespaceSetLocationSuiteBa
with CommandSuiteBase {
override def commandVersion: String = super[AlterNamespaceSetLocationSuiteBase].commandVersion
- test("Hive catalog not supported") {
+ // Hive catalog is supported
+ ignore("Hive catalog not supported") {
val ns = s"$catalog.$namespace"
withNamespace(ns) {
sql(s"CREATE NAMESPACE $ns")
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala
index 8e654d28cd03..9aed44129b2d 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala
@@ -131,7 +131,7 @@ class ShowCreateTableSuite extends v1.ShowCreateTableSuiteBase with CommandSuite
val expected = s"CREATE TABLE $nsTable ( c1 INT COMMENT 'bla', c2 STRING)" +
" ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'" +
" WITH SERDEPROPERTIES (" +
- " 'colelction.delim' = '@'," +
+ " 'collection.delim' = '@'," +
" 'field.delim' = ','," +
" 'mapkey.delim' = '#'," +
" 'serialization.format' = ',')" +
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala
index c1084dd4ee7f..e4cea1c7954f 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala
@@ -19,6 +19,9 @@ package org.apache.spark.sql.hive.orc
import java.io.File
+import org.apache.hadoop.mapred.FileInputFormat
+import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks
+
import org.apache.spark.sql.{AnalysisException, Row}
import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT}
import org.apache.spark.sql.execution.datasources.orc.OrcSuite
@@ -27,7 +30,8 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
-class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
+class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton
+ with ScalaCheckDrivenPropertyChecks {
override val orcImp: String = "hive"
@@ -197,8 +201,10 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
}
test("SPARK-25993 CREATE EXTERNAL TABLE with subdirectories") {
- Seq(true, false).foreach { convertMetastore =>
- withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") {
+ forAll { (convertMetastore: Boolean, inputDirRecursive: Boolean) =>
+ withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore",
+ // HIVE-19258 and MAPREDUCE-7086: Hive does not include subdirectories by default
+ FileInputFormat.INPUT_DIR_RECURSIVE -> s"$inputDirRecursive") {
withTempDir { dir =>
withTable("orc_tbl1", "orc_tbl2", "orc_tbl3") {
val orcTblStatement1 =
@@ -254,7 +260,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
|LOCATION '${s"${dir.getCanonicalPath}"}'""".stripMargin
sql(topDirStatement)
val topDirSqlStatement = s"SELECT * FROM tbl1"
- if (convertMetastore) {
+ if (convertMetastore || !inputDirRecursive) {
checkAnswer(sql(topDirSqlStatement), Nil)
} else {
checkAnswer(sql(topDirSqlStatement), (1 to 6).map(i => Row(i, i, s"orc$i")))
@@ -270,7 +276,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
|LOCATION '${s"${dir.getCanonicalPath}/l1/"}'""".stripMargin
sql(l1DirStatement)
val l1DirSqlStatement = s"SELECT * FROM tbl2"
- if (convertMetastore) {
+ if (convertMetastore || !inputDirRecursive) {
checkAnswer(sql(l1DirSqlStatement), (1 to 2).map(i => Row(i, i, s"orc$i")))
} else {
checkAnswer(sql(l1DirSqlStatement), (1 to 6).map(i => Row(i, i, s"orc$i")))
@@ -286,7 +292,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
|LOCATION '${s"${dir.getCanonicalPath}/l1/l2/"}'""".stripMargin
sql(l2DirStatement)
val l2DirSqlStatement = s"SELECT * FROM tbl3"
- if (convertMetastore) {
+ if (convertMetastore || !inputDirRecursive) {
checkAnswer(sql(l2DirSqlStatement), (3 to 4).map(i => Row(i, i, s"orc$i")))
} else {
checkAnswer(sql(l2DirSqlStatement), (3 to 6).map(i => Row(i, i, s"orc$i")))
@@ -305,6 +311,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
if (convertMetastore) {
checkAnswer(sql(wildcardTopDirSqlStatement), (1 to 2).map(i => Row(i, i, s"orc$i")))
} else {
+ // * in the path never includes subdirectories
checkAnswer(sql(wildcardTopDirSqlStatement), Nil)
}
@@ -321,6 +328,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
if (convertMetastore) {
checkAnswer(sql(wildcardL1DirSqlStatement), (1 to 4).map(i => Row(i, i, s"orc$i")))
} else {
+ // * in the path never includes subdirectories
checkAnswer(sql(wildcardL1DirSqlStatement), Nil)
}
@@ -337,6 +345,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton {
if (convertMetastore) {
checkAnswer(sql(wildcardL2SqlStatement), (3 to 6).map(i => Row(i, i, s"orc$i")))
} else {
+ // * in the path never includes subdirectories
checkAnswer(sql(wildcardL2SqlStatement), Nil)
}
}