-
Notifications
You must be signed in to change notification settings - Fork 29.1k
[SPARK-17190] [SQL] Removal of HiveSharedState #14757
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
57a952a
eedd8b5
1c27954
1faaa2d
1632b01
0c9fbf3
f63826e
d61ced8
4a14bed
9b7152f
2942f85
7775ba5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
This file was deleted.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | |
| import org.apache.spark.sql.execution.QueryExecution | ||
| import org.apache.spark.sql.execution.command.CacheTableCommand | ||
| import org.apache.spark.sql.hive._ | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.internal.{SharedState, SQLConf} | ||
| import org.apache.spark.util.{ShutdownHookManager, Utils} | ||
|
|
||
| // SPARK-3729: Test key required to check for initialization errors with config. | ||
|
|
@@ -108,13 +108,13 @@ class TestHiveContext( | |
| * A [[SparkSession]] used in [[TestHiveContext]]. | ||
| * | ||
| * @param sc SparkContext | ||
| * @param existingSharedState optional [[HiveSharedState]] | ||
| * @param existingSharedState optional [[SharedState]] | ||
| * @param loadTestTables if true, load the test tables. They can only be loaded when running | ||
| * in the JVM, i.e when calling from Python this flag has to be false. | ||
| */ | ||
| private[hive] class TestHiveSparkSession( | ||
| @transient private val sc: SparkContext, | ||
| @transient private val existingSharedState: Option[HiveSharedState], | ||
| @transient private val existingSharedState: Option[SharedState], | ||
| private val loadTestTables: Boolean) | ||
| extends SparkSession(sc) with Logging { self => | ||
|
|
||
|
|
@@ -139,14 +139,13 @@ private[hive] class TestHiveSparkSession( | |
|
|
||
| assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") | ||
|
|
||
| // TODO: Let's remove HiveSharedState and TestHiveSessionState. Otherwise, | ||
| // we are not really testing the reflection logic based on the setting of | ||
| // CATALOG_IMPLEMENTATION. | ||
| @transient | ||
| override lazy val sharedState: HiveSharedState = { | ||
| existingSharedState.getOrElse(new HiveSharedState(sc)) | ||
| override lazy val sharedState: SharedState = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need to override this?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. But is
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Although Actually, I tried it. It breaks one test case BTW, it works if
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i see, thanks |
||
| existingSharedState.getOrElse(new SharedState(sc)) | ||
| } | ||
|
|
||
| // TODO: Let's remove TestHiveSessionState. Otherwise, we are not really testing the reflection | ||
| // logic based on the setting of CATALOG_IMPLEMENTATION. | ||
| @transient | ||
| override lazy val sessionState: TestHiveSessionState = | ||
| new TestHiveSessionState(self) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,7 +31,7 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton { | |
| } | ||
|
|
||
| test("SPARK-15887: hive-site.xml should be loaded") { | ||
| val hiveClient = spark.sharedState.asInstanceOf[HiveSharedState].metadataHive | ||
| val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need to change this line?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. oh sorry I misread the code, |
||
| assert(hiveClient.getConf("hive.in.test", "") == "true") | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,26 +21,26 @@ import org.apache.hadoop.conf.Configuration | |
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.sql.catalyst.catalog._ | ||
| import org.apache.spark.sql.hive.client.HiveClient | ||
|
|
||
| /** | ||
| * Test suite for the [[HiveExternalCatalog]]. | ||
| */ | ||
| class HiveExternalCatalogSuite extends ExternalCatalogSuite { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why the changes in this file make hive client connect to a metastore that already has data?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Before the PR, what |
||
|
|
||
| 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. | ||
| HiveUtils.newClientForExecution(new SparkConf, new Configuration) | ||
| private val externalCatalog: HiveExternalCatalog = { | ||
| val catalog = new HiveExternalCatalog(new SparkConf, new Configuration) | ||
| catalog.client.reset() | ||
| catalog | ||
| } | ||
|
|
||
| protected override val utils: CatalogTestUtils = new CatalogTestUtils { | ||
| override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" | ||
| override val tableOutputFormat: String = "org.apache.hadoop.mapred.SequenceFileOutputFormat" | ||
| override def newEmptyCatalog(): ExternalCatalog = | ||
| new HiveExternalCatalog(client, new Configuration()) | ||
| override def newEmptyCatalog(): ExternalCatalog = externalCatalog | ||
| } | ||
|
|
||
| protected override def resetState(): Unit = client.reset() | ||
| protected override def resetState(): Unit = { | ||
| externalCatalog.client.reset() | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -378,10 +378,9 @@ object SetMetastoreURLTest extends Logging { | |
| s"spark.sql.test.expectedMetastoreURL should be set.") | ||
| } | ||
|
|
||
| // HiveSharedState is used when Hive support is enabled. | ||
| // HiveExternalCatalog is used when Hive support is enabled. | ||
| val actualMetastoreURL = | ||
| spark.sharedState.asInstanceOf[HiveSharedState] | ||
| .metadataHive | ||
| spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need to change this line?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. See the above answer. |
||
| .getConf("javax.jdo.option.ConnectionURL", "this_is_a_wrong_URL") | ||
| logInfo(s"javax.jdo.option.ConnectionURL is $actualMetastoreURL") | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -51,7 +51,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv | |
|
|
||
| // To test `HiveExternalCatalog`, we need to read the raw table metadata(schema, partition | ||
| // columns and bucket specification are still in table properties) from hive client. | ||
| private def hiveClient: HiveClient = sharedState.asInstanceOf[HiveSharedState].metadataHive | ||
| private def hiveClient: HiveClient = | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need to change this line?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. See the above answer. |
||
| sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client | ||
|
|
||
| test("persistent JSON table") { | ||
| withTable("jsonTable") { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -266,7 +266,7 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing | |
| } | ||
|
|
||
| private def createRawHiveTable(ddl: String): Unit = { | ||
| hiveContext.sharedState.asInstanceOf[HiveSharedState].metadataHive.runSqlHive(ddl) | ||
| hiveContext.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.runSqlHive(ddl) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need to change this line?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. See the above answer. |
||
| } | ||
|
|
||
| private def checkCreateTable(table: String): Unit = { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
is
TestHiveSessionStatealready removed?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah. Let me move it back.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The comment is moved to https://github.com/gatorsmile/spark/blob/d61ced8a2e81cef13afa5e0aa6203e505dd67570/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala#L147-L148
Thanks!