You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by liancheng <gi...@git.apache.org> on 2014/10/31 18:09:38 UTC

[GitHub] spark pull request: [SPARK-2189][SQL] Adds unregisterTempTable API

GitHub user liancheng opened a pull request:

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

    [SPARK-2189][SQL] Adds unregisterTempTable API

    This PR adds an API for unregistering temporary tables. If a temporary table has been cached before, user can unpersist it as well.

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

    $ git pull https://github.com/liancheng/spark unregister-temp-table

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

    https://github.com/apache/spark/pull/3039.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 #3039
    
----
commit 7d4fb2bf1364832a0faf3f26ae23264d0389e64b
Author: Cheng Lian <li...@gmail.com>
Date:   2014-10-31T16:54:44Z

    Adds unregisterTempTable API

commit aca41d364147ccdc2dfe2a0591a7965f7ad4400e
Author: Cheng Lian <li...@gmail.com>
Date:   2014-10-31T17:03:28Z

    Ensures thread safety

----


---
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: [SPARK-2189][SQL] Adds dropTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61354499
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22674/
    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: [SPARK-2189][SQL] Adds dropTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61354420
  
      [Test build #22674 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22674/consoleFull) for   PR 3039 at commit [`1948c14`](https://github.com/apache/spark/commit/1948c14455c97ec1c0b0f3ab3d96911f757bd157).
     * This patch merges cleanly.


---
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: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#discussion_r19685910
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -267,6 +267,23 @@ class SQLContext(@transient val sparkContext: SparkContext)
       }
     
       /**
    +   * Unregisters the temporary table with the given table name in the catalog. If the table has been
    +   * cached/persisted before, it can be unpersisted if required.
    +   *
    +   * @param tableName the name of the table to be unregistered.
    +   * @param unpersist whether to unpersist the table if it has been cached/persisted before.
    +   *
    +   * @group userf
    +   */
    +  def unregisterTempTable(tableName: String, unpersist: Boolean = false): Unit = {
    --- End diff --
    
    Why have an option to leave the data around after a table has been dropped?  Also what about "DROP [TEMPORARY?] TABLE"?


---
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: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61294919
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22614/
    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: [SPARK-2189][SQL] Adds dropTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61360615
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22686/
    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: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#discussion_r19699722
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -267,6 +267,23 @@ class SQLContext(@transient val sparkContext: SparkContext)
       }
     
       /**
    +   * Unregisters the temporary table with the given table name in the catalog. If the table has been
    +   * cached/persisted before, it can be unpersisted if required.
    +   *
    +   * @param tableName the name of the table to be unregistered.
    +   * @param unpersist whether to unpersist the table if it has been cached/persisted before.
    +   *
    +   * @group userf
    +   */
    +  def unregisterTempTable(tableName: String, unpersist: Boolean = false): Unit = {
    --- End diff --
    
    But indeed I think this API is not intuitive. Another choice is to ask users to uncache the table explicitly before call `dropTempTable`, and simply don't care about caching in `dropTempTable`. But this calling sequence is not thread-safe, plus it's verbose and error prone from user's perspective.
    
    The ultimate solution should be another refinement of caching semantics: namely making cached columnar RDDs "reference counted", a cached columnar RDD shouldn't be removed as long as there's at least one temporary table uses it.
    
    Maybe for now, we can just unpersist the cache if any and accept the risk of losing cache of multiple 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 pull request: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#discussion_r19679677
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala ---
    @@ -103,6 +103,17 @@ private[sql] trait CacheManager {
         cachedData.remove(dataIndex)
       }
     
    +  /** Tries to remove the data for the given SchemaRDD from the cache if it's cached */
    +  private[sql] def tryUncacheQuery(query: SchemaRDD, blocking: Boolean = true): Boolean = writeLock {
    +    val planToCache = query.queryExecution.analyzed
    +    val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan))
    +    val found = dataIndex >= 0
    +    if (found) {
    +      cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking)
    +      cachedData.remove(dataIndex)
    +    }
    +    found
    +  }
    --- End diff --
    
    This method is added to ensure `CacheManager` thread safety. (However, the catalog within `SQLContext` is not thread safe, which should be fixed in another PR.)


