You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by kunalkhamar <gi...@git.apache.org> on 2017/02/06 23:25:37 UTC

[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

GitHub user kunalkhamar opened a pull request:

    https://github.com/apache/spark/pull/16826

    Fork SparkSession with option to inherit a copy of the SessionState.

    ## What changes were proposed in this pull request?
    
    Forking a newSession() from SparkSession currently makes a new SparkSession that does not retain SessionState (i.e. temporary tables, SQL config, registered functions etc.) This change adds a flag newSession(inheritSessionState: Boolean) which can create a new SparkSession with a copy of the parent's SessionState.
    
    ## How was this patch tested?
    
    Unit tests

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/kunalkhamar/spark fork-sparksession

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/16826.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #16826
    
----
commit 18ce1b82403cfb1e2b4fcac2dd5875f289d61ed3
Author: Kunal Khamar <kk...@outlook.com>
Date:   2017-02-03T00:31:05Z

    Add capability to inherit SessionState (SQL conf, temp tables, registered functions) when forking a new SparkSession.

commit 9beb78d82d7777bc17aed047d818fdd4fca1b0d6
Author: Kunal Khamar <kk...@outlook.com>
Date:   2017-02-06T22:55:56Z

    Add tests for forking new session with inherit config enabled. Update overloaded functions for Java bytecode compatibility.

commit a343d8af9c577158042e4af9f8832f46aeecd509
Author: Kunal Khamar <kk...@outlook.com>
Date:   2017-02-06T23:24:02Z

    Fix constructor default args for bytecode compatibility.

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102137396
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -217,6 +217,7 @@ class HiveSparkSubmitSuite
         runSparkSubmit(args)
       }
     
    +  /*  TODO: SPARK-19540 re-enable this test
       test("set hive.metastore.warehouse.dir") {
    --- End diff --
    
    updated to use `ignore`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][Spark-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72679 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72679/testReport)** for PR 16826 at commit [`4210079`](https://github.com/apache/spark/commit/4210079c0dfe4eb3d606606ca06ac28ccdbf2ae1).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][Spark-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73027 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73027/testReport)** for PR 16826 at commit [`e2bbfa8`](https://github.com/apache/spark/commit/e2bbfa8c81f91c57f5628e771f42d414a1031d57).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103534150
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1181,34 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary views and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, do not need deep
    +   * copy. `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def clone(
    --- End diff --
    
    I think it's worth to use a better name rather than `clone`. I thought it should clone all necessary fields inside `SessionCatalog`. But `conf` and `functionRegistry` are copied by the caller.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618883
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +85,120 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(sparkSession: SparkSession): SessionState = {
    +    val sparkContext = sparkSession.sparkContext
    +    val confCopy = conf.clone()
    +    val hadoopConfCopy = SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy)
    +    val functionRegistryCopy = functionRegistry.clone()
    +    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    +    val catalogCopy = catalog.clone(confCopy, hadoopConfCopy, functionRegistryCopy, sqlParser)
    +    val queryExecution = (plan: LogicalPlan) => new QueryExecution(sparkSession, plan)
    +
    +    sparkContext.getConf.getAll.foreach { case (k, v) =>
    +      confCopy.setConfString(k, v)
    +    }
    +
    +    new SessionState(
    +      sparkContext,
    +      sparkSession.sharedState,
    +      confCopy,
    +      experimentalMethods.copy,
    +      functionRegistryCopy,
    +      catalogCopy,
    +      sqlParser,
    +      SessionState.createAnalyzer(sparkSession, catalogCopy, confCopy),
    +      new StreamingQueryManager(sparkSession),
    +      queryExecution)
    +  }
    +
    +  // ------------------------------------------------------
    +  //  Helper methods, partially leftover from pre-2.0 days
    +  // ------------------------------------------------------
    +
    +  def executePlan(plan: LogicalPlan): QueryExecution = queryExecutionCreator(plan)
    +
    +  def refreshTable(tableName: String): Unit = {
    +    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    +  }
    +
    +  def addJar(path: String): Unit = sharedState.addJar(path)
    +
    --- End diff --
    
    extra line.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103331408
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -17,89 +17,50 @@
     
     package org.apache.spark.sql.hive
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.analysis.Analyzer
    -import org.apache.spark.sql.execution.SparkPlanner
    +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanner, SparkSqlParser}
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.hive.client.HiveClient
    -import org.apache.spark.sql.internal.SessionState
    +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
      */
    -private[hive] class HiveSessionState(sparkSession: SparkSession)
    -  extends SessionState(sparkSession) {
    -
    -  self =>
    -
    -  /**
    -   * A Hive client used for interacting with the metastore.
    -   */
    -  lazy val metadataHive: HiveClient =
    -    sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.newSession()
    -
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  override lazy val catalog = {
    -    new HiveSessionCatalog(
    -      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    -      sparkSession.sharedState.globalTempViewManager,
    -      sparkSession,
    -      functionResourceLoader,
    -      functionRegistry,
    +private[hive] class HiveSessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    conf: SQLConf,
    +    experimentalMethods: ExperimentalMethods,
    +    functionRegistry: FunctionRegistry,
    +    override val catalog: HiveSessionCatalog,
    +    sqlParser: ParserInterface,
    +    val metadataHive: HiveClient,
    +    override val analyzer: Analyzer,
    --- End diff --
    
    Previous implementation needed it to be that way. But can remove `override` now. Good catch.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73388/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72970 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72970/testReport)** for PR 16826 at commit [`8c00344`](https://github.com/apache/spark/commit/8c003449a910a5909fd0b80509ef7b3ef96d698b).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74134/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103062876
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +82,118 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(newSparkSession: SparkSession): SessionState = {
    +    val sparkContext = newSparkSession.sparkContext
    +    val confCopy = conf.clone()
    +    val functionRegistryCopy = functionRegistry.clone()
    +    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    +    val catalogCopy = catalog.clone(
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      functionRegistryCopy,
    +      sqlParser)
    +    val queryExecutionCreator = (plan: LogicalPlan) => new QueryExecution(newSparkSession, plan)
    +
    +    SessionState.mergeSparkConf(confCopy, sparkContext.getConf)
    +
    +    new SessionState(
    +      sparkContext,
    +      newSparkSession.sharedState,
    +      confCopy,
    +      experimentalMethods.clone(),
    +      functionRegistryCopy,
    +      catalogCopy,
    +      sqlParser,
    +      SessionState.createAnalyzer(newSparkSession, catalogCopy, confCopy),
    +      new StreamingQueryManager(newSparkSession),
    +      queryExecutionCreator)
    +  }
    +
    +  // ------------------------------------------------------
    +  //  Helper methods, partially leftover from pre-2.0 days
    +  // ------------------------------------------------------
    +
    +  def executePlan(plan: LogicalPlan): QueryExecution = queryExecutionCreator(plan)
    +
    +  def refreshTable(tableName: String): Unit = {
    +    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    +  }
    +
    +  def addJar(path: String): Unit = sharedState.addJar(path)
    +}
    +
    +
    +object SessionState {
    +
    +  def apply(sparkSession: SparkSession): SessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): SessionState = {
    --- End diff --
    
    why option? the external api should allow user to do 
    ```
    SessionState(session)
    SessionState(session, existingConf)
    ```
    Doesnt make sense if the user has to write an additional `Some(existingConf)`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618467
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -870,6 +902,7 @@ object SparkSession {
               if (!sc.conf.contains("spark.app.name")) {
                 sc.conf.setAppName(randomAppName)
               }
    +          logDebug("3 using new sparkContext")
    --- End diff --
    
    remove.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103065284
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +class SessionStateSuite extends SparkFunSuite with BeforeAndAfterEach {
    +
    +  protected var activeSession: SparkSession = _
    +
    +  protected def createSession(): Unit = {
    +    activeSession = SparkSession.builder().master("local").getOrCreate()
    +  }
    +
    +  override def beforeEach(): Unit = {
    +    createSession()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    activeSession.stop()
    +  }
    +
    +  test("fork new session and inherit RuntimeConfig options") {
    +    val key = "spark-config-clone"
    +    activeSession.conf.set(key, "active")
    +
    +    // inheritance
    +    val forkedSession = activeSession.cloneSession()
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.conf ne activeSession.conf)
    +    assert(forkedSession.conf.get(key) == "active")
    +
    +    // independence
    +    forkedSession.conf.set(key, "forked")
    +    assert(activeSession.conf.get(key) == "active")
    +    activeSession.conf.set(key, "dontcopyme")
    +    assert(forkedSession.conf.get(key) == "forked")
    +  }
    +
    +  test("fork new session and inherit function registry and udf") {
    +    activeSession.udf.register("strlenScala", (_: String).length + (_: Int))
    +    val forkedSession = activeSession.cloneSession()
    +
    +    // inheritance
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.sessionState.functionRegistry ne
    +      activeSession.sessionState.functionRegistry)
    +    assert(forkedSession.sessionState.functionRegistry.lookupFunction("strlenScala").nonEmpty)
    +
    +    // independence
    +    forkedSession.sessionState.functionRegistry.dropFunction("strlenScala")
    +    assert(activeSession.sessionState.functionRegistry.lookupFunction("strlenScala").nonEmpty)
    +    activeSession.udf.register("addone", (_: Int) + 1)
    +    assert(forkedSession.sessionState.functionRegistry.lookupFunction("addone").isEmpty)
    +  }
    +
    +  test("fork new session and inherit experimental methods") {
    +    object DummyRule1 extends Rule[LogicalPlan] {
    +      def apply(p: LogicalPlan): LogicalPlan = p
    +    }
    +    object DummyRule2 extends Rule[LogicalPlan] {
    +      def apply(p: LogicalPlan): LogicalPlan = p
    +    }
    +    val optimizations = List(DummyRule1, DummyRule2)
    +
    +    activeSession.experimental.extraOptimizations = optimizations
    +
    +    val forkedSession = activeSession.cloneSession()
    +
    +    // inheritance
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.experimental ne activeSession.experimental)
    +    assert(forkedSession.experimental.extraOptimizations.toSet ==
    +      activeSession.experimental.extraOptimizations.toSet)
    +
    +    // independence
    +    forkedSession.experimental.extraOptimizations = List(DummyRule2)
    +    assert(activeSession.experimental.extraOptimizations == optimizations)
    +    activeSession.experimental.extraOptimizations = List(DummyRule1)
    +    assert(forkedSession.experimental.extraOptimizations == List(DummyRule2))
    +  }
    +
    +  test("fork new sessions and run query on inherited table") {
    +    def checkTableExists(sparkSession: SparkSession): Unit = {
    +      QueryTest.checkAnswer(sparkSession.sql(
    +        """
    +          |SELECT x.str, COUNT(*)
    +          |FROM df x JOIN df y ON x.str = y.str
    +          |GROUP BY x.str
    +        """.stripMargin),
    +        Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil)
    +    }
    +
    +    SparkSession.setActiveSession(activeSession)
    --- End diff --
    
    well.. why do you need to set this? other tests in this suite does not set it. if there is a reason, its not obvious and should be commented about.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104604870
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -17,89 +17,55 @@
     
     package org.apache.spark.sql.hive
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.analysis.Analyzer
    -import org.apache.spark.sql.execution.SparkPlanner
    +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanner, SparkSqlParser}
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.hive.client.HiveClient
    -import org.apache.spark.sql.internal.SessionState
    +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
    + * @param catalog A Hive client used for interacting with the metastore.
    + * @param analyzer An analyzer that uses the Hive metastore.
    + * @param plannerCreator Lambda to create a [[SparkPlanner]] that converts optimized logical
    + *                       plans to physical plans.
      */
    -private[hive] class HiveSessionState(sparkSession: SparkSession)
    -  extends SessionState(sparkSession) {
    -
    -  self =>
    -
    -  /**
    -   * A Hive client used for interacting with the metastore.
    -   */
    -  lazy val metadataHive: HiveClient =
    -    sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.newSession()
    -
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  override lazy val catalog = {
    -    new HiveSessionCatalog(
    -      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    -      sparkSession.sharedState.globalTempViewManager,
    -      sparkSession,
    -      functionResourceLoader,
    -      functionRegistry,
    +private[hive] class HiveSessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    conf: SQLConf,
    +    experimentalMethods: ExperimentalMethods,
    +    functionRegistry: FunctionRegistry,
    +    override val catalog: HiveSessionCatalog,
    +    sqlParser: ParserInterface,
    +    val metadataHive: HiveClient,
    --- End diff --
    
    This is for avoiding using `lazy val`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74039/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102620162
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala ---
    @@ -34,9 +40,32 @@ private[sql] class TestSparkSession(sc: SparkContext) extends SparkSession(sc) {
         this(new SparkConf)
       }
     
    -  @transient
    -  override lazy val sessionState: SessionState = new SessionState(self) {
    -    override lazy val conf: SQLConf = {
    +  class TestSessionState(
    +      sparkContext: SparkContext,
    +      sharedState: SharedState,
    +      conf: SQLConf,
    +      experimentalMethods: ExperimentalMethods,
    +      functionRegistry: FunctionRegistry,
    +      catalog: SessionCatalog,
    +      sqlParser: ParserInterface,
    +      analyzer: Analyzer,
    +      streamingQueryManager: StreamingQueryManager,
    +      queryExecution: LogicalPlan => QueryExecution)
    +    extends SessionState(
    +        sparkContext,
    +        sharedState,
    +        conf,
    +        experimentalMethods,
    +        functionRegistry,
    +        catalog,
    +        sqlParser,
    +        analyzer,
    +        streamingQueryManager,
    +        queryExecution) {}
    +
    +  object TestSessionState {
    +
    +    def createTestConf: SQLConf = {
    --- End diff --
    
    testConf


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73302 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73302/testReport)** for PR 16826 at commit [`4f70d12`](https://github.com/apache/spark/commit/4f70d12dba6f89a6548ffea40879663a42902811).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064055
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -17,43 +17,60 @@
     
     package org.apache.spark.sql.internal
     
    -import java.io.File
    -
     import org.apache.hadoop.conf.Configuration
    -import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.{SparkConf, SparkContext}
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution) {
    --- End diff --
    
    nit: earlier all of there were vals and had docs on them. now they dont. i see that you have converted those docs to inline comments in `apply()` but thats does help developers as they wont show up as docs. So i would add those docs in the class docs with `@ param`. This is more necessary than inline comments in `apply`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104607606
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -17,89 +17,55 @@
     
     package org.apache.spark.sql.hive
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.analysis.Analyzer
    -import org.apache.spark.sql.execution.SparkPlanner
    +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanner, SparkSqlParser}
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.hive.client.HiveClient
    -import org.apache.spark.sql.internal.SessionState
    +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
    + * @param catalog A Hive client used for interacting with the metastore.
    + * @param analyzer An analyzer that uses the Hive metastore.
    + * @param plannerCreator Lambda to create a [[SparkPlanner]] that converts optimized logical
    + *                       plans to physical plans.
      */
    -private[hive] class HiveSessionState(sparkSession: SparkSession)
    -  extends SessionState(sparkSession) {
    -
    -  self =>
    -
    -  /**
    -   * A Hive client used for interacting with the metastore.
    -   */
    -  lazy val metadataHive: HiveClient =
    -    sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.newSession()
    -
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  override lazy val catalog = {
    -    new HiveSessionCatalog(
    -      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    -      sparkSession.sharedState.globalTempViewManager,
    -      sparkSession,
    -      functionResourceLoader,
    -      functionRegistry,
    +private[hive] class HiveSessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    conf: SQLConf,
    +    experimentalMethods: ExperimentalMethods,
    +    functionRegistry: FunctionRegistry,
    +    override val catalog: HiveSessionCatalog,
    +    sqlParser: ParserInterface,
    +    val metadataHive: HiveClient,
    +    analyzer: Analyzer,
    +    streamingQueryManager: StreamingQueryManager,
    +    queryExecutionCreator: LogicalPlan => QueryExecution,
    +    val plannerCreator: () => SparkPlanner)
    --- End diff --
    
    uh, I see


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102140755
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution,
    +    val jarClassLoader: NonClosableMutableURLClassLoader) {
    --- End diff --
    
    `jarClassLoader ` is from SharedState. Do we still need it as an input parm for SessionState?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103306666
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +class SessionStateSuite extends SparkFunSuite with BeforeAndAfterEach {
    +
    +  protected var activeSession: SparkSession = _
    +
    +  protected def createSession(): Unit = {
    +    activeSession = SparkSession.builder().master("local").getOrCreate()
    +  }
    +
    +  override def beforeEach(): Unit = {
    +    createSession()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    activeSession.stop()
    +  }
    +
    +  test("fork new session and inherit RuntimeConfig options") {
    +    val key = "spark-config-clone"
    +    activeSession.conf.set(key, "active")
    +
    +    // inheritance
    +    val forkedSession = activeSession.cloneSession()
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.conf ne activeSession.conf)
    +    assert(forkedSession.conf.get(key) == "active")
    +
    +    // independence
    +    forkedSession.conf.set(key, "forked")
    +    assert(activeSession.conf.get(key) == "active")
    +    activeSession.conf.set(key, "dontcopyme")
    +    assert(forkedSession.conf.get(key) == "forked")
    +  }
    +
    +  test("fork new session and inherit function registry and udf") {
    +    activeSession.udf.register("strlenScala", (_: String).length + (_: Int))
    +    val forkedSession = activeSession.cloneSession()
    +
    +    // inheritance
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.sessionState.functionRegistry ne
    +      activeSession.sessionState.functionRegistry)
    +    assert(forkedSession.sessionState.functionRegistry.lookupFunction("strlenScala").nonEmpty)
    +
    +    // independence
    +    forkedSession.sessionState.functionRegistry.dropFunction("strlenScala")
    +    assert(activeSession.sessionState.functionRegistry.lookupFunction("strlenScala").nonEmpty)
    +    activeSession.udf.register("addone", (_: Int) + 1)
    +    assert(forkedSession.sessionState.functionRegistry.lookupFunction("addone").isEmpty)
    +  }
    +
    +  test("fork new session and inherit experimental methods") {
    +    object DummyRule1 extends Rule[LogicalPlan] {
    +      def apply(p: LogicalPlan): LogicalPlan = p
    +    }
    +    object DummyRule2 extends Rule[LogicalPlan] {
    +      def apply(p: LogicalPlan): LogicalPlan = p
    +    }
    +    val optimizations = List(DummyRule1, DummyRule2)
    +
    +    activeSession.experimental.extraOptimizations = optimizations
    +
    --- End diff --
    
    removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102595563
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala ---
    @@ -46,4 +46,10 @@ class ExperimentalMethods private[sql]() {
     
       @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil
     
    +  def copy: ExperimentalMethods = {
    --- End diff --
    
    Changed to `clone()`. It does not seem straightforward to override `copy()` in case classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
     Merged build triggered.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100195324
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -213,6 +218,24 @@ class SparkSession private(
         new SparkSession(sparkContext, Some(sharedState))
       }
     
    +  /**
    +   * Start a new session, sharing the underlying `SparkContext` and cached data.
    +   * If inheritSessionState is enabled, then SQL configurations, temporary tables,
    +   * registered functions are copied over from parent `SparkSession`.
    +   *
    +   * @note Other than the `SparkContext`, all shared state is initialized lazily.
    +   * This method will force the initialization of the shared state to ensure that parent
    +   * and child sessions are set up with the same shared state. If the underlying catalog
    +   * implementation is Hive, this will initialize the metastore, which may take some time.
    --- End diff --
    
    nit: add `@since 2.1.1`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102602517
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -217,7 +217,8 @@ class HiveSparkSubmitSuite
         runSparkSubmit(args)
       }
     
    -  test("set hive.metastore.warehouse.dir") {
    +  // TODO: SPARK-19540 re-enable this test
    +  ignore("set hive.metastore.warehouse.dir") {
    --- End diff --
    
    This was intended to be temporary, working on fixing these tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104606859
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -278,6 +278,8 @@ private[hive] class HiveClientImpl(
         state.getConf.setClassLoader(clientLoader.classLoader)
         // Set the thread local metastore client to the client associated with this HiveClientImpl.
         Hive.set(client)
    +    // Replace conf in the thread local Hive with current conf
    +    Hive.get(conf)
    --- End diff --
    
    Because of reusing`IsolatedClientLoader.cachedHive`. Without this line, it may use an out-of-date HiveConf. The failed test is `HiveSparkSubmitSuite.test("SPARK-18360: default table path of tables in default database should depend on the " +
        "location of default database")`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73195/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73026 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73026/testReport)** for PR 16826 at commit [`2cee190`](https://github.com/apache/spark/commit/2cee190eb6c6902d39f68c25d928fbd5aaa522bc).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100195293
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -213,6 +218,24 @@ class SparkSession private(
         new SparkSession(sparkContext, Some(sharedState))
       }
     
    +  /**
    +   * Start a new session, sharing the underlying `SparkContext` and cached data.
    --- End diff --
    
    nit: add `:: Experimental ::`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103328320
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala ---
    @@ -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.spark.sql.hive
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +
    +class HiveSessionStateSuite extends SessionStateSuite
    +  with TestHiveSingleton with BeforeAndAfterEach {
    +
    +  override def beforeEach(): Unit = {
    +    createSession()
    +  }
    +
    +  override def afterEach(): Unit = {}
    +
    +  override def createSession(): Unit = {
    +    activeSession = spark.newSession()
    +  }
    +
    --- End diff --
    
    removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73190/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103065045
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionCatalogSuite.scala ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.hive
    +
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.SimpleFunctionRegistry
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
    +import org.apache.spark.sql.catalyst.plans.logical.Range
    +import org.apache.spark.sql.internal.SQLConf
    +
    +class HiveSessionCatalogSuite extends SessionCatalogSuite {
    --- End diff --
    
    what is the point of inheriting the test suite? the tests in SessionCatalogSuite seems to test SessionCatalog and not HiveCatalog. that is, i dont see any code that customizes this suite to run existing tests with a HiveSessionCatalog.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104605419
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -17,89 +17,55 @@
     
     package org.apache.spark.sql.hive
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.analysis.Analyzer
    -import org.apache.spark.sql.execution.SparkPlanner
    +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanner, SparkSqlParser}
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.hive.client.HiveClient
    -import org.apache.spark.sql.internal.SessionState
    +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
    + * @param catalog A Hive client used for interacting with the metastore.
    + * @param analyzer An analyzer that uses the Hive metastore.
    + * @param plannerCreator Lambda to create a [[SparkPlanner]] that converts optimized logical
    + *                       plans to physical plans.
      */
    -private[hive] class HiveSessionState(sparkSession: SparkSession)
    -  extends SessionState(sparkSession) {
    -
    -  self =>
    -
    -  /**
    -   * A Hive client used for interacting with the metastore.
    -   */
    -  lazy val metadataHive: HiveClient =
    -    sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.newSession()
    -
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  override lazy val catalog = {
    -    new HiveSessionCatalog(
    -      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    -      sparkSession.sharedState.globalTempViewManager,
    -      sparkSession,
    -      functionResourceLoader,
    -      functionRegistry,
    +private[hive] class HiveSessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    conf: SQLConf,
    +    experimentalMethods: ExperimentalMethods,
    +    functionRegistry: FunctionRegistry,
    +    override val catalog: HiveSessionCatalog,
    +    sqlParser: ParserInterface,
    +    val metadataHive: HiveClient,
    +    analyzer: Analyzer,
    +    streamingQueryManager: StreamingQueryManager,
    +    queryExecutionCreator: LogicalPlan => QueryExecution,
    +    val plannerCreator: () => SparkPlanner)
    --- End diff --
    
    How about adding `val planner` to `SessionState`? So far, the interface of `HiveSessionState` looks a little bit complex to me.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103065209
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +class SessionStateSuite extends SparkFunSuite with BeforeAndAfterEach {
    +
    +  protected var activeSession: SparkSession = _
    +
    +  protected def createSession(): Unit = {
    --- End diff --
    
    nit: this is creating a SparkContext every time. rather use SharedSparkContext trait, to share that, and create sessions using the same spark context.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72897 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72897/testReport)** for PR 16826 at commit [`579d0b7`](https://github.com/apache/spark/commit/579d0b77738e2de53c06725e55f6a0de905325a5).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102622705
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    --- End diff --
    
    double check whether all of these need to be vals


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73302/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    @zsxwing @tdas @viirya @gatorsmile 
    This is ready for another iteration.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73026 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73026/testReport)** for PR 16826 at commit [`2cee190`](https://github.com/apache/spark/commit/2cee190eb6c6902d39f68c25d928fbd5aaa522bc).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100195235
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -213,6 +218,24 @@ class SparkSession private(
         new SparkSession(sparkContext, Some(sharedState))
       }
     
    +  /**
    +   * Start a new session, sharing the underlying `SparkContext` and cached data.
    +   * If inheritSessionState is enabled, then SQL configurations, temporary tables,
    +   * registered functions are copied over from parent `SparkSession`.
    +   *
    +   * @note Other than the `SparkContext`, all shared state is initialized lazily.
    +   * This method will force the initialization of the shared state to ensure that parent
    +   * and child sessions are set up with the same shared state. If the underlying catalog
    +   * implementation is Hive, this will initialize the metastore, which may take some time.
    +   */
    --- End diff --
    
    nit: please add `@Experimental` and `@InterfaceStability.Evolving`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064415
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -146,4 +107,153 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
     
    +  /**
    +   * 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.
    +   */
    +  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.clone(
    +      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))
    +  }
    +
    +}
    +
    +object HiveSessionState {
    +
    +  def apply(sparkSession: SparkSession): HiveSessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): HiveSessionState = {
    +
    +    val initHelper = SessionState(sparkSession, conf)
    +
    +    val sparkContext = sparkSession.sparkContext
    +
    +    val catalog = HiveSessionCatalog(
    +      sparkSession,
    +      SessionState.createFunctionResourceLoader(sparkContext, sparkSession.sharedState),
    +      initHelper.functionRegistry,
    +      initHelper.conf,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, initHelper.conf),
    +      initHelper.sqlParser)
    +
    +    // A Hive client used for interacting with the metastore.
    +    val metadataHive: HiveClient =
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
    +        .newSession()
    +
    +    // An analyzer that uses the Hive metastore.
    +    val analyzer: Analyzer = createAnalyzer(sparkSession, catalog, initHelper.conf)
    +
    +    val plannerCreator = createPlannerCreator(
    +      sparkSession,
    +      initHelper.conf,
    +      initHelper.experimentalMethods)
    +
    +    new HiveSessionState(
    +      sparkContext,
    +      sparkSession.sharedState,
    +      initHelper.conf,
    +      initHelper.experimentalMethods,
    +      initHelper.functionRegistry,
    +      catalog,
    +      initHelper.sqlParser,
    +      metadataHive,
    +      analyzer,
    +      initHelper.streamingQueryManager,
    +      initHelper.queryExecutionCreator,
    +      plannerCreator)
    +  }
    +
    +  def createAnalyzer(
    --- End diff --
    
    private?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102622403
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -212,3 +240,31 @@ private[sql] class HiveSessionCatalog(
         "histogram_numeric"
       )
     }
    +
    +private[sql] object HiveSessionCatalog {
    +
    +  def apply(
    --- End diff --
    
    Is this apply used anywhere other than in `clone`? if not, then there is no point creating an object with an `apply`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103063348
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
    @@ -136,6 +139,26 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
         }
         SparkSession.sqlListener.get()
       }
    +
    +  /*
    +   * This belongs here more than in `SessionState`. However, does not seem that it can be
    --- End diff --
    
    also i think its more obvious here (since it uses jarClassloaded from sharedState). so no reasoning necessary at all. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618239
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -212,6 +219,29 @@ class SparkSession private(
         new SparkSession(sparkContext, Some(sharedState))
       }
     
    +  /**
    +   * :: Experimental ::
    +   * Create an identical copy of this `SparkSession`, sharing the underlying `SparkContext`
    +   * and cached data. All the state of this session (i.e. SQL configurations, temporary tables,
    +   * registered functions) is also copied over.
    +   * Changes to base session are not propagated to cloned session, cloned is independent
    --- End diff --
    
    The cloned session is independent of this session, that is, any changes in either session is not reflected in the other.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73388 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73388/testReport)** for PR 16826 at commit [`16824f9`](https://github.com/apache/spark/commit/16824f916e87fd90706f9dfd7b7dd81d87b732dd).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73448 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73448/testReport)** for PR 16826 at commit [`437b0bc`](https://github.com/apache/spark/commit/437b0bca7bc29809083f26b8a4848d53d999d097).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103307491
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -146,4 +107,153 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
     
    +  /**
    +   * 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.
    +   */
    +  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.clone(
    +      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))
    +  }
    +
    +}
    +
    +object HiveSessionState {
    +
    +  def apply(sparkSession: SparkSession): HiveSessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): HiveSessionState = {
    +
    +    val initHelper = SessionState(sparkSession, conf)
    +
    +    val sparkContext = sparkSession.sparkContext
    +
    +    val catalog = HiveSessionCatalog(
    +      sparkSession,
    +      SessionState.createFunctionResourceLoader(sparkContext, sparkSession.sharedState),
    +      initHelper.functionRegistry,
    +      initHelper.conf,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, initHelper.conf),
    +      initHelper.sqlParser)
    +
    +    // A Hive client used for interacting with the metastore.
    +    val metadataHive: HiveClient =
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
    +        .newSession()
    +
    +    // An analyzer that uses the Hive metastore.
    +    val analyzer: Analyzer = createAnalyzer(sparkSession, catalog, initHelper.conf)
    +
    +    val plannerCreator = createPlannerCreator(
    +      sparkSession,
    +      initHelper.conf,
    +      initHelper.experimentalMethods)
    +
    +    new HiveSessionState(
    +      sparkContext,
    +      sparkSession.sharedState,
    +      initHelper.conf,
    +      initHelper.experimentalMethods,
    +      initHelper.functionRegistry,
    +      catalog,
    +      initHelper.sqlParser,
    +      metadataHive,
    +      analyzer,
    +      initHelper.streamingQueryManager,
    +      initHelper.queryExecutionCreator,
    +      plannerCreator)
    +  }
    +
    +  def createAnalyzer(
    +      sparkSession: SparkSession,
    +      catalog: HiveSessionCatalog,
    +      sqlConf: SQLConf): Analyzer = {
    +
    +    new Analyzer(catalog, sqlConf) {
    +      override val extendedResolutionRules =
    +        new ResolveHiveSerdeTable(sparkSession) ::
    +        new FindDataSourceTable(sparkSession) ::
    +        new FindHiveSerdeTable(sparkSession) ::
    +        new ResolveSQLOnFile(sparkSession) :: Nil
    +
    +      override val postHocResolutionRules =
    +        catalog.ParquetConversions ::
    +        catalog.OrcConversions ::
    +        PreprocessTableCreation(sparkSession) ::
    +        PreprocessTableInsertion(sqlConf) ::
    +        DataSourceAnalysis(sqlConf) ::
    +        HiveAnalysis :: Nil
    +
    +      override val extendedCheckRules = Seq(PreWriteCheck)
    +    }
    +  }
    +
    +  def createPlannerCreator(
    +      associatedSparkSession: SparkSession,
    +      sqlConf: SQLConf,
    +      experimentalMethods: ExperimentalMethods): () => SparkPlanner = {
    +
    --- End diff --
    
    Removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73448/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: Fork SparkSession with option to inherit a copy of the S...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    @kunalkhamar you should create a JIRA ticket for this.
    
    In addition, I'm not a big fan of the design to pass a base session in. It'd be simpler if there is just a clone method on sessionstate and the associated states we store, and then cloning a new sparksession is just creating a new sparksession with cloned sessionstate.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74056 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74056/testReport)** for PR 16826 at commit [`2f0b1ad`](https://github.com/apache/spark/commit/2f0b1adf98807f0ca9f48ab46b1f5f7521c365c2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103061563
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1196,4 +1198,28 @@ class SessionCatalogSuite extends PlanTest {
           catalog.listFunctions("unknown_db", "func*")
         }
       }
    +
    +  test("copy SessionCatalog") {
    --- End diff --
    
    nit: copy -> clone


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103533034
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1181,34 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    --- End diff --
    
    This is not correct. I would like to say
    ```
    Create a new SessionCatalog with `currentDb` and the existing temp tables.
    ``` 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: Fork SparkSession with option to inherit a copy of the S...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Can one of the admins verify this patch?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72727 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72727/testReport)** for PR 16826 at commit [`6da6bda`](https://github.com/apache/spark/commit/6da6bdadc526d71ce887b0bcbef57272a713d6b2).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104565854
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
    @@ -136,6 +139,21 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
         }
         SparkSession.sqlListener.get()
       }
    +
    +  def addJar(path: String): Unit = {
    --- End diff --
    
    Moving this back to SessionState to avoid that someone calls it via SharedState by mistake in future.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103307776
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -146,4 +107,153 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
     
    +  /**
    +   * 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.
    +   */
    +  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.clone(
    +      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))
    +  }
    +
    +}
    +
    +object HiveSessionState {
    +
    +  def apply(sparkSession: SparkSession): HiveSessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): HiveSessionState = {
    +
    +    val initHelper = SessionState(sparkSession, conf)
    +
    +    val sparkContext = sparkSession.sparkContext
    +
    +    val catalog = HiveSessionCatalog(
    +      sparkSession,
    +      SessionState.createFunctionResourceLoader(sparkContext, sparkSession.sharedState),
    +      initHelper.functionRegistry,
    +      initHelper.conf,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, initHelper.conf),
    +      initHelper.sqlParser)
    +
    +    // A Hive client used for interacting with the metastore.
    +    val metadataHive: HiveClient =
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
    +        .newSession()
    +
    +    // An analyzer that uses the Hive metastore.
    +    val analyzer: Analyzer = createAnalyzer(sparkSession, catalog, initHelper.conf)
    +
    +    val plannerCreator = createPlannerCreator(
    +      sparkSession,
    +      initHelper.conf,
    +      initHelper.experimentalMethods)
    +
    +    new HiveSessionState(
    +      sparkContext,
    +      sparkSession.sharedState,
    +      initHelper.conf,
    +      initHelper.experimentalMethods,
    +      initHelper.functionRegistry,
    +      catalog,
    +      initHelper.sqlParser,
    +      metadataHive,
    +      analyzer,
    +      initHelper.streamingQueryManager,
    +      initHelper.queryExecutionCreator,
    +      plannerCreator)
    +  }
    +
    +  def createAnalyzer(
    +      sparkSession: SparkSession,
    +      catalog: HiveSessionCatalog,
    +      sqlConf: SQLConf): Analyzer = {
    +
    +    new Analyzer(catalog, sqlConf) {
    +      override val extendedResolutionRules =
    +        new ResolveHiveSerdeTable(sparkSession) ::
    +        new FindDataSourceTable(sparkSession) ::
    +        new FindHiveSerdeTable(sparkSession) ::
    +        new ResolveSQLOnFile(sparkSession) :: Nil
    +
    +      override val postHocResolutionRules =
    +        catalog.ParquetConversions ::
    +        catalog.OrcConversions ::
    +        PreprocessTableCreation(sparkSession) ::
    +        PreprocessTableInsertion(sqlConf) ::
    +        DataSourceAnalysis(sqlConf) ::
    +        HiveAnalysis :: Nil
    +
    +      override val extendedCheckRules = Seq(PreWriteCheck)
    +    }
    +  }
    +
    +  def createPlannerCreator(
    +      associatedSparkSession: SparkSession,
    +      sqlConf: SQLConf,
    +      experimentalMethods: ExperimentalMethods): () => SparkPlanner = {
    +
    +    () =>
    +      new SparkPlanner(
    +          associatedSparkSession.sparkContext,
    +          sqlConf,
    +          experimentalMethods.extraStrategies)
    +        with HiveStrategies {
    +
    +        override val sparkSession: SparkSession = associatedSparkSession
    +
    +        override def strategies: Seq[Strategy] = {
    +          experimentalMethods.extraStrategies ++ Seq(
    +            FileSourceStrategy,
    +            DataSourceStrategy,
    +            SpecialLimits,
    +            InMemoryScans,
    +            HiveTableScans,
    +            Scripts,
    +            Aggregation,
    +            JoinSelection,
    +            BasicOperators
    +          )
    +        }
    +      }
    +  }
    +
    --- End diff --
    
    According to style guide, these are neither encouraged nor discouraged.
    To be consistent, will delete all extra lines after last member of class/object.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [WIP][Spark-19540][SQL] Add ability to clone Spar...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100463301
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -213,6 +218,24 @@ class SparkSession private(
         new SparkSession(sparkContext, Some(sharedState))
       }
     
    +  /**
    +   * Start a new session, sharing the underlying `SparkContext` and cached data.
    +   * If inheritSessionState is enabled, then SQL configurations, temporary tables,
    +   * registered functions are copied over from parent `SparkSession`.
    +   *
    +   * @note Other than the `SparkContext`, all shared state is initialized lazily.
    +   * This method will force the initialization of the shared state to ensure that parent
    +   * and child sessions are set up with the same shared state. If the underlying catalog
    +   * implementation is Hive, this will initialize the metastore, which may take some time.
    +   */
    --- End diff --
    
    That seems cleaner, fixed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103295692
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1181,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep
    +   * copy. `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def clone(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    synchronized {
    +      catalog.currentDb = currentDb
    +      // copy over temporary tables
    +      tempTables.foreach(kv => catalog.tempTables.put(kv._1, kv._2))
    +    }
    +
    +    catalog
    +  }
    +
    --- End diff --
    
    Removed


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73357 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73357/testReport)** for PR 16826 at commit [`8a8d47b`](https://github.com/apache/spark/commit/8a8d47b517a308b3fd79c226865cc7e78042575c).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103062936
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +82,118 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(newSparkSession: SparkSession): SessionState = {
    +    val sparkContext = newSparkSession.sparkContext
    +    val confCopy = conf.clone()
    +    val functionRegistryCopy = functionRegistry.clone()
    +    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    +    val catalogCopy = catalog.clone(
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      functionRegistryCopy,
    +      sqlParser)
    +    val queryExecutionCreator = (plan: LogicalPlan) => new QueryExecution(newSparkSession, plan)
    +
    +    SessionState.mergeSparkConf(confCopy, sparkContext.getConf)
    +
    +    new SessionState(
    +      sparkContext,
    +      newSparkSession.sharedState,
    +      confCopy,
    +      experimentalMethods.clone(),
    +      functionRegistryCopy,
    +      catalogCopy,
    +      sqlParser,
    +      SessionState.createAnalyzer(newSparkSession, catalogCopy, confCopy),
    +      new StreamingQueryManager(newSparkSession),
    +      queryExecutionCreator)
    +  }
    +
    +  // ------------------------------------------------------
    +  //  Helper methods, partially leftover from pre-2.0 days
    +  // ------------------------------------------------------
    +
    +  def executePlan(plan: LogicalPlan): QueryExecution = queryExecutionCreator(plan)
    +
    +  def refreshTable(tableName: String): Unit = {
    +    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    +  }
    +
    +  def addJar(path: String): Unit = sharedState.addJar(path)
    +}
    +
    +
    +object SessionState {
    +
    +  def apply(sparkSession: SparkSession): SessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): SessionState = {
    +
    --- End diff --
    
    nit: extra line.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618757
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution) {
    +
    +  /*
    +   * Interface exposed to the user for registering user-defined functions.
    +   * Note that the user-defined functions must be deterministic.
    +   */
    +  val udf: UDFRegistration = new UDFRegistration(functionRegistry)
     
    -  // Note: These are all lazy vals because they depend on each other (e.g. conf) and we
    -  // want subclasses to override some of the fields. Otherwise, we would get a lot of NPEs.
    +  // Logical query plan optimizer.
    +  val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods)
    +
    +  /*
    --- End diff --
    
    Use /** ... */


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103060863
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1178,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def copy(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    synchronized {
    --- End diff --
    
    Yeah, it should be `synchronized`. I was wrong.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73319 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73319/testReport)** for PR 16826 at commit [`dd2dedd`](https://github.com/apache/spark/commit/dd2dedd6d578a3b5a75359be72677e61eea751e3).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102541237
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -146,4 +108,144 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
     
    +  override def copy(sparkSession: SparkSession): HiveSessionState = {
    +    val sparkContext = sparkSession.sparkContext
    +    val confCopy = conf.clone
    +    val copyHelper = SessionState(sparkSession, Some(confCopy))
    +    val catalogCopy = catalog.copy(
    +      sparkSession,
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      copyHelper.functionRegistry,
    +      copyHelper.sqlParser)
    +    val hiveClient =
    --- End diff --
    
    I'm not sure if we should create a new HiveClient or not.
    
    @yhuai do you know what states in the HiveClient related to a SparkSession? In this method, we want to create a new HiveSessionState to inherit all states in the current HiveSessionState.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72968/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103060717
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -217,7 +217,8 @@ class HiveSparkSubmitSuite
         runSparkSubmit(args)
       }
     
    -  test("set hive.metastore.warehouse.dir") {
    +  // TODO: SPARK-19540 re-enable this test
    +  ignore("set hive.metastore.warehouse.dir") {
    --- End diff --
    
    These tests are now fixed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73027 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73027/testReport)** for PR 16826 at commit [`e2bbfa8`](https://github.com/apache/spark/commit/e2bbfa8c81f91c57f5628e771f42d414a1031d57).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103062140
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -90,110 +203,37 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         }
       }
     
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  lazy val catalog = new SessionCatalog(
    -    sparkSession.sharedState.externalCatalog,
    -    sparkSession.sharedState.globalTempViewManager,
    -    functionResourceLoader,
    -    functionRegistry,
    -    conf,
    -    newHadoopConf(),
    -    sqlParser)
    +  def newHadoopConf(copyHadoopConf: Configuration, sqlConf: SQLConf): Configuration = {
    --- End diff --
    
    nit: `hadoopConfToCopy` or just `hadoopConf` (consistent with `sqlConf`)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102633274
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -90,110 +208,29 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         }
       }
     
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  lazy val catalog = new SessionCatalog(
    -    sparkSession.sharedState.externalCatalog,
    -    sparkSession.sharedState.globalTempViewManager,
    -    functionResourceLoader,
    -    functionRegistry,
    -    conf,
    -    newHadoopConf(),
    -    sqlParser)
    +  def newHadoopConf(copyHadoopConf: Configuration, sqlConf: SQLConf): Configuration = {
    +    val hadoopConf = new Configuration(copyHadoopConf)
    +    sqlConf.getAllConfs.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) }
    +    hadoopConf
    +  }
     
    -  /**
    -   * Interface exposed to the user for registering user-defined functions.
    -   * Note that the user-defined functions must be deterministic.
    -   */
    -  lazy val udf: UDFRegistration = new UDFRegistration(functionRegistry)
    +  def createAnalyzer(
    +      sparkSession: SparkSession,
    +      catalog: SessionCatalog,
    +      sqlConf: SQLConf): Analyzer = {
     
    -  /**
    -   * Logical query plan analyzer for resolving unresolved attributes and relations.
    -   */
    -  lazy val analyzer: Analyzer = {
    -    new Analyzer(catalog, conf) {
    +    new Analyzer(catalog, sqlConf) {
           override val extendedResolutionRules =
             new FindDataSourceTable(sparkSession) ::
             new ResolveSQLOnFile(sparkSession) :: Nil
     
           override val postHocResolutionRules =
             PreprocessTableCreation(sparkSession) ::
    -        PreprocessTableInsertion(conf) ::
    -        DataSourceAnalysis(conf) :: Nil
    +        PreprocessTableInsertion(sqlConf) ::
    +        DataSourceAnalysis(sqlConf) :: Nil
     
           override val extendedCheckRules = Seq(PreWriteCheck, HiveOnlyCheck)
         }
       }
     
    -  /**
    -   * Logical query plan optimizer.
    -   */
    -  lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods)
    -
    -  /**
    -   * Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    -   */
    -  lazy val sqlParser: ParserInterface = new SparkSqlParser(conf)
    -
    -  /**
    -   * Planner that converts optimized logical plans to physical plans.
    -   */
    -  def planner: SparkPlanner =
    -    new SparkPlanner(sparkSession.sparkContext, conf, experimentalMethods.extraStrategies)
    -
    -  /**
    -   * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s
    -   * that listen for execution metrics.
    -   */
    -  lazy val listenerManager: ExecutionListenerManager = new ExecutionListenerManager
    -
    -  /**
    -   * Interface to start and stop [[StreamingQuery]]s.
    -   */
    -  lazy val streamingQueryManager: StreamingQueryManager = {
    -    new StreamingQueryManager(sparkSession)
    -  }
    -
    -  private val jarClassLoader: NonClosableMutableURLClassLoader =
    -    sparkSession.sharedState.jarClassLoader
    -
    -  // Automatically extract all entries and put it in our SQLConf
    -  // We need to call it after all of vals have been initialized.
    -  sparkSession.sparkContext.getConf.getAll.foreach { case (k, v) =>
    -    conf.setConfString(k, v)
    -  }
    -
    -  // ------------------------------------------------------
    -  //  Helper methods, partially leftover from pre-2.0 days
    -  // ------------------------------------------------------
    -
    -  def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan)
    -
    -  def refreshTable(tableName: String): Unit = {
    -    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    -  }
    -
    -  def addJar(path: String): Unit = {
    -    sparkSession.sparkContext.addJar(path)
    -
    -    val uri = new Path(path).toUri
    -    val jarURL = if (uri.getScheme == null) {
    -      // `path` is a local file path without a URL scheme
    -      new File(path).toURI.toURL
    -    } else {
    -      // `path` is a URL with a scheme
    -      uri.toURL
    -    }
    -    jarClassLoader.addURL(jarURL)
    -    Thread.currentThread().setContextClassLoader(jarClassLoader)
    -  }
    -
    -  /**
    -   * Analyzes the given table in the current database to generate statistics, which will be
    -   * used in query optimizations.
    -   */
    -  def analyze(tableIdent: TableIdentifier, noscan: Boolean = true): Unit = {
    --- End diff --
    
    It seems it was never used in our code, should it still be kept around?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102862069
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -146,4 +108,144 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
     
    +  override def copy(sparkSession: SparkSession): HiveSessionState = {
    +    val sparkContext = sparkSession.sparkContext
    +    val confCopy = conf.clone
    +    val copyHelper = SessionState(sparkSession, Some(confCopy))
    +    val catalogCopy = catalog.copy(
    +      sparkSession,
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      copyHelper.functionRegistry,
    +      copyHelper.sqlParser)
    +    val hiveClient =
    --- End diff --
    
    I checked with @yhuai in person, new `HiveClient` should be fine.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73379 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73379/testReport)** for PR 16826 at commit [`ffc2058`](https://github.com/apache/spark/commit/ffc205826de95c9a467bc68509f9915137440799).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103305709
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -90,110 +208,29 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         }
       }
     
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  lazy val catalog = new SessionCatalog(
    -    sparkSession.sharedState.externalCatalog,
    -    sparkSession.sharedState.globalTempViewManager,
    -    functionResourceLoader,
    -    functionRegistry,
    -    conf,
    -    newHadoopConf(),
    -    sqlParser)
    +  def newHadoopConf(copyHadoopConf: Configuration, sqlConf: SQLConf): Configuration = {
    +    val hadoopConf = new Configuration(copyHadoopConf)
    +    sqlConf.getAllConfs.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) }
    +    hadoopConf
    +  }
     
    -  /**
    -   * Interface exposed to the user for registering user-defined functions.
    -   * Note that the user-defined functions must be deterministic.
    -   */
    -  lazy val udf: UDFRegistration = new UDFRegistration(functionRegistry)
    +  def createAnalyzer(
    +      sparkSession: SparkSession,
    +      catalog: SessionCatalog,
    +      sqlConf: SQLConf): Analyzer = {
     
    -  /**
    -   * Logical query plan analyzer for resolving unresolved attributes and relations.
    -   */
    -  lazy val analyzer: Analyzer = {
    -    new Analyzer(catalog, conf) {
    +    new Analyzer(catalog, sqlConf) {
           override val extendedResolutionRules =
             new FindDataSourceTable(sparkSession) ::
             new ResolveSQLOnFile(sparkSession) :: Nil
     
           override val postHocResolutionRules =
             PreprocessTableCreation(sparkSession) ::
    -        PreprocessTableInsertion(conf) ::
    -        DataSourceAnalysis(conf) :: Nil
    +        PreprocessTableInsertion(sqlConf) ::
    +        DataSourceAnalysis(sqlConf) :: Nil
     
           override val extendedCheckRules = Seq(PreWriteCheck, HiveOnlyCheck)
         }
       }
     
    -  /**
    -   * Logical query plan optimizer.
    -   */
    -  lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods)
    -
    -  /**
    -   * Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    -   */
    -  lazy val sqlParser: ParserInterface = new SparkSqlParser(conf)
    -
    -  /**
    -   * Planner that converts optimized logical plans to physical plans.
    -   */
    -  def planner: SparkPlanner =
    -    new SparkPlanner(sparkSession.sparkContext, conf, experimentalMethods.extraStrategies)
    -
    -  /**
    -   * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s
    -   * that listen for execution metrics.
    -   */
    -  lazy val listenerManager: ExecutionListenerManager = new ExecutionListenerManager
    -
    -  /**
    -   * Interface to start and stop [[StreamingQuery]]s.
    -   */
    -  lazy val streamingQueryManager: StreamingQueryManager = {
    -    new StreamingQueryManager(sparkSession)
    -  }
    -
    -  private val jarClassLoader: NonClosableMutableURLClassLoader =
    -    sparkSession.sharedState.jarClassLoader
    -
    -  // Automatically extract all entries and put it in our SQLConf
    -  // We need to call it after all of vals have been initialized.
    -  sparkSession.sparkContext.getConf.getAll.foreach { case (k, v) =>
    -    conf.setConfString(k, v)
    -  }
    -
    -  // ------------------------------------------------------
    -  //  Helper methods, partially leftover from pre-2.0 days
    -  // ------------------------------------------------------
    -
    -  def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan)
    -
    -  def refreshTable(tableName: String): Unit = {
    -    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    -  }
    -
    -  def addJar(path: String): Unit = {
    -    sparkSession.sparkContext.addJar(path)
    -
    -    val uri = new Path(path).toUri
    -    val jarURL = if (uri.getScheme == null) {
    -      // `path` is a local file path without a URL scheme
    -      new File(path).toURI.toURL
    -    } else {
    -      // `path` is a URL with a scheme
    -      uri.toURL
    -    }
    -    jarClassLoader.addURL(jarURL)
    -    Thread.currentThread().setContextClassLoader(jarClassLoader)
    -  }
    -
    -  /**
    -   * Analyzes the given table in the current database to generate statistics, which will be
    -   * used in query optimizations.
    -   */
    -  def analyze(tableIdent: TableIdentifier, noscan: Boolean = true): Unit = {
    --- End diff --
    
    Cool.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102130841
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -217,6 +217,7 @@ class HiveSparkSubmitSuite
         runSparkSubmit(args)
       }
     
    +  /*  TODO: SPARK-19540 re-enable this test
       test("set hive.metastore.warehouse.dir") {
    --- End diff --
    
    Is it better to use `ignore`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73197 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73197/testReport)** for PR 16826 at commit [`b027412`](https://github.com/apache/spark/commit/b02741293939bf34e0a976b775b4e21511d71ba9).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103063053
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +82,118 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(newSparkSession: SparkSession): SessionState = {
    +    val sparkContext = newSparkSession.sparkContext
    +    val confCopy = conf.clone()
    +    val functionRegistryCopy = functionRegistry.clone()
    +    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    +    val catalogCopy = catalog.clone(
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      functionRegistryCopy,
    +      sqlParser)
    +    val queryExecutionCreator = (plan: LogicalPlan) => new QueryExecution(newSparkSession, plan)
    +
    +    SessionState.mergeSparkConf(confCopy, sparkContext.getConf)
    +
    +    new SessionState(
    +      sparkContext,
    +      newSparkSession.sharedState,
    +      confCopy,
    +      experimentalMethods.clone(),
    +      functionRegistryCopy,
    +      catalogCopy,
    +      sqlParser,
    +      SessionState.createAnalyzer(newSparkSession, catalogCopy, confCopy),
    +      new StreamingQueryManager(newSparkSession),
    +      queryExecutionCreator)
    +  }
    +
    +  // ------------------------------------------------------
    +  //  Helper methods, partially leftover from pre-2.0 days
    +  // ------------------------------------------------------
    +
    +  def executePlan(plan: LogicalPlan): QueryExecution = queryExecutionCreator(plan)
    +
    +  def refreshTable(tableName: String): Unit = {
    +    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    +  }
    +
    +  def addJar(path: String): Unit = sharedState.addJar(path)
    +}
    +
    +
    +object SessionState {
    +
    +  def apply(sparkSession: SparkSession): SessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): SessionState = {
    --- End diff --
    
    I think you can simply do 
    `def apply(session: SparkSession, conf = new SqlConf) {...}`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102142543
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -117,9 +122,11 @@ class SparkSession private(
       @InterfaceStability.Unstable
       @transient
       lazy val sessionState: SessionState = {
    -    SparkSession.reflect[SessionState, SparkSession](
    -      SparkSession.sessionStateClassName(sparkContext.conf),
    -      self)
    +    parentSessionState
    +      .map(_.copy(this))
    --- End diff --
    
    As this is a lazy val, the cloned `SparkSession` will not copy parent `SessionState` immediately. So if the parent `SessionState` is changed, e.g., new functions registered, before this lazy variable initialization, the cloned one will also have new functions.
    
     It will not match the description `...Changes to base session are not propagated to cloned session, cloned is independent after creation...` below.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102635568
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -446,6 +449,7 @@ object SetWarehouseLocationTest extends Logging {
         val providedExpectedWarehouseLocation =
           sparkConf.getOption("spark.sql.test.expectedWarehouseDir")
     
    +    println(s"kunal providedExpectedWarehouseLocation: $providedExpectedWarehouseLocation")
    --- End diff --
    
    ??


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104749905
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -17,89 +17,55 @@
     
     package org.apache.spark.sql.hive
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.analysis.Analyzer
    -import org.apache.spark.sql.execution.SparkPlanner
    +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanner, SparkSqlParser}
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.hive.client.HiveClient
    -import org.apache.spark.sql.internal.SessionState
    +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
    + * @param catalog A Hive client used for interacting with the metastore.
    + * @param analyzer An analyzer that uses the Hive metastore.
    + * @param plannerCreator Lambda to create a [[SparkPlanner]] that converts optimized logical
    + *                       plans to physical plans.
      */
    -private[hive] class HiveSessionState(sparkSession: SparkSession)
    -  extends SessionState(sparkSession) {
    -
    -  self =>
    -
    -  /**
    -   * A Hive client used for interacting with the metastore.
    -   */
    -  lazy val metadataHive: HiveClient =
    -    sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.newSession()
    -
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  override lazy val catalog = {
    -    new HiveSessionCatalog(
    -      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    -      sparkSession.sharedState.globalTempViewManager,
    -      sparkSession,
    -      functionResourceLoader,
    -      functionRegistry,
    +private[hive] class HiveSessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    conf: SQLConf,
    +    experimentalMethods: ExperimentalMethods,
    +    functionRegistry: FunctionRegistry,
    +    override val catalog: HiveSessionCatalog,
    +    sqlParser: ParserInterface,
    +    val metadataHive: HiveClient,
    --- End diff --
    
    Avoid using SparkSession in this class.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73026/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73022 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73022/testReport)** for PR 16826 at commit [`f423f74`](https://github.com/apache/spark/commit/f423f7481348c021d9a27986064dfbe389c5de77).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73022/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102779157
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -77,10 +71,44 @@ private[sql] class HiveSessionCatalog(
         metastoreCatalog.hiveDefaultTableFilePath(name)
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with `sparkSession`.
    +   * This should ideally override `SessionCatalog.copy` but does not at present, since
    +   * `HiveMetastoreCatalog` is dependent on `SparkSession`.
    +   */
    +  def clone(
    --- End diff --
    
    This should ideally override `SessionCatalog.clone` but does not at present, since `HiveMetastoreCatalog` is dependent on `SparkSession`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103060937
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala ---
    @@ -34,9 +40,32 @@ private[sql] class TestSparkSession(sc: SparkContext) extends SparkSession(sc) {
         this(new SparkConf)
       }
     
    -  @transient
    -  override lazy val sessionState: SessionState = new SessionState(self) {
    -    override lazy val conf: SQLConf = {
    +  class TestSessionState(
    +      sparkContext: SparkContext,
    +      sharedState: SharedState,
    +      conf: SQLConf,
    +      experimentalMethods: ExperimentalMethods,
    +      functionRegistry: FunctionRegistry,
    +      catalog: SessionCatalog,
    +      sqlParser: ParserInterface,
    +      analyzer: Analyzer,
    +      streamingQueryManager: StreamingQueryManager,
    +      queryExecution: LogicalPlan => QueryExecution)
    +    extends SessionState(
    +        sparkContext,
    +        sharedState,
    +        conf,
    +        experimentalMethods,
    +        functionRegistry,
    +        catalog,
    +        sqlParser,
    +        analyzer,
    +        streamingQueryManager,
    +        queryExecution) {}
    +
    +  object TestSessionState {
    +
    +    def createTestConf: SQLConf = {
    --- End diff --
    
    Changed


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73027/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103336696
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +82,118 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(newSparkSession: SparkSession): SessionState = {
    +    val sparkContext = newSparkSession.sparkContext
    +    val confCopy = conf.clone()
    +    val functionRegistryCopy = functionRegistry.clone()
    +    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    +    val catalogCopy = catalog.clone(
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      functionRegistryCopy,
    +      sqlParser)
    +    val queryExecutionCreator = (plan: LogicalPlan) => new QueryExecution(newSparkSession, plan)
    +
    +    SessionState.mergeSparkConf(confCopy, sparkContext.getConf)
    +
    +    new SessionState(
    +      sparkContext,
    +      newSparkSession.sharedState,
    +      confCopy,
    +      experimentalMethods.clone(),
    +      functionRegistryCopy,
    +      catalogCopy,
    +      sqlParser,
    +      SessionState.createAnalyzer(newSparkSession, catalogCopy, confCopy),
    +      new StreamingQueryManager(newSparkSession),
    +      queryExecutionCreator)
    +  }
    +
    +  // ------------------------------------------------------
    +  //  Helper methods, partially leftover from pre-2.0 days
    +  // ------------------------------------------------------
    +
    +  def executePlan(plan: LogicalPlan): QueryExecution = queryExecutionCreator(plan)
    +
    +  def refreshTable(tableName: String): Unit = {
    +    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    +  }
    +
    +  def addJar(path: String): Unit = sharedState.addJar(path)
    +}
    +
    +
    +object SessionState {
    +
    +  def apply(sparkSession: SparkSession): SessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): SessionState = {
    +
    +    val sparkContext = sparkSession.sparkContext
    +
    +    // SQL-specific key-value configurations.
    --- End diff --
    
    changed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r101879811
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala ---
    @@ -123,4 +126,95 @@ class SparkSessionBuilderSuite extends SparkFunSuite {
           session.stop()
         }
       }
    +
    +  test("fork new session and inherit a copy of the session state") {
    +    val activeSession = SparkSession.builder().master("local").getOrCreate()
    +    val forkedSession = activeSession.cloneSession()
    +
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.sessionState ne activeSession.sessionState)
    +
    +    forkedSession.stop()
    +    activeSession.stop()
    +  }
    +
    +  test("fork new session and inherit sql config options") {
    --- End diff --
    
    For each other these state, you have to test the following. 
    
    - whether the state gets copied over and is usable
    - whether the original session is unaffected by changes to clones session, AND vice versa.
    
    I think the last one is missing in these tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102621016
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala ---
    @@ -34,9 +40,32 @@ private[sql] class TestSparkSession(sc: SparkContext) extends SparkSession(sc) {
         this(new SparkConf)
       }
     
    -  @transient
    -  override lazy val sessionState: SessionState = new SessionState(self) {
    -    override lazy val conf: SQLConf = {
    +  class TestSessionState(
    --- End diff --
    
    i think you can probably put all of these inside a lazy val as before.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618844
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +85,120 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(sparkSession: SparkSession): SessionState = {
    --- End diff --
    
    sparkSession -> newSparkSession


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73202/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103065116
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala ---
    @@ -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.spark.sql.hive
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +
    +class HiveSessionStateSuite extends SessionStateSuite
    +  with TestHiveSingleton with BeforeAndAfterEach {
    +
    +  override def beforeEach(): Unit = {
    +    createSession()
    +  }
    +
    +  override def afterEach(): Unit = {}
    +
    +  override def createSession(): Unit = {
    +    activeSession = spark.newSession()
    --- End diff --
    
    is this a hive session? if so ... please comment .. its confusing.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103073487
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala ---
    @@ -46,4 +46,10 @@ class ExperimentalMethods private[sql]() {
     
       @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil
     
    +  override def clone(): ExperimentalMethods = {
    --- End diff --
    
    It sounds like we also need to add sync for both `extraStrategies ` and `extraOptimizations `


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103347559
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala ---
    @@ -493,6 +493,28 @@ class CatalogSuite
         }
       }
     
    +  test("clone Catalog") {
    +    // need to test tempTables are cloned
    +    assert(spark.catalog.listTables().collect().isEmpty)
    +
    +    createTempTable("my_temp_table")
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    // inheritance
    +    val forkedSession = spark.cloneSession()
    +    assert(spark ne forkedSession)
    +    assert(spark.catalog ne forkedSession.catalog)
    +    assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    // independence
    +    dropTable("my_temp_table") // drop table in original session
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set())
    +    assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +    forkedSession.sessionState.catalog
    +      .createTempView("fork_table", Range(1, 2, 3, 4), overrideIfExists = true)
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set())
    +  }
    +
       // TODO: add tests for the rest of them
    --- End diff --
    
    After that TODO was added, there have been many additions of tests.
    This is a large interface, so the tests are scattered over this suite and `GlobalTempViewSuite`, `CachedTableSuite`, `PartitionProviderCompatibilitySuite`, `(Hive)MetadataCacheSuite`, `DDLSuite`, `ParquetQuerySuite` to name a few.
    Though I do see at least one test for every method in the trait `Catalog`.
    Checked with @cloud-fan in person, we should be fine with removing this TODO.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102292237
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1178,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def copy(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    synchronized {
    --- End diff --
    
    `synchronized` -> `catalog .synchronized`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103528893
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala ---
    @@ -46,4 +46,10 @@ class ExperimentalMethods private[sql]() {
     
       @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil
     
    +  override def clone(): ExperimentalMethods = {
    --- End diff --
    
    @gatorsmile any reason you want to add `synchronized` here? `extraStrategies` and `extraOptimizations` are both `@volatile`. If you want to update them atomically, then we also need to add `synchronized` to other places that change `extraStrategies` and `extraOptimizations`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73202 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73202/testReport)** for PR 16826 at commit [`847b484`](https://github.com/apache/spark/commit/847b484ca1ef416ae16952c7de156c6cade23cf1).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74148 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74148/testReport)** for PR 16826 at commit [`05abcf8`](https://github.com/apache/spark/commit/05abcf801f57c861435600fcc5fb2cecf6f3e11f).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72727/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73083 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73083/testReport)** for PR 16826 at commit [`8ac778a`](https://github.com/apache/spark/commit/8ac778ab444f90eadd22d36b91889d81ef593d44).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74124 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74124/testReport)** for PR 16826 at commit [`5eb6733`](https://github.com/apache/spark/commit/5eb6733d7e49ff64bcd4d6fbd21acca1d4c3187f).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73319/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102779367
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -446,6 +449,7 @@ object SetWarehouseLocationTest extends Logging {
         val providedExpectedWarehouseLocation =
           sparkConf.getOption("spark.sql.test.expectedWarehouseDir")
     
    +    println(s"kunal providedExpectedWarehouseLocation: $providedExpectedWarehouseLocation")
    --- End diff --
    
    That file was not supposed to be committed, my mistake.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73357 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73357/testReport)** for PR 16826 at commit [`8a8d47b`](https://github.com/apache/spark/commit/8a8d47b517a308b3fd79c226865cc7e78042575c).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618715
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution) {
    +
    +  /*
    +   * Interface exposed to the user for registering user-defined functions.
    +   * Note that the user-defined functions must be deterministic.
    +   */
    +  val udf: UDFRegistration = new UDFRegistration(functionRegistry)
    --- End diff --
    
    Use /** ... */


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72968 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72968/testReport)** for PR 16826 at commit [`2837e73`](https://github.com/apache/spark/commit/2837e73e00374fe20c2713138815cb539468996f).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064184
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
    @@ -136,6 +139,26 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
         }
         SparkSession.sqlListener.get()
       }
    +
    +  /*
    +   * This belongs here more than in `SessionState`. However, does not seem that it can be
    +   * removed from `SessionState` and `HiveSessionState` without using reflection in
    +   * `AddJarCommand`.
    +   */
    +  def addJar(path: String): Unit = {
    +    sparkContext.addJar(path)
    +
    --- End diff --
    
    nit: extra line.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: Fork SparkSession with option to inherit a copy of the S...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72606 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72606/testReport)** for PR 16826 at commit [`a343d8a`](https://github.com/apache/spark/commit/a343d8af9c577158042e4af9f8832f46aeecd509).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r101879805
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala ---
    @@ -123,4 +126,95 @@ class SparkSessionBuilderSuite extends SparkFunSuite {
           session.stop()
         }
       }
    +
    +  test("fork new session and inherit a copy of the session state") {
    --- End diff --
    
    this test doesnt test anything other than `ne` which is already covered by other tests


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102619583
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala ---
    @@ -493,6 +493,21 @@ class CatalogSuite
         }
       }
     
    +  test("clone SessionCatalog") {
    +    // need to test tempTables are cloned
    +    assert(spark.catalog.listTables().collect().isEmpty)
    +
    +    createTempTable("my_temp_table")
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    val forkedSession = spark.cloneSession()
    --- End diff --
    
    why is this testing cloning of catalog using cloning of session. 
    clone just the catalog and test it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r107283640
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -17,43 +17,70 @@
     
     package org.apache.spark.sql.internal
     
    -import java.io.File
    -
     import org.apache.hadoop.conf.Configuration
    -import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.{SparkConf, SparkContext}
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
    + * @param functionRegistry Internal catalog for managing functions registered by the user.
    + * @param catalog Internal catalog for managing table and database states.
    + * @param sqlParser Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    + * @param analyzer Logical query plan analyzer for resolving unresolved attributes and relations.
    + * @param streamingQueryManager Interface to start and stop
    + *                              [[org.apache.spark.sql.streaming.StreamingQuery]]s.
    + * @param queryExecutionCreator Lambda to create a [[QueryExecution]] from a [[LogicalPlan]]
    --- End diff --
    
    @rxin Removing the redundant comments in [SPARK-20048](https://github.com/apache/spark/pull/17379).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103066371
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -17,43 +17,60 @@
     
     package org.apache.spark.sql.internal
     
    -import java.io.File
    -
     import org.apache.hadoop.conf.Configuration
    -import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.{SparkConf, SparkContext}
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution) {
    --- End diff --
    
    +1


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][Spark-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72679 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72679/testReport)** for PR 16826 at commit [`4210079`](https://github.com/apache/spark/commit/4210079c0dfe4eb3d606606ca06ac28ccdbf2ae1).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102632895
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -212,6 +219,29 @@ class SparkSession private(
         new SparkSession(sparkContext, Some(sharedState))
       }
     
    +  /**
    +   * :: Experimental ::
    +   * Create an identical copy of this `SparkSession`, sharing the underlying `SparkContext`
    +   * and cached data. All the state of this session (i.e. SQL configurations, temporary tables,
    +   * registered functions) is also copied over.
    --- End diff --
    
    Right, added


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73024 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73024/testReport)** for PR 16826 at commit [`b1371d8`](https://github.com/apache/spark/commit/b1371d8a3a44107d7d35ee40ff6e6f9878e1cd99).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102635875
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -77,10 +71,44 @@ private[sql] class HiveSessionCatalog(
         metastoreCatalog.hiveDefaultTableFilePath(name)
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with `sparkSession`.
    +   * This should ideally override `SessionCatalog.copy` but does not at present, since
    +   * `HiveMetastoreCatalog` is dependent on `SparkSession`.
    +   */
    +  def clone(
    --- End diff --
    
    override?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74056 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74056/testReport)** for PR 16826 at commit [`2f0b1ad`](https://github.com/apache/spark/commit/2f0b1adf98807f0ca9f48ab46b1f5f7521c365c2).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102619029
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
    @@ -136,6 +139,21 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
         }
         SparkSession.sqlListener.get()
       }
    +
    +  def addJar(path: String): Unit = {
    --- End diff --
    
    add docs. 
    why was this moved here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104596281
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -17,43 +17,70 @@
     
     package org.apache.spark.sql.internal
     
    -import java.io.File
    -
     import org.apache.hadoop.conf.Configuration
    -import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.{SparkConf, SparkContext}
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
    + * @param functionRegistry Internal catalog for managing functions registered by the user.
    + * @param catalog Internal catalog for managing table and database states.
    + * @param sqlParser Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    + * @param analyzer Logical query plan analyzer for resolving unresolved attributes and relations.
    + * @param streamingQueryManager Interface to start and stop
    + *                              [[org.apache.spark.sql.streaming.StreamingQuery]]s.
    + * @param queryExecutionCreator Lambda to create a [[QueryExecution]] from a [[LogicalPlan]]
    --- End diff --
    
    Let us document all parms?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102129943
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1178,34 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def copy(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    catalog.currentDb = currentDb
    --- End diff --
    
    synchronized?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104753243
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -17,43 +17,70 @@
     
     package org.apache.spark.sql.internal
     
    -import java.io.File
    -
     import org.apache.hadoop.conf.Configuration
    -import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.{SparkConf, SparkContext}
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
    + * @param functionRegistry Internal catalog for managing functions registered by the user.
    + * @param catalog Internal catalog for managing table and database states.
    + * @param sqlParser Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    + * @param analyzer Logical query plan analyzer for resolving unresolved attributes and relations.
    + * @param streamingQueryManager Interface to start and stop
    + *                              [[org.apache.spark.sql.streaming.StreamingQuery]]s.
    + * @param queryExecutionCreator Lambda to create a [[QueryExecution]] from a [[LogicalPlan]]
    --- End diff --
    
    done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103059989
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1178,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def copy(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    synchronized {
    --- End diff --
    
    doesnt look like. its still `synchronized`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104781325
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala ---
    @@ -113,19 +132,26 @@ class SessionStateSuite extends SparkFunSuite
             Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil)
         }
     
    -    implicit val enc = Encoders.tuple(Encoders.scalaInt, Encoders.STRING)
    +    val spark = activeSession
    +    // Cannot use `import activeSession.implicits._` due to the compiler limitation.
    +    import spark.implicits._
    +
         activeSession
    --- End diff --
    
    Should create temp view be inside try block?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74024/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102788016
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    --- End diff --
    
    I added both of them, so should be ok as private.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74215 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74215/testReport)** for PR 16826 at commit [`4c23e7a`](https://github.com/apache/spark/commit/4c23e7a0caaff5aa2df3dbbf5e6a688b3a4d442d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  public static class LongWrapper `
      * `  public static class IntWrapper `
      * `case class CostBasedJoinReorder(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper `
      * `  case class JoinPlan(itemIds: Set[Int], plan: LogicalPlan, joinConds: Set[Expression], cost: Cost)`
      * `case class Cost(rows: BigInt, size: BigInt) `
      * `abstract class RepartitionOperation extends UnaryNode `
      * `trait WatermarkSupport extends UnaryExecNode `


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103308317
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala ---
    @@ -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.spark.sql.hive
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +
    +class HiveSessionStateSuite extends SessionStateSuite
    +  with TestHiveSingleton with BeforeAndAfterEach {
    +
    +  override def beforeEach(): Unit = {
    +    createSession()
    +  }
    +
    +  override def afterEach(): Unit = {}
    +
    +  override def createSession(): Unit = {
    +    activeSession = spark.newSession()
    --- End diff --
    
    Added comment.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74039 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74039/testReport)** for PR 16826 at commit [`2740c63`](https://github.com/apache/spark/commit/2740c63634b41de54a8913ea71e3c8927a9969ca).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104593862
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1197,6 +1199,65 @@ class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("clone SessionCatalog - temp views") {
    +    val externalCatalog = newEmptyCatalog()
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    original.createTempView("copytest1", tempTable1, overrideIfExists = false)
    +
    +    // check if tables copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(original ne clone)
    +    assert(clone.getTempView("copytest1") == Option(tempTable1))
    +
    +    // check if clone and original independent
    +    clone.dropTable(TableIdentifier("copytest1"), ignoreIfNotExists = false, purge = false)
    +    assert(original.getTempView("copytest1") == Option(tempTable1))
    +
    +    val tempTable2 = Range(1, 20, 2, 10)
    +    original.createTempView("copytest2", tempTable2, overrideIfExists = false)
    +    assert(clone.getTempView("copytest2").isEmpty)
    +  }
    +
    +  test("clone SessionCatalog - current db") {
    +    val externalCatalog = newEmptyCatalog()
    +    externalCatalog.createDatabase(newDb("copytest1"), true)
    +    externalCatalog.createDatabase(newDb("copytest2"), true)
    +    externalCatalog.createDatabase(newDb("copytest3"), true)
    +
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    val db1 = "copytest1"
    +    original.createTempView(db1, tempTable1, overrideIfExists = false)
    +    original.setCurrentDatabase(db1)
    +
    +    // check if current db copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(original ne clone)
    +    assert(clone.getCurrentDatabase == db1)
    +
    +    // check if clone and original independent
    +    val db2 = "copytest2"
    +    val tempTable2 = Range(1, 20, 2, 20)
    +    clone.createTempView(db2, tempTable2, overrideIfExists = false)
    --- End diff --
    
    the same here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r101878757
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1180,4 +1180,26 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   */
    +  def copy: SessionCatalog = {
    +    val catalog = new SessionCatalog(
    --- End diff --
    
    if you are copying... shouldnt all the fields be copied?
    you should test that any updates to the copied catalog should not get reflected in the earlier catalog.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][Spark-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72679/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73302 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73302/testReport)** for PR 16826 at commit [`4f70d12`](https://github.com/apache/spark/commit/4f70d12dba6f89a6548ffea40879663a42902811).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104600696
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -278,6 +278,8 @@ private[hive] class HiveClientImpl(
         state.getConf.setClassLoader(clientLoader.classLoader)
         // Set the thread local metastore client to the client associated with this HiveClientImpl.
         Hive.set(client)
    +    // Replace conf in the thread local Hive with current conf
    +    Hive.get(conf)
    --- End diff --
    
    Because `IsolatedClientLoader` is shared? If we do not make this change, any test case failed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73197 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73197/testReport)** for PR 16826 at commit [`b027412`](https://github.com/apache/spark/commit/b02741293939bf34e0a976b775b4e21511d71ba9).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74165 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74165/testReport)** for PR 16826 at commit [`05abcf8`](https://github.com/apache/spark/commit/05abcf801f57c861435600fcc5fb2cecf6f3e11f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100194351
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,12 +80,29 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    +  lazy val experimentalMethods: ExperimentalMethods = {
    +    existingSessionState
    +      .map(_.experimentalMethods.clone)
    +      .getOrElse(new ExperimentalMethods)
    +  }
     
       /**
        * Internal catalog for managing functions registered by the user.
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  lazy val functionRegistry: FunctionRegistry = {
    --- End diff --
    
    It's better to just add a `copy` method to `FunctionRegistry` to simplify these codes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102293636
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution,
    +    val jarClassLoader: NonClosableMutableURLClassLoader) {
    --- End diff --
    
    Either `jarClassLoader` or `SharedState`. Both look good to me.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064464
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala ---
    @@ -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.spark.sql.hive
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.sql._
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +
    +class HiveSessionStateSuite extends SessionStateSuite
    +  with TestHiveSingleton with BeforeAndAfterEach {
    +
    +  override def beforeEach(): Unit = {
    +    createSession()
    +  }
    +
    +  override def afterEach(): Unit = {}
    +
    +  override def createSession(): Unit = {
    +    activeSession = spark.newSession()
    +  }
    +
    --- End diff --
    
    nit: extra line.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102616690
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1196,4 +1198,27 @@ class SessionCatalogSuite extends PlanTest {
           catalog.listFunctions("unknown_db", "func*")
         }
       }
    +
    +  test("copy SessionCatalog") {
    +    val externalCatalog = newEmptyCatalog()
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    original.createTempView("copytest1", tempTable1, overrideIfExists = false)
    +
    +    // check if tables copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(clone.getTempView("copytest1") == Option(tempTable1))
    --- End diff --
    
    `assert(!original.eq(clone))`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104489629
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -274,6 +274,8 @@ private[hive] class HiveClientImpl(
         val original = Thread.currentThread().getContextClassLoader
         // Set the thread local metastore client to the client associated with this HiveClientImpl.
         Hive.set(client)
    +    // replace conf in cached hive with current conf
    --- End diff --
    
    @viirya `Hive.get(conf)` replaces conf in the **thread local** Hive with current conf, so it must be called after  `Hive.set(client)` which sets the current thread local Hive.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102595359
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution,
    +    val jarClassLoader: NonClosableMutableURLClassLoader) {
    --- End diff --
    
    Removed `jarClassLoader`, replaced with `SharedState`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064301
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -212,3 +247,31 @@ private[sql] class HiveSessionCatalog(
         "histogram_numeric"
       )
     }
    +
    +private[sql] object HiveSessionCatalog {
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      functionResourceLoader: FunctionResourceLoader,
    +      functionRegistry: FunctionRegistry,
    +      conf: SQLConf,
    +      hadoopConf: Configuration,
    +      parser: ParserInterface): HiveSessionCatalog = {
    +
    +    // Catalog for handling data source tables. TODO: This really doesn't belong here since it is
    +    // essentially a cache for metastore tables. However, it relies on a lot of session-specific
    +    // things so it would be a lot of work to split its functionality between HiveSessionCatalog
    +    // and HiveCatalog. We should still do it at some point...
    +    val metastoreCatalog = new HiveMetastoreCatalog(sparkSession)
    +
    +    new HiveSessionCatalog(
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    +      sparkSession.sharedState.globalTempViewManager,
    +      metastoreCatalog,
    +      functionResourceLoader: FunctionResourceLoader,
    --- End diff --
    
    i think i mentioned before, this is a weird way to specify params. this is used *only* when you want to coerce the param into a particular class type to fit the required param type. i dont think that is the case here. please remove this if not necessary.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104607180
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -278,6 +278,8 @@ private[hive] class HiveClientImpl(
         state.getConf.setClassLoader(clientLoader.classLoader)
         // Set the thread local metastore client to the client associated with this HiveClientImpl.
         Hive.set(client)
    +    // Replace conf in the thread local Hive with current conf
    +    Hive.get(conf)
    --- End diff --
    
    Thank you!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104607483
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -17,89 +17,55 @@
     
     package org.apache.spark.sql.hive
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.analysis.Analyzer
    -import org.apache.spark.sql.execution.SparkPlanner
    +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanner, SparkSqlParser}
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.hive.client.HiveClient
    -import org.apache.spark.sql.internal.SessionState
    +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
    + * @param catalog A Hive client used for interacting with the metastore.
    + * @param analyzer An analyzer that uses the Hive metastore.
    + * @param plannerCreator Lambda to create a [[SparkPlanner]] that converts optimized logical
    + *                       plans to physical plans.
      */
    -private[hive] class HiveSessionState(sparkSession: SparkSession)
    -  extends SessionState(sparkSession) {
    -
    -  self =>
    -
    -  /**
    -   * A Hive client used for interacting with the metastore.
    -   */
    -  lazy val metadataHive: HiveClient =
    -    sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.newSession()
    -
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  override lazy val catalog = {
    -    new HiveSessionCatalog(
    -      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    -      sparkSession.sharedState.globalTempViewManager,
    -      sparkSession,
    -      functionResourceLoader,
    -      functionRegistry,
    +private[hive] class HiveSessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    conf: SQLConf,
    +    experimentalMethods: ExperimentalMethods,
    +    functionRegistry: FunctionRegistry,
    +    override val catalog: HiveSessionCatalog,
    +    sqlParser: ParserInterface,
    +    val metadataHive: HiveClient,
    +    analyzer: Analyzer,
    +    streamingQueryManager: StreamingQueryManager,
    +    queryExecutionCreator: LogicalPlan => QueryExecution,
    +    val plannerCreator: () => SparkPlanner)
    --- End diff --
    
    Right now, SessionState.planner is a method. So it will return a new SparkPlanner using the latest `experimentalMethods.extraStrategies` every time. Changing it to a `val` is a breaking change.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72727 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72727/testReport)** for PR 16826 at commit [`6da6bda`](https://github.com/apache/spark/commit/6da6bdadc526d71ce887b0bcbef57272a713d6b2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103295639
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1181,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    --- End diff --
    
    Changed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102143747
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala ---
    @@ -217,7 +217,8 @@ class HiveSparkSubmitSuite
         runSparkSubmit(args)
       }
     
    -  test("set hive.metastore.warehouse.dir") {
    +  // TODO: SPARK-19540 re-enable this test
    +  ignore("set hive.metastore.warehouse.dir") {
    --- End diff --
    
    What is the reasons we need to ignore them? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103307299
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -212,3 +247,31 @@ private[sql] class HiveSessionCatalog(
         "histogram_numeric"
       )
     }
    +
    +private[sql] object HiveSessionCatalog {
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      functionResourceLoader: FunctionResourceLoader,
    +      functionRegistry: FunctionRegistry,
    +      conf: SQLConf,
    +      hadoopConf: Configuration,
    +      parser: ParserInterface): HiveSessionCatalog = {
    +
    +    // Catalog for handling data source tables. TODO: This really doesn't belong here since it is
    +    // essentially a cache for metastore tables. However, it relies on a lot of session-specific
    +    // things so it would be a lot of work to split its functionality between HiveSessionCatalog
    +    // and HiveCatalog. We should still do it at some point...
    +    val metastoreCatalog = new HiveMetastoreCatalog(sparkSession)
    +
    +    new HiveSessionCatalog(
    +      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    +      sparkSession.sharedState.globalTempViewManager,
    +      metastoreCatalog,
    +      functionResourceLoader: FunctionResourceLoader,
    --- End diff --
    
    Removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74056/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72897 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72897/testReport)** for PR 16826 at commit [`579d0b7`](https://github.com/apache/spark/commit/579d0b77738e2de53c06725e55f6a0de905325a5).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build started.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104786618
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -32,10 +32,19 @@ import org.apache.spark.sql.streaming.StreamingQueryManager
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
    - * @param catalog A Hive client used for interacting with the metastore.
    - * @param analyzer An analyzer that uses the Hive metastore.
    - * @param plannerCreator Lambda to create a [[SparkPlanner]] that converts optimized logical
    - *                       plans to physical plans.
    + * @param sparkContext The [[SparkContext]].
    + * @param sharedState The shared state.
    + * @param conf SQL-specific key-value configurations.
    + * @param experimentalMethods The experimental methods.
    + * @param functionRegistry Internal catalog for managing functions registered by the user.
    + * @param catalog Internal catalog for managing table and database states.
    --- End diff --
    
    Add comment on difference from `SessionCatalog`: `HiveSessionCatalog` uses Hive client for interacting with metastore.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102635390
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala ---
    @@ -493,6 +493,21 @@ class CatalogSuite
         }
       }
     
    +  test("clone SessionCatalog") {
    +    // need to test tempTables are cloned
    +    assert(spark.catalog.listTables().collect().isEmpty)
    +
    +    createTempTable("my_temp_table")
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    val forkedSession = spark.cloneSession()
    +    assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    dropTable("my_temp_table")
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set())
    +    assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    --- End diff --
    
    Yes!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102615602
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -74,7 +74,10 @@ class SessionCatalog(
     
       // For testing only.
       def this(externalCatalog: ExternalCatalog) {
    -    this(externalCatalog, new SimpleFunctionRegistry, new SimpleCatalystConf(true))
    +    this(
    +      externalCatalog,
    +      new SimpleFunctionRegistry,
    +      SimpleCatalystConf(caseSensitiveAnalysis = true))
    --- End diff --
    
    +1 for adding the param name


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103061069
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala ---
    @@ -77,10 +71,44 @@ private[sql] class HiveSessionCatalog(
         metastoreCatalog.hiveDefaultTableFilePath(name)
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with `sparkSession`.
    +   * This should ideally override `SessionCatalog.copy` but does not at present, since
    +   * `HiveMetastoreCatalog` is dependent on `SparkSession`.
    +   */
    +  def clone(
    --- End diff --
    
    Added an overridden version, `HiveSessionCatalog.clone` that throws an exception, to prevent future user from using incorrect api.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102632860
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala ---
    @@ -66,7 +66,7 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] {
      * Preprocess [[CreateTable]], to do some normalization and checking.
      */
     case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[LogicalPlan] {
    -  private val catalog = sparkSession.sessionState.catalog
    +  private def catalog = sparkSession.sessionState.catalog
    --- End diff --
    
    Good point


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103063173
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
    @@ -136,6 +139,26 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
         }
         SparkSession.sqlListener.get()
       }
    +
    +  /*
    +   * This belongs here more than in `SessionState`. However, does not seem that it can be
    --- End diff --
    
    not how you write docs. docs first start with what the class/method does. other points to note comes later.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    jenkins test this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73024/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102596195
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.hive
    +
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.sql.{QueryTest, Row, SparkSession}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class HiveSessionStateSuite  extends SparkFunSuite with SharedSQLContext {
    --- End diff --
    
    Added more tests, and moved this to `HiveSessionCatalog` which inherits from `SessionCatalog` to cover all tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73447 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73447/testReport)** for PR 16826 at commit [`fd11ee2`](https://github.com/apache/spark/commit/fd11ee2289ae26b3061659dc26b1f09ded32d039).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74024 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74024/testReport)** for PR 16826 at commit [`c3f052f`](https://github.com/apache/spark/commit/c3f052ff791c4af4dc9aa25aa87aca1262bf176f).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102632920
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ---
    @@ -212,6 +219,29 @@ class SparkSession private(
         new SparkSession(sparkContext, Some(sharedState))
       }
     
    +  /**
    +   * :: Experimental ::
    +   * Create an identical copy of this `SparkSession`, sharing the underlying `SparkContext`
    +   * and cached data. All the state of this session (i.e. SQL configurations, temporary tables,
    +   * registered functions) is also copied over.
    +   * Changes to base session are not propagated to cloned session, cloned is independent
    --- End diff --
    
    Changed


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104596342
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -17,43 +17,70 @@
     
     package org.apache.spark.sql.internal
     
    -import java.io.File
    -
     import org.apache.hadoop.conf.Configuration
    -import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.{SparkConf, SparkContext}
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
    + * @param functionRegistry Internal catalog for managing functions registered by the user.
    + * @param catalog Internal catalog for managing table and database states.
    + * @param sqlParser Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    + * @param analyzer Logical query plan analyzer for resolving unresolved attributes and relations.
    + * @param streamingQueryManager Interface to start and stop
    + *                              [[org.apache.spark.sql.streaming.StreamingQuery]]s.
    + * @param queryExecutionCreator Lambda to create a [[QueryExecution]] from a [[LogicalPlan]]
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution) {
     
    -  // Note: These are all lazy vals because they depend on each other (e.g. conf) and we
    -  // want subclasses to override some of the fields. Otherwise, we would get a lot of NPEs.
    +  /**
    +   * Interface exposed to the user for registering user-defined functions.
    +   * Note that the user-defined functions must be deterministic.
    +   */
    +  val udf: UDFRegistration = new UDFRegistration(functionRegistry)
     
       /**
    -   * SQL-specific key-value configurations.
    +   *   Logical query plan optimizer.
    --- End diff --
    
    Nit: remove the extra space


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103303272
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -90,110 +203,37 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         }
       }
     
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  lazy val catalog = new SessionCatalog(
    -    sparkSession.sharedState.externalCatalog,
    -    sparkSession.sharedState.globalTempViewManager,
    -    functionResourceLoader,
    -    functionRegistry,
    -    conf,
    -    newHadoopConf(),
    -    sqlParser)
    +  def newHadoopConf(copyHadoopConf: Configuration, sqlConf: SQLConf): Configuration = {
    --- End diff --
    
    Changed name.
    This is also used in `HiveSessionState.apply`, would be rendered inaccessible if `private`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    retest this please. The running one will be failed because master was broken.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102634640
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala ---
    @@ -493,6 +493,21 @@ class CatalogSuite
         }
       }
     
    +  test("clone SessionCatalog") {
    +    // need to test tempTables are cloned
    +    assert(spark.catalog.listTables().collect().isEmpty)
    +
    +    createTempTable("my_temp_table")
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    val forkedSession = spark.cloneSession()
    --- End diff --
    
    This is just testing the user-facing wrappers. There are also tests for the actual implementation which is `SessionCatalog`. This tests `Catalog(CatalogImpl(SessionCatalog))`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100194640
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -115,16 +113,22 @@ class TestHiveContext(
     private[hive] class TestHiveSparkSession(
         @transient private val sc: SparkContext,
         @transient private val existingSharedState: Option[SharedState],
    +    existingSessionState: Option[SessionState],
    --- End diff --
    
    Looks like you don't need to change this file?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74044/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103307383
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -146,4 +107,153 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
     
    +  /**
    +   * 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.
    +   */
    +  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.clone(
    +      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))
    +  }
    +
    +}
    +
    +object HiveSessionState {
    +
    +  def apply(sparkSession: SparkSession): HiveSessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    --- End diff --
    
    Changed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74148/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: Fork SparkSession with option to inherit a copy of the S...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    @kunalkhamar  by the way, please add the JIRA number to the title.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73442 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73442/testReport)** for PR 16826 at commit [`fd11ee2`](https://github.com/apache/spark/commit/fd11ee2289ae26b3061659dc26b1f09ded32d039).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103306212
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
    @@ -136,6 +139,26 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
         }
         SparkSession.sqlListener.get()
       }
    +
    +  /*
    +   * This belongs here more than in `SessionState`. However, does not seem that it can be
    +   * removed from `SessionState` and `HiveSessionState` without using reflection in
    +   * `AddJarCommand`.
    +   */
    +  def addJar(path: String): Unit = {
    +    sparkContext.addJar(path)
    +
    --- End diff --
    
    removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103062959
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +82,118 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(newSparkSession: SparkSession): SessionState = {
    +    val sparkContext = newSparkSession.sparkContext
    +    val confCopy = conf.clone()
    +    val functionRegistryCopy = functionRegistry.clone()
    +    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    +    val catalogCopy = catalog.clone(
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      functionRegistryCopy,
    +      sqlParser)
    +    val queryExecutionCreator = (plan: LogicalPlan) => new QueryExecution(newSparkSession, plan)
    +
    +    SessionState.mergeSparkConf(confCopy, sparkContext.getConf)
    +
    +    new SessionState(
    +      sparkContext,
    +      newSparkSession.sharedState,
    +      confCopy,
    +      experimentalMethods.clone(),
    +      functionRegistryCopy,
    +      catalogCopy,
    +      sqlParser,
    +      SessionState.createAnalyzer(newSparkSession, catalogCopy, confCopy),
    +      new StreamingQueryManager(newSparkSession),
    +      queryExecutionCreator)
    +  }
    +
    +  // ------------------------------------------------------
    +  //  Helper methods, partially leftover from pre-2.0 days
    +  // ------------------------------------------------------
    +
    +  def executePlan(plan: LogicalPlan): QueryExecution = queryExecutionCreator(plan)
    +
    +  def refreshTable(tableName: String): Unit = {
    +    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    +  }
    +
    +  def addJar(path: String): Unit = sharedState.addJar(path)
    +}
    +
    +
    +object SessionState {
    +
    +  def apply(sparkSession: SparkSession): SessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): SessionState = {
    +
    +    val sparkContext = sparkSession.sparkContext
    +
    +    // SQL-specific key-value configurations.
    --- End diff --
    
    super nit: unnecessary comment. and extra line.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103297916
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1196,4 +1198,28 @@ class SessionCatalogSuite extends PlanTest {
           catalog.listFunctions("unknown_db", "func*")
         }
       }
    +
    +  test("copy SessionCatalog") {
    +    val externalCatalog = newEmptyCatalog()
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    original.createTempView("copytest1", tempTable1, overrideIfExists = false)
    +
    +    // check if tables copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(original ne clone)
    +    assert(clone.getTempView("copytest1") == Option(tempTable1))
    +
    +    // check if clone and original independent
    +    clone.dropTable(TableIdentifier("copytest1"), ignoreIfNotExists = false, purge = false)
    +    assert(original.getTempView("copytest1") == Option(tempTable1))
    +
    +    val tempTable2 = Range(1, 20, 2, 10)
    --- End diff --
    
    Added a test for this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618617
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala ---
    @@ -66,7 +66,7 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] {
      * Preprocess [[CreateTable]], to do some normalization and checking.
      */
     case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[LogicalPlan] {
    -  private val catalog = sparkSession.sessionState.catalog
    +  private def catalog = sparkSession.sessionState.catalog
    --- End diff --
    
    Add comments saying that this should be a def and not a val as this would introduce circular references..


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103063798
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -144,11 +145,37 @@ private[hive] class TestHiveSparkSession(
         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.
    +  private def createHiveSessionState: HiveSessionState = {
    --- End diff --
    
    why separate function. why not just put inline. 
    ```
    override lazy val sessionState: HiveSessionState = {
       val testConf = ....
    ....
    }
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064385
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -17,89 +17,50 @@
     
     package org.apache.spark.sql.hive
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.analysis.Analyzer
    -import org.apache.spark.sql.execution.SparkPlanner
    +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanner, SparkSqlParser}
     import org.apache.spark.sql.execution.datasources._
     import org.apache.spark.sql.hive.client.HiveClient
    -import org.apache.spark.sql.internal.SessionState
    +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]] backed by Hive.
      */
    -private[hive] class HiveSessionState(sparkSession: SparkSession)
    -  extends SessionState(sparkSession) {
    -
    -  self =>
    -
    -  /**
    -   * A Hive client used for interacting with the metastore.
    -   */
    -  lazy val metadataHive: HiveClient =
    -    sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.newSession()
    -
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  override lazy val catalog = {
    -    new HiveSessionCatalog(
    -      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
    -      sparkSession.sharedState.globalTempViewManager,
    -      sparkSession,
    -      functionResourceLoader,
    -      functionRegistry,
    +private[hive] class HiveSessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    conf: SQLConf,
    +    experimentalMethods: ExperimentalMethods,
    +    functionRegistry: FunctionRegistry,
    +    override val catalog: HiveSessionCatalog,
    +    sqlParser: ParserInterface,
    +    val metadataHive: HiveClient,
    +    override val analyzer: Analyzer,
    --- End diff --
    
    why is this a `override val`? keep it consistent with other params


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100190232
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala ---
    @@ -46,4 +48,23 @@ class ExperimentalMethods private[sql]() {
     
       @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil
     
    +  /**
    +   * Get an identical copy of this `ExperimentalMethods` instance.
    +   * @note This is used when forking a `SparkSession`.
    +   * `clone` is provided here instead of implementing equivalent functionality
    +   * in `SparkSession.clone` since it needs to be updated
    +   * as the class `ExperimentalMethods` is extended or  modified.
    +   */
    +  override def clone: ExperimentalMethods = {
    +    def cloneSeq[T](seq: Seq[T]): Seq[T] = {
    +      val newSeq = new ListBuffer[T]
    +      newSeq ++= seq
    +      newSeq
    +    }
    +
    +    val result = new ExperimentalMethods
    +    result.extraStrategies = cloneSeq(extraStrategies)
    --- End diff --
    
    You don't need to copy these two `Seq`s since they are not mutable `Seq`s.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74165 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74165/testReport)** for PR 16826 at commit [`05abcf8`](https://github.com/apache/spark/commit/05abcf801f57c861435600fcc5fb2cecf6f3e11f).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104595290
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1197,6 +1199,65 @@ class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("clone SessionCatalog - temp views") {
    +    val externalCatalog = newEmptyCatalog()
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    original.createTempView("copytest1", tempTable1, overrideIfExists = false)
    +
    +    // check if tables copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(original ne clone)
    +    assert(clone.getTempView("copytest1") == Option(tempTable1))
    +
    +    // check if clone and original independent
    +    clone.dropTable(TableIdentifier("copytest1"), ignoreIfNotExists = false, purge = false)
    +    assert(original.getTempView("copytest1") == Option(tempTable1))
    +
    +    val tempTable2 = Range(1, 20, 2, 10)
    +    original.createTempView("copytest2", tempTable2, overrideIfExists = false)
    +    assert(clone.getTempView("copytest2").isEmpty)
    +  }
    +
    +  test("clone SessionCatalog - current db") {
    +    val externalCatalog = newEmptyCatalog()
    +    externalCatalog.createDatabase(newDb("copytest1"), true)
    +    externalCatalog.createDatabase(newDb("copytest2"), true)
    +    externalCatalog.createDatabase(newDb("copytest3"), true)
    +
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    val db1 = "copytest1"
    +    original.createTempView(db1, tempTable1, overrideIfExists = false)
    +    original.setCurrentDatabase(db1)
    +
    +    // check if current db copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(original ne clone)
    +    assert(clone.getCurrentDatabase == db1)
    +
    +    // check if clone and original independent
    +    val db2 = "copytest2"
    +    val tempTable2 = Range(1, 20, 2, 20)
    +    clone.createTempView(db2, tempTable2, overrideIfExists = false)
    +    clone.setCurrentDatabase(db2)
    +    assert(original.getCurrentDatabase == db1)
    +
    +    val db3 = "copytest3"
    +    val tempTable3 = Range(1, 30, 2, 30)
    +    original.createTempView(db3, tempTable3, overrideIfExists = false)
    +    original.setCurrentDatabase(db3)
    +    assert(clone.getCurrentDatabase == db2)
    +  }
    --- End diff --
    
    How about?
    
    ```Scala
      test("clone SessionCatalog - current db") {
        val externalCatalog = newEmptyCatalog()
        val db1 = "db1"
        val db2 = "db2"
        val db3 = "db3"
        val data = Range(1, 10, 1, 10)
    
        externalCatalog.createDatabase(newDb(db1), ignoreIfExists = true)
        externalCatalog.createDatabase(newDb(db2), ignoreIfExists = true)
        externalCatalog.createDatabase(newDb(db3), ignoreIfExists = true)
    
        val original = new SessionCatalog(externalCatalog)
        original.createTempView("view1", data, overrideIfExists = false)
        original.setCurrentDatabase(db1)
    
        // check if current db copied over
        val clone = original.clone(
          SimpleCatalystConf(caseSensitiveAnalysis = true),
          new Configuration(),
          new SimpleFunctionRegistry,
          CatalystSqlParser)
        assert(original != clone)
        assert(clone.getCurrentDatabase == db1)
    
        // check if clone and original independent
        clone.createTempView("view2", data, overrideIfExists = false)
        clone.setCurrentDatabase(db2)
        assert(original.getCurrentDatabase == db1)
        original.setCurrentDatabase(db3)
        assert(clone.getCurrentDatabase == db2)
      }
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74079 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74079/testReport)** for PR 16826 at commit [`c41e7bc`](https://github.com/apache/spark/commit/c41e7bc1fbebf46655b7414b6c63b93853d1b843).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103061676
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1196,4 +1198,28 @@ class SessionCatalogSuite extends PlanTest {
           catalog.listFunctions("unknown_db", "func*")
         }
       }
    +
    +  test("copy SessionCatalog") {
    +    val externalCatalog = newEmptyCatalog()
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    original.createTempView("copytest1", tempTable1, overrideIfExists = false)
    +
    +    // check if tables copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(original ne clone)
    +    assert(clone.getTempView("copytest1") == Option(tempTable1))
    +
    +    // check if clone and original independent
    +    clone.dropTable(TableIdentifier("copytest1"), ignoreIfNotExists = false, purge = false)
    +    assert(original.getTempView("copytest1") == Option(tempTable1))
    +
    +    val tempTable2 = Range(1, 20, 2, 10)
    --- End diff --
    
    should also test that `currentDb` get copied over and is independent.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: Fork SparkSession with option to inherit a copy of the S...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74044 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74044/testReport)** for PR 16826 at commit [`0f167db`](https://github.com/apache/spark/commit/0f167db45f8393065ebeb43ff6d7014d642c0267).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103067630
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala ---
    @@ -274,6 +274,8 @@ private[hive] class HiveClientImpl(
         val original = Thread.currentThread().getContextClassLoader
         // Set the thread local metastore client to the client associated with this HiveClientImpl.
         Hive.set(client)
    +    // replace conf in cached hive with current conf
    --- End diff --
    
    A `get` after a `set` looks weird. Actually I need to go back to check Hive source to know `get` will replace conf with provided one. Shall we move this into `private def client: Hive` and add comment for it?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103329699
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -144,11 +145,37 @@ private[hive] class TestHiveSparkSession(
         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.
    +  private def createHiveSessionState: HiveSessionState = {
    --- End diff --
    
    Neat, changed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74124 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74124/testReport)** for PR 16826 at commit [`5eb6733`](https://github.com/apache/spark/commit/5eb6733d7e49ff64bcd4d6fbd21acca1d4c3187f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HiveSessionCatalogSuite extends TestHiveSingleton `


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    jenkins retest this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102616133
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1181,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    --- End diff --
    
    nit: 
    deep copy*.*


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    LGTM. Merging to master.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103295794
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1196,4 +1198,28 @@ class SessionCatalogSuite extends PlanTest {
           catalog.listFunctions("unknown_db", "func*")
         }
       }
    +
    +  test("copy SessionCatalog") {
    --- End diff --
    
    changed


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103063544
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala ---
    @@ -493,6 +493,28 @@ class CatalogSuite
         }
       }
     
    +  test("clone Catalog") {
    +    // need to test tempTables are cloned
    +    assert(spark.catalog.listTables().collect().isEmpty)
    +
    +    createTempTable("my_temp_table")
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    // inheritance
    +    val forkedSession = spark.cloneSession()
    +    assert(spark ne forkedSession)
    +    assert(spark.catalog ne forkedSession.catalog)
    +    assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +
    +    // independence
    +    dropTable("my_temp_table") // drop table in original session
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set())
    +    assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table"))
    +    forkedSession.sessionState.catalog
    +      .createTempView("fork_table", Range(1, 2, 3, 4), overrideIfExists = true)
    +    assert(spark.catalog.listTables().collect().map(_.name).toSet == Set())
    +  }
    +
       // TODO: add tests for the rest of them
    --- End diff --
    
    what is this comment for? remove it if its just dead comment.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103060505
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1178,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def copy(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    synchronized {
    --- End diff --
    
    Right, I should mention we decided to keep it `synchronized`. `catalog` is not yet accessible outside local scope.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103062258
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -90,110 +208,29 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         }
       }
     
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  lazy val catalog = new SessionCatalog(
    -    sparkSession.sharedState.externalCatalog,
    -    sparkSession.sharedState.globalTempViewManager,
    -    functionResourceLoader,
    -    functionRegistry,
    -    conf,
    -    newHadoopConf(),
    -    sqlParser)
    +  def newHadoopConf(copyHadoopConf: Configuration, sqlConf: SQLConf): Configuration = {
    +    val hadoopConf = new Configuration(copyHadoopConf)
    +    sqlConf.getAllConfs.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) }
    +    hadoopConf
    +  }
     
    -  /**
    -   * Interface exposed to the user for registering user-defined functions.
    -   * Note that the user-defined functions must be deterministic.
    -   */
    -  lazy val udf: UDFRegistration = new UDFRegistration(functionRegistry)
    +  def createAnalyzer(
    +      sparkSession: SparkSession,
    +      catalog: SessionCatalog,
    +      sqlConf: SQLConf): Analyzer = {
     
    -  /**
    -   * Logical query plan analyzer for resolving unresolved attributes and relations.
    -   */
    -  lazy val analyzer: Analyzer = {
    -    new Analyzer(catalog, conf) {
    +    new Analyzer(catalog, sqlConf) {
           override val extendedResolutionRules =
             new FindDataSourceTable(sparkSession) ::
             new ResolveSQLOnFile(sparkSession) :: Nil
     
           override val postHocResolutionRules =
             PreprocessTableCreation(sparkSession) ::
    -        PreprocessTableInsertion(conf) ::
    -        DataSourceAnalysis(conf) :: Nil
    +        PreprocessTableInsertion(sqlConf) ::
    +        DataSourceAnalysis(sqlConf) :: Nil
     
           override val extendedCheckRules = Seq(PreWriteCheck, HiveOnlyCheck)
         }
       }
     
    -  /**
    -   * Logical query plan optimizer.
    -   */
    -  lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods)
    -
    -  /**
    -   * Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    -   */
    -  lazy val sqlParser: ParserInterface = new SparkSqlParser(conf)
    -
    -  /**
    -   * Planner that converts optimized logical plans to physical plans.
    -   */
    -  def planner: SparkPlanner =
    -    new SparkPlanner(sparkSession.sparkContext, conf, experimentalMethods.extraStrategies)
    -
    -  /**
    -   * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s
    -   * that listen for execution metrics.
    -   */
    -  lazy val listenerManager: ExecutionListenerManager = new ExecutionListenerManager
    -
    -  /**
    -   * Interface to start and stop [[StreamingQuery]]s.
    -   */
    -  lazy val streamingQueryManager: StreamingQueryManager = {
    -    new StreamingQueryManager(sparkSession)
    -  }
    -
    -  private val jarClassLoader: NonClosableMutableURLClassLoader =
    -    sparkSession.sharedState.jarClassLoader
    -
    -  // Automatically extract all entries and put it in our SQLConf
    -  // We need to call it after all of vals have been initialized.
    -  sparkSession.sparkContext.getConf.getAll.foreach { case (k, v) =>
    -    conf.setConfString(k, v)
    -  }
    -
    -  // ------------------------------------------------------
    -  //  Helper methods, partially leftover from pre-2.0 days
    -  // ------------------------------------------------------
    -
    -  def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan)
    -
    -  def refreshTable(tableName: String): Unit = {
    -    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    -  }
    -
    -  def addJar(path: String): Unit = {
    -    sparkSession.sparkContext.addJar(path)
    -
    -    val uri = new Path(path).toUri
    -    val jarURL = if (uri.getScheme == null) {
    -      // `path` is a local file path without a URL scheme
    -      new File(path).toURI.toURL
    -    } else {
    -      // `path` is a URL with a scheme
    -      uri.toURL
    -    }
    -    jarClassLoader.addURL(jarURL)
    -    Thread.currentThread().setContextClassLoader(jarClassLoader)
    -  }
    -
    -  /**
    -   * Analyzes the given table in the current database to generate statistics, which will be
    -   * used in query optimizations.
    -   */
    -  def analyze(tableIdent: TableIdentifier, noscan: Boolean = true): Unit = {
    --- End diff --
    
    if its not used, lets remove it. no point keeping around dead code.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    @rxin 
    The change from `SessionState(SparkSession,parentSessionState)` to `SessionState(SparkSession,SQLConf,SessionCatalog,...)` is mostly done. Still have 4 failing tests (OOMs) and will create some more tests.
    
    Also, currently `SparkSession` has-a `SessionState` and `SessionState` has-a `SparkSession`. Investigating into removing the `SparkSession` reference from inside `SessionState`. This would change the constructor from `SessionState(SparkSession,SQLConf,SessionCatalog,...)`to `SessionState(SQLConf,SessionCatalog,...)`.
    
    There are fields inside `SessionState` that depend on `SparkSession`, e.g. `Analyzer` and `QueryExecution`. But these can be created and passed in to `SessionState` during creation. It seems we might be able to pull up all such references to `SparkSession` to initialization (i.e. inside `SessionState.apply`.) Same goes for `HiveSessionState`.
    
    Any thoughts on this?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73442 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73442/testReport)** for PR 16826 at commit [`fd11ee2`](https://github.com/apache/spark/commit/fd11ee2289ae26b3061659dc26b1f09ded32d039).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73388 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73388/testReport)** for PR 16826 at commit [`16824f9`](https://github.com/apache/spark/commit/16824f916e87fd90706f9dfd7b7dd81d87b732dd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r101879685
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -64,6 +64,8 @@ trait FunctionRegistry {
       /** Clear all registered functions. */
       def clear(): Unit
     
    +  /** Create a copy of this registry with identical functions as this registry. */
    +  def copy: FunctionRegistry
    --- End diff --
    
    should it be `copy()`? @rxin 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102143663
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.hive
    +
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.sql.{QueryTest, Row, SparkSession}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class HiveSessionStateSuite  extends SparkFunSuite with SharedSQLContext {
    --- End diff --
    
    I am worrying about the test case coverage. How about also extending the test suite for `SessionState`? It can automatically running them without copying and pasting the test cases here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103337066
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala ---
    @@ -136,6 +139,26 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
         }
         SparkSession.sqlListener.get()
       }
    +
    +  /*
    +   * This belongs here more than in `SessionState`. However, does not seem that it can be
    --- End diff --
    
    Removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74039 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74039/testReport)** for PR 16826 at commit [`2740c63`](https://github.com/apache/spark/commit/2740c63634b41de54a8913ea71e3c8927a9969ca).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064447
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala ---
    @@ -146,4 +107,153 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
         conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC)
       }
     
    +  /**
    +   * 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.
    +   */
    +  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.clone(
    +      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))
    +  }
    +
    +}
    +
    +object HiveSessionState {
    +
    +  def apply(sparkSession: SparkSession): HiveSessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    --- End diff --
    
    same comment as `SessionState.apply`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: Fork SparkSession with option to inherit a copy of the S...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #72606 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72606/testReport)** for PR 16826 at commit [`a343d8a`](https://github.com/apache/spark/commit/a343d8af9c577158042e4af9f8832f46aeecd509).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/16826


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103064704
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.catalyst.rules.Rule
    +
    +class SessionStateSuite extends SparkFunSuite with BeforeAndAfterEach {
    +
    +  protected var activeSession: SparkSession = _
    +
    +  protected def createSession(): Unit = {
    +    activeSession = SparkSession.builder().master("local").getOrCreate()
    +  }
    +
    +  override def beforeEach(): Unit = {
    +    createSession()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    activeSession.stop()
    +  }
    +
    +  test("fork new session and inherit RuntimeConfig options") {
    +    val key = "spark-config-clone"
    +    activeSession.conf.set(key, "active")
    +
    +    // inheritance
    +    val forkedSession = activeSession.cloneSession()
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.conf ne activeSession.conf)
    +    assert(forkedSession.conf.get(key) == "active")
    +
    +    // independence
    +    forkedSession.conf.set(key, "forked")
    +    assert(activeSession.conf.get(key) == "active")
    +    activeSession.conf.set(key, "dontcopyme")
    +    assert(forkedSession.conf.get(key) == "forked")
    +  }
    +
    +  test("fork new session and inherit function registry and udf") {
    +    activeSession.udf.register("strlenScala", (_: String).length + (_: Int))
    +    val forkedSession = activeSession.cloneSession()
    +
    +    // inheritance
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.sessionState.functionRegistry ne
    +      activeSession.sessionState.functionRegistry)
    +    assert(forkedSession.sessionState.functionRegistry.lookupFunction("strlenScala").nonEmpty)
    +
    +    // independence
    +    forkedSession.sessionState.functionRegistry.dropFunction("strlenScala")
    +    assert(activeSession.sessionState.functionRegistry.lookupFunction("strlenScala").nonEmpty)
    +    activeSession.udf.register("addone", (_: Int) + 1)
    +    assert(forkedSession.sessionState.functionRegistry.lookupFunction("addone").isEmpty)
    +  }
    +
    +  test("fork new session and inherit experimental methods") {
    +    object DummyRule1 extends Rule[LogicalPlan] {
    +      def apply(p: LogicalPlan): LogicalPlan = p
    +    }
    +    object DummyRule2 extends Rule[LogicalPlan] {
    +      def apply(p: LogicalPlan): LogicalPlan = p
    +    }
    +    val optimizations = List(DummyRule1, DummyRule2)
    +
    +    activeSession.experimental.extraOptimizations = optimizations
    +
    +    val forkedSession = activeSession.cloneSession()
    +
    +    // inheritance
    +    assert(forkedSession ne activeSession)
    +    assert(forkedSession.experimental ne activeSession.experimental)
    +    assert(forkedSession.experimental.extraOptimizations.toSet ==
    +      activeSession.experimental.extraOptimizations.toSet)
    +
    +    // independence
    +    forkedSession.experimental.extraOptimizations = List(DummyRule2)
    +    assert(activeSession.experimental.extraOptimizations == optimizations)
    +    activeSession.experimental.extraOptimizations = List(DummyRule1)
    +    assert(forkedSession.experimental.extraOptimizations == List(DummyRule2))
    +  }
    +
    +  test("fork new sessions and run query on inherited table") {
    +    def checkTableExists(sparkSession: SparkSession): Unit = {
    +      QueryTest.checkAnswer(sparkSession.sql(
    +        """
    +          |SELECT x.str, COUNT(*)
    +          |FROM df x JOIN df y ON x.str = y.str
    +          |GROUP BY x.str
    +        """.stripMargin),
    +        Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil)
    +    }
    +
    +    SparkSession.setActiveSession(activeSession)
    --- End diff --
    
    the setting and clearing of active session should be in a try-finally so that errors in this test (that does not clear the session) does not cascade to future tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103061761
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1181,36 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep
    +   * copy. `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def clone(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    synchronized {
    +      catalog.currentDb = currentDb
    +      // copy over temporary tables
    +      tempTables.foreach(kv => catalog.tempTables.put(kv._1, kv._2))
    +    }
    +
    +    catalog
    +  }
    +
    --- End diff --
    
    nit: extra line.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102618748
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -22,38 +22,58 @@ import java.io.File
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
     
    +import org.apache.spark.SparkContext
     import org.apache.spark.sql._
    -import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
     import org.apache.spark.sql.catalyst.catalog._
     import org.apache.spark.sql.catalyst.optimizer.Optimizer
     import org.apache.spark.sql.catalyst.parser.ParserInterface
     import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
     import org.apache.spark.sql.execution._
    -import org.apache.spark.sql.execution.command.AnalyzeTableCommand
     import org.apache.spark.sql.execution.datasources._
    -import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager}
    +import org.apache.spark.sql.streaming.StreamingQueryManager
     import org.apache.spark.sql.util.ExecutionListenerManager
     
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkContext: SparkContext,
    +    sharedState: SharedState,
    +    val conf: SQLConf,
    +    val experimentalMethods: ExperimentalMethods,
    +    val functionRegistry: FunctionRegistry,
    +    val catalog: SessionCatalog,
    +    val sqlParser: ParserInterface,
    +    val analyzer: Analyzer,
    +    val streamingQueryManager: StreamingQueryManager,
    +    val queryExecutionCreator: LogicalPlan => QueryExecution) {
    +
    +  /*
    +   * Interface exposed to the user for registering user-defined functions.
    --- End diff --
    
    Use /** ... */


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    jenkins retest this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103336676
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -65,22 +82,118 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         hadoopConf
       }
     
    -  lazy val experimentalMethods = new ExperimentalMethods
    -
       /**
    -   * Internal catalog for managing functions registered by the user.
    +   * Get an identical copy of the `SessionState` and associate it with the given `SparkSession`
        */
    -  lazy val functionRegistry: FunctionRegistry = FunctionRegistry.builtin.copy()
    +  def clone(newSparkSession: SparkSession): SessionState = {
    +    val sparkContext = newSparkSession.sparkContext
    +    val confCopy = conf.clone()
    +    val functionRegistryCopy = functionRegistry.clone()
    +    val sqlParser: ParserInterface = new SparkSqlParser(confCopy)
    +    val catalogCopy = catalog.clone(
    +      confCopy,
    +      SessionState.newHadoopConf(sparkContext.hadoopConfiguration, confCopy),
    +      functionRegistryCopy,
    +      sqlParser)
    +    val queryExecutionCreator = (plan: LogicalPlan) => new QueryExecution(newSparkSession, plan)
    +
    +    SessionState.mergeSparkConf(confCopy, sparkContext.getConf)
    +
    +    new SessionState(
    +      sparkContext,
    +      newSparkSession.sharedState,
    +      confCopy,
    +      experimentalMethods.clone(),
    +      functionRegistryCopy,
    +      catalogCopy,
    +      sqlParser,
    +      SessionState.createAnalyzer(newSparkSession, catalogCopy, confCopy),
    +      new StreamingQueryManager(newSparkSession),
    +      queryExecutionCreator)
    +  }
    +
    +  // ------------------------------------------------------
    +  //  Helper methods, partially leftover from pre-2.0 days
    +  // ------------------------------------------------------
    +
    +  def executePlan(plan: LogicalPlan): QueryExecution = queryExecutionCreator(plan)
    +
    +  def refreshTable(tableName: String): Unit = {
    +    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    +  }
    +
    +  def addJar(path: String): Unit = sharedState.addJar(path)
    +}
    +
    +
    +object SessionState {
    +
    +  def apply(sparkSession: SparkSession): SessionState = {
    +    apply(sparkSession, None)
    +  }
    +
    +  def apply(
    +      sparkSession: SparkSession,
    +      conf: Option[SQLConf]): SessionState = {
    +
    --- End diff --
    
    Removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: Fork SparkSession with option to inherit a copy of the S...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72606/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102619000
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -90,110 +208,29 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         }
       }
     
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  lazy val catalog = new SessionCatalog(
    -    sparkSession.sharedState.externalCatalog,
    -    sparkSession.sharedState.globalTempViewManager,
    -    functionResourceLoader,
    -    functionRegistry,
    -    conf,
    -    newHadoopConf(),
    -    sqlParser)
    +  def newHadoopConf(copyHadoopConf: Configuration, sqlConf: SQLConf): Configuration = {
    +    val hadoopConf = new Configuration(copyHadoopConf)
    +    sqlConf.getAllConfs.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) }
    +    hadoopConf
    +  }
     
    -  /**
    -   * Interface exposed to the user for registering user-defined functions.
    -   * Note that the user-defined functions must be deterministic.
    -   */
    -  lazy val udf: UDFRegistration = new UDFRegistration(functionRegistry)
    +  def createAnalyzer(
    +      sparkSession: SparkSession,
    +      catalog: SessionCatalog,
    +      sqlConf: SQLConf): Analyzer = {
     
    -  /**
    -   * Logical query plan analyzer for resolving unresolved attributes and relations.
    -   */
    -  lazy val analyzer: Analyzer = {
    -    new Analyzer(catalog, conf) {
    +    new Analyzer(catalog, sqlConf) {
           override val extendedResolutionRules =
             new FindDataSourceTable(sparkSession) ::
             new ResolveSQLOnFile(sparkSession) :: Nil
     
           override val postHocResolutionRules =
             PreprocessTableCreation(sparkSession) ::
    -        PreprocessTableInsertion(conf) ::
    -        DataSourceAnalysis(conf) :: Nil
    +        PreprocessTableInsertion(sqlConf) ::
    +        DataSourceAnalysis(sqlConf) :: Nil
     
           override val extendedCheckRules = Seq(PreWriteCheck, HiveOnlyCheck)
         }
       }
     
    -  /**
    -   * Logical query plan optimizer.
    -   */
    -  lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods)
    -
    -  /**
    -   * Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
    -   */
    -  lazy val sqlParser: ParserInterface = new SparkSqlParser(conf)
    -
    -  /**
    -   * Planner that converts optimized logical plans to physical plans.
    -   */
    -  def planner: SparkPlanner =
    -    new SparkPlanner(sparkSession.sparkContext, conf, experimentalMethods.extraStrategies)
    -
    -  /**
    -   * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s
    -   * that listen for execution metrics.
    -   */
    -  lazy val listenerManager: ExecutionListenerManager = new ExecutionListenerManager
    -
    -  /**
    -   * Interface to start and stop [[StreamingQuery]]s.
    -   */
    -  lazy val streamingQueryManager: StreamingQueryManager = {
    -    new StreamingQueryManager(sparkSession)
    -  }
    -
    -  private val jarClassLoader: NonClosableMutableURLClassLoader =
    -    sparkSession.sharedState.jarClassLoader
    -
    -  // Automatically extract all entries and put it in our SQLConf
    -  // We need to call it after all of vals have been initialized.
    -  sparkSession.sparkContext.getConf.getAll.foreach { case (k, v) =>
    -    conf.setConfString(k, v)
    -  }
    -
    -  // ------------------------------------------------------
    -  //  Helper methods, partially leftover from pre-2.0 days
    -  // ------------------------------------------------------
    -
    -  def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan)
    -
    -  def refreshTable(tableName: String): Unit = {
    -    catalog.refreshTable(sqlParser.parseTableIdentifier(tableName))
    -  }
    -
    -  def addJar(path: String): Unit = {
    -    sparkSession.sparkContext.addJar(path)
    -
    -    val uri = new Path(path).toUri
    -    val jarURL = if (uri.getScheme == null) {
    -      // `path` is a local file path without a URL scheme
    -      new File(path).toURI.toURL
    -    } else {
    -      // `path` is a URL with a scheme
    -      uri.toURL
    -    }
    -    jarClassLoader.addURL(jarURL)
    -    Thread.currentThread().setContextClassLoader(jarClassLoader)
    -  }
    -
    -  /**
    -   * Analyzes the given table in the current database to generate statistics, which will be
    -   * used in query optimizations.
    -   */
    -  def analyze(tableIdent: TableIdentifier, noscan: Boolean = true): Unit = {
    --- End diff --
    
    Where did this function go?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103062312
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -90,110 +203,37 @@ private[sql] class SessionState(sparkSession: SparkSession) {
         }
       }
     
    -  /**
    -   * Internal catalog for managing table and database states.
    -   */
    -  lazy val catalog = new SessionCatalog(
    -    sparkSession.sharedState.externalCatalog,
    -    sparkSession.sharedState.globalTempViewManager,
    -    functionResourceLoader,
    -    functionRegistry,
    -    conf,
    -    newHadoopConf(),
    -    sqlParser)
    +  def newHadoopConf(copyHadoopConf: Configuration, sqlConf: SQLConf): Configuration = {
    +    val hadoopConf = new Configuration(copyHadoopConf)
    +    sqlConf.getAllConfs.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) }
    +    hadoopConf
    +  }
     
    -  /**
    -   * Interface exposed to the user for registering user-defined functions.
    -   * Note that the user-defined functions must be deterministic.
    -   */
    -  lazy val udf: UDFRegistration = new UDFRegistration(functionRegistry)
    +  def createAnalyzer(
    --- End diff --
    
    docs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [WIP][SPARK-19540][SQL] Add ability to clone SparkSessio...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72970/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73195 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73195/testReport)** for PR 16826 at commit [`292011a`](https://github.com/apache/spark/commit/292011a2ad06c46ba58756a5bde19e9e24cfc9b2).
     * This patch **fails to build**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:
      * `trait Logging `
      * `case class UnresolvedRelation(tableIdentifier: TableIdentifier) extends LeafNode `
      * `case class SaveIntoDataSourceCommand(`
      * `abstract class JsonDataSource[T] extends Serializable `


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104589951
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala ---
    @@ -46,4 +46,10 @@ class ExperimentalMethods private[sql]() {
     
       @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil
     
    +  override def clone(): ExperimentalMethods = {
    --- End diff --
    
    When I reviewing the code, I just found they are not thread-safe, although they have already been declared volatile. 
    
    I am fine to keep them unchanged. How about leaving a comment in the code to emphasize that it is not thread-safe.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #73542 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73542/testReport)** for PR 16826 at commit [`300d3a0`](https://github.com/apache/spark/commit/300d3a05e43dee853b452973eea8a707d486dd61).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: Fork SparkSession with option to inherit a copy o...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r100192972
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -38,16 +38,31 @@ import org.apache.spark.sql.util.ExecutionListenerManager
     
     /**
      * A class that holds all session-specific state in a given [[SparkSession]].
    + * If an `existingSessionState` is supplied, then its members will be copied over.
      */
    -private[sql] class SessionState(sparkSession: SparkSession) {
    +private[sql] class SessionState(
    +    sparkSession: SparkSession,
    +    existingSessionState: Option[SessionState]) {
    --- End diff --
    
    nit: `existingSessionState` -> `parentSessionState` to indicate we should copy its internal states.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    add to whitelist


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    **[Test build #74215 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74215/testReport)** for PR 16826 at commit [`4c23e7a`](https://github.com/apache/spark/commit/4c23e7a0caaff5aa2df3dbbf5e6a688b3a4d442d).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r104593552
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1197,6 +1199,65 @@ class SessionCatalogSuite extends PlanTest {
         }
       }
     
    +  test("clone SessionCatalog - temp views") {
    +    val externalCatalog = newEmptyCatalog()
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    original.createTempView("copytest1", tempTable1, overrideIfExists = false)
    +
    +    // check if tables copied over
    +    val clone = original.clone(
    +      SimpleCatalystConf(caseSensitiveAnalysis = true),
    +      new Configuration(),
    +      new SimpleFunctionRegistry,
    +      CatalystSqlParser)
    +    assert(original ne clone)
    +    assert(clone.getTempView("copytest1") == Option(tempTable1))
    +
    +    // check if clone and original independent
    +    clone.dropTable(TableIdentifier("copytest1"), ignoreIfNotExists = false, purge = false)
    +    assert(original.getTempView("copytest1") == Option(tempTable1))
    +
    +    val tempTable2 = Range(1, 20, 2, 10)
    +    original.createTempView("copytest2", tempTable2, overrideIfExists = false)
    +    assert(clone.getTempView("copytest2").isEmpty)
    +  }
    +
    +  test("clone SessionCatalog - current db") {
    +    val externalCatalog = newEmptyCatalog()
    +    externalCatalog.createDatabase(newDb("copytest1"), true)
    +    externalCatalog.createDatabase(newDb("copytest2"), true)
    +    externalCatalog.createDatabase(newDb("copytest3"), true)
    +
    +    val original = new SessionCatalog(externalCatalog)
    +    val tempTable1 = Range(1, 10, 1, 10)
    +    val db1 = "copytest1"
    --- End diff --
    
    `db1`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102786174
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala ---
    @@ -34,9 +40,32 @@ private[sql] class TestSparkSession(sc: SparkContext) extends SparkSession(sc) {
         this(new SparkConf)
       }
     
    -  @transient
    -  override lazy val sessionState: SessionState = new SessionState(self) {
    -    override lazy val conf: SQLConf = {
    +  class TestSessionState(
    --- End diff --
    
    Yes, will simplify this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16826: [SPARK-19540][SQL] Add ability to clone SparkSession whe...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16826
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74165/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r101878697
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -150,6 +152,9 @@ object EmptyFunctionRegistry extends FunctionRegistry {
         throw new UnsupportedOperationException
       }
     
    +  override def copy: FunctionRegistry = {
    +    throw new UnsupportedOperationException
    --- End diff --
    
    this ideally should be supported, and just returns itself.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r103308243
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionCatalogSuite.scala ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.hive
    +
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.SimpleFunctionRegistry
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
    +import org.apache.spark.sql.catalyst.plans.logical.Range
    +import org.apache.spark.sql.internal.SQLConf
    +
    +class HiveSessionCatalogSuite extends SessionCatalogSuite {
    --- End diff --
    
    Good point, removed extends.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16826: [SPARK-19540][SQL] Add ability to clone SparkSess...

Posted by kunalkhamar <gi...@git.apache.org>.
Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16826#discussion_r102137355
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---
    @@ -1178,4 +1178,34 @@ class SessionCatalog(
         }
       }
     
    +  /**
    +   * Get an identical copy of the `SessionCatalog`.
    +   * The temporary tables and function registry are retained.
    +   * The table relation cache will not be populated.
    +   * @note `externalCatalog` and `globalTempViewManager` are from shared state, don't need deep copy
    +   * `FunctionResourceLoader` is effectively stateless, also does not need deep copy.
    +   * All arguments passed in should be associated with a particular `SparkSession`.
    +   */
    +  def copy(
    +      conf: CatalystConf,
    +      hadoopConf: Configuration,
    +      functionRegistry: FunctionRegistry,
    +      parser: ParserInterface): SessionCatalog = {
    +
    +    val catalog = new SessionCatalog(
    +      externalCatalog,
    +      globalTempViewManager,
    +      functionResourceLoader,
    +      functionRegistry,
    +      conf,
    +      hadoopConf,
    +      parser)
    +
    +    catalog.currentDb = currentDb
    --- End diff --
    
    yes!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org