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) } }