You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by clockfly <gi...@git.apache.org> on 2016/05/31 17:42:10 UTC

[GitHub] spark pull request: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

GitHub user clockfly opened a pull request:

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

    [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE USING...", uses "CREAT TEMPORARY VIEW USING..." instead

    ## What changes were proposed in this pull request?
    
    The current implementation of "CREATE TEMPORARY TABLE USING..." is actually creating a temporary VIEW behind the scene. We should just use "CREATE TEMPORARY VIEW USING..." instead.
    
    Example:
    
    ```
    scala> spark.sql("CREATE temporary view  my_tab7 (c1: String, c2: String)  USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat  OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')")
    scala> spark.sql("select c1, c2 from my_tab7").show()
    +----+-----+
    |  c1|   c2|
    +----+-----+
    |year| make|
    |2012|Tesla|
    ...
    ```
    
    It NOW prints a **deprecation warning** if "CREATE TEMPORARY TABLE USING..." is used.
    
    ```
    scala> spark.sql("CREATE temporary table  my_tab7 (c1: String, c2: String)  USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat  OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')")
    16/05/31 10:39:27 WARN SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE tableName USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead
    ```
    
    
    ## How was this patch tested?
    
    Unit test.


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

    $ git pull https://github.com/clockfly/spark create_temp_view_using

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

    https://github.com/apache/spark/pull/13414.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 #13414
    
----
commit 94d66c20c23cbb286495be70988677d4b5f29b3d
Author: Sean Zhong <se...@databricks.com>
Date:   2016-05-31T17:32:13Z

    SPARK-15674: Deprecates "CREATE TEMPORARY TABLE USING...", use "CREATE TEMPORARY VIEW USING..." instead

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY T...

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

    https://github.com/apache/spark/pull/13414#discussion_r65967943
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---
    @@ -115,6 +115,10 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
         if (!path.endsWith(File.separator)) path + File.separator else path
       }
     
    +  private def testFile(fileName: String): String = {
    +    Thread.currentThread().getContextClassLoader.getResource(fileName).toString
    --- End diff --
    
    Why is this needed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #60081 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60081/consoleFull)** for PR 13414 at commit [`127c309`](https://github.com/apache/spark/commit/127c30962bcea93a26ed9572c5e680e3d01e0430).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    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: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    @hvanhovell 
    
    > create temp table ... using statement describes the access to a physical storage; which in my book is a table.
    
    We still allow `create table using...`, what we deprecate is "create **temporary** table using...". We still treats external data source as tables. To wrap the external data source as a table, we can use "create table using..." .
    
    temporary views and temporary tables are intermediate layers between user and the actual table:
    
    ```
    User --> Temporary view/table --> External data source table (Can be wrapped by create table using...)
    ```
    
    Currently, in our implementation, we don't support temporary table, we only supports temporary view. The difference is that:
    1. Temporary view is backed by a SQL string, which acts like a pointer. So every time we use the temporary view, the SQL is RE-executed again, which will asks data from the original data source.
    2. Temporary table is supposed to execute the SQL for ONLY once, and store the result in a temporary HDFS directory. Every time you use the temporary table, we are actually using the data in the temporary HDFS directory directly, without bothering the original data source.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY T...

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

    https://github.com/apache/spark/pull/13414#discussion_r65966588
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -344,6 +344,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
       }
     
       /**
    +   * Creates a [[CreateTempViewUsing]] logical plan.
    +   */
    +  override def visitCreateTempViewUsing(ctx: CreateTempViewUsingContext)
    --- End diff --
    
    NIT: Style move the `ctx` to the next 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: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #59663 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/59663/consoleFull)** for PR 13414 at commit [`c2a29b3`](https://github.com/apache/spark/commit/c2a29b3fb8d5f8f0ceb12c0e62ac1d71cd3d1dd0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60081/
    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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    @clockfly this looks pretty good. I have left some (minor) comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    @clockfly the description is getting there. IIUC the problem we are solving is the following:
    
    `CREATE TEMPORARY TABLE ... USING ...` allows us to create a temporary (session bound) connection to a (potentially) permanent data store. When the session finishes, the table **definition** (connection) is dropped, but the **data** is not. This is more-or-less the behavior you expect with a `TEMPORARY EXTERNAL` table (do we have those?), and this actually violates the common definition of a temporary table in which both the table definition and the data are session bound.
    
    Using `CREATE TEMPORARY VIEW ... USING ...` accomplishes two things:
    - It doesn't make assumption about the underlying data (it can both be permanent or session bound).
    - It doesn't allow user to write to the datasource.
    
    I do have a couple of issues with this:
    - Using `CREATE TEMPORARY TABLE ... USING ...` should still be allowed to use if you are using an actual session local temporary table. We could detect these by checking if a schema is defined (the location is also an issue). How do we deal with this use case?
    - I would support creating a `CREATE TEMPORARY EXTERNAL TABLE ... USING ...` to retain the current behavior.
    
    What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #59662 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/59662/consoleFull)** for PR 13414 at commit [`94d66c2`](https://github.com/apache/spark/commit/94d66c20c23cbb286495be70988677d4b5f29b3d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #60081 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60081/consoleFull)** for PR 13414 at commit [`127c309`](https://github.com/apache/spark/commit/127c30962bcea93a26ed9572c5e680e3d01e0430).
     * 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    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: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    So I am not sure I understand this one. Why should we deprecate this in favour of creating a view? A `create temp table ... using` statement describes the access to a physical storage; which in my book is a table.
    
    Could you elaborate on why we need 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: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    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: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/59662/
    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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    LGTM pending Jenkins


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY T...

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

    https://github.com/apache/spark/pull/13414#discussion_r65967765
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala ---
    @@ -375,9 +375,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
       object DDLStrategy extends Strategy {
         def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
           case c: CreateTableUsing if c.temporary && !c.allowExisting =>
    +        logWarning(
    +          s"CREATE TEMPORARY TABLE tableName USING... is deprecated, please use CREATE " +
    --- End diff --
    
    `tableName` is not interpolated at the moment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    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: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    I think the name of the `SessionCatalog.createTempView` is a misnomer - this is strengthened by the fact that the documentation and usage all refer to create temp tables...
    
    I am pretty sure that no query is executed in this case. It will just scan the data. For example the following REPL code:
    ```scala
    import java.nio.file.Files
    val location = Files.createTempDirectory("data").resolve("src")
    spark.range(0, 100000).
      select($"id".as("key"), rand().as("value")).
      write.parquet(location.toString)
    spark.sql(s"create temporary table my_src using parquet options(path '$location')")
    spark.table("my_src").explain(true)
    ```
    Yields the following plan:
    ```
    == Parsed Logical Plan ==
    SubqueryAlias my_src
    +- Relation[key#14L,value#15] parquet
    
    == Analyzed Logical Plan ==
    key: bigint, value: double
    SubqueryAlias my_src
    +- Relation[key#14L,value#15] parquet
    
    == Optimized Logical Plan ==
    Relation[key#14L,value#15] parquet
    
    == Physical Plan ==
    *BatchedScan parquet [key#14L,value#15] Format: ParquetFormat, InputPaths: file:/tmp/data8602759574255545993/src, PushedFilters: [], ReadSchema: struct<key:bigint,value:double>
    ```
    
    Am I missing something?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60079/
    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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    @hvanhovell  Thanks for the review. 
    
    Updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    @hvanhovell 
    
    I updated the description, please check whether it makes more sense now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #59663 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/59663/consoleFull)** for PR 13414 at commit [`c2a29b3`](https://github.com/apache/spark/commit/c2a29b3fb8d5f8f0ceb12c0e62ac1d71cd3d1dd0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class CreateTempViewUsing(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY T...

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

    https://github.com/apache/spark/pull/13414#discussion_r65966463
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
    @@ -90,6 +90,8 @@ statement
             identifierCommentList? (COMMENT STRING)?
             (PARTITIONED ON identifierList)?
             (TBLPROPERTIES tablePropertyList)? AS query                    #createView
    +    | CREATE (OR REPLACE)? TEMPORARY VIEW tableIdentifier ('(' colTypeList ')')? tableProvider
    --- End diff --
    
    NIT: Could you break this line up so we keep all `#...` hooks on the same column...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY T...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY T...

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

    https://github.com/apache/spark/pull/13414#discussion_r65966717
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -344,6 +344,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
       }
     
       /**
    +   * Creates a [[CreateTempViewUsing]] logical plan.
    +   */
    +  override def visitCreateTempViewUsing(ctx: CreateTempViewUsingContext)
    +    : LogicalPlan = withOrigin(ctx) {
    +    val options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)
    --- End diff --
    
    NIT: you could create the `CreateTempViewUsing` immediately 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/59663/
    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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    Thanks! Merging to master/2.0


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #60079 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60079/consoleFull)** for PR 13414 at commit [`e521310`](https://github.com/apache/spark/commit/e5213100ebf9ce6c7ec96c6b9700ca2ce4e8de34).
     * 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-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #59662 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/59662/consoleFull)** for PR 13414 at commit [`94d66c2`](https://github.com/apache/spark/commit/94d66c20c23cbb286495be70988677d4b5f29b3d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class CreateTempViewUsing(`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    **[Test build #60079 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60079/consoleFull)** for PR 13414 at commit [`e521310`](https://github.com/apache/spark/commit/e5213100ebf9ce6c7ec96c6b9700ca2ce4e8de34).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA 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 #13414: [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE US...

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

    https://github.com/apache/spark/pull/13414
  
    @hvanhovell Probably we can talk more face to face next week.


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

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