Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17433#discussion_r108282976
  
    --- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -121,150 +124,115 @@ private[hive] class HiveSessionState(
       def hiveThriftServerAsync: Boolean = {
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
    +}
     
    +private[hive] object HiveSessionState {
       /**
    -   * Get an identical copy of the `HiveSessionState`.
    -   * This should ideally reuse the `SessionState.clone` but cannot do so.
    -   * Doing that will throw an exception when trying to clone the catalog.
    +   * Create a new [[HiveSessionState]] for the given session.
        */
    -  override def clone(newSparkSession: SparkSession): HiveSessionState = {
    -    val sparkContext = newSparkSession.sparkContext
    -    val confCopy = conf.clone()
    -    val functionRegistryCopy = functionRegistry.clone()
    -    val experimentalMethodsCopy = experimentalMethods.clone()
    -    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    -    val catalogCopy = catalog.newSessionCatalogWith(
    -      newSparkSession,
    -      confCopy,
    -      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, 
confCopy),
    -      functionRegistryCopy,
    -      sqlParser)
    -    val queryExecutionCreator = (plan: LogicalPlan) => new 
QueryExecution(newSparkSession, plan)
    -
    -    val hiveClient =
    -      
newSparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
    -        .newSession()
    -
    -    SessionState.mergeSparkConf(confCopy, sparkContext.getConf)
    -
    -    new HiveSessionState(
    -      sparkContext,
    -      newSparkSession.sharedState,
    -      confCopy,
    -      experimentalMethodsCopy,
    -      functionRegistryCopy,
    -      catalogCopy,
    -      sqlParser,
    -      hiveClient,
    -      HiveSessionState.createAnalyzer(newSparkSession, catalogCopy, 
confCopy),
    -      new StreamingQueryManager(newSparkSession),
    -      queryExecutionCreator,
    -      HiveSessionState.createPlannerCreator(
    -        newSparkSession,
    -        confCopy,
    -        experimentalMethodsCopy))
    +  def apply(session: SparkSession): SessionState = {
    +    new HiveSessionStateBuilder(session).build()
       }
    -
     }
     
    -private[hive] object HiveSessionState {
    -
    -  def apply(sparkSession: SparkSession): HiveSessionState = {
    -    apply(sparkSession, new SQLConf)
    -  }
    -
    -  def apply(sparkSession: SparkSession, conf: SQLConf): HiveSessionState = 
{
    -    val initHelper = SessionState(sparkSession, conf)
    -
    -    val sparkContext = sparkSession.sparkContext
    -
    -    val catalog = HiveSessionCatalog(
    -      sparkSession,
    -      initHelper.functionRegistry,
    -      initHelper.conf,
    -      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, 
initHelper.conf),
    -      initHelper.sqlParser)
    -
    -    val metadataHive: HiveClient =
    -      
sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
    -        .newSession()
    -
    -    val analyzer: Analyzer = createAnalyzer(sparkSession, catalog, 
initHelper.conf)
    +/**
    + * Builder that produces a [[HiveSessionState]].
    + */
    +@Experimental
    [email protected]
    +class HiveSessionStateBuilder(session: SparkSession, parentState: 
Option[SessionState] = None)
    +  extends BaseSessionStateBuilder(session, parentState) {
     
    -    val plannerCreator = createPlannerCreator(
    -      sparkSession,
    -      initHelper.conf,
    -      initHelper.experimentalMethods)
    +  private def externalCatalog: HiveExternalCatalog =
    +    session.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
     
    -    val hiveSessionState = new HiveSessionState(
    -      sparkContext,
    -      sparkSession.sharedState,
    -      initHelper.conf,
    -      initHelper.experimentalMethods,
    -      initHelper.functionRegistry,
    -      catalog,
    -      initHelper.sqlParser,
    -      metadataHive,
    -      analyzer,
    -      initHelper.streamingQueryManager,
    -      initHelper.queryExecutionCreator,
    -      plannerCreator)
    -    catalog.functionResourceLoader = 
hiveSessionState.functionResourceLoader
    -    hiveSessionState
    +  /**
    +   * Create a [[HiveSessionCatalog]].
    +   */
    +  override protected lazy val catalog: HiveSessionCatalog = {
    +    val catalog = new HiveSessionCatalog(
    +      externalCatalog,
    +      session.sharedState.globalTempViewManager,
    +      new HiveMetastoreCatalog(session),
    +      functionRegistry,
    +      conf,
    +      SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, 
conf),
    +      sqlParser,
    +      new SessionFunctionResourceLoader(session))
    +    parentState.foreach(_.catalog.copyStateTo(catalog))
    +    catalog
       }
     
       /**
    -   * Create an logical query plan `Analyzer` with rules specific to a 
`HiveSessionState`.
    +   * An logical query plan `Analyzer` with rules specific to Hive.
    --- End diff --
    
    super nit: "A logical ..."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to