---
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: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#discussion_r19700892
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -267,6 +267,23 @@ class SQLContext(@transient val sparkContext: SparkContext)
       }
     
       /**
    +   * Unregisters the temporary table with the given table name in the catalog. If the table has been
    +   * cached/persisted before, it can be unpersisted if required.
    +   *
    +   * @param tableName the name of the table to be unregistered.
    +   * @param unpersist whether to unpersist the table if it has been cached/persisted before.
    +   *
    +   * @group userf
    +   */
    +  def unregisterTempTable(tableName: String, unpersist: Boolean = false): Unit = {
    --- End diff --
    
    Yeah, I think for now we should just remove the cache no matter what.  Eventually we'll probably add names to the cached data so we can provide clear semantics here, but I think its a pretty uncommon case to cache the same data twice under two names and then uncache one of them.
    
    Also, I wasn't arguing we should change the name of the function.  I think its good to have symmetry with register/unregister.  I was wondering if we could have SQL DDL for this same operation since we already have DDL for creating cached 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 pull request: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#discussion_r19698963
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -267,6 +267,23 @@ class SQLContext(@transient val sparkContext: SparkContext)
       }
     
       /**
    +   * Unregisters the temporary table with the given table name in the catalog. If the table has been
    +   * cached/persisted before, it can be unpersisted if required.
    +   *
    +   * @param tableName the name of the table to be unregistered.
    +   * @param unpersist whether to unpersist the table if it has been cached/persisted before.
    +   *
    +   * @group userf
    +   */
    +  def unregisterTempTable(tableName: String, unpersist: Boolean = false): Unit = {
    --- End diff --
    
    Because currently a single cached query plan can be shared by multiple tables:
    
    ```scala
    testData.select('key).registerTempTable("t1")
    testData.select('key).registerTempTable("t2")
    
    cacheTable("t1")
    println(isCached("t1"))  // true
    println(isCached("t2"))  // true
    
    unregisterTempTable("t2", unpersist = true)
    println(isCached("t1"))  // false
    println(isCached("t2"))  // false
    ```
    
    Maybe we should unpersist by default? Thoughts?
    
    (Agree with renaming to `dropTempTable`.)


---
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: [SPARK-2189][SQL] Adds dropTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61360612
  
      [Test build #22686 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22686/consoleFull) for   PR 3039 at commit [`54ae99f`](https://github.com/apache/spark/commit/54ae99f16087c6a23f6fd57cee27a1e604727b53).
     * 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: [SPARK-2189][SQL] Adds dropTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61359778
  
      [Test build #22686 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22686/consoleFull) for   PR 3039 at commit [`54ae99f`](https://github.com/apache/spark/commit/54ae99f16087c6a23f6fd57cee27a1e604727b53).
     * This patch merges cleanly.


---
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: [SPARK-2189][SQL] Adds dropTempTable API

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61294754
  
      [Test build #22614 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22614/consoleFull) for   PR 3039 at commit [`aca41d3`](https://github.com/apache/spark/commit/aca41d364147ccdc2dfe2a0591a7965f7ad4400e).
     * This patch merges cleanly.


---
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: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#discussion_r19700967
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
    @@ -267,6 +267,23 @@ class SQLContext(@transient val sparkContext: SparkContext)
       }
     
       /**
    +   * Unregisters the temporary table with the given table name in the catalog. If the table has been
    +   * cached/persisted before, it can be unpersisted if required.
    +   *
    +   * @param tableName the name of the table to be unregistered.
    +   * @param unpersist whether to unpersist the table if it has been cached/persisted before.
    +   *
    +   * @group userf
    +   */
    +  def unregisterTempTable(tableName: String, unpersist: Boolean = false): Unit = {
    --- End diff --
    
    Agree with the DDL part.


---
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: [SPARK-2189][SQL] Adds unregisterTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61294913
  
      [Test build #22614 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22614/consoleFull) for   PR 3039 at commit [`aca41d3`](https://github.com/apache/spark/commit/aca41d364147ccdc2dfe2a0591a7965f7ad4400e).
     * 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: [SPARK-2189][SQL] Adds dropTempTable API

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

    https://github.com/apache/spark/pull/3039#issuecomment-61354498
  
      [Test build #22674 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22674/consoleFull) for   PR 3039 at commit [`1948c14`](https://github.com/apache/spark/commit/1948c14455c97ec1c0b0f3ab3d96911f757bd157).
     * 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