diff options
Diffstat (limited to 'sql')
20 files changed, 55 insertions, 55 deletions
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 d89c3b4ab2..3025660301 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 @@ -35,7 +35,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow} import org.apache.spark.sql.execution.command.SetCommand -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes, HiveUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.{Utils => SparkUtils} @@ -98,7 +98,7 @@ private[hive] class SparkExecuteStatementOperation( case TimestampType => to += from.getAs[Timestamp](ordinal) case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal))) + val hiveString = HiveUtils.toHiveString((from.get(ordinal), dataTypes(ordinal))) to += hiveString } } 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 57693284b0..057fbbe6d9 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 @@ -39,7 +39,7 @@ import org.apache.thrift.transport.TSocket import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.util.ShutdownHookManager /** @@ -82,7 +82,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { val cliConf = new HiveConf(classOf[SessionState]) // Override the location of the metastore since this is only used for local execution. - HiveContext.newTemporaryConfiguration(useInMemoryDerby = false).foreach { + HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false).foreach { case (key, value) => cliConf.set(key, value) } val sessionState = new CliSessionState(cliConf) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 2679ac1854..465457f1ba 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging import org.apache.spark.scheduler.StatsReportListener -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.{HiveContext, HiveUtils} import org.apache.spark.util.Utils /** A singleton object for the master program. The slaves should not access this. */ @@ -62,7 +62,7 @@ private[hive] object SparkSQLEnv extends Logging { hiveContext.sessionState.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) hiveContext.sessionState.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) - hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) + hiveContext.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) if (log.isDebugEnabled) { hiveContext.sessionState.hiveconf.getAllProperties.asScala.toSeq.sorted diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index f492b5656c..a0beffdaa2 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -27,7 +27,7 @@ import org.apache.hive.service.cli.session.SessionManager import org.apache.hive.service.cli.thrift.TProtocolVersion import org.apache.hive.service.server.HiveServer2 -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.{HiveContext, HiveUtils} import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager @@ -76,7 +76,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext: } else { hiveContext.newSession() } - ctx.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) + ctx.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) sparkSqlOperationManager.sessionToContexts += sessionHandle -> ctx sessionHandle } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index ee14b6dc8d..bc45334036 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -42,7 +42,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -115,7 +115,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val resultSet = statement.executeQuery("SET spark.sql.hive.version") resultSet.next() assert(resultSet.getString(1) === "spark.sql.hive.version") - assert(resultSet.getString(2) === HiveContext.hiveExecutionVersion) + assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) } } @@ -624,7 +624,7 @@ class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { val resultSet = statement.executeQuery("SET spark.sql.hive.version") resultSet.next() assert(resultSet.getString(1) === "spark.sql.hive.version") - assert(resultSet.getString(2) === HiveContext.hiveExecutionVersion) + assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) } } } 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 49fd198730..b7d6c26fbe 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 @@ -23,7 +23,7 @@ import java.util.{Locale, TimeZone} import org.scalatest.BeforeAndAfter import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.internal.SQLConf @@ -60,7 +60,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.sessionState.functionRegistry.unregisterFunction("hash") // Ensures that the plans generation use metastore relation and not OrcRelation // Was done because SqlBuilder does not work with plans having logical relation - TestHive.setConf(HiveContext.CONVERT_METASTORE_ORC, false) + TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, false) RuleExecutor.resetTime() } @@ -71,7 +71,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { Locale.setDefault(originalLocale) TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) - TestHive.setConf(HiveContext.CONVERT_METASTORE_ORC, originalConvertMetastoreOrc) + TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, originalConvertMetastoreOrc) TestHive.sessionState.functionRegistry.restore() // For debugging dump some statistics about how much time was spent in various optimizer rules. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala index 0ee34f07fd..ed1340dccf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala @@ -53,7 +53,7 @@ protected[hive] class HiveQueryExecution(ctx: SQLContext, logicalPlan: LogicalPl // We need the types so we can output struct field names val types = analyzed.output.map(_.dataType) // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq + result.map(_.zip(types).map(HiveUtils.toHiveString)).map(_.mkString("\t")).toSeq } override def simpleString: String = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 4db0d78cfc..d8cc057fe2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -190,7 +190,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * SerDe. */ def convertMetastoreParquet: Boolean = { - conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET) + conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET) } /** @@ -200,7 +200,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true. */ def convertMetastoreParquetWithSchemaMerging: Boolean = { - conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING) + conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING) } /** @@ -209,7 +209,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * SerDe. */ def convertMetastoreOrc: Boolean = { - conf.getConf(HiveContext.CONVERT_METASTORE_ORC) + conf.getConf(HiveUtils.CONVERT_METASTORE_ORC) } /** @@ -225,14 +225,14 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) * and no SerDe is specified (no ROW FORMAT SERDE clause). */ def convertCTAS: Boolean = { - conf.getConf(HiveContext.CONVERT_CTAS) + conf.getConf(HiveUtils.CONVERT_CTAS) } /** * When true, Hive Thrift server will execute SQL queries asynchronously using a thread pool." */ def hiveThriftServerAsync: Boolean = { - conf.getConf(HiveContext.HIVE_THRIFT_SERVER_ASYNC) + conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC) } def hiveThriftServerSingleSession: Boolean = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala index 11097c33df..1d8ce3099d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala @@ -34,7 +34,7 @@ private[hive] class HiveSharedState(override val sparkContext: SparkContext) * A Hive client used for execution. */ val executionHive: HiveClientImpl = { - HiveContext.newClientForExecution(sparkContext.conf, sparkContext.hadoopConfiguration) + HiveUtils.newClientForExecution(sparkContext.conf, sparkContext.hadoopConfiguration) } /** @@ -42,7 +42,7 @@ private[hive] class HiveSharedState(override val sparkContext: SparkContext) */ // This needs to be a lazy val at here because TestHiveSharedState is overriding it. lazy val metadataHive: HiveClient = { - HiveContext.newClientForMetadata(sparkContext.conf, sparkContext.hadoopConfiguration) + HiveUtils.newClientForMetadata(sparkContext.conf, sparkContext.hadoopConfiguration) } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index b2ce3e0df2..44d3cc257b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -59,7 +59,7 @@ class HiveContext private[hive]( self => def this(sc: SparkContext) = { - this(new SparkSession(HiveContext.withHiveExternalCatalog(sc)), true) + this(new SparkSession(HiveUtils.withHiveExternalCatalog(sc)), true) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -84,7 +84,7 @@ class HiveContext private[hive]( } -private[hive] object HiveContext extends Logging { +private[spark] object HiveUtils extends Logging { def withHiveExternalCatalog(sc: SparkContext): SparkContext = { sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive") @@ -315,10 +315,10 @@ private[hive] object HiveContext extends Logging { configurations: Map[String, String]): HiveClient = { val sqlConf = new SQLConf sqlConf.setConf(SQLContext.getSQLProperties(conf)) - val hiveMetastoreVersion = HiveContext.hiveMetastoreVersion(sqlConf) - val hiveMetastoreJars = HiveContext.hiveMetastoreJars(sqlConf) - val hiveMetastoreSharedPrefixes = HiveContext.hiveMetastoreSharedPrefixes(sqlConf) - val hiveMetastoreBarrierPrefixes = HiveContext.hiveMetastoreBarrierPrefixes(sqlConf) + val hiveMetastoreVersion = HiveUtils.hiveMetastoreVersion(sqlConf) + val hiveMetastoreJars = HiveUtils.hiveMetastoreJars(sqlConf) + val hiveMetastoreSharedPrefixes = HiveUtils.hiveMetastoreSharedPrefixes(sqlConf) + val hiveMetastoreBarrierPrefixes = HiveUtils.hiveMetastoreBarrierPrefixes(sqlConf) val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) val defaultWarehouseLocation = hiveConf.get("hive.metastore.warehouse.dir") 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 6a20d7c25b..e95069e830 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 @@ -23,8 +23,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.metadata.{HiveUtils, Partition => HivePartition, - Table => HiveTable} +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, @@ -300,7 +299,8 @@ private[hive] object HiveTableUtil { def configureJobPropertiesForStorageHandler( tableDesc: TableDesc, jobConf: JobConf, input: Boolean) { val property = tableDesc.getProperties.getProperty(META_TABLE_STORAGE) - val storageHandler = HiveUtils.getStorageHandler(jobConf, property) + val storageHandler = + org.apache.hadoop.hive.ql.metadata.HiveUtils.getStorageHandler(jobConf, property) if (storageHandler != null) { val jobProperties = new util.LinkedHashMap[String, String] if (input) { 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 7e0d1b446f..0380d2342b 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 @@ -32,7 +32,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkSubmitUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.util.{MutableURLClassLoader, Utils} /** Factory for `IsolatedClientLoader` with specific versions of hive. */ @@ -263,7 +263,7 @@ private[hive] class IsolatedClientLoader( throw new ClassNotFoundException( s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" + "Please make sure that jars for your version of hive and hadoop are included in the " + - s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.", e) + s"paths passed to ${HiveUtils.HIVE_METASTORE_JARS}.", e) } else { throw e } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 741e3bdd18..7f8f6292cb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -74,7 +74,7 @@ class TestHiveContext(@transient val sparkSession: TestHiveSparkSession, isRootC extends HiveContext(sparkSession, isRootContext) { def this(sc: SparkContext) { - this(new TestHiveSparkSession(HiveContext.withHiveExternalCatalog(sc)), true) + this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc)), true) } override def newSession(): TestHiveContext = { @@ -117,7 +117,7 @@ private[hive] class TestHiveSparkSession( sc, Utils.createTempDir(namePrefix = "warehouse"), TestHiveContext.makeScratchDir(), - HiveContext.newTemporaryConfiguration(useInMemoryDerby = false), + HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false), None) } @@ -576,7 +576,7 @@ private[hive] object TestHiveContext { scratchDirPath: File, metastoreTemporaryConf: Map[String, String]): HiveClient = { val hiveConf = new HiveConf(hadoopConf, classOf[HiveConf]) - HiveContext.newClientForMetadata( + HiveUtils.newClientForMetadata( conf, hiveConf, hadoopConf, @@ -591,7 +591,7 @@ private[hive] object TestHiveContext { warehousePath: File, scratchDirPath: File, metastoreTemporaryConf: Map[String, String]): Map[String, String] = { - HiveContext.hiveClientConfigurations(hiveconf) ++ metastoreTemporaryConf ++ Map( + HiveUtils.hiveClientConfigurations(hiveconf) ++ metastoreTemporaryConf ++ Map( ConfVars.METASTOREWAREHOUSE.varname -> warehousePath.toURI.toString, ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN.varname -> "true", ConfVars.SCRATCHDIR.varname -> scratchDirPath.toURI.toString, 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 84285b7f40..cb60a2c8cf 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 @@ -31,7 +31,7 @@ class HiveExternalCatalogSuite extends CatalogTestCases { private val client: HiveClient = { // We create a metastore at a temp location to avoid any potential // conflict of having multiple connections to a single derby instance. - HiveContext.newClientForExecution(new SparkConf, new Configuration) + HiveUtils.newClientForExecution(new SparkConf, new Configuration) } protected override val utils: CatalogTestUtils = new CatalogTestUtils { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index bbe135b2d6..dc87daae72 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -550,7 +550,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } test("scan a parquet table created through a CTAS statement") { - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "true") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { withTempTable("jt") { (1 to 10).map(i => i -> s"str$i").toDF("a", "b").registerTempTable("jt") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index a9823ae262..d78914505a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -53,7 +53,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val rows = row :: Row(Seq.fill(row.length)(null): _*) :: Nil // Don't convert Hive metastore Parquet tables to let Hive write those Parquet files. - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { withTempTable("data") { val fields = hiveTypes.zipWithIndex.map { case (typ, index) => s" col_$index $typ" } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 8b0719209d..e0288ff98f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal, NamedExpression} import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.types.IntegerType import org.apache.spark.tags.ExtendedHiveTest import org.apache.spark.util.Utils @@ -62,7 +62,7 @@ class VersionsSuite extends SparkFunSuite with Logging { test("success sanity check") { val badClient = IsolatedClientLoader.forVersion( - hiveMetastoreVersion = HiveContext.hiveExecutionVersion, + hiveMetastoreVersion = HiveUtils.hiveExecutionVersion, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, hadoopConf = new Configuration(), @@ -76,7 +76,7 @@ class VersionsSuite extends SparkFunSuite with Logging { val hadoopConf = new Configuration(); hadoopConf.set("test", "success") val client = IsolatedClientLoader.forVersion( - hiveMetastoreVersion = HiveContext.hiveExecutionVersion, + hiveMetastoreVersion = HiveUtils.hiveExecutionVersion, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, hadoopConf = hadoopConf, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 345ee8ef28..2e14aaa6d7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} +import org.apache.spark.sql.hive.{HiveUtils, MetastoreRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -351,7 +351,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val originalConf = sessionState.convertCTAS - setConf(HiveContext.CONVERT_CTAS, true) + setConf(HiveUtils.CONVERT_CTAS, true) try { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") @@ -395,7 +395,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkRelation("ctas1", false) sql("DROP TABLE ctas1") } finally { - setConf(HiveContext.CONVERT_CTAS, originalConf) + setConf(HiveUtils.CONVERT_CTAS, originalConf) sql("DROP TABLE IF EXISTS ctas1") } } @@ -470,7 +470,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { | FROM src | ORDER BY key, value""".stripMargin).collect() - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { checkExistence(sql("DESC EXTENDED ctas5"), true, "name:key", "type:string", "name:value", "ctas5", "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", @@ -481,7 +481,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } // use the Hive SerDe for parquet tables - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { checkAnswer( sql("SELECT key, value FROM ctas5 ORDER BY key, value"), sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) @@ -732,7 +732,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) read.json(rdd).registerTempTable("data") val originalConf = sessionState.convertCTAS - setConf(HiveContext.CONVERT_CTAS, false) + setConf(HiveUtils.CONVERT_CTAS, false) try { sql("CREATE TABLE explodeTest (key bigInt)") @@ -751,7 +751,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql("DROP TABLE explodeTest") dropTempTable("data") } finally { - setConf(HiveContext.CONVERT_CTAS, originalConf) + setConf(HiveUtils.CONVERT_CTAS, originalConf) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 5ef8194f28..4fb78ac02c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -27,7 +27,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.internal.SQLConf @@ -406,7 +406,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("SPARK-14070 Use ORC data source for SQL queries on ORC tables") { withTempPath { dir => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true", - HiveContext.CONVERT_METASTORE_ORC.key -> "true") { + HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { val path = dir.getCanonicalPath withTable("dummy_orc") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 6fa4c3334f..2984ee99be 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -174,7 +174,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").registerTempTable("jt") (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a").registerTempTable("jt_array") - setConf(HiveContext.CONVERT_METASTORE_PARQUET, true) + setConf(HiveUtils.CONVERT_METASTORE_PARQUET, true) } override def afterAll(): Unit = { @@ -186,7 +186,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { "jt", "jt_array", "test_parquet") - setConf(HiveContext.CONVERT_METASTORE_PARQUET, false) + setConf(HiveUtils.CONVERT_METASTORE_PARQUET, false) } test(s"conversion is working") { @@ -619,7 +619,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { withTable("array_of_struct") { val conf = Seq( - HiveContext.CONVERT_METASTORE_PARQUET.key -> "false", + HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false", SQLConf.PARQUET_BINARY_AS_STRING.key -> "true", SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") |